You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ne...@apache.org on 2022/08/11 07:15:12 UTC

[iotdb] branch master updated: [IOTDB-4052] Add annotations in confignode.thrift and move threadname to ThreadName.java

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

neuyilan 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 40532ef5d3 [IOTDB-4052] Add annotations in confignode.thrift and move threadname to ThreadName.java
40532ef5d3 is described below

commit 40532ef5d3f98add18b0ba394e513a449b87d6d4
Author: 任宇华 <79...@users.noreply.github.com>
AuthorDate: Thu Aug 11 15:15:08 2022 +0800

    [IOTDB-4052] Add annotations in confignode.thrift and move threadname to ThreadName.java
---
 .../iotdb/confignode/manager/NodeManager.java      |  1 -
 .../iotdb/commons/client/ClientPoolFactory.java    | 13 +++-----
 .../iotdb/commons/concurrent/ThreadName.java       |  9 ++++--
 .../iotdb/db/client/DataNodeClientPoolFactory.java |  9 ++----
 .../src/main/thrift/confignode.thrift              | 37 ++++++++++++++++++++--
 5 files changed, 49 insertions(+), 20 deletions(-)

diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/NodeManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/NodeManager.java
index fd92da65c8..b3519d7eaa 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/NodeManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/NodeManager.java
@@ -205,7 +205,6 @@ public class NodeManager {
   /**
    * Only leader use this interface
    *
-   * @param dataNodeId Specific DataNodeId
    * @return All registered DataNodes
    */
   public List<TDataNodeConfiguration> getRegisteredDataNodes() {
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java b/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java
index 1b64be7657..0361f4598d 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java
@@ -24,6 +24,7 @@ import org.apache.iotdb.commons.client.async.AsyncConfigNodeHeartbeatServiceClie
 import org.apache.iotdb.commons.client.async.AsyncDataNodeHeartbeatServiceClient;
 import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient;
 import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient;
+import org.apache.iotdb.commons.concurrent.ThreadName;
 import org.apache.iotdb.commons.conf.CommonConfig;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
 
@@ -33,12 +34,6 @@ import org.apache.commons.pool2.impl.GenericKeyedObjectPool;
 public class ClientPoolFactory {
 
   private static final CommonConfig conf = CommonDescriptor.getInstance().getConfig();
-  private static final String DATA_NODE_CLIENT_POOL_THREAD_NAME =
-      "AsyncDataNodeInternalServiceClientPool";
-  private static final String CONFIG_NODE_HEARTBEAT_CLIENT_POOL_THREAD_NAME =
-      "AsyncConfigNodeHeartbeatServiceClientPool";
-  private static final String DATA_NODE_HEARTBEAT_CLIENT_POOL_THREAD_NAME =
-      "AsyncDataNodeHeartbeatServiceClientPool";
 
   private ClientPoolFactory() {}
 
@@ -72,7 +67,7 @@ public class ClientPoolFactory {
                   .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled())
                   .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager())
                   .build(),
-              DATA_NODE_CLIENT_POOL_THREAD_NAME),
+              ThreadName.DATA_NODE_CLIENT_POOL_THREAD_NAME.getName()),
           new ClientPoolProperty.Builder<AsyncDataNodeInternalServiceClient>().build().getConfig());
     }
   }
@@ -90,7 +85,7 @@ public class ClientPoolFactory {
                   .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled())
                   .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager())
                   .build(),
-              CONFIG_NODE_HEARTBEAT_CLIENT_POOL_THREAD_NAME),
+              ThreadName.CONFIG_NODE_HEARTBEAT_CLIENT_POOL_THREAD_NAME.getName()),
           new ClientPoolProperty.Builder<AsyncConfigNodeHeartbeatServiceClient>()
               .build()
               .getConfig());
@@ -110,7 +105,7 @@ public class ClientPoolFactory {
                   .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled())
                   .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager())
                   .build(),
-              DATA_NODE_HEARTBEAT_CLIENT_POOL_THREAD_NAME),
+              ThreadName.DATA_NODE_HEARTBEAT_CLIENT_POOL_THREAD_NAME.getName()),
           new ClientPoolProperty.Builder<AsyncDataNodeHeartbeatServiceClient>()
               .build()
               .getConfig());
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
index 66f312fad8..971f04ab2f 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
@@ -78,12 +78,17 @@ public enum ThreadName {
   MULTI_LEADER_CONSENSUS_RPC_SERVER("MultiLeaderConsensusRPC-Server"),
   DATA_NODE_MANAGEMENT_RPC_SERVER("DataNodeManagementRPC"),
   DATA_NODE_MANAGEMENT_RPC_CLIENT("DataNodeManagementRPC-Client"),
-  Cluster_Monitor("ClusterMonitor"),
   MPP_DATA_EXCHANGE_RPC_SERVER("MPPDataExchangeRPC"),
   MPP_DATA_EXCHANGE_RPC_CLIENT("MPPDataExchangeRPC-Client"),
   INTERNAL_SERVICE_RPC_SERVER("InternalServiceRPC"),
   INTERNAL_SERVICE_RPC_CLIENT("InternalServiceRPC-Client"),
-  EXT_PIPE_PLUGIN_WORKER("ExtPipePlugin-Worker");
+  EXT_PIPE_PLUGIN_WORKER("ExtPipePlugin-Worker"),
+  DATA_NODE_CLIENT_POOL_THREAD_NAME("AsyncDataNodeInternalServiceClientPool"),
+  CONFIG_NODE_HEARTBEAT_CLIENT_POOL_THREAD_NAME("AsyncConfigNodeHeartbeatServiceClientPool"),
+  DATA_NODE_HEARTBEAT_CLIENT_POOL_THREAD_NAME("AsyncDataNodeHeartbeatServiceClientPool"),
+  CONFIG_NODE_CLIENT_POOL_THREAD_NAME("AsyncConfigNodeIServiceClientPool"),
+  DATA_NODE_MPP_DATA_EXCHANGE_CLIENT_POOL_THREAD_NAME(
+      "AsyncDataNodeMPPDataExchangeServiceClientPool");
 
   private final String name;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/client/DataNodeClientPoolFactory.java b/server/src/main/java/org/apache/iotdb/db/client/DataNodeClientPoolFactory.java
index b405101b2c..8f3e0d1ae7 100644
--- a/server/src/main/java/org/apache/iotdb/db/client/DataNodeClientPoolFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/client/DataNodeClientPoolFactory.java
@@ -29,6 +29,7 @@ import org.apache.iotdb.commons.client.async.AsyncDataNodeMPPDataExchangeService
 import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient;
 import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient;
 import org.apache.iotdb.commons.client.sync.SyncDataNodeMPPDataExchangeServiceClient;
+import org.apache.iotdb.commons.concurrent.ThreadName;
 import org.apache.iotdb.commons.consensus.PartitionRegionId;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -39,10 +40,6 @@ import org.apache.commons.pool2.impl.GenericKeyedObjectPool;
 public class DataNodeClientPoolFactory {
 
   private static final IoTDBConfig conf = IoTDBDescriptor.getInstance().getConfig();
-  private static final String CONFIG_NODE_CLIENT_POOL_THREAD_NAME =
-      "AsyncConfigNodeIServiceClientPool";
-  private static final String DATA_NODE_MPP_DATA_EXCHANGE_CLIENT_POOL_THREAD_NAME =
-      "AsyncDataNodeMPPDataExchangeServiceClientPool";
 
   private DataNodeClientPoolFactory() {}
 
@@ -76,7 +73,7 @@ public class DataNodeClientPoolFactory {
                   .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnable())
                   .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager())
                   .build(),
-              CONFIG_NODE_CLIENT_POOL_THREAD_NAME),
+              ThreadName.CONFIG_NODE_CLIENT_POOL_THREAD_NAME.getName()),
           new ClientPoolProperty.Builder<AsyncConfigNodeIServiceClient>().build().getConfig());
     }
   }
@@ -130,7 +127,7 @@ public class DataNodeClientPoolFactory {
                   .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnable())
                   .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager())
                   .build(),
-              DATA_NODE_MPP_DATA_EXCHANGE_CLIENT_POOL_THREAD_NAME),
+              ThreadName.DATA_NODE_MPP_DATA_EXCHANGE_CLIENT_POOL_THREAD_NAME.getName()),
           new ClientPoolProperty.Builder<AsyncDataNodeMPPDataExchangeServiceClient>()
               .build()
               .getConfig());
diff --git a/thrift-confignode/src/main/thrift/confignode.thrift b/thrift-confignode/src/main/thrift/confignode.thrift
index 3b59f48bd0..52de985075 100644
--- a/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/thrift-confignode/src/main/thrift/confignode.thrift
@@ -496,15 +496,45 @@ service IConfigNodeRPCService {
   TDataPartitionTableResp getOrCreateDataPartitionTable(TDataPartitionReq req)
 
   // ======================================================
-  // Authorize TODO: @RYH61 add interface annotation
+  // Authorize
   // ======================================================
 
+  /**
+   * Execute permission write operations such as create user, create role, and grant permission.
+   * There is no need to update the cache information of the DataNode for creating users and roles
+   *
+   * @return SUCCESS_STATUS if the permission write operation is executed successfully
+   *         INVALIDATE_PERMISSION_CACHE_ERROR if the update cache of the permission information in the datanode fails
+   *         EXECUTE_PERMISSION_EXCEPTION_ERROR if the permission write operation fails, like the user doesn't exist
+   *         INTERNAL_SERVER_ERROR if the permission type does not exist
+   */
   common.TSStatus operatePermission(TAuthorizerReq req)
 
+  /**
+   * Execute permission read operations such as list user
+   *
+   * @return SUCCESS_STATUS if the permission read operation is executed successfully
+   *         ROLE_NOT_EXIST_ERROR if the role does not exist
+   *         USER_NOT_EXIST_ERROR if the user does not exist
+   *         INTERNAL_SERVER_ERROR if the permission type does not exist
+   */
   TAuthorizerResp queryPermission(TAuthorizerReq req)
 
+  /**
+   * Authenticate user login
+   *
+   * @return SUCCESS_STATUS if the user exists and the correct username and password are entered
+   *         WRONG_LOGIN_PASSWORD_ERROR if the user enters the wrong username or password
+   */
   TPermissionInfoResp login(TLoginReq req)
 
+  /**
+   * Permission checking for user operations
+   *
+   * @return SUCCESS_STATUS if the user has the permission
+   *         EXECUTE_PERMISSION_EXCEPTION_ERROR if the seriesPath or the privilege is illegal.
+   *         NO_PERMISSION_ERROR if the user does not have this permission
+   */
   TPermissionInfoResp checkUserPrivileges(TCheckUserPrivilegesReq req)
 
   // ======================================================
@@ -572,13 +602,16 @@ service IConfigNodeRPCService {
   common.TSStatus dropFunction(TDropFunctionReq req)
 
   // ======================================================
-  // Maintenance Tools TODO: @RYH61 add interface annotation
+  // Maintenance Tools
   // ======================================================
 
+  /** Execute Level Compaction and unsequence Compaction task on all DataNodes */
   common.TSStatus merge(TMergeReq req)
 
+  /** Persist all the data points in the memory table of the storage group to the disk, and seal the data file on all DataNodes */
   common.TSStatus flush(common.TFlushReq req)
 
+  /** Clear the cache of chunk, chunk metadata and timeseries metadata to release the memory footprint on all DataNodes */
   common.TSStatus clearCache(TClearCacheReq req)
 
   // ======================================================