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

[iotdb] branch master updated: [IOTDB-2841] add permission manager model (#5401)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new a8f7439db9 [IOTDB-2841] add permission manager model (#5401)
a8f7439db9 is described below

commit a8f7439db9e2419e3c7443082ce7d476394651d8
Author: 任宇华 <79...@users.noreply.github.com>
AuthorDate: Sat Apr 9 07:13:59 2022 +0800

    [IOTDB-2841] add permission manager model (#5401)
    
    Co-authored-by: renyuhua <ry...@163.com>
---
 confignode/pom.xml                                 |   5 +
 .../response/PermissionInfoDataSet.java}           |  18 +-
 .../statemachine/PartitionRegionStateMachine.java  |   5 +-
 .../iotdb/confignode/manager/ConfigManager.java    |  12 +
 .../apache/iotdb/confignode/manager/Manager.java   |   8 +
 .../PermissionManager.java}                        |  32 ++-
 .../persistence/AuthorInfoPersistence.java         | 146 +++++++++++
 .../iotdb/confignode/physical/PhysicalPlan.java    |  20 ++
 .../confignode/physical/PhysicalPlanType.java      |  20 +-
 .../iotdb/confignode/physical/sys/AuthorPlan.java  | 243 +++++++++++++++++++
 .../confignode/service/executor/PlanExecutor.java  |  36 ++-
 .../server/ConfigNodeRPCServerProcessor.java       |  25 ++
 .../apache/iotdb/confignode/auth/AuthorTest.java}  |  36 ++-
 .../db/auth/authorizer/AuthorizerManager.java      | 190 +++++++++++++++
 .../iotdb/db/auth/authorizer/BasicAuthorizer.java  |   1 +
 .../db/auth/authorizer/ClusterAuthorizer.java      |  15 +-
 .../iotdb/db/metadata/LocalConfigManager.java      |   1 -
 .../apache/iotdb/db/mpp/sql/analyze/Analyzer.java  | 142 +++++++++++
 .../apache/iotdb/db/mpp/sql/parser/ASTVisitor.java | 223 ++++++++++++++++-
 .../iotdb/db/mpp/sql/planner/LogicalPlanner.java   | 144 ++++++++++-
 .../db/mpp/sql/planner/plan/node/PlanNodeType.java |   6 +-
 .../plan/node/metedata/write/AuthorNode.java       | 268 +++++++++++++++++++++
 .../db/mpp/sql/statement/StatementVisitor.java     |  98 ++++++++
 .../db/mpp/sql/statement/sys/AuthorStatement.java  | 152 ++++++++++++
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  53 ++--
 .../iotdb/db/mpp/sql/plan/LogicalPlannerTest.java  | 149 ++++++++++++
 .../src/main/thrift/confignode.thrift              |  13 +
 27 files changed, 1972 insertions(+), 89 deletions(-)

diff --git a/confignode/pom.xml b/confignode/pom.xml
index 01c496b516..24fecb0a8e 100644
--- a/confignode/pom.xml
+++ b/confignode/pom.xml
@@ -55,6 +55,11 @@
             <artifactId>iotdb-consensus</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-server</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <dependency>
             <groupId>io.dropwizard.metrics</groupId>
             <artifactId>metrics-core</artifactId>
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/PermissionInfoDataSet.java
similarity index 73%
copy from confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java
copy to confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/PermissionInfoDataSet.java
index d7606b4787..a60a2377fb 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/PermissionInfoDataSet.java
@@ -16,17 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.confignode.physical;
 
-public enum PhysicalPlanType {
-  RegisterDataNode,
-  QueryDataNodeInfo,
-  SetStorageGroup,
-  DeleteStorageGroup,
-  QueryStorageGroupSchema,
-  CreateRegion,
-  QueryDataPartition,
-  ApplyDataPartition,
-  QuerySchemaPartition,
-  ApplySchemaPartition
+package org.apache.iotdb.confignode.consensus.response;
+
+import org.apache.iotdb.consensus.common.DataSet;
+
+public class PermissionInfoDataSet implements DataSet {
+  // TODO: Store the returned result
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/PartitionRegionStateMachine.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/PartitionRegionStateMachine.java
index 2c63b40946..9eb7fdfde0 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/PartitionRegionStateMachine.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/PartitionRegionStateMachine.java
@@ -25,6 +25,7 @@ import org.apache.iotdb.consensus.common.DataSet;
 import org.apache.iotdb.consensus.common.request.ByteBufferConsensusRequest;
 import org.apache.iotdb.consensus.common.request.IConsensusRequest;
 import org.apache.iotdb.consensus.statemachine.IStateMachine;
+import org.apache.iotdb.db.auth.AuthException;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.service.rpc.thrift.TSStatus;
 
@@ -68,7 +69,7 @@ public class PartitionRegionStateMachine implements IStateMachine {
     TSStatus result;
     try {
       result = executor.executorNonQueryPlan(plan);
-    } catch (UnknownPhysicalPlanTypeException e) {
+    } catch (UnknownPhysicalPlanTypeException | AuthException e) {
       LOGGER.error(e.getMessage());
       result = new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode());
     }
@@ -99,7 +100,7 @@ public class PartitionRegionStateMachine implements IStateMachine {
     DataSet result;
     try {
       result = executor.executorQueryPlan(plan);
-    } catch (UnknownPhysicalPlanTypeException e) {
+    } catch (UnknownPhysicalPlanTypeException | AuthException e) {
       LOGGER.error(e.getMessage());
       result = null;
     }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
index 55f8f41f98..0090974f0a 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
@@ -23,6 +23,7 @@ import org.apache.iotdb.confignode.conf.ConfigNodeConf;
 import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
 import org.apache.iotdb.confignode.consensus.response.DataNodesInfoDataSet;
 import org.apache.iotdb.confignode.physical.PhysicalPlan;
+import org.apache.iotdb.confignode.physical.sys.AuthorPlan;
 import org.apache.iotdb.confignode.physical.sys.DataPartitionPlan;
 import org.apache.iotdb.confignode.physical.sys.QueryDataNodeInfoPlan;
 import org.apache.iotdb.confignode.physical.sys.RegisterDataNodePlan;
@@ -53,11 +54,14 @@ public class ConfigManager implements Manager {
   /** manager assign schema region and data region */
   private final RegionManager regionManager;
 
+  private final PermissionManager permissionManager;
+
   public ConfigManager() throws IOException {
     this.dataNodeManager = new DataNodeManager(this);
     this.partitionManager = new PartitionManager(this);
     this.regionManager = new RegionManager(this);
     this.consensusManager = new ConsensusManager();
+    this.permissionManager = new PermissionManager(this);
   }
 
   public void close() throws IOException {
@@ -150,4 +154,12 @@ public class ConfigManager implements Manager {
   public ConsensusManager getConsensusManager() {
     return consensusManager;
   }
+
+  @Override
+  public TSStatus operatePermission(PhysicalPlan physicalPlan) {
+    if (physicalPlan instanceof AuthorPlan) {
+      return permissionManager.operatePermission((AuthorPlan) physicalPlan);
+    }
+    return ERROR_TSSTATUS;
+  }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/Manager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/Manager.java
index a0228f1afc..292bf8d10d 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/Manager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/Manager.java
@@ -116,4 +116,12 @@ public interface Manager {
   DeviceGroupHashInfo getDeviceGroupHashInfo();
 
   ConsensusManager getConsensusManager();
+
+  /**
+   * operate permission
+   *
+   * @param physicalPlan
+   * @return
+   */
+  TSStatus operatePermission(PhysicalPlan physicalPlan);
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java
similarity index 58%
copy from confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java
copy to confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java
index d7606b4787..a884b577ae 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java
@@ -16,17 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.confignode.physical;
 
-public enum PhysicalPlanType {
-  RegisterDataNode,
-  QueryDataNodeInfo,
-  SetStorageGroup,
-  DeleteStorageGroup,
-  QueryStorageGroupSchema,
-  CreateRegion,
-  QueryDataPartition,
-  ApplyDataPartition,
-  QuerySchemaPartition,
-  ApplySchemaPartition
+package org.apache.iotdb.confignode.manager;
+
+import org.apache.iotdb.confignode.physical.sys.AuthorPlan;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+
+public class PermissionManager {
+
+  private Manager configNodeManager;
+
+  public PermissionManager(Manager configManager) {
+    this.configNodeManager = configManager;
+  }
+
+  public TSStatus operatePermission(AuthorPlan authorPlan) {
+    return getConsensusManager().write(authorPlan).getStatus();
+  }
+
+  private ConsensusManager getConsensusManager() {
+    return configNodeManager.getConsensusManager();
+  }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfoPersistence.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfoPersistence.java
new file mode 100644
index 0000000000..a900f817ed
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfoPersistence.java
@@ -0,0 +1,146 @@
+/*
+ * 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.persistence;
+
+import org.apache.iotdb.confignode.consensus.response.PermissionInfoDataSet;
+import org.apache.iotdb.confignode.physical.PhysicalPlanType;
+import org.apache.iotdb.confignode.physical.sys.AuthorPlan;
+import org.apache.iotdb.db.auth.AuthException;
+import org.apache.iotdb.db.auth.authorizer.BasicAuthorizer;
+import org.apache.iotdb.db.auth.authorizer.IAuthorizer;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Set;
+
+public class AuthorInfoPersistence {
+
+  private static final Logger logger = LoggerFactory.getLogger(AuthorInfoPersistence.class);
+
+  private IAuthorizer authorizer;
+
+  {
+    try {
+      authorizer = BasicAuthorizer.getInstance();
+    } catch (AuthException e) {
+      logger.error("get user or role info failed", e);
+    }
+  }
+
+  public TSStatus authorNonQuery(AuthorPlan authorPlan) throws AuthException {
+    PhysicalPlanType authorType = authorPlan.getAuthorType();
+    String userName = authorPlan.getUserName();
+    String roleName = authorPlan.getRoleName();
+    String password = authorPlan.getPassword();
+    String newPassword = authorPlan.getNewPassword();
+    Set<Integer> permissions = authorPlan.getPermissions();
+    String nodeName = authorPlan.getNodeName();
+    try {
+      switch (authorType) {
+        case UPDATE_USER:
+          authorizer.updateUserPassword(userName, newPassword);
+          break;
+        case CREATE_USER:
+          authorizer.createUser(userName, password);
+          break;
+        case CREATE_ROLE:
+          authorizer.createRole(roleName);
+          break;
+        case DROP_USER:
+          authorizer.deleteUser(userName);
+          break;
+        case DROP_ROLE:
+          authorizer.deleteRole(roleName);
+          break;
+        case GRANT_ROLE:
+          for (int i : permissions) {
+            authorizer.grantPrivilegeToRole(roleName, nodeName, i);
+          }
+          break;
+        case GRANT_USER:
+          for (int i : permissions) {
+            authorizer.grantPrivilegeToUser(userName, nodeName, i);
+          }
+          break;
+        case GRANT_ROLE_TO_USER:
+          authorizer.grantRoleToUser(roleName, userName);
+          break;
+        case REVOKE_USER:
+          for (int i : permissions) {
+            authorizer.revokePrivilegeFromUser(userName, nodeName, i);
+          }
+          break;
+        case REVOKE_ROLE:
+          for (int i : permissions) {
+            authorizer.revokePrivilegeFromRole(roleName, nodeName, i);
+          }
+          break;
+        case REVOKE_ROLE_FROM_USER:
+          authorizer.revokeRoleFromUser(roleName, userName);
+          break;
+        default:
+          throw new AuthException("execute " + authorPlan + " failed");
+      }
+    } catch (AuthException e) {
+      throw new AuthException("execute " + authorPlan + " failed: ", e);
+    }
+    return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
+  }
+
+  /** TODO: Construct the query result as a DataSet and return it */
+  public PermissionInfoDataSet executeListRole(AuthorPlan plan) throws AuthException {
+    return null;
+  }
+
+  public PermissionInfoDataSet executeListUser(AuthorPlan plan) throws AuthException {
+    return null;
+  }
+
+  public PermissionInfoDataSet executeListRoleUsers(AuthorPlan plan) throws AuthException {
+    return null;
+  }
+
+  public PermissionInfoDataSet executeListUserRoles(AuthorPlan plan) throws AuthException {
+    return null;
+  }
+
+  public PermissionInfoDataSet executeListRolePrivileges(AuthorPlan plan) throws AuthException {
+    return null;
+  }
+
+  public PermissionInfoDataSet executeListUserPrivileges(AuthorPlan plan) throws AuthException {
+    return null;
+  }
+
+  private static class AuthorInfoPersistenceHolder {
+
+    private static final AuthorInfoPersistence INSTANCE = new AuthorInfoPersistence();
+
+    private AuthorInfoPersistenceHolder() {
+      // empty constructor
+    }
+  }
+
+  public static AuthorInfoPersistence getInstance() {
+    return AuthorInfoPersistence.AuthorInfoPersistenceHolder.INSTANCE;
+  }
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlan.java
index fefc3f54b6..1c71cf70ff 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlan.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.confignode.physical;
 
+import org.apache.iotdb.confignode.physical.sys.AuthorPlan;
 import org.apache.iotdb.confignode.physical.sys.DataPartitionPlan;
 import org.apache.iotdb.confignode.physical.sys.QueryDataNodeInfoPlan;
 import org.apache.iotdb.confignode.physical.sys.QueryStorageGroupSchemaPlan;
@@ -109,6 +110,25 @@ public abstract class PhysicalPlan implements IConsensusRequest {
         case ApplySchemaPartition:
           plan = new SchemaPartitionPlan(PhysicalPlanType.ApplySchemaPartition);
           break;
+        case LIST_USER:
+        case LIST_ROLE:
+        case LIST_USER_PRIVILEGE:
+        case LIST_ROLE_PRIVILEGE:
+        case LIST_USER_ROLES:
+        case LIST_ROLE_USERS:
+        case CREATE_USER:
+        case CREATE_ROLE:
+        case DROP_USER:
+        case DROP_ROLE:
+        case GRANT_ROLE:
+        case GRANT_USER:
+        case GRANT_ROLE_TO_USER:
+        case REVOKE_USER:
+        case REVOKE_ROLE:
+        case REVOKE_ROLE_FROM_USER:
+        case UPDATE_USER:
+          plan = new AuthorPlan(type);
+          break;
         default:
           throw new IOException("unknown PhysicalPlan type: " + typeNum);
       }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java
index d7606b4787..540bf0bdd6 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java
@@ -28,5 +28,23 @@ public enum PhysicalPlanType {
   QueryDataPartition,
   ApplyDataPartition,
   QuerySchemaPartition,
-  ApplySchemaPartition
+  ApplySchemaPartition,
+  AUTHOR,
+  CREATE_USER,
+  CREATE_ROLE,
+  DROP_USER,
+  DROP_ROLE,
+  GRANT_ROLE,
+  GRANT_USER,
+  GRANT_ROLE_TO_USER,
+  REVOKE_USER,
+  REVOKE_ROLE,
+  REVOKE_ROLE_FROM_USER,
+  UPDATE_USER,
+  LIST_USER,
+  LIST_ROLE,
+  LIST_USER_PRIVILEGE,
+  LIST_ROLE_PRIVILEGE,
+  LIST_USER_ROLES,
+  LIST_ROLE_USERS
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/AuthorPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/AuthorPlan.java
new file mode 100644
index 0000000000..32ac83dc42
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/AuthorPlan.java
@@ -0,0 +1,243 @@
+/*
+ * 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.physical.sys;
+
+import org.apache.iotdb.confignode.physical.PhysicalPlan;
+import org.apache.iotdb.confignode.physical.PhysicalPlanType;
+import org.apache.iotdb.db.auth.AuthException;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+public class AuthorPlan extends PhysicalPlan {
+
+  private PhysicalPlanType authorType;
+  private String roleName;
+  private String password;
+  private String newPassword;
+  private Set<Integer> permissions;
+  private String nodeName;
+  private String userName;
+
+  public AuthorPlan(PhysicalPlanType type) {
+    super(type);
+  }
+
+  /**
+   * AuthorPlan Constructor.
+   *
+   * @param authorType author type
+   * @param userName user name
+   * @param roleName role name
+   * @param password password
+   * @param newPassword new password
+   * @param permissions permissions
+   * @param nodeName node name in Path structure
+   * @throws AuthException Authentication Exception
+   */
+  public AuthorPlan(
+      PhysicalPlanType authorType,
+      String userName,
+      String roleName,
+      String password,
+      String newPassword,
+      Set<Integer> permissions,
+      String nodeName)
+      throws AuthException {
+    this(authorType);
+    this.authorType = authorType;
+    this.userName = userName;
+    this.roleName = roleName;
+    this.password = password;
+    this.newPassword = newPassword;
+    this.permissions = permissions;
+    this.nodeName = nodeName;
+  }
+
+  public PhysicalPlanType getAuthorType() {
+    return authorType;
+  }
+
+  public void setAuthorType(PhysicalPlanType authorType) {
+    this.authorType = authorType;
+  }
+
+  public String getRoleName() {
+    return roleName;
+  }
+
+  public void setRoleName(String roleName) {
+    this.roleName = roleName;
+  }
+
+  public String getPassword() {
+    return password;
+  }
+
+  public void setPassword(String password) {
+    this.password = password;
+  }
+
+  public String getNewPassword() {
+    return newPassword;
+  }
+
+  public void setNewPassword(String newPassword) {
+    this.newPassword = newPassword;
+  }
+
+  public Set<Integer> getPermissions() {
+    return permissions;
+  }
+
+  public void setPermissions(Set<Integer> permissions) {
+    this.permissions = permissions;
+  }
+
+  public String getNodeName() {
+    return nodeName;
+  }
+
+  public void setNodeName(String nodeName) {
+    this.nodeName = nodeName;
+  }
+
+  public String getUserName() {
+    return userName;
+  }
+
+  public void setUserName(String userName) {
+    this.userName = userName;
+  }
+
+  @Override
+  protected void serializeImpl(ByteBuffer buffer) {
+    buffer.putInt(getPlanTypeOrdinal(authorType));
+    buffer.putInt(userName.length());
+    buffer.put(userName.getBytes());
+    buffer.putInt(roleName.length());
+    buffer.put(roleName.getBytes());
+    buffer.putInt(password.length());
+    buffer.put(password.getBytes());
+    buffer.putInt(newPassword.length());
+    buffer.put(newPassword.getBytes());
+    if (permissions == null && permissions.size() == 0) {
+      buffer.put("false".getBytes());
+    } else {
+      buffer.put("true".getBytes());
+      buffer.putInt(permissions.size());
+      for (Integer permission : permissions) {
+        buffer.putInt(permission);
+      }
+    }
+    if (nodeName == null && nodeName.equals("")) {
+      buffer.put("false".getBytes());
+    } else {
+      buffer.put("true".getBytes());
+      buffer.putInt(nodeName.length());
+      buffer.put(nodeName.getBytes());
+    }
+  }
+
+  @Override
+  protected void deserializeImpl(ByteBuffer buffer) {
+    userName = getAuthorInfo(buffer);
+    roleName = getAuthorInfo(buffer);
+    password = getAuthorInfo(buffer);
+    newPassword = getAuthorInfo(buffer);
+    String permissionIsNull = getAuthorInfo(buffer);
+    if ("true".equals(permissionIsNull)) {
+      int permissionSize = buffer.getInt();
+      for (int i = 0; i < permissionSize; i++) {
+        permissions.add(buffer.getInt());
+      }
+    }
+    String nodeNameIsNull = getAuthorInfo(buffer);
+    if ("true".equals(nodeNameIsNull)) {
+      nodeName = getAuthorInfo(buffer);
+    }
+  }
+
+  private String getAuthorInfo(ByteBuffer buffer) {
+    int infoSize = buffer.getInt();
+    byte[] byteInfo = new byte[infoSize];
+    buffer.get(byteInfo, 0, infoSize);
+    return new String(byteInfo, 0, infoSize);
+  }
+
+  private int getPlanTypeOrdinal(PhysicalPlanType physicalPlanType) {
+    int type;
+    switch (physicalPlanType) {
+      case CREATE_USER:
+        type = PhysicalPlanType.CREATE_USER.ordinal();
+        break;
+      case CREATE_ROLE:
+        type = PhysicalPlanType.CREATE_ROLE.ordinal();
+        break;
+      case DROP_USER:
+        type = PhysicalPlanType.DROP_USER.ordinal();
+        break;
+      case DROP_ROLE:
+        type = PhysicalPlanType.DROP_ROLE.ordinal();
+        break;
+      case GRANT_ROLE:
+        type = PhysicalPlanType.GRANT_ROLE.ordinal();
+        break;
+      case GRANT_USER:
+        type = PhysicalPlanType.GRANT_USER.ordinal();
+        break;
+      case GRANT_ROLE_TO_USER:
+        type = PhysicalPlanType.GRANT_ROLE_TO_USER.ordinal();
+        break;
+      case REVOKE_USER:
+        type = PhysicalPlanType.REVOKE_USER.ordinal();
+        break;
+      case REVOKE_ROLE:
+        type = PhysicalPlanType.REVOKE_ROLE.ordinal();
+        break;
+      case REVOKE_ROLE_FROM_USER:
+        type = PhysicalPlanType.REVOKE_ROLE_FROM_USER.ordinal();
+        break;
+      case UPDATE_USER:
+        type = PhysicalPlanType.UPDATE_USER.ordinal();
+        break;
+      case LIST_USER:
+        type = PhysicalPlanType.LIST_USER.ordinal();
+        break;
+      case LIST_ROLE:
+        type = PhysicalPlanType.LIST_ROLE.ordinal();
+        break;
+      case LIST_USER_PRIVILEGE:
+        type = PhysicalPlanType.LIST_USER_PRIVILEGE.ordinal();
+        break;
+      case LIST_ROLE_PRIVILEGE:
+        type = PhysicalPlanType.LIST_ROLE_PRIVILEGE.ordinal();
+        break;
+      case LIST_USER_ROLES:
+        type = PhysicalPlanType.LIST_USER_ROLES.ordinal();
+        break;
+      case LIST_ROLE_USERS:
+        type = PhysicalPlanType.LIST_ROLE_USERS.ordinal();
+        break;
+      default:
+        throw new IllegalArgumentException("Unknown operator: " + physicalPlanType);
+    }
+    return type;
+  }
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/executor/PlanExecutor.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/executor/PlanExecutor.java
index 6f4c14ca6e..88e036a620 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/executor/PlanExecutor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/executor/PlanExecutor.java
@@ -19,16 +19,19 @@
 package org.apache.iotdb.confignode.service.executor;
 
 import org.apache.iotdb.confignode.exception.physical.UnknownPhysicalPlanTypeException;
+import org.apache.iotdb.confignode.persistence.AuthorInfoPersistence;
 import org.apache.iotdb.confignode.persistence.DataNodeInfoPersistence;
 import org.apache.iotdb.confignode.persistence.PartitionInfoPersistence;
 import org.apache.iotdb.confignode.persistence.RegionInfoPersistence;
 import org.apache.iotdb.confignode.physical.PhysicalPlan;
+import org.apache.iotdb.confignode.physical.sys.AuthorPlan;
 import org.apache.iotdb.confignode.physical.sys.DataPartitionPlan;
 import org.apache.iotdb.confignode.physical.sys.QueryDataNodeInfoPlan;
 import org.apache.iotdb.confignode.physical.sys.RegisterDataNodePlan;
 import org.apache.iotdb.confignode.physical.sys.SchemaPartitionPlan;
 import org.apache.iotdb.confignode.physical.sys.SetStorageGroupPlan;
 import org.apache.iotdb.consensus.common.DataSet;
+import org.apache.iotdb.db.auth.AuthException;
 import org.apache.iotdb.service.rpc.thrift.TSStatus;
 
 public class PlanExecutor {
@@ -39,13 +42,17 @@ public class PlanExecutor {
 
   private final PartitionInfoPersistence partitionInfoPersistence;
 
+  private final AuthorInfoPersistence authorInfoPersistence;
+
   public PlanExecutor() {
     this.dataNodeInfoPersistence = DataNodeInfoPersistence.getInstance();
     this.regionInfoPersistence = RegionInfoPersistence.getInstance();
     this.partitionInfoPersistence = PartitionInfoPersistence.getInstance();
+    this.authorInfoPersistence = AuthorInfoPersistence.getInstance();
   }
 
-  public DataSet executorQueryPlan(PhysicalPlan plan) throws UnknownPhysicalPlanTypeException {
+  public DataSet executorQueryPlan(PhysicalPlan plan)
+      throws UnknownPhysicalPlanTypeException, AuthException {
     switch (plan.getType()) {
       case QueryDataNodeInfo:
         return dataNodeInfoPersistence.getDataNodeInfo((QueryDataNodeInfoPlan) plan);
@@ -59,17 +66,42 @@ public class PlanExecutor {
         return partitionInfoPersistence.applySchemaPartition((SchemaPartitionPlan) plan);
       case ApplyDataPartition:
         return partitionInfoPersistence.applyDataPartition((DataPartitionPlan) plan);
+      case LIST_USER:
+        return authorInfoPersistence.executeListUser((AuthorPlan) plan);
+      case LIST_ROLE:
+        return authorInfoPersistence.executeListRole((AuthorPlan) plan);
+      case LIST_USER_PRIVILEGE:
+        return authorInfoPersistence.executeListUserPrivileges((AuthorPlan) plan);
+      case LIST_ROLE_PRIVILEGE:
+        return authorInfoPersistence.executeListRolePrivileges((AuthorPlan) plan);
+      case LIST_USER_ROLES:
+        return authorInfoPersistence.executeListUserRoles((AuthorPlan) plan);
+      case LIST_ROLE_USERS:
+        return authorInfoPersistence.executeListRoleUsers((AuthorPlan) plan);
       default:
         throw new UnknownPhysicalPlanTypeException(plan.getType());
     }
   }
 
-  public TSStatus executorNonQueryPlan(PhysicalPlan plan) throws UnknownPhysicalPlanTypeException {
+  public TSStatus executorNonQueryPlan(PhysicalPlan plan)
+      throws UnknownPhysicalPlanTypeException, AuthException {
     switch (plan.getType()) {
       case RegisterDataNode:
         return dataNodeInfoPersistence.registerDataNode((RegisterDataNodePlan) plan);
       case SetStorageGroup:
         return regionInfoPersistence.setStorageGroup((SetStorageGroupPlan) plan);
+      case CREATE_USER:
+      case CREATE_ROLE:
+      case DROP_USER:
+      case DROP_ROLE:
+      case GRANT_ROLE:
+      case GRANT_USER:
+      case GRANT_ROLE_TO_USER:
+      case REVOKE_USER:
+      case REVOKE_ROLE:
+      case REVOKE_ROLE_FROM_USER:
+      case UPDATE_USER:
+        return authorInfoPersistence.authorNonQuery((AuthorPlan) plan);
       default:
         throw new UnknownPhysicalPlanTypeException(plan.getType());
     }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessor.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessor.java
index f9e4741a00..396510b523 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessor.java
@@ -26,10 +26,12 @@ import org.apache.iotdb.confignode.consensus.response.StorageGroupSchemaDataSet;
 import org.apache.iotdb.confignode.manager.ConfigManager;
 import org.apache.iotdb.confignode.partition.StorageGroupSchema;
 import org.apache.iotdb.confignode.physical.PhysicalPlanType;
+import org.apache.iotdb.confignode.physical.sys.AuthorPlan;
 import org.apache.iotdb.confignode.physical.sys.QueryDataNodeInfoPlan;
 import org.apache.iotdb.confignode.physical.sys.RegisterDataNodePlan;
 import org.apache.iotdb.confignode.physical.sys.SchemaPartitionPlan;
 import org.apache.iotdb.confignode.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.confignode.rpc.thrift.AuthorizerReq;
 import org.apache.iotdb.confignode.rpc.thrift.ConfigIService;
 import org.apache.iotdb.confignode.rpc.thrift.DataNodeMessage;
 import org.apache.iotdb.confignode.rpc.thrift.DataNodeRegisterReq;
@@ -49,6 +51,7 @@ import org.apache.iotdb.confignode.rpc.thrift.SchemaPartitionInfoResp;
 import org.apache.iotdb.confignode.rpc.thrift.SetStorageGroupReq;
 import org.apache.iotdb.confignode.rpc.thrift.StorageGroupMessage;
 import org.apache.iotdb.consensus.common.DataSet;
+import org.apache.iotdb.db.auth.AuthException;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.service.rpc.thrift.EndPoint;
 import org.apache.iotdb.service.rpc.thrift.TSStatus;
@@ -156,6 +159,28 @@ public class ConfigNodeRPCServerProcessor implements ConfigIService.Iface {
     return configManager.getDeviceGroupHashInfo();
   }
 
+  @Override
+  public TSStatus operatePermission(AuthorizerReq req) throws TException {
+    if (req.getAuthorType() < 0 || req.getAuthorType() >= PhysicalPlanType.values().length) {
+      throw new IndexOutOfBoundsException("Invalid ordinal");
+    }
+    AuthorPlan plan = null;
+    try {
+      plan =
+          new AuthorPlan(
+              PhysicalPlanType.values()[req.getAuthorType()],
+              req.getUserName(),
+              req.getRoleName(),
+              req.getPassword(),
+              req.getNewPassword(),
+              req.getPermissions(),
+              req.getNodeName());
+    } catch (AuthException e) {
+      LOGGER.error(e.getMessage());
+    }
+    return configManager.operatePermission(plan);
+  }
+
   @Override
   public DataPartitionInfo applyDataPartition(GetDataPartitionReq req) throws TException {
     return null;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java b/confignode/src/test/java/org/apache/iotdb/confignode/auth/AuthorTest.java
similarity index 57%
copy from confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java
copy to confignode/src/test/java/org/apache/iotdb/confignode/auth/AuthorTest.java
index d7606b4787..7d86edfabd 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/auth/AuthorTest.java
@@ -16,17 +16,29 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.confignode.physical;
+package org.apache.iotdb.confignode.auth;
 
-public enum PhysicalPlanType {
-  RegisterDataNode,
-  QueryDataNodeInfo,
-  SetStorageGroup,
-  DeleteStorageGroup,
-  QueryStorageGroupSchema,
-  CreateRegion,
-  QueryDataPartition,
-  ApplyDataPartition,
-  QuerySchemaPartition,
-  ApplySchemaPartition
+import org.apache.iotdb.confignode.utils.ConfigNodeEnvironmentUtils;
+import org.apache.iotdb.db.auth.authorizer.BasicAuthorizer;
+import org.apache.iotdb.db.auth.authorizer.IAuthorizer;
+
+import org.junit.After;
+import org.junit.Before;
+
+public class AuthorTest {
+
+  IAuthorizer authorizer;
+
+  @Before
+  public void setUp() throws Exception {
+    ConfigNodeEnvironmentUtils.envSetUp();
+    authorizer = BasicAuthorizer.getInstance();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    ConfigNodeEnvironmentUtils.cleanEnv();
+  }
+
+  // TODO: ADD TEST
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/auth/authorizer/AuthorizerManager.java b/server/src/main/java/org/apache/iotdb/db/auth/authorizer/AuthorizerManager.java
new file mode 100644
index 0000000000..495e23725a
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/auth/authorizer/AuthorizerManager.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.auth.authorizer;
+
+import org.apache.iotdb.db.auth.AuthException;
+import org.apache.iotdb.db.auth.entity.Role;
+import org.apache.iotdb.db.auth.entity.User;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class AuthorizerManager implements IAuthorizer {
+
+  private static final Logger logger = LoggerFactory.getLogger(AuthorizerManager.class);
+
+  IAuthorizer iAuthorizer;
+
+  public AuthorizerManager() {
+    try {
+      iAuthorizer = BasicAuthorizer.getInstance();
+    } catch (AuthException e) {
+      logger.error("Authorizer uninitialized");
+    }
+  }
+
+  /** SingleTone */
+  private static class AuthorizerManagerHolder {
+    private static final AuthorizerManager INSTANCE = new AuthorizerManager();
+
+    private AuthorizerManagerHolder() {}
+  }
+
+  public static AuthorizerManager getInstance() {
+    return AuthorizerManager.AuthorizerManagerHolder.INSTANCE;
+  }
+
+  @Override
+  public boolean login(String username, String password) throws AuthException {
+    return iAuthorizer.login(username, password);
+  }
+
+  @Override
+  public void createUser(String username, String password) throws AuthException {
+    iAuthorizer.createUser(username, password);
+  }
+
+  @Override
+  public void deleteUser(String username) throws AuthException {
+    iAuthorizer.deleteUser(username);
+  }
+
+  @Override
+  public void grantPrivilegeToUser(String username, String path, int privilegeId)
+      throws AuthException {
+    iAuthorizer.grantPrivilegeToUser(username, path, privilegeId);
+  }
+
+  @Override
+  public void revokePrivilegeFromUser(String username, String path, int privilegeId)
+      throws AuthException {
+    iAuthorizer.revokePrivilegeFromUser(username, path, privilegeId);
+  }
+
+  @Override
+  public void createRole(String roleName) throws AuthException {
+    iAuthorizer.createRole(roleName);
+  }
+
+  @Override
+  public void deleteRole(String roleName) throws AuthException {
+    iAuthorizer.deleteRole(roleName);
+  }
+
+  @Override
+  public void grantPrivilegeToRole(String roleName, String path, int privilegeId)
+      throws AuthException {
+    iAuthorizer.grantPrivilegeToRole(roleName, path, privilegeId);
+  }
+
+  @Override
+  public void revokePrivilegeFromRole(String roleName, String path, int privilegeId)
+      throws AuthException {
+    iAuthorizer.revokePrivilegeFromRole(roleName, path, privilegeId);
+  }
+
+  @Override
+  public void grantRoleToUser(String roleName, String username) throws AuthException {
+    iAuthorizer.grantRoleToUser(roleName, username);
+  }
+
+  @Override
+  public void revokeRoleFromUser(String roleName, String username) throws AuthException {
+    iAuthorizer.revokeRoleFromUser(roleName, username);
+  }
+
+  @Override
+  public Set<Integer> getPrivileges(String username, String path) throws AuthException {
+    return iAuthorizer.getPrivileges(username, path);
+  }
+
+  @Override
+  public void updateUserPassword(String username, String newPassword) throws AuthException {
+    iAuthorizer.updateUserPassword(username, newPassword);
+  }
+
+  @Override
+  public boolean checkUserPrivileges(String username, String path, int privilegeId)
+      throws AuthException {
+    return iAuthorizer.checkUserPrivileges(username, path, privilegeId);
+  }
+
+  @Override
+  public void reset() throws AuthException {}
+
+  @Override
+  public List<String> listAllUsers() {
+    return iAuthorizer.listAllUsers();
+  }
+
+  @Override
+  public List<String> listAllRoles() {
+    return iAuthorizer.listAllRoles();
+  }
+
+  @Override
+  public Role getRole(String roleName) throws AuthException {
+    return iAuthorizer.getRole(roleName);
+  }
+
+  @Override
+  public User getUser(String username) throws AuthException {
+    return iAuthorizer.getUser(username);
+  }
+
+  @Override
+  public boolean isUserUseWaterMark(String userName) throws AuthException {
+    return iAuthorizer.isUserUseWaterMark(userName);
+  }
+
+  @Override
+  public void setUserUseWaterMark(String userName, boolean useWaterMark) throws AuthException {
+    iAuthorizer.setUserUseWaterMark(userName, useWaterMark);
+  }
+
+  @Override
+  public Map<String, Boolean> getAllUserWaterMarkStatus() {
+    return iAuthorizer.getAllUserWaterMarkStatus();
+  }
+
+  @Override
+  public Map<String, User> getAllUsers() {
+    return iAuthorizer.getAllUsers();
+  }
+
+  @Override
+  public Map<String, Role> getAllRoles() {
+    return iAuthorizer.getAllRoles();
+  }
+
+  @Override
+  public void replaceAllUsers(Map<String, User> users) throws AuthException {
+    iAuthorizer.replaceAllUsers(users);
+  }
+
+  @Override
+  public void replaceAllRoles(Map<String, Role> roles) throws AuthException {
+    iAuthorizer.replaceAllRoles(roles);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/auth/authorizer/BasicAuthorizer.java b/server/src/main/java/org/apache/iotdb/db/auth/authorizer/BasicAuthorizer.java
index f244dd6745..e856ec8d25 100644
--- a/server/src/main/java/org/apache/iotdb/db/auth/authorizer/BasicAuthorizer.java
+++ b/server/src/main/java/org/apache/iotdb/db/auth/authorizer/BasicAuthorizer.java
@@ -47,6 +47,7 @@ public abstract class BasicAuthorizer implements IAuthorizer, IService {
   private static final String NO_SUCH_ROLE_EXCEPTION = "No such role : %s";
   private static final String NO_SUCH_USER_EXCEPTION = "No such user : %s";
 
+  // TODO: add cache
   static {
     ADMIN_PRIVILEGES = new HashSet<>();
     for (int i = 0; i < PrivilegeType.values().length; i++) {
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java b/server/src/main/java/org/apache/iotdb/db/auth/authorizer/ClusterAuthorizer.java
similarity index 73%
copy from confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java
copy to server/src/main/java/org/apache/iotdb/db/auth/authorizer/ClusterAuthorizer.java
index d7606b4787..321ea0c243 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/PhysicalPlanType.java
+++ b/server/src/main/java/org/apache/iotdb/db/auth/authorizer/ClusterAuthorizer.java
@@ -16,17 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.confignode.physical;
+package org.apache.iotdb.db.auth.authorizer;
 
-public enum PhysicalPlanType {
-  RegisterDataNode,
-  QueryDataNodeInfo,
-  SetStorageGroup,
-  DeleteStorageGroup,
-  QueryStorageGroupSchema,
-  CreateRegion,
-  QueryDataPartition,
-  ApplyDataPartition,
-  QuerySchemaPartition,
-  ApplySchemaPartition
+public class ClusterAuthorizer {
+  // TODO: send rpc to confignode
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/LocalConfigManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/LocalConfigManager.java
index e033b47c25..bd8c000446 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/LocalConfigManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/LocalConfigManager.java
@@ -782,5 +782,4 @@ public class LocalConfigManager {
   }
 
   // endregion
-
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/Analyzer.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/Analyzer.java
index ed4da79235..a8a61e7f9f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/Analyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/Analyzer.java
@@ -48,6 +48,7 @@ import org.apache.iotdb.db.mpp.sql.statement.crud.QueryStatement;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.AlterTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.CreateAlignedTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.CreateTimeSeriesStatement;
+import org.apache.iotdb.db.mpp.sql.statement.sys.AuthorStatement;
 
 import java.util.*;
 
@@ -222,6 +223,146 @@ public class Analyzer {
       return analysis;
     }
 
+    @Override
+    public Analysis visitCreateUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitCreateRole(AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitAlterUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitGrantUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitGrantRole(AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitGrantRoleToUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitRevokeUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitRevokeRole(AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitRevokeRoleFromUser(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitDropUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitDropRole(AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitListUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitListRole(AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitListPrivilegesUser(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitListPrivilegesRole(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitListUserPrivileges(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitListRolePrivileges(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitListAllRoleOfUser(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
+    @Override
+    public Analysis visitListAllUserOfRole(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      Analysis analysis = new Analysis();
+      analysis.setStatement(authorStatement);
+      return analysis;
+    }
+
     public Analysis visitInsertRow(InsertRowStatement insertRowStatement, MPPQueryContext context) {
       DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam();
       dataPartitionQueryParam.setDevicePath(insertRowStatement.getDevicePath().getFullPath());
@@ -254,6 +395,7 @@ public class Analyzer {
       analysis.setStatement(insertRowStatement);
       analysis.setDataPartitionInfo(partitionInfo.getDataPartitionInfo());
       analysis.setSchemaPartitionInfo(partitionInfo.getSchemaPartitionInfo());
+
       return analysis;
     }
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/parser/ASTVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/parser/ASTVisitor.java
index d291dc84d3..6ebbf4846a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/parser/ASTVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/parser/ASTVisitor.java
@@ -34,7 +34,17 @@ import org.apache.iotdb.db.mpp.common.filter.QueryFilter;
 import org.apache.iotdb.db.mpp.common.filter.RegexpFilter;
 import org.apache.iotdb.db.mpp.sql.constant.FilterConstant;
 import org.apache.iotdb.db.mpp.sql.statement.Statement;
-import org.apache.iotdb.db.mpp.sql.statement.component.*;
+import org.apache.iotdb.db.mpp.sql.statement.component.FillComponent;
+import org.apache.iotdb.db.mpp.sql.statement.component.FilterNullComponent;
+import org.apache.iotdb.db.mpp.sql.statement.component.FilterNullPolicy;
+import org.apache.iotdb.db.mpp.sql.statement.component.FromComponent;
+import org.apache.iotdb.db.mpp.sql.statement.component.GroupByLevelComponent;
+import org.apache.iotdb.db.mpp.sql.statement.component.GroupByTimeComponent;
+import org.apache.iotdb.db.mpp.sql.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.sql.statement.component.ResultColumn;
+import org.apache.iotdb.db.mpp.sql.statement.component.ResultSetFormat;
+import org.apache.iotdb.db.mpp.sql.statement.component.SelectComponent;
+import org.apache.iotdb.db.mpp.sql.statement.component.WhereCondition;
 import org.apache.iotdb.db.mpp.sql.statement.crud.AggregationQueryStatement;
 import org.apache.iotdb.db.mpp.sql.statement.crud.FillQueryStatement;
 import org.apache.iotdb.db.mpp.sql.statement.crud.GroupByFillQueryStatement;
@@ -49,7 +59,9 @@ import org.apache.iotdb.db.mpp.sql.statement.metadata.CreateAlignedTimeSeriesSta
 import org.apache.iotdb.db.mpp.sql.statement.metadata.CreateTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.ShowDevicesStatement;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.ShowTimeSeriesStatement;
+import org.apache.iotdb.db.mpp.sql.statement.sys.AuthorStatement;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.logical.sys.AuthorOperator;
 import org.apache.iotdb.db.qp.sql.IoTDBSqlParser;
 import org.apache.iotdb.db.qp.sql.IoTDBSqlParserBaseVisitor;
 import org.apache.iotdb.db.qp.utils.DatetimeUtils;
@@ -76,7 +88,14 @@ import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.commons.lang.StringEscapeUtils;
 
 import java.time.ZoneId;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
 import java.util.regex.Pattern;
 
 import static org.apache.iotdb.db.index.common.IndexConstant.PATTERN;
@@ -1341,6 +1360,206 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     return src;
   }
 
+  /** Data Control Language (DCL) */
+
+  // Create User
+
+  @Override
+  public Statement visitCreateUser(IoTDBSqlParser.CreateUserContext ctx) {
+    AuthorStatement authorStatement = new AuthorStatement(AuthorOperator.AuthorType.CREATE_USER);
+    authorStatement.setUserName(ctx.userName.getText());
+    authorStatement.setPassWord(parseStringLiteral(ctx.password.getText()));
+    return authorStatement;
+  }
+
+  // Create Role
+
+  @Override
+  public Statement visitCreateRole(IoTDBSqlParser.CreateRoleContext ctx) {
+    AuthorStatement authorStatement = new AuthorStatement(AuthorOperator.AuthorType.CREATE_ROLE);
+    authorStatement.setRoleName(ctx.roleName.getText());
+    return authorStatement;
+  }
+
+  // Alter Password
+
+  @Override
+  public Statement visitAlterUser(IoTDBSqlParser.AlterUserContext ctx) {
+    AuthorStatement authorStatement = new AuthorStatement(AuthorOperator.AuthorType.UPDATE_USER);
+    authorStatement.setUserName(ctx.userName.getText());
+    authorStatement.setNewPassword(parseStringLiteral(ctx.password.getText()));
+    return authorStatement;
+  }
+
+  // Grant User Privileges
+
+  @Override
+  public Statement visitGrantUser(IoTDBSqlParser.GrantUserContext ctx) {
+    AuthorStatement authorStatement = new AuthorStatement(AuthorOperator.AuthorType.GRANT_USER);
+    authorStatement.setUserName(ctx.userName.getText());
+    authorStatement.setPrivilegeList(parsePrivilege(ctx.privileges()));
+    authorStatement.setNodeNameList(parsePrefixPath(ctx.prefixPath()));
+    return authorStatement;
+  }
+
+  // Grant Role Privileges
+
+  @Override
+  public Statement visitGrantRole(IoTDBSqlParser.GrantRoleContext ctx) {
+    AuthorStatement authorStatement = new AuthorStatement(AuthorOperator.AuthorType.GRANT_ROLE);
+    authorStatement.setRoleName(ctx.roleName.getText());
+    authorStatement.setPrivilegeList(parsePrivilege(ctx.privileges()));
+    authorStatement.setNodeNameList(parsePrefixPath(ctx.prefixPath()));
+    return authorStatement;
+  }
+
+  // Grant User Role
+
+  @Override
+  public Statement visitGrantRoleToUser(IoTDBSqlParser.GrantRoleToUserContext ctx) {
+    AuthorStatement authorStatement =
+        new AuthorStatement(AuthorOperator.AuthorType.GRANT_ROLE_TO_USER);
+    authorStatement.setRoleName(ctx.roleName.getText());
+    authorStatement.setUserName(ctx.userName.getText());
+    return authorStatement;
+  }
+
+  // Revoke User Privileges
+
+  @Override
+  public Statement visitRevokeUser(IoTDBSqlParser.RevokeUserContext ctx) {
+    AuthorStatement authorStatement = new AuthorStatement(AuthorOperator.AuthorType.REVOKE_USER);
+    authorStatement.setUserName(ctx.userName.getText());
+    authorStatement.setPrivilegeList(parsePrivilege(ctx.privileges()));
+    authorStatement.setNodeNameList(parsePrefixPath(ctx.prefixPath()));
+    return authorStatement;
+  }
+
+  // Revoke Role Privileges
+
+  @Override
+  public Statement visitRevokeRole(IoTDBSqlParser.RevokeRoleContext ctx) {
+    AuthorStatement authorStatement = new AuthorStatement(AuthorOperator.AuthorType.REVOKE_ROLE);
+    authorStatement.setRoleName(ctx.roleName.getText());
+    authorStatement.setPrivilegeList(parsePrivilege(ctx.privileges()));
+    authorStatement.setNodeNameList(parsePrefixPath(ctx.prefixPath()));
+    return authorStatement;
+  }
+
+  // Revoke Role From User
+
+  @Override
+  public Statement visitRevokeRoleFromUser(IoTDBSqlParser.RevokeRoleFromUserContext ctx) {
+    AuthorStatement authorStatement =
+        new AuthorStatement(AuthorOperator.AuthorType.REVOKE_ROLE_FROM_USER);
+    authorStatement.setRoleName(ctx.roleName.getText());
+    authorStatement.setUserName(ctx.userName.getText());
+    return authorStatement;
+  }
+
+  // Drop User
+
+  @Override
+  public Statement visitDropUser(IoTDBSqlParser.DropUserContext ctx) {
+    AuthorStatement authorStatement = new AuthorStatement(AuthorOperator.AuthorType.DROP_USER);
+    authorStatement.setUserName(ctx.userName.getText());
+    return authorStatement;
+  }
+
+  // Drop Role
+
+  @Override
+  public Statement visitDropRole(IoTDBSqlParser.DropRoleContext ctx) {
+    AuthorStatement authorStatement = new AuthorStatement(AuthorOperator.AuthorType.DROP_ROLE);
+    authorStatement.setRoleName(ctx.roleName.getText());
+    return authorStatement;
+  }
+
+  // List Users
+
+  @Override
+  public Statement visitListUser(IoTDBSqlParser.ListUserContext ctx) {
+    return new AuthorStatement(AuthorOperator.AuthorType.LIST_USER);
+  }
+
+  // List Roles
+
+  @Override
+  public Statement visitListRole(IoTDBSqlParser.ListRoleContext ctx) {
+    return new AuthorStatement(AuthorOperator.AuthorType.LIST_ROLE);
+  }
+
+  // List Privileges
+
+  @Override
+  public Statement visitListPrivilegesUser(IoTDBSqlParser.ListPrivilegesUserContext ctx) {
+    AuthorStatement authorStatement =
+        new AuthorStatement(AuthorOperator.AuthorType.LIST_USER_PRIVILEGE);
+    authorStatement.setUserName(ctx.userName.getText());
+    authorStatement.setNodeNameList(parsePrefixPath(ctx.prefixPath()));
+    return authorStatement;
+  }
+
+  // List Privileges of Roles On Specific Path
+
+  @Override
+  public Statement visitListPrivilegesRole(IoTDBSqlParser.ListPrivilegesRoleContext ctx) {
+    AuthorStatement authorStatement =
+        new AuthorStatement(AuthorOperator.AuthorType.LIST_ROLE_PRIVILEGE);
+    authorStatement.setRoleName(ctx.roleName.getText());
+    authorStatement.setNodeNameList(parsePrefixPath(ctx.prefixPath()));
+    return authorStatement;
+  }
+
+  // List Privileges of Users
+
+  @Override
+  public Statement visitListUserPrivileges(IoTDBSqlParser.ListUserPrivilegesContext ctx) {
+    AuthorStatement authorStatement =
+        new AuthorStatement(AuthorOperator.AuthorType.LIST_USER_PRIVILEGE);
+    authorStatement.setUserName(ctx.userName.getText());
+    return authorStatement;
+  }
+
+  // List Privileges of Roles
+
+  @Override
+  public Statement visitListRolePrivileges(IoTDBSqlParser.ListRolePrivilegesContext ctx) {
+    AuthorStatement authorStatement =
+        new AuthorStatement(AuthorOperator.AuthorType.LIST_ROLE_PRIVILEGE);
+    authorStatement.setRoleName(ctx.roleName.getText());
+    return authorStatement;
+  }
+
+  // List Roles of Users
+
+  @Override
+  public Statement visitListAllRoleOfUser(IoTDBSqlParser.ListAllRoleOfUserContext ctx) {
+    AuthorStatement authorStatement =
+        new AuthorStatement(AuthorOperator.AuthorType.LIST_USER_ROLES);
+    authorStatement.setUserName(ctx.userName.getText());
+    return authorStatement;
+  }
+
+  // List Users of Role
+
+  @Override
+  public Statement visitListAllUserOfRole(IoTDBSqlParser.ListAllUserOfRoleContext ctx) {
+    AuthorStatement authorStatement =
+        new AuthorStatement(AuthorOperator.AuthorType.LIST_ROLE_USERS);
+    authorStatement.setRoleName(ctx.roleName.getText());
+    return authorStatement;
+  }
+
+  private String[] parsePrivilege(IoTDBSqlParser.PrivilegesContext ctx) {
+    List<IoTDBSqlParser.PrivilegeValueContext> privilegeList = ctx.privilegeValue();
+    List<String> privileges = new ArrayList<>();
+    for (IoTDBSqlParser.PrivilegeValueContext privilegeValue : privilegeList) {
+      privileges.add(privilegeValue.getText());
+    }
+    return privileges.toArray(new String[0]);
+  }
+
   /** function for parsing file path used by LOAD statement. */
   public String parseFilePath(String src) {
     return src.substring(1, src.length() - 1);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/LogicalPlanner.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/LogicalPlanner.java
index f8c2a38d90..b3a63494ae 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/LogicalPlanner.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/LogicalPlanner.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.sql.planner;
 
+import org.apache.iotdb.db.auth.AuthException;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
 import org.apache.iotdb.db.mpp.common.filter.QueryFilter;
 import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
@@ -25,14 +26,26 @@ import org.apache.iotdb.db.mpp.sql.optimization.PlanOptimizer;
 import org.apache.iotdb.db.mpp.sql.planner.plan.LogicalQueryPlan;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.AlterTimeSeriesNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.AuthorNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.CreateAlignedTimeSeriesNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.CreateTimeSeriesNode;
-import org.apache.iotdb.db.mpp.sql.planner.plan.node.process.*;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.process.DeviceMergeNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.process.FilterNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.process.FilterNullNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.process.GroupByLevelNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.process.LimitNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.process.OffsetNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.process.SortNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.process.TimeJoinNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.source.SourceNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.write.InsertTabletNode;
 import org.apache.iotdb.db.mpp.sql.statement.StatementVisitor;
-import org.apache.iotdb.db.mpp.sql.statement.component.*;
+import org.apache.iotdb.db.mpp.sql.statement.component.FillComponent;
+import org.apache.iotdb.db.mpp.sql.statement.component.FilterNullComponent;
+import org.apache.iotdb.db.mpp.sql.statement.component.GroupByLevelComponent;
+import org.apache.iotdb.db.mpp.sql.statement.component.OrderBy;
+import org.apache.iotdb.db.mpp.sql.statement.component.ResultColumn;
 import org.apache.iotdb.db.mpp.sql.statement.crud.AggregationQueryStatement;
 import org.apache.iotdb.db.mpp.sql.statement.crud.FillQueryStatement;
 import org.apache.iotdb.db.mpp.sql.statement.crud.InsertRowStatement;
@@ -41,10 +54,17 @@ import org.apache.iotdb.db.mpp.sql.statement.crud.QueryStatement;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.AlterTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.CreateAlignedTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.CreateTimeSeriesStatement;
+import org.apache.iotdb.db.mpp.sql.statement.sys.AuthorStatement;
 import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import java.util.stream.Collectors;
 
 /** Generate a logical plan for the statement. */
@@ -325,6 +345,124 @@ public class LogicalPlanner {
           insertRowStatement.getTime(),
           insertRowStatement.getValues());
     }
+
+    @Override
+    public PlanNode visitCreateUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitCreateRole(AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitAlterUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitGrantUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitGrantRole(AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitGrantRoleToUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitRevokeUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitRevokeRole(AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitRevokeRoleFromUser(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitDropUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitDropRole(AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitListUser(AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitListRole(AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitListPrivilegesUser(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitListPrivilegesRole(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitListUserPrivileges(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitListRolePrivileges(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitListAllRoleOfUser(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    @Override
+    public PlanNode visitListAllUserOfRole(
+        AuthorStatement authorStatement, MPPQueryContext context) {
+      return getNewAuthorNode(authorStatement, context);
+    }
+
+    public AuthorNode getNewAuthorNode(AuthorStatement authorStatement, MPPQueryContext context) {
+      try {
+        return new AuthorNode(
+            context.getQueryId().genPlanNodeId(),
+            authorStatement.getAuthorType(),
+            authorStatement.getUserName(),
+            authorStatement.getRoleName(),
+            authorStatement.getPassWord(),
+            authorStatement.getNewPassword(),
+            authorStatement.getPrivilegeList(),
+            authorStatement.getNodeName());
+      } catch (AuthException e) {
+        return null;
+      }
+    }
   }
 
   private class PlanBuilder {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanNodeType.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanNodeType.java
index 340f8a431f..2b0df3b0aa 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanNodeType.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanNodeType.java
@@ -19,6 +19,7 @@
 package org.apache.iotdb.db.mpp.sql.planner.plan.node;
 
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.read.ShowDevicesNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.AuthorNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.CreateTimeSeriesNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.process.AggregateNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.process.DeviceMergeNode;
@@ -63,7 +64,8 @@ public enum PlanNodeType {
   INSERT_MULTI_TABLET((short) 17),
   SHOW_DEVICES((short) 18),
   CREATE_TIME_SERIES((short) 19),
-  EXCHANGE((short) 20);
+  EXCHANGE((short) 20),
+  AUTHOR((short) 21);
 
   private final short nodeType;
 
@@ -120,6 +122,8 @@ public enum PlanNodeType {
         return CreateTimeSeriesNode.deserialize(buffer);
       case 20:
         return ExchangeNode.deserialize(buffer);
+      case 21:
+        return AuthorNode.deserialize(buffer);
       default:
         throw new IllegalArgumentException("Invalid node type: " + nodeType);
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/write/AuthorNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/write/AuthorNode.java
new file mode 100644
index 0000000000..ea2c5b64d9
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/write/AuthorNode.java
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write;
+
+import org.apache.iotdb.db.auth.AuthException;
+import org.apache.iotdb.db.auth.entity.PrivilegeType;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
+import org.apache.iotdb.db.qp.logical.sys.AuthorOperator;
+
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class AuthorNode extends PlanNode {
+
+  private AuthorOperator.AuthorType authorType;
+  private String userName;
+  private String roleName;
+  private String password;
+  private String newPassword;
+  private Set<Integer> permissions;
+  private PartialPath nodeName;
+
+  public AuthorNode(
+      PlanNodeId id,
+      AuthorOperator.AuthorType authorType,
+      String userName,
+      String roleName,
+      String password,
+      String newPassword,
+      String[] privilegeList,
+      PartialPath nodeName)
+      throws AuthException {
+    super(id);
+    this.authorType = authorType;
+    this.userName = userName;
+    this.roleName = roleName;
+    this.password = password;
+    this.newPassword = newPassword;
+    this.permissions = strToPermissions(privilegeList);
+    this.nodeName = nodeName;
+  }
+
+  public AuthorOperator.AuthorType getAuthorType() {
+    return authorType;
+  }
+
+  public void setAuthorType(AuthorOperator.AuthorType authorType) {
+    this.authorType = authorType;
+  }
+
+  public String getUserName() {
+    return userName;
+  }
+
+  public void setUserName(String userName) {
+    this.userName = userName;
+  }
+
+  public String getRoleName() {
+    return roleName;
+  }
+
+  public void setRoleName(String roleName) {
+    this.roleName = roleName;
+  }
+
+  public String getPassword() {
+    return password;
+  }
+
+  public void setPassword(String password) {
+    this.password = password;
+  }
+
+  public String getNewPassword() {
+    return newPassword;
+  }
+
+  public void setNewPassword(String newPassword) {
+    this.newPassword = newPassword;
+  }
+
+  public Set<Integer> getPermissions() {
+    return permissions;
+  }
+
+  public void setPermissions(Set<Integer> permissions) {
+    this.permissions = permissions;
+  }
+
+  public PartialPath getNodeName() {
+    return nodeName;
+  }
+
+  public void setNodeName(PartialPath nodeName) {
+    this.nodeName = nodeName;
+  }
+
+  @Override
+  public List<PlanNode> getChildren() {
+    return null;
+  }
+
+  @Override
+  public void addChild(PlanNode child) {}
+
+  @Override
+  public PlanNode clone() {
+    return null;
+  }
+
+  @Override
+  public PlanNode cloneWithChildren(List<PlanNode> children) {
+    return null;
+  }
+
+  @Override
+  public int allowedChildCount() {
+    return 0;
+  }
+
+  @Override
+  public List<String> getOutputColumnNames() {
+    return null;
+  }
+
+  @Override
+  public void serialize(ByteBuffer buffer) {
+    buffer.putInt(getPlanType(authorType));
+    buffer.putInt(userName.length());
+    buffer.put(userName.getBytes());
+    buffer.putInt(roleName.length());
+    buffer.put(roleName.getBytes());
+    buffer.putInt(password.length());
+    buffer.put(password.getBytes());
+    buffer.putInt(newPassword.length());
+    buffer.put(newPassword.getBytes());
+    if (permissions == null && permissions.size() == 0) {
+      buffer.put("false".getBytes());
+    } else {
+      buffer.put("true".getBytes());
+      buffer.putInt(permissions.size());
+      for (Integer permission : permissions) {
+        buffer.putInt(permission);
+      }
+    }
+    if (nodeName == null && nodeName.equals("")) {
+      buffer.put("false".getBytes());
+    } else {
+      buffer.put("true".getBytes());
+      buffer.putInt(nodeName.getFullPath().length());
+      buffer.put(nodeName.getFullPath().getBytes());
+    }
+  }
+
+  public static AuthorNode deserialize(ByteBuffer buffer) {
+    return null;
+  }
+
+  public Set<Integer> strToPermissions(String[] authorizationList) throws AuthException {
+    Set<Integer> result = new HashSet<>();
+    if (authorizationList == null) {
+      return result;
+    }
+    for (String s : authorizationList) {
+      PrivilegeType[] types = PrivilegeType.values();
+      boolean legal = false;
+      for (PrivilegeType privilegeType : types) {
+        if (s.equalsIgnoreCase(privilegeType.name())) {
+          result.add(privilegeType.ordinal());
+          legal = true;
+          break;
+        }
+      }
+      if (!legal) {
+        throw new AuthException("No such privilege " + s);
+      }
+    }
+    return result;
+  }
+
+  private static String getAuthorInfo(ByteBuffer buffer) {
+    int infoSize = buffer.getInt();
+    byte[] byteInfo = new byte[infoSize];
+    buffer.get(byteInfo, 0, infoSize);
+    return new String(byteInfo, 0, infoSize);
+  }
+
+  private int getPlanType(AuthorOperator.AuthorType physicalPlanType) {
+    int type;
+    switch (physicalPlanType) {
+      case CREATE_USER:
+        type = AuthorOperator.AuthorType.CREATE_USER.ordinal();
+        break;
+      case CREATE_ROLE:
+        type = AuthorOperator.AuthorType.CREATE_ROLE.ordinal();
+        break;
+      case DROP_USER:
+        type = AuthorOperator.AuthorType.DROP_USER.ordinal();
+        break;
+      case DROP_ROLE:
+        type = AuthorOperator.AuthorType.DROP_ROLE.ordinal();
+        break;
+      case GRANT_ROLE:
+        type = AuthorOperator.AuthorType.GRANT_ROLE.ordinal();
+        break;
+      case GRANT_USER:
+        type = AuthorOperator.AuthorType.GRANT_USER.ordinal();
+        break;
+      case GRANT_ROLE_TO_USER:
+        type = AuthorOperator.AuthorType.GRANT_ROLE_TO_USER.ordinal();
+        break;
+      case REVOKE_USER:
+        type = AuthorOperator.AuthorType.REVOKE_USER.ordinal();
+        break;
+      case REVOKE_ROLE:
+        type = AuthorOperator.AuthorType.REVOKE_ROLE.ordinal();
+        break;
+      case REVOKE_ROLE_FROM_USER:
+        type = AuthorOperator.AuthorType.REVOKE_ROLE_FROM_USER.ordinal();
+        break;
+      case UPDATE_USER:
+        type = AuthorOperator.AuthorType.UPDATE_USER.ordinal();
+        break;
+      case LIST_USER:
+        type = AuthorOperator.AuthorType.LIST_USER.ordinal();
+        break;
+      case LIST_ROLE:
+        type = AuthorOperator.AuthorType.LIST_ROLE.ordinal();
+        break;
+      case LIST_USER_PRIVILEGE:
+        type = AuthorOperator.AuthorType.LIST_USER_PRIVILEGE.ordinal();
+        break;
+      case LIST_ROLE_PRIVILEGE:
+        type = AuthorOperator.AuthorType.LIST_ROLE_PRIVILEGE.ordinal();
+        break;
+      case LIST_USER_ROLES:
+        type = AuthorOperator.AuthorType.LIST_USER_ROLES.ordinal();
+        break;
+      case LIST_ROLE_USERS:
+        type = AuthorOperator.AuthorType.LIST_ROLE_USERS.ordinal();
+        break;
+      default:
+        throw new IllegalArgumentException("Unknown operator: " + physicalPlanType);
+    }
+    return type;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/StatementVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/StatementVisitor.java
index cee9a7980c..9cd14429cc 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/StatementVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/StatementVisitor.java
@@ -26,6 +26,7 @@ import org.apache.iotdb.db.mpp.sql.statement.crud.QueryStatement;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.AlterTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.CreateAlignedTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.CreateTimeSeriesStatement;
+import org.apache.iotdb.db.mpp.sql.statement.sys.AuthorStatement;
 
 /**
  * This class provides a visitor of {@link org.apache.iotdb.db.mpp.sql.statement.StatementNode},
@@ -84,6 +85,103 @@ public abstract class StatementVisitor<R, C> {
     return visitStatement(insertTabletStatement, context);
   }
 
+  /** Data Control Language (DCL) */
+
+  // Create User
+  public R visitCreateUser(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // Create Role
+  public R visitCreateRole(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // Alter Password
+  public R visitAlterUser(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // Grant User Privileges
+  public R visitGrantUser(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // Grant Role Privileges
+  public R visitGrantRole(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // Grant User Role
+  public R visitGrantRoleToUser(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // Revoke User Privileges
+  public R visitRevokeUser(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // Revoke Role Privileges
+  public R visitRevokeRole(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // Revoke Role From User
+  public R visitRevokeRoleFromUser(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // Drop User
+  public R visitDropUser(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // Drop Role
+  public R visitDropRole(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // List Users
+  public R visitListUser(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // List Roles
+  public R visitListRole(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // List Privileges
+  public R visitListPrivilegesUser(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // List Privileges of Roles On Specific Path
+  public R visitListPrivilegesRole(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // List Privileges of Users
+  public R visitListUserPrivileges(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // List Privileges of Roles
+  public R visitListRolePrivileges(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // List Roles of Users
+  public R visitListAllRoleOfUser(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
+  // List Users of Role
+  public R visitListAllUserOfRole(AuthorStatement authorStatement, C context) {
+    return visitStatement(authorStatement, context);
+  }
+
   public R visitInsertRow(InsertRowStatement insertRowStatement, C context) {
     return visitStatement(insertRowStatement, context);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/sys/AuthorStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/sys/AuthorStatement.java
new file mode 100644
index 0000000000..045dd2bc13
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/sys/AuthorStatement.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.mpp.sql.statement.sys;
+
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.mpp.sql.constant.StatementType;
+import org.apache.iotdb.db.mpp.sql.statement.Statement;
+import org.apache.iotdb.db.mpp.sql.statement.StatementVisitor;
+import org.apache.iotdb.db.qp.logical.sys.AuthorOperator;
+
+public class AuthorStatement extends Statement {
+
+  private final AuthorOperator.AuthorType authorType;
+  private String userName;
+  private String roleName;
+  private String password;
+  private String newPassword;
+  private String[] privilegeList;
+  private PartialPath nodeName;
+
+  /**
+   * AuthorOperator Constructor with AuthorType.
+   *
+   * @param type author type
+   */
+  public AuthorStatement(AuthorOperator.AuthorType type) {
+    super();
+    authorType = type;
+    statementType = StatementType.AUTHOR;
+  }
+
+  /**
+   * AuthorOperator Constructor with OperatorType.
+   *
+   * @param type statement type
+   */
+  public AuthorStatement(StatementType type) {
+    super();
+    authorType = null;
+    statementType = type;
+  }
+
+  public AuthorOperator.AuthorType getAuthorType() {
+    return authorType;
+  }
+
+  public String getUserName() {
+    return userName;
+  }
+
+  public void setUserName(String userName) {
+    this.userName = userName;
+  }
+
+  public String getRoleName() {
+    return roleName;
+  }
+
+  public void setRoleName(String roleName) {
+    this.roleName = roleName;
+  }
+
+  public String getPassWord() {
+    return password;
+  }
+
+  public void setPassWord(String password) {
+    this.password = password;
+  }
+
+  public String getNewPassword() {
+    return newPassword;
+  }
+
+  public void setNewPassword(String newPassword) {
+    this.newPassword = newPassword;
+  }
+
+  public String[] getPrivilegeList() {
+    return privilegeList;
+  }
+
+  public void setPrivilegeList(String[] authorizationList) {
+    this.privilegeList = authorizationList;
+  }
+
+  public PartialPath getNodeName() {
+    return nodeName;
+  }
+
+  public void setNodeNameList(PartialPath nodePath) {
+    this.nodeName = nodePath;
+  }
+
+  @Override
+  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
+    switch (this.authorType) {
+      case CREATE_USER:
+        return visitor.visitCreateUser(this, context);
+      case CREATE_ROLE:
+        return visitor.visitCreateRole(this, context);
+      case DROP_USER:
+        return visitor.visitDropUser(this, context);
+      case DROP_ROLE:
+        return visitor.visitDropRole(this, context);
+      case GRANT_ROLE:
+        return visitor.visitGrantRole(this, context);
+      case GRANT_USER:
+        return visitor.visitGrantUser(this, context);
+      case GRANT_ROLE_TO_USER:
+        return visitor.visitGrantRoleToUser(this, context);
+      case REVOKE_USER:
+        return visitor.visitRevokeUser(this, context);
+      case REVOKE_ROLE:
+        return visitor.visitRevokeRole(this, context);
+      case REVOKE_ROLE_FROM_USER:
+        return visitor.visitRevokeRoleFromUser(this, context);
+      case UPDATE_USER:
+        return visitor.visitAlterUser(this, context);
+      case LIST_USER:
+        return visitor.visitListUser(this, context);
+      case LIST_ROLE:
+        return visitor.visitListRole(this, context);
+      case LIST_USER_PRIVILEGE:
+        return visitor.visitListUserPrivileges(this, context);
+      case LIST_ROLE_PRIVILEGE:
+        return visitor.visitListRolePrivileges(this, context);
+      case LIST_USER_ROLES:
+        return visitor.visitListAllUserOfRole(this, context);
+      case LIST_ROLE_USERS:
+        return visitor.visitListAllRoleOfUser(this, context);
+      default:
+        return null;
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
index 482ae16278..3a480afd09 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
@@ -23,8 +23,7 @@ import org.apache.iotdb.commons.concurrent.ThreadName;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.db.auth.AuthException;
 import org.apache.iotdb.db.auth.AuthorityChecker;
-import org.apache.iotdb.db.auth.authorizer.BasicAuthorizer;
-import org.apache.iotdb.db.auth.authorizer.IAuthorizer;
+import org.apache.iotdb.db.auth.authorizer.AuthorizerManager;
 import org.apache.iotdb.db.auth.entity.PathPrivilege;
 import org.apache.iotdb.db.auth.entity.PrivilegeType;
 import org.apache.iotdb.db.auth.entity.Role;
@@ -236,7 +235,7 @@ public class PlanExecutor implements IPlanExecutor {
   // for data query
   protected IQueryRouter queryRouter;
   // for administration
-  private final IAuthorizer authorizer;
+  private final AuthorizerManager authorizerManager;
 
   private ThreadPoolExecutor insertionPool;
 
@@ -244,11 +243,7 @@ public class PlanExecutor implements IPlanExecutor {
 
   public PlanExecutor() throws QueryProcessException {
     queryRouter = new QueryRouter();
-    try {
-      authorizer = BasicAuthorizer.getInstance();
-    } catch (AuthException e) {
-      throw new QueryProcessException(e.getMessage());
-    }
+    authorizerManager = AuthorizerManager.getInstance();
   }
 
   @Override
@@ -1779,45 +1774,45 @@ public class PlanExecutor implements IPlanExecutor {
     try {
       switch (authorType) {
         case UPDATE_USER:
-          authorizer.updateUserPassword(userName, newPassword);
+          authorizerManager.updateUserPassword(userName, newPassword);
           break;
         case CREATE_USER:
-          authorizer.createUser(userName, password);
+          authorizerManager.createUser(userName, password);
           break;
         case CREATE_ROLE:
-          authorizer.createRole(roleName);
+          authorizerManager.createRole(roleName);
           break;
         case DROP_USER:
-          authorizer.deleteUser(userName);
+          authorizerManager.deleteUser(userName);
           break;
         case DROP_ROLE:
-          authorizer.deleteRole(roleName);
+          authorizerManager.deleteRole(roleName);
           break;
         case GRANT_ROLE:
           for (int i : permissions) {
-            authorizer.grantPrivilegeToRole(roleName, nodeName.getFullPath(), i);
+            authorizerManager.grantPrivilegeToRole(roleName, nodeName.getFullPath(), i);
           }
           break;
         case GRANT_USER:
           for (int i : permissions) {
-            authorizer.grantPrivilegeToUser(userName, nodeName.getFullPath(), i);
+            authorizerManager.grantPrivilegeToUser(userName, nodeName.getFullPath(), i);
           }
           break;
         case GRANT_ROLE_TO_USER:
-          authorizer.grantRoleToUser(roleName, userName);
+          authorizerManager.grantRoleToUser(roleName, userName);
           break;
         case REVOKE_USER:
           for (int i : permissions) {
-            authorizer.revokePrivilegeFromUser(userName, nodeName.getFullPath(), i);
+            authorizerManager.revokePrivilegeFromUser(userName, nodeName.getFullPath(), i);
           }
           break;
         case REVOKE_ROLE:
           for (int i : permissions) {
-            authorizer.revokePrivilegeFromRole(roleName, nodeName.getFullPath(), i);
+            authorizerManager.revokePrivilegeFromRole(roleName, nodeName.getFullPath(), i);
           }
           break;
         case REVOKE_ROLE_FROM_USER:
-          authorizer.revokeRoleFromUser(roleName, userName);
+          authorizerManager.revokeRoleFromUser(roleName, userName);
           break;
         default:
           throw new QueryProcessException("Unsupported operation " + authorType);
@@ -1832,7 +1827,7 @@ public class PlanExecutor implements IPlanExecutor {
       throws QueryProcessException {
     try {
       for (String user : users) {
-        authorizer.setUserUseWaterMark(user, useWatermark);
+        authorizerManager.setUserUseWaterMark(user, useWatermark);
       }
     } catch (AuthException e) {
       throw new QueryProcessException(e.getMessage());
@@ -2063,7 +2058,7 @@ public class PlanExecutor implements IPlanExecutor {
       return dataSet;
     }
 
-    List<String> roleList = authorizer.listAllRoles();
+    List<String> roleList = authorizerManager.listAllRoles();
     addToDataSet(roleList, dataSet);
     return dataSet;
   }
@@ -2096,13 +2091,13 @@ public class PlanExecutor implements IPlanExecutor {
       return dataSet;
     }
 
-    List<String> userList = authorizer.listAllUsers();
+    List<String> userList = authorizerManager.listAllUsers();
     addToDataSet(userList, dataSet);
     return dataSet;
   }
 
   private ListDataSet executeListRoleUsers(String roleName) throws AuthException {
-    Role role = authorizer.getRole(roleName);
+    Role role = authorizerManager.getRole(roleName);
     if (role == null) {
       throw new AuthException("No such role : " + roleName);
     }
@@ -2110,10 +2105,10 @@ public class PlanExecutor implements IPlanExecutor {
         new ListDataSet(
             Collections.singletonList(new PartialPath(COLUMN_USER, false)),
             Collections.singletonList(TSDataType.TEXT));
-    List<String> userList = authorizer.listAllUsers();
+    List<String> userList = authorizerManager.listAllUsers();
     int index = 0;
     for (String userN : userList) {
-      User userObj = authorizer.getUser(userN);
+      User userObj = authorizerManager.getUser(userN);
       if (userObj != null && userObj.hasRole(roleName)) {
         RowRecord record = new RowRecord(index++);
         Field field = new Field(TSDataType.TEXT);
@@ -2126,7 +2121,7 @@ public class PlanExecutor implements IPlanExecutor {
   }
 
   private ListDataSet executeListUserRoles(String userName) throws AuthException {
-    User user = authorizer.getUser(userName);
+    User user = authorizerManager.getUser(userName);
     if (user != null) {
       ListDataSet dataSet =
           new ListDataSet(
@@ -2148,7 +2143,7 @@ public class PlanExecutor implements IPlanExecutor {
 
   private ListDataSet executeListRolePrivileges(String roleName, PartialPath path)
       throws AuthException {
-    Role role = authorizer.getRole(roleName);
+    Role role = authorizerManager.getRole(roleName);
     if (role != null) {
       List<PartialPath> headerList = new ArrayList<>();
       List<TSDataType> typeList = new ArrayList<>();
@@ -2173,7 +2168,7 @@ public class PlanExecutor implements IPlanExecutor {
 
   private ListDataSet executeListUserPrivileges(String userName, PartialPath path)
       throws AuthException {
-    User user = authorizer.getUser(userName);
+    User user = authorizerManager.getUser(userName);
     if (user == null) {
       throw new AuthException("No such user : " + userName);
     }
@@ -2211,7 +2206,7 @@ public class PlanExecutor implements IPlanExecutor {
         }
       }
       for (String roleN : user.getRoleList()) {
-        Role role = authorizer.getRole(roleN);
+        Role role = authorizerManager.getRole(roleN);
         if (role == null) {
           continue;
         }
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/LogicalPlannerTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/LogicalPlannerTest.java
index 403af0d743..9f20df2c9c 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/LogicalPlannerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/LogicalPlannerTest.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.mpp.sql.plan;
 
+import org.apache.iotdb.db.auth.AuthException;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
@@ -28,10 +29,12 @@ import org.apache.iotdb.db.mpp.sql.parser.StatementGenerator;
 import org.apache.iotdb.db.mpp.sql.planner.LogicalPlanner;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.AlterTimeSeriesNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.AuthorNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.CreateAlignedTimeSeriesNode;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.CreateTimeSeriesNode;
 import org.apache.iotdb.db.mpp.sql.statement.Statement;
 import org.apache.iotdb.db.mpp.sql.statement.metadata.AlterTimeSeriesStatement;
+import org.apache.iotdb.db.qp.logical.sys.AuthorOperator;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
@@ -369,4 +372,150 @@ public class LogicalPlannerTest {
     }
     return planNode;
   }
+
+  @Test
+  public void authorTest() throws AuthException {
+
+    String sql = null;
+    AuthorNode authorNode = null;
+    String[] privilegesList = {"DELETE_TIMESERIES"};
+
+    // create user
+    sql = "CREATE USER thulab 'passwd';";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.CREATE_USER, authorNode.getAuthorType());
+    Assert.assertEquals("thulab", authorNode.getUserName());
+    Assert.assertEquals("passwd", authorNode.getPassword());
+
+    // create role
+    sql = "CREATE ROLE admin;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.CREATE_ROLE, authorNode.getAuthorType());
+    Assert.assertEquals("admin", authorNode.getRoleName());
+
+    // alter user
+    sql = "ALTER USER tempuser SET PASSWORD 'newpwd';";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.UPDATE_USER, authorNode.getAuthorType());
+    Assert.assertEquals("tempuser", authorNode.getUserName());
+    Assert.assertEquals("newpwd", authorNode.getNewPassword());
+
+    // grant user
+    sql = "GRANT USER tempuser PRIVILEGES DELETE_TIMESERIES on root.ln;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.GRANT_USER, authorNode.getAuthorType());
+    Assert.assertEquals("tempuser", authorNode.getUserName());
+    Assert.assertEquals(authorNode.strToPermissions(privilegesList), authorNode.getPermissions());
+    Assert.assertEquals("root.ln", authorNode.getNodeName().getFullPath());
+
+    // grant role
+    sql = "GRANT ROLE temprole PRIVILEGES DELETE_TIMESERIES ON root.ln;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.GRANT_ROLE, authorNode.getAuthorType());
+    Assert.assertEquals("temprole", authorNode.getRoleName());
+    Assert.assertEquals(authorNode.strToPermissions(privilegesList), authorNode.getPermissions());
+    Assert.assertEquals("root.ln", authorNode.getNodeName().getFullPath());
+
+    // grant role to user
+    sql = "GRANT temprole TO tempuser;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.GRANT_ROLE_TO_USER, authorNode.getAuthorType());
+    Assert.assertEquals("temprole", authorNode.getRoleName());
+    Assert.assertEquals("tempuser", authorNode.getUserName());
+
+    // revoke user
+    sql = "REVOKE USER tempuser PRIVILEGES DELETE_TIMESERIES on root.ln;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.REVOKE_USER, authorNode.getAuthorType());
+    Assert.assertEquals("tempuser", authorNode.getUserName());
+    Assert.assertEquals(authorNode.strToPermissions(privilegesList), authorNode.getPermissions());
+    Assert.assertEquals("root.ln", authorNode.getNodeName().getFullPath());
+
+    // revoke role
+    sql = "REVOKE ROLE temprole PRIVILEGES DELETE_TIMESERIES ON root.ln;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.REVOKE_ROLE, authorNode.getAuthorType());
+    Assert.assertEquals("temprole", authorNode.getRoleName());
+    Assert.assertEquals(authorNode.strToPermissions(privilegesList), authorNode.getPermissions());
+    Assert.assertEquals("root.ln", authorNode.getNodeName().getFullPath());
+
+    // revoke role from user
+    sql = "REVOKE temprole FROM tempuser;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(
+        AuthorOperator.AuthorType.REVOKE_ROLE_FROM_USER, authorNode.getAuthorType());
+    Assert.assertEquals("temprole", authorNode.getRoleName());
+    Assert.assertEquals("tempuser", authorNode.getUserName());
+
+    // drop user
+    sql = "DROP USER xiaoming;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.DROP_USER, authorNode.getAuthorType());
+    Assert.assertEquals("xiaoming", authorNode.getUserName());
+
+    // drop role
+    sql = "DROP ROLE admin;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.DROP_ROLE, authorNode.getAuthorType());
+    Assert.assertEquals("admin", authorNode.getRoleName());
+
+    // list user
+    sql = "LIST USER";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.LIST_USER, authorNode.getAuthorType());
+
+    // list role
+    sql = "LIST ROLE";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.LIST_ROLE, authorNode.getAuthorType());
+
+    // list privileges user
+    sql = "LIST PRIVILEGES USER sgcc_wirte_user ON root.sgcc;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.LIST_USER_PRIVILEGE, authorNode.getAuthorType());
+
+    // list privileges role
+    sql = "LIST PRIVILEGES ROLE wirte_role ON root.sgcc;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.LIST_ROLE_PRIVILEGE, authorNode.getAuthorType());
+
+    // list user privileges
+    sql = "LIST USER PRIVILEGES tempuser;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.LIST_USER_PRIVILEGE, authorNode.getAuthorType());
+
+    // list role privileges
+    sql = "LIST ROLE PRIVILEGES actor;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.LIST_ROLE_PRIVILEGE, authorNode.getAuthorType());
+
+    // list all role of user
+    sql = "LIST ALL ROLE OF USER tempuser;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.LIST_USER_ROLES, authorNode.getAuthorType());
+
+    // list all user of role
+    sql = "LIST ALL USER OF ROLE roleuser;";
+    authorNode = (AuthorNode) parseSQLToPlanNode(sql);
+    Assert.assertNotNull(authorNode);
+    Assert.assertEquals(AuthorOperator.AuthorType.LIST_ROLE_USERS, authorNode.getAuthorType());
+  }
 }
diff --git a/thrift-confignode/src/main/thrift/confignode.thrift b/thrift-confignode/src/main/thrift/confignode.thrift
index 2e3fccbf5f..b56029fdce 100644
--- a/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/thrift-confignode/src/main/thrift/confignode.thrift
@@ -101,6 +101,16 @@ struct DataPartitionInfoResp {
     1: required map<string, map<i32, map<i64, list<RegionInfo>>>> dataPartitionMap
 }
 
+struct AuthorizerReq{
+    1: required i32 authorType
+    2: required string userName
+    3: required string roleName
+    4: required string password
+    5: required string newPassword
+    6: required set<i32> permissions
+    7: required string nodeName
+}
+
 struct SchemaPartitionInfoResp {
     // Map<StorageGroup, Map<DeviceGroupID, SchemaRegionPlaceInfo>>
     1: required map<string, map<i32, RegionInfo>> schemaPartitionInfo
@@ -145,4 +155,7 @@ service ConfigIService {
   SchemaPartitionInfoResp fetchSchemaPartitionInfo(FetchSchemaPartitionReq req)
 
   PartitionInfoResp fetchPartitionInfo(FetchPartitionReq req)
+
+   rpc.TSStatus operatePermission(AuthorizerReq req)
+
 }
\ No newline at end of file