diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/runtime/RequestDelegate.java b/integration-test/src/main/java/org/apache/iotdb/itbase/runtime/RequestDelegate.java index bc32c31e7932..47f31004e6f2 100644 --- a/integration-test/src/main/java/org/apache/iotdb/itbase/runtime/RequestDelegate.java +++ b/integration-test/src/main/java/org/apache/iotdb/itbase/runtime/RequestDelegate.java @@ -76,14 +76,10 @@ public final void addRequest(Callable request) { */ public final T requestAllAndCompare() throws SQLException { List results = requestAll(); - T data = results.get(0); - for (int i = 1; i < results.size(); i++) { - T anotherData = results.get(i); - if (!compare(data, anotherData)) { - throw new InconsistentDataException(results, endpoints); - } + if (!results.stream().allMatch(result -> compare(results.get(0), result))) { + throw new InconsistentDataException(results, endpoints); } - return data; + return results.get(0); } private boolean compare(T data, T anotherData) { diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBVerifyConnectionIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBVerifyConnectionIT.java new file mode 100644 index 000000000000..e442264eb7bf --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBVerifyConnectionIT.java @@ -0,0 +1,123 @@ +/* + * 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.it; + +import org.apache.iotdb.commons.service.ThriftService; +import org.apache.iotdb.consensus.ConsensusFactory; +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.ClusterIT; + +import com.google.common.collect.ImmutableMap; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.HashMap; +import java.util.Map; + +@RunWith(IoTDBTestRunner.class) +@Category({ClusterIT.class}) +public class IoTDBVerifyConnectionIT { + private static final String testConsensusProtocolClass = ConsensusFactory.RATIS_CONSENSUS; + + @Before + public void setUp() throws Exception { + EnvFactory.getEnv() + .getConfig() + .getCommonConfig() + .setConfigNodeConsensusProtocolClass(testConsensusProtocolClass) + .setSchemaRegionConsensusProtocolClass(testConsensusProtocolClass) + .setDataRegionConsensusProtocolClass(testConsensusProtocolClass); + } + + @After + public void tearDown() throws Exception { + EnvFactory.getEnv().cleanClusterEnvironment(); + } + + @Test + public void verifyConnectionAllUpTest() throws Exception { + EnvFactory.getEnv().initClusterEnvironment(3, 3); + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + ResultSet resultSet = statement.executeQuery("verify connection"); + Assert.assertEquals( + ImmutableMap.of(ThriftService.STATUS_UP, 12), collectConnectionResult(resultSet)); + resultSet = statement.executeQuery("verify connection details"); + Assert.assertEquals( + ImmutableMap.of(ThriftService.STATUS_UP, 54), collectConnectionResult(resultSet)); + } + } + + @Test + public void verifyConnectionWithNodeCorruptTest() throws Exception { + EnvFactory.getEnv().initClusterEnvironment(5, 1); + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + ResultSet resultSet = statement.executeQuery("verify connection details"); + Assert.assertEquals( + ImmutableMap.of(ThriftService.STATUS_UP, 38), collectConnectionResult(resultSet)); + EnvFactory.getEnv().getConfigNodeWrapperList().get(0).stopForcibly(); + resultSet = statement.executeQuery("verify connection"); + Assert.assertEquals( + ImmutableMap.of(ThriftService.STATUS_UP, 2, ThriftService.STATUS_DOWN, 11), + collectConnectionResult(resultSet)); + resultSet = statement.executeQuery("verify connection details"); + Assert.assertEquals( + ImmutableMap.of(ThriftService.STATUS_UP, 27, ThriftService.STATUS_DOWN, 11), + collectConnectionResult(resultSet)); + EnvFactory.getEnv().getConfigNodeWrapperList().get(1).stopForcibly(); + Thread.sleep(10000); + resultSet = statement.executeQuery("verify connection"); + Assert.assertEquals( + ImmutableMap.of(ThriftService.STATUS_UP, 2, ThriftService.STATUS_DOWN, 20), + collectConnectionResult(resultSet)); + resultSet = statement.executeQuery("verify connection details"); + Assert.assertEquals( + ImmutableMap.of(ThriftService.STATUS_UP, 18, ThriftService.STATUS_DOWN, 20), + collectConnectionResult(resultSet)); + } + } + + private static Map collectConnectionResult(ResultSet resultSet) + throws Exception { + Map map = new HashMap<>(); + while (resultSet.next()) { + String result = resultSet.getString(3).split(" ")[0]; + map.computeIfPresent(result, (key, value) -> value + 1); + map.putIfAbsent(result, 1); + } + return map; + } + + private static void showResult(ResultSet resultSet) throws Exception { + while (resultSet.next()) { + System.out.printf( + "%s %s %s\n", resultSet.getString(1), resultSet.getString(2), resultSet.getString(3)); + } + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index 41ac7bf47f8f..0d281a0bf3a1 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -158,6 +158,7 @@ public enum TSStatusCode { CAN_NOT_CONNECT_DATANODE(1007), TRANSFER_LEADER_ERROR(1008), GET_CLUSTER_ID_ERROR(1009), + CAN_NOT_CONNECT_CONFIGNODE(1010), // Sync, Load TsFile LOAD_FILE_ERROR(1100), diff --git a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4 b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4 index 29c36e881e7d..c9cfc400dcee 100644 --- a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4 +++ b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4 @@ -63,6 +63,7 @@ keyWords | CONDITION | CONFIGNODES | CONFIGURATION + | CONNECTION | CONNECTOR | CONTAIN | CONTAINS diff --git a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 index 6d3ffe0b837d..191c18889fb9 100644 --- a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 +++ b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 @@ -64,7 +64,7 @@ ddlStatement | createContinuousQuery | dropContinuousQuery | showContinuousQueries // Cluster | showVariables | showCluster | showRegions | showDataNodes | showConfigNodes | showClusterId - | getRegionId | getTimeSlotList | countTimeSlotList | getSeriesSlotList | migrateRegion + | getRegionId | getTimeSlotList | countTimeSlotList | getSeriesSlotList | migrateRegion | verifyConnection // Quota | setSpaceQuota | showSpaceQuota | setThrottleQuota | showThrottleQuota // View @@ -524,6 +524,10 @@ migrateRegion : MIGRATE REGION regionId=INTEGER_LITERAL FROM fromId=INTEGER_LITERAL TO toId=INTEGER_LITERAL ; +verifyConnection + : VERIFY CONNECTION (DETAILS)? + ; + // Pipe Task ========================================================================================= createPipe : CREATE PIPE pipeName=identifier diff --git a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 index 29cbcdcd17df..d1865a1e2048 100644 --- a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 +++ b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 @@ -165,6 +165,10 @@ CONFIGURATION : C O N F I G U R A T I O N ; +CONNECTION + : C O N N E C T I O N + ; + CONNECTOR : C O N N E C T O R ; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/ConfigNodeRequestType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/CnToCnNodeRequestType.java similarity index 92% rename from iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/ConfigNodeRequestType.java rename to iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/CnToCnNodeRequestType.java index 5f3064b4eed7..3864f26cbc8e 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/ConfigNodeRequestType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/CnToCnNodeRequestType.java @@ -19,7 +19,7 @@ package org.apache.iotdb.confignode.client; -public enum ConfigNodeRequestType { +public enum CnToCnNodeRequestType { ADD_CONSENSUS_GROUP, NOTIFY_REGISTER_SUCCESS, REGISTER_CONFIG_NODE, @@ -30,4 +30,6 @@ public enum ConfigNodeRequestType { STOP_CONFIG_NODE, SET_CONFIGURATION, SHOW_CONFIGURATION, + SUBMIT_TEST_CONNECTION_TASK, + TEST_CONNECTION, } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/DataNodeRequestType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/CnToDnRequestType.java similarity index 97% rename from iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/DataNodeRequestType.java rename to iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/CnToDnRequestType.java index 6f22341e3958..bc7307282573 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/DataNodeRequestType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/CnToDnRequestType.java @@ -19,7 +19,7 @@ package org.apache.iotdb.confignode.client; -public enum DataNodeRequestType { +public enum CnToDnRequestType { // Node Maintenance DISABLE_DATA_NODE, @@ -35,6 +35,9 @@ public enum DataNodeRequestType { SET_CONFIGURATION, SHOW_CONFIGURATION, + SUBMIT_TEST_CONNECTION_TASK, + TEST_CONNECTION, + // Region Maintenance CREATE_DATA_REGION, CREATE_SCHEMA_REGION, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncConfigNodeHeartbeatClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncConfigNodeHeartbeatClientPool.java index fdcd3b0d1e29..7b6bca5d0d9e 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncConfigNodeHeartbeatClientPool.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncConfigNodeHeartbeatClientPool.java @@ -22,17 +22,17 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.client.ClientPoolFactory; import org.apache.iotdb.commons.client.IClientManager; -import org.apache.iotdb.commons.client.async.AsyncConfigNodeIServiceClient; +import org.apache.iotdb.commons.client.async.AsyncConfigNodeInternalServiceClient; import org.apache.iotdb.confignode.client.async.handlers.heartbeat.ConfigNodeHeartbeatHandler; import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeHeartbeatReq; public class AsyncConfigNodeHeartbeatClientPool { - private final IClientManager clientManager; + private final IClientManager clientManager; private AsyncConfigNodeHeartbeatClientPool() { clientManager = - new IClientManager.Factory() + new IClientManager.Factory() .createClientManager( new ClientPoolFactory.AsyncConfigNodeHeartbeatServiceClientPoolFactory()); } @@ -53,7 +53,6 @@ public void getConfigNodeHeartBeat( } } - // TODO: Is the ClientPool must be a singleton? private static class AsyncConfigNodeHeartbeatClientPoolHolder { private static final AsyncConfigNodeHeartbeatClientPool INSTANCE = diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncDataNodeClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncDataNodeClientPool.java deleted file mode 100644 index 155597ceb43d..000000000000 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncDataNodeClientPool.java +++ /dev/null @@ -1,552 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.confignode.client.async; - -import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; -import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; -import org.apache.iotdb.common.rpc.thrift.TEndPoint; -import org.apache.iotdb.common.rpc.thrift.TFlushReq; -import org.apache.iotdb.common.rpc.thrift.TSetConfigurationReq; -import org.apache.iotdb.common.rpc.thrift.TSetSpaceQuotaReq; -import org.apache.iotdb.common.rpc.thrift.TSetTTLReq; -import org.apache.iotdb.common.rpc.thrift.TSetThrottleQuotaReq; -import org.apache.iotdb.commons.client.ClientPoolFactory; -import org.apache.iotdb.commons.client.IClientManager; -import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient; -import org.apache.iotdb.commons.client.exception.ClientManagerException; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.AsyncTSStatusRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.CheckTimeSeriesExistenceRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.CountPathsUsingTemplateRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.FetchSchemaBlackListRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.PipeHeartbeatRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.PipePushMetaRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.SchemaUpdateRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.TransferLeaderRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.CheckSchemaRegionUsingTemplateRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.ConsumerGroupPushMetaRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.TopicPushMetaRPCHandler; -import org.apache.iotdb.mpp.rpc.thrift.TActiveTriggerInstanceReq; -import org.apache.iotdb.mpp.rpc.thrift.TAlterViewReq; -import org.apache.iotdb.mpp.rpc.thrift.TCheckSchemaRegionUsingTemplateReq; -import org.apache.iotdb.mpp.rpc.thrift.TCheckTimeSeriesExistenceReq; -import org.apache.iotdb.mpp.rpc.thrift.TConstructSchemaBlackListReq; -import org.apache.iotdb.mpp.rpc.thrift.TConstructSchemaBlackListWithTemplateReq; -import org.apache.iotdb.mpp.rpc.thrift.TConstructViewSchemaBlackListReq; -import org.apache.iotdb.mpp.rpc.thrift.TCountPathsUsingTemplateReq; -import org.apache.iotdb.mpp.rpc.thrift.TCreateDataRegionReq; -import org.apache.iotdb.mpp.rpc.thrift.TCreateFunctionInstanceReq; -import org.apache.iotdb.mpp.rpc.thrift.TCreatePipePluginInstanceReq; -import org.apache.iotdb.mpp.rpc.thrift.TCreateSchemaRegionReq; -import org.apache.iotdb.mpp.rpc.thrift.TCreateTriggerInstanceReq; -import org.apache.iotdb.mpp.rpc.thrift.TDeactivateTemplateReq; -import org.apache.iotdb.mpp.rpc.thrift.TDeleteDataForDeleteSchemaReq; -import org.apache.iotdb.mpp.rpc.thrift.TDeleteTimeSeriesReq; -import org.apache.iotdb.mpp.rpc.thrift.TDeleteViewSchemaReq; -import org.apache.iotdb.mpp.rpc.thrift.TDropFunctionInstanceReq; -import org.apache.iotdb.mpp.rpc.thrift.TDropPipePluginInstanceReq; -import org.apache.iotdb.mpp.rpc.thrift.TDropTriggerInstanceReq; -import org.apache.iotdb.mpp.rpc.thrift.TFetchSchemaBlackListReq; -import org.apache.iotdb.mpp.rpc.thrift.TInactiveTriggerInstanceReq; -import org.apache.iotdb.mpp.rpc.thrift.TInvalidateMatchedSchemaCacheReq; -import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatReq; -import org.apache.iotdb.mpp.rpc.thrift.TPushConsumerGroupMetaReq; -import org.apache.iotdb.mpp.rpc.thrift.TPushMultiPipeMetaReq; -import org.apache.iotdb.mpp.rpc.thrift.TPushMultiTopicMetaReq; -import org.apache.iotdb.mpp.rpc.thrift.TPushPipeMetaReq; -import org.apache.iotdb.mpp.rpc.thrift.TPushSingleConsumerGroupMetaReq; -import org.apache.iotdb.mpp.rpc.thrift.TPushSinglePipeMetaReq; -import org.apache.iotdb.mpp.rpc.thrift.TPushSingleTopicMetaReq; -import org.apache.iotdb.mpp.rpc.thrift.TPushTopicMetaReq; -import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeReq; -import org.apache.iotdb.mpp.rpc.thrift.TRegionRouteReq; -import org.apache.iotdb.mpp.rpc.thrift.TResetPeerListReq; -import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListReq; -import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListWithTemplateReq; -import org.apache.iotdb.mpp.rpc.thrift.TRollbackViewSchemaBlackListReq; -import org.apache.iotdb.mpp.rpc.thrift.TUpdateTemplateReq; -import org.apache.iotdb.mpp.rpc.thrift.TUpdateTriggerLocationReq; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.concurrent.TimeUnit; - -/** Asynchronously send RPC requests to DataNodes. See queryengine.thrift for more details. */ -public class AsyncDataNodeClientPool { - - private static final Logger LOGGER = LoggerFactory.getLogger(AsyncDataNodeClientPool.class); - - private final IClientManager clientManager; - - private static final int MAX_RETRY_NUM = 6; - - private AsyncDataNodeClientPool() { - clientManager = - new IClientManager.Factory() - .createClientManager( - new ClientPoolFactory.AsyncDataNodeInternalServiceClientPoolFactory()); - } - - /** - * Send asynchronous requests to the specified DataNodes with default retry num - * - *

Notice: The DataNodes that failed to receive the requests will be reconnected - * - * @param clientHandler which will also contain the result - * @param timeoutInMs timeout in milliseconds - */ - public void sendAsyncRequestToDataNodeWithRetryAndTimeoutInMs( - AsyncClientHandler clientHandler, long timeoutInMs) { - sendAsyncRequest(clientHandler, MAX_RETRY_NUM, timeoutInMs); - } - - /** - * Send asynchronous requests to the specified DataNodes with default retry num - * - *

Notice: The DataNodes that failed to receive the requests will be reconnected - * - * @param clientHandler which will also contain the result - */ - public void sendAsyncRequestToDataNodeWithRetry(AsyncClientHandler clientHandler) { - sendAsyncRequest(clientHandler, MAX_RETRY_NUM, null); - } - - public void sendAsyncRequestToDataNode(AsyncClientHandler clientHandler) { - sendAsyncRequest(clientHandler, 1, null); - } - - private void sendAsyncRequest( - AsyncClientHandler clientHandler, int retryNum, Long timeoutInMs) { - if (clientHandler.getRequestIndices().isEmpty()) { - return; - } - - DataNodeRequestType requestType = clientHandler.getRequestType(); - for (int retry = 0; retry < retryNum; retry++) { - // Always Reset CountDownLatch first - clientHandler.resetCountDownLatch(); - - // Send requests to all targetDataNodes - for (int requestId : clientHandler.getRequestIndices()) { - TDataNodeLocation targetDataNode = clientHandler.getDataNodeLocation(requestId); - sendAsyncRequestToDataNode(clientHandler, requestId, targetDataNode, retry); - } - - // Wait for this batch of asynchronous RPC requests finish - try { - if (timeoutInMs == null) { - clientHandler.getCountDownLatch().await(); - } else { - if (!clientHandler.getCountDownLatch().await(timeoutInMs, TimeUnit.MILLISECONDS)) { - LOGGER.warn( - "Timeout during {} on ConfigNode. Retry: {}/{}", requestType, retry, retryNum); - } - } - } catch (InterruptedException e) { - LOGGER.error( - "Interrupted during {} on ConfigNode. Retry: {}/{}", requestType, retry, retryNum); - Thread.currentThread().interrupt(); - } - - // Check if there is a DataNode that fails to execute the request, and retry if there exists - if (clientHandler.getRequestIndices().isEmpty()) { - return; - } - } - - if (!clientHandler.getRequestIndices().isEmpty()) { - LOGGER.warn( - "Failed to {} on ConfigNode after {} retries, requestIndices: {}", - requestType, - retryNum, - clientHandler.getRequestIndices()); - } - } - - private void sendAsyncRequestToDataNode( - AsyncClientHandler clientHandler, - int requestId, - TDataNodeLocation targetDataNode, - int retryCount) { - - try { - AsyncDataNodeInternalServiceClient client; - client = clientManager.borrowClient(targetDataNode.getInternalEndPoint()); - - switch (clientHandler.getRequestType()) { - case SET_TTL: - client.setTTL( - (TSetTTLReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CREATE_DATA_REGION: - client.createDataRegion( - (TCreateDataRegionReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case DELETE_REGION: - client.deleteRegion( - (TConsensusGroupId) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CREATE_SCHEMA_REGION: - client.createSchemaRegion( - (TCreateSchemaRegionReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CREATE_FUNCTION: - client.createFunction( - (TCreateFunctionInstanceReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case DROP_FUNCTION: - client.dropFunction( - (TDropFunctionInstanceReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CREATE_TRIGGER_INSTANCE: - client.createTriggerInstance( - (TCreateTriggerInstanceReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case DROP_TRIGGER_INSTANCE: - client.dropTriggerInstance( - (TDropTriggerInstanceReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case ACTIVE_TRIGGER_INSTANCE: - client.activeTriggerInstance( - (TActiveTriggerInstanceReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case INACTIVE_TRIGGER_INSTANCE: - client.inactiveTriggerInstance( - (TInactiveTriggerInstanceReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case UPDATE_TRIGGER_LOCATION: - client.updateTriggerLocation( - (TUpdateTriggerLocationReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CREATE_PIPE_PLUGIN: - client.createPipePlugin( - (TCreatePipePluginInstanceReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case DROP_PIPE_PLUGIN: - client.dropPipePlugin( - (TDropPipePluginInstanceReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case PIPE_PUSH_ALL_META: - client.pushPipeMeta( - (TPushPipeMetaReq) clientHandler.getRequest(requestId), - (PipePushMetaRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case PIPE_PUSH_SINGLE_META: - client.pushSinglePipeMeta( - (TPushSinglePipeMetaReq) clientHandler.getRequest(requestId), - (PipePushMetaRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case PIPE_PUSH_MULTI_META: - client.pushMultiPipeMeta( - (TPushMultiPipeMetaReq) clientHandler.getRequest(requestId), - (PipePushMetaRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case TOPIC_PUSH_ALL_META: - client.pushTopicMeta( - (TPushTopicMetaReq) clientHandler.getRequest(requestId), - (TopicPushMetaRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case TOPIC_PUSH_SINGLE_META: - client.pushSingleTopicMeta( - (TPushSingleTopicMetaReq) clientHandler.getRequest(requestId), - (TopicPushMetaRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case TOPIC_PUSH_MULTI_META: - client.pushMultiTopicMeta( - (TPushMultiTopicMetaReq) clientHandler.getRequest(requestId), - (TopicPushMetaRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CONSUMER_GROUP_PUSH_ALL_META: - client.pushConsumerGroupMeta( - (TPushConsumerGroupMetaReq) clientHandler.getRequest(requestId), - (ConsumerGroupPushMetaRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CONSUMER_GROUP_PUSH_SINGLE_META: - client.pushSingleConsumerGroupMeta( - (TPushSingleConsumerGroupMetaReq) clientHandler.getRequest(requestId), - (ConsumerGroupPushMetaRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case PIPE_HEARTBEAT: - client.pipeHeartbeat( - (TPipeHeartbeatReq) clientHandler.getRequest(requestId), - (PipeHeartbeatRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case MERGE: - case FULL_MERGE: - client.merge( - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case FLUSH: - client.flush( - (TFlushReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CLEAR_CACHE: - client.clearCache( - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case SET_CONFIGURATION: - client.setConfiguration( - (TSetConfigurationReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case START_REPAIR_DATA: - client.startRepairData( - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case STOP_REPAIR_DATA: - client.stopRepairData( - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case LOAD_CONFIGURATION: - client.loadConfiguration( - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case SET_SYSTEM_STATUS: - client.setSystemStatus( - (String) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case UPDATE_REGION_ROUTE_MAP: - client.updateRegionCache( - (TRegionRouteReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CHANGE_REGION_LEADER: - client.changeRegionLeader( - (TRegionLeaderChangeReq) clientHandler.getRequest(requestId), - (TransferLeaderRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CONSTRUCT_SCHEMA_BLACK_LIST: - client.constructSchemaBlackList( - (TConstructSchemaBlackListReq) clientHandler.getRequest(requestId), - (SchemaUpdateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case ROLLBACK_SCHEMA_BLACK_LIST: - client.rollbackSchemaBlackList( - (TRollbackSchemaBlackListReq) clientHandler.getRequest(requestId), - (SchemaUpdateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case FETCH_SCHEMA_BLACK_LIST: - client.fetchSchemaBlackList( - (TFetchSchemaBlackListReq) clientHandler.getRequest(requestId), - (FetchSchemaBlackListRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case INVALIDATE_MATCHED_SCHEMA_CACHE: - client.invalidateMatchedSchemaCache( - (TInvalidateMatchedSchemaCacheReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case DELETE_DATA_FOR_DELETE_SCHEMA: - client.deleteDataForDeleteSchema( - (TDeleteDataForDeleteSchemaReq) clientHandler.getRequest(requestId), - (SchemaUpdateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case DELETE_TIMESERIES: - client.deleteTimeSeries( - (TDeleteTimeSeriesReq) clientHandler.getRequest(requestId), - (SchemaUpdateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CONSTRUCT_SCHEMA_BLACK_LIST_WITH_TEMPLATE: - client.constructSchemaBlackListWithTemplate( - (TConstructSchemaBlackListWithTemplateReq) clientHandler.getRequest(requestId), - (SchemaUpdateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case ROLLBACK_SCHEMA_BLACK_LIST_WITH_TEMPLATE: - client.rollbackSchemaBlackListWithTemplate( - (TRollbackSchemaBlackListWithTemplateReq) clientHandler.getRequest(requestId), - (SchemaUpdateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case DEACTIVATE_TEMPLATE: - client.deactivateTemplate( - (TDeactivateTemplateReq) clientHandler.getRequest(requestId), - (SchemaUpdateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case UPDATE_TEMPLATE: - client.updateTemplate( - (TUpdateTemplateReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case COUNT_PATHS_USING_TEMPLATE: - client.countPathsUsingTemplate( - (TCountPathsUsingTemplateReq) clientHandler.getRequest(requestId), - (CountPathsUsingTemplateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CHECK_SCHEMA_REGION_USING_TEMPLATE: - client.checkSchemaRegionUsingTemplate( - (TCheckSchemaRegionUsingTemplateReq) clientHandler.getRequest(requestId), - (CheckSchemaRegionUsingTemplateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CHECK_TIMESERIES_EXISTENCE: - client.checkTimeSeriesExistence( - (TCheckTimeSeriesExistenceReq) clientHandler.getRequest(requestId), - (CheckTimeSeriesExistenceRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case CONSTRUCT_VIEW_SCHEMA_BLACK_LIST: - client.constructViewSchemaBlackList( - (TConstructViewSchemaBlackListReq) clientHandler.getRequest(requestId), - (SchemaUpdateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case ROLLBACK_VIEW_SCHEMA_BLACK_LIST: - client.rollbackViewSchemaBlackList( - (TRollbackViewSchemaBlackListReq) clientHandler.getRequest(requestId), - (SchemaUpdateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case DELETE_VIEW: - client.deleteViewSchema( - (TDeleteViewSchemaReq) clientHandler.getRequest(requestId), - (SchemaUpdateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case ALTER_VIEW: - client.alterView( - (TAlterViewReq) clientHandler.getRequest(requestId), - (SchemaUpdateRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case KILL_QUERY_INSTANCE: - client.killQueryInstance( - (String) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case SET_SPACE_QUOTA: - client.setSpaceQuota( - (TSetSpaceQuotaReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case SET_THROTTLE_QUOTA: - client.setThrottleQuota( - (TSetThrottleQuotaReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - case RESET_PEER_LIST: - client.resetPeerList( - (TResetPeerListReq) clientHandler.getRequest(requestId), - (AsyncTSStatusRPCHandler) - clientHandler.createAsyncRPCHandler(requestId, targetDataNode)); - break; - default: - LOGGER.error( - "Unexpected DataNode Request Type: {} when sendAsyncRequestToDataNode", - clientHandler.getRequestType()); - } - } catch (Exception e) { - LOGGER.warn( - "{} failed on DataNode {}, because {}, retrying {}...", - clientHandler.getRequestType(), - targetDataNode.getInternalEndPoint(), - e.getMessage(), - retryCount); - } - } - - /** - * Always call this interface when a DataNode is restarted or removed. - * - * @param endPoint The specific DataNode - */ - public void resetClient(TEndPoint endPoint) { - clientManager.clear(endPoint); - } - - public AsyncDataNodeInternalServiceClient getAsyncClient(TDataNodeLocation targetDataNode) - throws ClientManagerException { - return clientManager.borrowClient(targetDataNode.getInternalEndPoint()); - } - - // TODO: Is the ClientPool must be a singleton? - private static class ClientPoolHolder { - - private static final AsyncDataNodeClientPool INSTANCE = new AsyncDataNodeClientPool(); - - private ClientPoolHolder() { - // Empty constructor - } - } - - public static AsyncDataNodeClientPool getInstance() { - return ClientPoolHolder.INSTANCE; - } -} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToCnInternalServiceAsyncRequestManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToCnInternalServiceAsyncRequestManager.java new file mode 100644 index 000000000000..bd5ee3fe4995 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToCnInternalServiceAsyncRequestManager.java @@ -0,0 +1,74 @@ +/* + * 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.client.async; + +import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TNodeLocations; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; +import org.apache.iotdb.commons.client.request.ConfigNodeInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.CnToCnNodeRequestType; +import org.apache.iotdb.confignode.client.async.handlers.rpc.ConfigNodeAsyncRequestRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.ConfigNodeTSStatusRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.SubmitTestConnectionTaskToConfigNodeRPCHandler; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CnToCnInternalServiceAsyncRequestManager + extends ConfigNodeInternalServiceAsyncRequestManager { + + private static final Logger LOGGER = + LoggerFactory.getLogger(CnToCnInternalServiceAsyncRequestManager.class); + + @Override + protected void initActionMapBuilder() { + actionMapBuilder.put( + CnToCnNodeRequestType.SUBMIT_TEST_CONNECTION_TASK, + (req, client, handler) -> + client.submitTestConnectionTask( + (TNodeLocations) req, (SubmitTestConnectionTaskToConfigNodeRPCHandler) handler)); + actionMapBuilder.put( + CnToCnNodeRequestType.TEST_CONNECTION, + (req, client, handler) -> + client.testConnectionEmptyRPC((ConfigNodeTSStatusRPCHandler) handler)); + } + + @Override + protected AsyncRequestRPCHandler buildHandler( + AsyncRequestContext requestContext, + int requestId, + TConfigNodeLocation targetNode) { + return ConfigNodeAsyncRequestRPCHandler.buildHandler(requestContext, requestId, targetNode); + } + + private static class ClientPoolHolder { + private static final CnToCnInternalServiceAsyncRequestManager INSTANCE = + new CnToCnInternalServiceAsyncRequestManager(); + + private ClientPoolHolder() { + // Empty constructor + } + } + + public static CnToCnInternalServiceAsyncRequestManager getInstance() { + return ClientPoolHolder.INSTANCE; + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnInternalServiceAsyncRequestManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnInternalServiceAsyncRequestManager.java new file mode 100644 index 000000000000..8f3e7c720fc6 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnInternalServiceAsyncRequestManager.java @@ -0,0 +1,376 @@ +/* + * 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.client.async; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TFlushReq; +import org.apache.iotdb.common.rpc.thrift.TNodeLocations; +import org.apache.iotdb.common.rpc.thrift.TSetConfigurationReq; +import org.apache.iotdb.common.rpc.thrift.TSetSpaceQuotaReq; +import org.apache.iotdb.common.rpc.thrift.TSetTTLReq; +import org.apache.iotdb.common.rpc.thrift.TSetThrottleQuotaReq; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; +import org.apache.iotdb.commons.client.request.DataNodeInternalServiceRequestManager; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.handlers.rpc.CheckTimeSeriesExistenceRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.CountPathsUsingTemplateRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.DataNodeAsyncRequestRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.DataNodeTSStatusRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.FetchSchemaBlackListRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.PipeHeartbeatRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.PipePushMetaRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.SchemaUpdateRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.SubmitTestConnectionTaskRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.TransferLeaderRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.CheckSchemaRegionUsingTemplateRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.ConsumerGroupPushMetaRPCHandler; +import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.TopicPushMetaRPCHandler; +import org.apache.iotdb.mpp.rpc.thrift.TActiveTriggerInstanceReq; +import org.apache.iotdb.mpp.rpc.thrift.TAlterViewReq; +import org.apache.iotdb.mpp.rpc.thrift.TCheckSchemaRegionUsingTemplateReq; +import org.apache.iotdb.mpp.rpc.thrift.TCheckTimeSeriesExistenceReq; +import org.apache.iotdb.mpp.rpc.thrift.TConstructSchemaBlackListReq; +import org.apache.iotdb.mpp.rpc.thrift.TConstructSchemaBlackListWithTemplateReq; +import org.apache.iotdb.mpp.rpc.thrift.TConstructViewSchemaBlackListReq; +import org.apache.iotdb.mpp.rpc.thrift.TCountPathsUsingTemplateReq; +import org.apache.iotdb.mpp.rpc.thrift.TCreateDataRegionReq; +import org.apache.iotdb.mpp.rpc.thrift.TCreateFunctionInstanceReq; +import org.apache.iotdb.mpp.rpc.thrift.TCreatePipePluginInstanceReq; +import org.apache.iotdb.mpp.rpc.thrift.TCreateSchemaRegionReq; +import org.apache.iotdb.mpp.rpc.thrift.TCreateTriggerInstanceReq; +import org.apache.iotdb.mpp.rpc.thrift.TDeactivateTemplateReq; +import org.apache.iotdb.mpp.rpc.thrift.TDeleteDataForDeleteSchemaReq; +import org.apache.iotdb.mpp.rpc.thrift.TDeleteTimeSeriesReq; +import org.apache.iotdb.mpp.rpc.thrift.TDeleteViewSchemaReq; +import org.apache.iotdb.mpp.rpc.thrift.TDropFunctionInstanceReq; +import org.apache.iotdb.mpp.rpc.thrift.TDropPipePluginInstanceReq; +import org.apache.iotdb.mpp.rpc.thrift.TDropTriggerInstanceReq; +import org.apache.iotdb.mpp.rpc.thrift.TFetchSchemaBlackListReq; +import org.apache.iotdb.mpp.rpc.thrift.TInactiveTriggerInstanceReq; +import org.apache.iotdb.mpp.rpc.thrift.TInvalidateMatchedSchemaCacheReq; +import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatReq; +import org.apache.iotdb.mpp.rpc.thrift.TPushConsumerGroupMetaReq; +import org.apache.iotdb.mpp.rpc.thrift.TPushMultiPipeMetaReq; +import org.apache.iotdb.mpp.rpc.thrift.TPushMultiTopicMetaReq; +import org.apache.iotdb.mpp.rpc.thrift.TPushPipeMetaReq; +import org.apache.iotdb.mpp.rpc.thrift.TPushSingleConsumerGroupMetaReq; +import org.apache.iotdb.mpp.rpc.thrift.TPushSinglePipeMetaReq; +import org.apache.iotdb.mpp.rpc.thrift.TPushSingleTopicMetaReq; +import org.apache.iotdb.mpp.rpc.thrift.TPushTopicMetaReq; +import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeReq; +import org.apache.iotdb.mpp.rpc.thrift.TRegionRouteReq; +import org.apache.iotdb.mpp.rpc.thrift.TResetPeerListReq; +import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListReq; +import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListWithTemplateReq; +import org.apache.iotdb.mpp.rpc.thrift.TRollbackViewSchemaBlackListReq; +import org.apache.iotdb.mpp.rpc.thrift.TUpdateTemplateReq; +import org.apache.iotdb.mpp.rpc.thrift.TUpdateTriggerLocationReq; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Asynchronously send RPC requests to DataNodes. See queryengine.thrift for more details. */ +public class CnToDnInternalServiceAsyncRequestManager + extends DataNodeInternalServiceRequestManager { + + private static final Logger LOGGER = + LoggerFactory.getLogger(CnToDnInternalServiceAsyncRequestManager.class); + + @Override + protected void initActionMapBuilder() { + actionMapBuilder.put( + CnToDnRequestType.SET_TTL, + (req, client, handler) -> + client.setTTL((TSetTTLReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CREATE_DATA_REGION, + (req, client, handler) -> + client.createDataRegion( + (TCreateDataRegionReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.DELETE_REGION, + (req, client, handler) -> + client.deleteRegion((TConsensusGroupId) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CREATE_SCHEMA_REGION, + (req, client, handler) -> + client.createSchemaRegion( + (TCreateSchemaRegionReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CREATE_FUNCTION, + (req, client, handler) -> + client.createFunction( + (TCreateFunctionInstanceReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.DROP_FUNCTION, + (req, client, handler) -> + client.dropFunction( + (TDropFunctionInstanceReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CREATE_TRIGGER_INSTANCE, + (req, client, handler) -> + client.createTriggerInstance( + (TCreateTriggerInstanceReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.DROP_TRIGGER_INSTANCE, + (req, client, handler) -> + client.dropTriggerInstance( + (TDropTriggerInstanceReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.ACTIVE_TRIGGER_INSTANCE, + (req, client, handler) -> + client.activeTriggerInstance( + (TActiveTriggerInstanceReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.INACTIVE_TRIGGER_INSTANCE, + (req, client, handler) -> + client.inactiveTriggerInstance( + (TInactiveTriggerInstanceReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.UPDATE_TRIGGER_LOCATION, + (req, client, handler) -> + client.updateTriggerLocation( + (TUpdateTriggerLocationReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CREATE_PIPE_PLUGIN, + (req, client, handler) -> + client.createPipePlugin( + (TCreatePipePluginInstanceReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.DROP_PIPE_PLUGIN, + (req, client, handler) -> + client.dropPipePlugin( + (TDropPipePluginInstanceReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.PIPE_PUSH_ALL_META, + (req, client, handler) -> + client.pushPipeMeta((TPushPipeMetaReq) req, (PipePushMetaRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.PIPE_PUSH_SINGLE_META, + (req, client, handler) -> + client.pushSinglePipeMeta( + (TPushSinglePipeMetaReq) req, (PipePushMetaRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.PIPE_PUSH_MULTI_META, + (req, client, handler) -> + client.pushMultiPipeMeta( + (TPushMultiPipeMetaReq) req, (PipePushMetaRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.TOPIC_PUSH_ALL_META, + (req, client, handler) -> + client.pushTopicMeta((TPushTopicMetaReq) req, (TopicPushMetaRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.TOPIC_PUSH_SINGLE_META, + (req, client, handler) -> + client.pushSingleTopicMeta( + (TPushSingleTopicMetaReq) req, (TopicPushMetaRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.TOPIC_PUSH_MULTI_META, + (req, client, handler) -> + client.pushMultiTopicMeta( + (TPushMultiTopicMetaReq) req, (TopicPushMetaRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CONSUMER_GROUP_PUSH_ALL_META, + (req, client, handler) -> + client.pushConsumerGroupMeta( + (TPushConsumerGroupMetaReq) req, (ConsumerGroupPushMetaRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CONSUMER_GROUP_PUSH_SINGLE_META, + (req, client, handler) -> + client.pushSingleConsumerGroupMeta( + (TPushSingleConsumerGroupMetaReq) req, (ConsumerGroupPushMetaRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.PIPE_HEARTBEAT, + (req, client, handler) -> + client.pipeHeartbeat((TPipeHeartbeatReq) req, (PipeHeartbeatRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.MERGE, + (req, client, handler) -> client.merge((DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.FULL_MERGE, + (req, client, handler) -> client.merge((DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.FLUSH, + (req, client, handler) -> + client.flush((TFlushReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CLEAR_CACHE, + (req, client, handler) -> client.clearCache((DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.START_REPAIR_DATA, + (req, client, handler) -> client.startRepairData((DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.STOP_REPAIR_DATA, + (req, client, handler) -> client.stopRepairData((DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.LOAD_CONFIGURATION, + (req, client, handler) -> client.loadConfiguration((DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.SET_SYSTEM_STATUS, + (req, client, handler) -> + client.setSystemStatus((String) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.SET_CONFIGURATION, + (req, client, handler) -> + client.setConfiguration( + (TSetConfigurationReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.UPDATE_REGION_ROUTE_MAP, + (req, client, handler) -> + client.updateRegionCache((TRegionRouteReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CHANGE_REGION_LEADER, + (req, client, handler) -> + client.changeRegionLeader( + (TRegionLeaderChangeReq) req, (TransferLeaderRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CONSTRUCT_SCHEMA_BLACK_LIST, + (req, client, handler) -> + client.constructSchemaBlackList( + (TConstructSchemaBlackListReq) req, (SchemaUpdateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.ROLLBACK_SCHEMA_BLACK_LIST, + (req, client, handler) -> + client.rollbackSchemaBlackList( + (TRollbackSchemaBlackListReq) req, (SchemaUpdateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.FETCH_SCHEMA_BLACK_LIST, + (req, client, handler) -> + client.fetchSchemaBlackList( + (TFetchSchemaBlackListReq) req, (FetchSchemaBlackListRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.INVALIDATE_MATCHED_SCHEMA_CACHE, + (req, client, handler) -> + client.invalidateMatchedSchemaCache( + (TInvalidateMatchedSchemaCacheReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.DELETE_DATA_FOR_DELETE_SCHEMA, + (req, client, handler) -> + client.deleteDataForDeleteSchema( + (TDeleteDataForDeleteSchemaReq) req, (SchemaUpdateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.DELETE_TIMESERIES, + (req, client, handler) -> + client.deleteTimeSeries((TDeleteTimeSeriesReq) req, (SchemaUpdateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CONSTRUCT_SCHEMA_BLACK_LIST_WITH_TEMPLATE, + (req, client, handler) -> + client.constructSchemaBlackListWithTemplate( + (TConstructSchemaBlackListWithTemplateReq) req, (SchemaUpdateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.ROLLBACK_SCHEMA_BLACK_LIST_WITH_TEMPLATE, + (req, client, handler) -> + client.rollbackSchemaBlackListWithTemplate( + (TRollbackSchemaBlackListWithTemplateReq) req, (SchemaUpdateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.DEACTIVATE_TEMPLATE, + (req, client, handler) -> + client.deactivateTemplate( + (TDeactivateTemplateReq) req, (SchemaUpdateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.UPDATE_TEMPLATE, + (req, client, handler) -> + client.updateTemplate((TUpdateTemplateReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.COUNT_PATHS_USING_TEMPLATE, + (req, client, handler) -> + client.countPathsUsingTemplate( + (TCountPathsUsingTemplateReq) req, (CountPathsUsingTemplateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CHECK_SCHEMA_REGION_USING_TEMPLATE, + (req, client, handler) -> + client.checkSchemaRegionUsingTemplate( + (TCheckSchemaRegionUsingTemplateReq) req, + (CheckSchemaRegionUsingTemplateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CHECK_TIMESERIES_EXISTENCE, + (req, client, handler) -> + client.checkTimeSeriesExistence( + (TCheckTimeSeriesExistenceReq) req, (CheckTimeSeriesExistenceRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.CONSTRUCT_VIEW_SCHEMA_BLACK_LIST, + (req, client, handler) -> + client.constructViewSchemaBlackList( + (TConstructViewSchemaBlackListReq) req, (SchemaUpdateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.ROLLBACK_VIEW_SCHEMA_BLACK_LIST, + (req, client, handler) -> + client.rollbackViewSchemaBlackList( + (TRollbackViewSchemaBlackListReq) req, (SchemaUpdateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.DELETE_VIEW, + (req, client, handler) -> + client.deleteViewSchema((TDeleteViewSchemaReq) req, (SchemaUpdateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.ALTER_VIEW, + (req, client, handler) -> + client.alterView((TAlterViewReq) req, (SchemaUpdateRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.KILL_QUERY_INSTANCE, + (req, client, handler) -> + client.killQueryInstance((String) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.SET_SPACE_QUOTA, + (req, client, handler) -> + client.setSpaceQuota((TSetSpaceQuotaReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.SET_THROTTLE_QUOTA, + (req, client, handler) -> + client.setThrottleQuota( + (TSetThrottleQuotaReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.RESET_PEER_LIST, + (req, client, handler) -> + client.resetPeerList((TResetPeerListReq) req, (DataNodeTSStatusRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.SUBMIT_TEST_CONNECTION_TASK, + (req, client, handler) -> + client.submitTestConnectionTask( + (TNodeLocations) req, (SubmitTestConnectionTaskRPCHandler) handler)); + actionMapBuilder.put( + CnToDnRequestType.TEST_CONNECTION, + (req, client, handler) -> + client.testConnectionEmptyRPC((DataNodeTSStatusRPCHandler) handler)); + } + + @Override + protected AsyncRequestRPCHandler buildHandler( + AsyncRequestContext requestContext, + int requestId, + TDataNodeLocation targetNode) { + return DataNodeAsyncRequestRPCHandler.buildHandler(requestContext, requestId, targetNode); + } + + private static class ClientPoolHolder { + + private static final CnToDnInternalServiceAsyncRequestManager INSTANCE = + new CnToDnInternalServiceAsyncRequestManager(); + + private ClientPoolHolder() { + // Empty constructor + } + } + + public static CnToDnInternalServiceAsyncRequestManager getInstance() { + return ClientPoolHolder.INSTANCE; + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/ConfigNodeAsyncRequestContext.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/ConfigNodeAsyncRequestContext.java new file mode 100644 index 000000000000..cde2cf1c8ec2 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/ConfigNodeAsyncRequestContext.java @@ -0,0 +1,43 @@ +/* + * 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.client.async.handlers; + +import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.confignode.client.CnToCnNodeRequestType; + +import java.util.Map; + +/** + * Asynchronous Client handler. + * + * @param ClassName of RPC request + * @param ClassName of RPC response + */ +public class ConfigNodeAsyncRequestContext + extends AsyncRequestContext { + + public ConfigNodeAsyncRequestContext( + CnToCnNodeRequestType configNodeRequestType, + Q q, + Map integerTConfigNodeLocationMap) { + super(configNodeRequestType, q, integerTConfigNodeLocationMap); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/DataNodeAsyncRequestContext.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/DataNodeAsyncRequestContext.java new file mode 100644 index 000000000000..2b813c081d7e --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/DataNodeAsyncRequestContext.java @@ -0,0 +1,50 @@ +/* + * 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.client.async.handlers; + +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.confignode.client.CnToDnRequestType; + +import java.util.Map; + +/** + * Asynchronous Client handler. + * + * @param ClassName of RPC request + * @param ClassName of RPC response + */ +public class DataNodeAsyncRequestContext + extends AsyncRequestContext { + + public DataNodeAsyncRequestContext(CnToDnRequestType requestType) { + super(requestType); + } + + public DataNodeAsyncRequestContext( + CnToDnRequestType requestType, Map dataNodeLocationMap) { + super(requestType, dataNodeLocationMap); + } + + public DataNodeAsyncRequestContext( + CnToDnRequestType requestType, Q q, Map dataNodeLocationMap) { + super(requestType, q, dataNodeLocationMap); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/CheckTimeSeriesExistenceRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/CheckTimeSeriesExistenceRPCHandler.java index 150947f1b97b..b12ba289f1c2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/CheckTimeSeriesExistenceRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/CheckTimeSeriesExistenceRPCHandler.java @@ -21,7 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.confignode.client.DataNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; import org.apache.iotdb.mpp.rpc.thrift.TCheckTimeSeriesExistenceResp; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -33,13 +33,13 @@ import java.util.concurrent.CountDownLatch; public class CheckTimeSeriesExistenceRPCHandler - extends AbstractAsyncRPCHandler { + extends DataNodeAsyncRequestRPCHandler { private static final Logger LOGGER = LoggerFactory.getLogger(CheckTimeSeriesExistenceRPCHandler.class); public CheckTimeSeriesExistenceRPCHandler( - DataNodeRequestType requestType, + CnToDnRequestType requestType, int requestId, TDataNodeLocation targetDataNode, Map dataNodeLocationMap, @@ -53,15 +53,13 @@ public void onComplete(TCheckTimeSeriesExistenceResp response) { TSStatus tsStatus = response.getStatus(); responseMap.put(requestId, response); if (tsStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - dataNodeLocationMap.remove(requestId); - LOGGER.info("Successfully check timeseries existence on DataNode: {}", targetDataNode); + nodeLocationMap.remove(requestId); + LOGGER.info("Successfully check timeseries existence on DataNode: {}", targetNode); } else if (tsStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { - dataNodeLocationMap.remove(requestId); - LOGGER.error( - "Failed to check timeseries existence on DataNode {}, {}", targetDataNode, tsStatus); + nodeLocationMap.remove(requestId); + LOGGER.error("Failed to check timeseries existence on DataNode {}, {}", targetNode, tsStatus); } else { - LOGGER.error( - "Failed to check timeseries existence on DataNode {}, {}", targetDataNode, tsStatus); + LOGGER.error("Failed to check timeseries existence on DataNode {}, {}", targetNode, tsStatus); } countDownLatch.countDown(); } @@ -70,9 +68,9 @@ public void onComplete(TCheckTimeSeriesExistenceResp response) { public void onError(Exception e) { String errorMsg = "Check timeseries existence error on DataNode: {id=" - + targetDataNode.getDataNodeId() + + targetNode.getDataNodeId() + ", internalEndPoint=" - + targetDataNode.getInternalEndPoint() + + targetNode.getInternalEndPoint() + "}" + e.getMessage(); LOGGER.error(errorMsg); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/ConfigNodeAsyncRequestRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/ConfigNodeAsyncRequestRPCHandler.java new file mode 100644 index 000000000000..4825629ee559 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/ConfigNodeAsyncRequestRPCHandler.java @@ -0,0 +1,88 @@ +/* + * 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.client.async.handlers.rpc; + +import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResp; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; +import org.apache.iotdb.confignode.client.CnToCnNodeRequestType; + +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +public abstract class ConfigNodeAsyncRequestRPCHandler + extends AsyncRequestRPCHandler { + + protected ConfigNodeAsyncRequestRPCHandler( + CnToCnNodeRequestType configNodeRequestType, + int requestId, + TConfigNodeLocation targetNode, + Map integerTConfigNodeLocationMap, + Map integerResponseMap, + CountDownLatch countDownLatch) { + super( + configNodeRequestType, + requestId, + targetNode, + integerTConfigNodeLocationMap, + integerResponseMap, + countDownLatch); + } + + @Override + protected String generateFormattedTargetLocation(TConfigNodeLocation configNodeLocation) { + return "{id=" + + targetNode.getConfigNodeId() + + ", internalEndPoint=" + + targetNode.getInternalEndPoint() + + "}"; + } + + public static ConfigNodeAsyncRequestRPCHandler buildHandler( + AsyncRequestContext context, + int requestId, + TConfigNodeLocation targetConfigNode) { + CnToCnNodeRequestType requestType = context.getRequestType(); + Map nodeLocationMap = context.getNodeLocationMap(); + Map responseMap = context.getResponseMap(); + CountDownLatch countDownLatch = context.getCountDownLatch(); + switch (requestType) { + case SUBMIT_TEST_CONNECTION_TASK: + return new SubmitTestConnectionTaskToConfigNodeRPCHandler( + requestType, + requestId, + targetConfigNode, + nodeLocationMap, + (Map) responseMap, + countDownLatch); + case TEST_CONNECTION: + default: + return new ConfigNodeTSStatusRPCHandler( + requestType, + requestId, + targetConfigNode, + nodeLocationMap, + (Map) responseMap, + countDownLatch); + } + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/ConfigNodeTSStatusRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/ConfigNodeTSStatusRPCHandler.java new file mode 100644 index 000000000000..a28acb349626 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/ConfigNodeTSStatusRPCHandler.java @@ -0,0 +1,95 @@ +/* + * 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.client.async.handlers.rpc; + +import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.confignode.client.CnToCnNodeRequestType; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +/** General RPC handler for TSStatus response type. */ +public class ConfigNodeTSStatusRPCHandler extends ConfigNodeAsyncRequestRPCHandler { + + private static final Logger LOGGER = LoggerFactory.getLogger(ConfigNodeTSStatusRPCHandler.class); + + public ConfigNodeTSStatusRPCHandler( + CnToCnNodeRequestType requestType, + int requestId, + TConfigNodeLocation targetConfigNode, + Map configNodeLocationMap, + Map responseMap, + CountDownLatch countDownLatch) { + super( + requestType, + requestId, + targetConfigNode, + configNodeLocationMap, + responseMap, + countDownLatch); + } + + @Override + public void onComplete(TSStatus response) { + // Put response + responseMap.put(requestId, response); + + if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + // Remove only if success + nodeLocationMap.remove(requestId); + LOGGER.info("Successfully {} on ConfigNode: {}", requestType, formattedTargetLocation); + } else { + LOGGER.error( + "Failed to {} on ConfigNode: {}, response: {}", + requestType, + formattedTargetLocation, + response); + } + + // Always CountDown + countDownLatch.countDown(); + } + + @Override + public void onError(Exception e) { + String errorMsg = + "Failed to " + + requestType + + " on ConfigNode: " + + formattedTargetLocation + + ", exception: " + + e.getMessage(); + LOGGER.error(errorMsg); + + responseMap.put( + requestId, + new TSStatus( + RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode(), errorMsg))); + + // Always CountDown + countDownLatch.countDown(); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/CountPathsUsingTemplateRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/CountPathsUsingTemplateRPCHandler.java index 168208d86b97..f2ed0cc425c5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/CountPathsUsingTemplateRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/CountPathsUsingTemplateRPCHandler.java @@ -21,7 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.confignode.client.DataNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; import org.apache.iotdb.mpp.rpc.thrift.TCountPathsUsingTemplateResp; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -33,13 +33,13 @@ import java.util.concurrent.CountDownLatch; public class CountPathsUsingTemplateRPCHandler - extends AbstractAsyncRPCHandler { + extends DataNodeAsyncRequestRPCHandler { private static final Logger LOGGER = LoggerFactory.getLogger(CountPathsUsingTemplateRPCHandler.class); public CountPathsUsingTemplateRPCHandler( - DataNodeRequestType requestType, + CnToDnRequestType requestType, int requestId, TDataNodeLocation targetDataNode, Map dataNodeLocationMap, @@ -53,15 +53,13 @@ public void onComplete(TCountPathsUsingTemplateResp response) { TSStatus tsStatus = response.getStatus(); responseMap.put(requestId, response); if (tsStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - dataNodeLocationMap.remove(requestId); - LOGGER.info("Successfully count paths using template on DataNode: {}", targetDataNode); + nodeLocationMap.remove(requestId); + LOGGER.info("Successfully count paths using template on DataNode: {}", targetNode); } else if (tsStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { - dataNodeLocationMap.remove(requestId); - LOGGER.error( - "Failed to count paths using template on DataNode {}, {}", targetDataNode, tsStatus); + nodeLocationMap.remove(requestId); + LOGGER.error("Failed to count paths using template on DataNode {}, {}", targetNode, tsStatus); } else { - LOGGER.error( - "Failed to count paths using template on DataNode {}, {}", targetDataNode, tsStatus); + LOGGER.error("Failed to count paths using template on DataNode {}, {}", targetNode, tsStatus); } countDownLatch.countDown(); } @@ -70,9 +68,9 @@ public void onComplete(TCountPathsUsingTemplateResp response) { public void onError(Exception e) { String errorMsg = "Count paths using template error on DataNode: {id=" - + targetDataNode.getDataNodeId() + + targetNode.getDataNodeId() + ", internalEndPoint=" - + targetDataNode.getInternalEndPoint() + + targetNode.getInternalEndPoint() + "}" + e.getMessage(); LOGGER.error(errorMsg); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/AsyncClientHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeAsyncRequestRPCHandler.java similarity index 58% rename from iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/AsyncClientHandler.java rename to iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeAsyncRequestRPCHandler.java index d4b0e7f52b71..f14a2365a9b3 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/AsyncClientHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeAsyncRequestRPCHandler.java @@ -17,20 +17,14 @@ * under the License. */ -package org.apache.iotdb.confignode.client.async.handlers; +package org.apache.iotdb.confignode.client.async.handlers.rpc; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.handlers.rpc.AbstractAsyncRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.AsyncTSStatusRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.CheckTimeSeriesExistenceRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.CountPathsUsingTemplateRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.FetchSchemaBlackListRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.PipeHeartbeatRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.PipePushMetaRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.SchemaUpdateRPCHandler; -import org.apache.iotdb.confignode.client.async.handlers.rpc.TransferLeaderRPCHandler; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResp; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.CheckSchemaRegionUsingTemplateRPCHandler; import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.ConsumerGroupPushMetaRPCHandler; import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.TopicPushMetaRPCHandler; @@ -44,129 +38,45 @@ import org.apache.iotdb.mpp.rpc.thrift.TPushTopicMetaResp; import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeResp; -import java.util.ArrayList; -import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; -/** - * Asynchronous Client handler. - * - * @param ClassName of RPC request - * @param ClassName of RPC response - */ -public class AsyncClientHandler { - - // Type of RPC request - protected final DataNodeRequestType requestType; - - /** - * Map key: The indices of asynchronous RPC requests. - * - *

Map value: The corresponding RPC request - */ - private final Map requestMap; - - /** - * Map key: The indices of asynchronous RPC requests. - * - *

Map value: The target DataNodes of corresponding indices - * - *

All kinds of AsyncHandler will remove its targetDataNode from the dataNodeLocationMap only - * if its corresponding RPC request success - */ - private final Map dataNodeLocationMap; - - /** - * Map key: The indices(targetDataNode's ID) of asynchronous RPC requests. - * - *

Map value: The response of corresponding indices - * - *

All kinds of AsyncHandler will add response to the responseMap after its corresponding RPC - * request finished - */ - private final Map responseMap; - - private CountDownLatch countDownLatch; - - /** Custom constructor. */ - public AsyncClientHandler(DataNodeRequestType requestType) { - this.requestType = requestType; - this.requestMap = new ConcurrentHashMap<>(); - this.dataNodeLocationMap = new ConcurrentHashMap<>(); - this.responseMap = new ConcurrentHashMap<>(); - } - - public void putRequest(int requestId, Q request) { - requestMap.put(requestId, request); - } - - public void putDataNodeLocation(int requestId, TDataNodeLocation dataNodeLocation) { - dataNodeLocationMap.put(requestId, dataNodeLocation); - } - - /** Constructor for null requests. */ - public AsyncClientHandler( - DataNodeRequestType requestType, Map dataNodeLocationMap) { - this.requestType = requestType; - this.dataNodeLocationMap = dataNodeLocationMap; - - this.requestMap = new ConcurrentHashMap<>(); - this.responseMap = new ConcurrentHashMap<>(); - } - - /** Constructor for unique request. */ - public AsyncClientHandler( - DataNodeRequestType requestType, - Q request, - Map dataNodeLocationMap) { - this.requestType = requestType; - this.dataNodeLocationMap = dataNodeLocationMap; - - this.requestMap = new ConcurrentHashMap<>(); - this.dataNodeLocationMap - .keySet() - .forEach(dataNodeId -> this.requestMap.put(dataNodeId, request)); - - this.responseMap = new ConcurrentHashMap<>(); +public abstract class DataNodeAsyncRequestRPCHandler + extends AsyncRequestRPCHandler { + + protected DataNodeAsyncRequestRPCHandler( + CnToDnRequestType requestType, + int requestId, + TDataNodeLocation targetNode, + Map dataNodeLocationMap, + Map integerResponseMap, + CountDownLatch countDownLatch) { + super( + requestType, + requestId, + targetNode, + dataNodeLocationMap, + integerResponseMap, + countDownLatch); } - public DataNodeRequestType getRequestType() { - return requestType; + @Override + protected String generateFormattedTargetLocation(TDataNodeLocation dataNodeLocation) { + return "{id=" + + targetNode.getDataNodeId() + + ", internalEndPoint=" + + targetNode.getInternalEndPoint() + + "}"; } - public List getRequestIndices() { - return new ArrayList<>(dataNodeLocationMap.keySet()); - } - - public Q getRequest(int requestId) { - return requestMap.get(requestId); - } - - public TDataNodeLocation getDataNodeLocation(int requestId) { - return dataNodeLocationMap.get(requestId); - } - - public List getResponseList() { - return new ArrayList<>(responseMap.values()); - } - - public Map getResponseMap() { - return responseMap; - } - - /** Always reset CountDownLatch before retry. */ - public void resetCountDownLatch() { - countDownLatch = new CountDownLatch(dataNodeLocationMap.size()); - } - - public CountDownLatch getCountDownLatch() { - return countDownLatch; - } - - public AbstractAsyncRPCHandler createAsyncRPCHandler( - int requestId, TDataNodeLocation targetDataNode) { + public static DataNodeAsyncRequestRPCHandler buildHandler( + AsyncRequestContext context, + int requestId, + TDataNodeLocation targetDataNode) { + CnToDnRequestType requestType = context.getRequestType(); + Map dataNodeLocationMap = context.getNodeLocationMap(); + Map responseMap = context.getResponseMap(); + CountDownLatch countDownLatch = context.getCountDownLatch(); switch (requestType) { case CONSTRUCT_SCHEMA_BLACK_LIST: case ROLLBACK_SCHEMA_BLACK_LIST: @@ -263,6 +173,14 @@ public AbstractAsyncRPCHandler createAsyncRPCHandler( dataNodeLocationMap, (Map) responseMap, countDownLatch); + case SUBMIT_TEST_CONNECTION_TASK: + return new SubmitTestConnectionTaskRPCHandler( + requestType, + requestId, + targetDataNode, + dataNodeLocationMap, + (Map) responseMap, + countDownLatch); case SET_TTL: case CREATE_DATA_REGION: case CREATE_SCHEMA_REGION: @@ -286,8 +204,9 @@ public AbstractAsyncRPCHandler createAsyncRPCHandler( case UPDATE_TEMPLATE: case KILL_QUERY_INSTANCE: case RESET_PEER_LIST: + case TEST_CONNECTION: default: - return new AsyncTSStatusRPCHandler( + return new DataNodeTSStatusRPCHandler( requestType, requestId, targetDataNode, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeTSStatusRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeTSStatusRPCHandler.java new file mode 100644 index 000000000000..19d451eb671f --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeTSStatusRPCHandler.java @@ -0,0 +1,89 @@ +/* + * 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.client.async.handlers.rpc; + +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +/** General RPC handler for TSStatus response type. */ +public class DataNodeTSStatusRPCHandler extends DataNodeAsyncRequestRPCHandler { + + private static final Logger LOGGER = LoggerFactory.getLogger(DataNodeTSStatusRPCHandler.class); + + public DataNodeTSStatusRPCHandler( + CnToDnRequestType requestType, + int requestId, + TDataNodeLocation targetDataNode, + Map dataNodeLocationMap, + Map responseMap, + CountDownLatch countDownLatch) { + super(requestType, requestId, targetDataNode, dataNodeLocationMap, responseMap, countDownLatch); + } + + @Override + public void onComplete(TSStatus response) { + // Put response + responseMap.put(requestId, response); + + if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + // Remove only if success + nodeLocationMap.remove(requestId); + LOGGER.info("Successfully {} on DataNode: {}", requestType, formattedTargetLocation); + } else { + LOGGER.error( + "Failed to {} on DataNode: {}, response: {}", + requestType, + formattedTargetLocation, + response); + } + + // Always CountDown + countDownLatch.countDown(); + } + + @Override + public void onError(Exception e) { + String errorMsg = + "Failed to " + + requestType + + " on DataNode: " + + formattedTargetLocation + + ", exception: " + + e.getMessage(); + LOGGER.error(errorMsg); + + responseMap.put( + requestId, + new TSStatus( + RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode(), errorMsg))); + + // Always CountDown + countDownLatch.countDown(); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/FetchSchemaBlackListRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/FetchSchemaBlackListRPCHandler.java index e73e41322065..45c659298ca3 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/FetchSchemaBlackListRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/FetchSchemaBlackListRPCHandler.java @@ -21,7 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.confignode.client.DataNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; import org.apache.iotdb.mpp.rpc.thrift.TFetchSchemaBlackListResp; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -33,13 +33,13 @@ import java.util.concurrent.CountDownLatch; public class FetchSchemaBlackListRPCHandler - extends AbstractAsyncRPCHandler { + extends DataNodeAsyncRequestRPCHandler { private static final Logger LOGGER = LoggerFactory.getLogger(FetchSchemaBlackListRPCHandler.class); public FetchSchemaBlackListRPCHandler( - DataNodeRequestType requestType, + CnToDnRequestType requestType, int requestId, TDataNodeLocation targetDataNode, Map dataNodeLocationMap, @@ -53,15 +53,15 @@ public void onComplete(TFetchSchemaBlackListResp tFetchSchemaBlackListResp) { TSStatus tsStatus = tFetchSchemaBlackListResp.getStatus(); responseMap.put(requestId, tFetchSchemaBlackListResp); if (tsStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - dataNodeLocationMap.remove(requestId); - LOGGER.info("Successfully fetch schemaengine black list on DataNode: {}", targetDataNode); + nodeLocationMap.remove(requestId); + LOGGER.info("Successfully fetch schemaengine black list on DataNode: {}", targetNode); } else if (tsStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { - dataNodeLocationMap.remove(requestId); + nodeLocationMap.remove(requestId); LOGGER.error( - "Failed to fetch schemaengine black list on DataNode {}, {}", targetDataNode, tsStatus); + "Failed to fetch schemaengine black list on DataNode {}, {}", targetNode, tsStatus); } else { LOGGER.error( - "Failed to fetch schemaengine black list on DataNode {}, {}", targetDataNode, tsStatus); + "Failed to fetch schemaengine black list on DataNode {}, {}", targetNode, tsStatus); } countDownLatch.countDown(); } @@ -70,9 +70,9 @@ public void onComplete(TFetchSchemaBlackListResp tFetchSchemaBlackListResp) { public void onError(Exception e) { String errorMsg = "Fetch schemaengine black list error on DataNode: {id=" - + targetDataNode.getDataNodeId() + + targetNode.getDataNodeId() + ", internalEndPoint=" - + targetDataNode.getInternalEndPoint() + + targetNode.getInternalEndPoint() + "}" + e.getMessage(); LOGGER.error(errorMsg); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipeHeartbeatRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipeHeartbeatRPCHandler.java index ff235a38bd08..ec4968cfa06d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipeHeartbeatRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipeHeartbeatRPCHandler.java @@ -20,7 +20,7 @@ package org.apache.iotdb.confignode.client.async.handlers.rpc; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; -import org.apache.iotdb.confignode.client.DataNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatResp; import org.slf4j.Logger; @@ -29,12 +29,12 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; -public class PipeHeartbeatRPCHandler extends AbstractAsyncRPCHandler { +public class PipeHeartbeatRPCHandler extends DataNodeAsyncRequestRPCHandler { private static final Logger LOGGER = LoggerFactory.getLogger(PipeHeartbeatRPCHandler.class); public PipeHeartbeatRPCHandler( - DataNodeRequestType requestType, + CnToDnRequestType requestType, int requestId, TDataNodeLocation targetDataNode, Map dataNodeLocationMap, @@ -47,7 +47,7 @@ public PipeHeartbeatRPCHandler( public void onComplete(TPipeHeartbeatResp response) { // Put response responseMap.put(requestId, response); - dataNodeLocationMap.remove(requestId); + nodeLocationMap.remove(requestId); LOGGER.info("Successfully {} on DataNode: {}", requestType, formattedTargetLocation); // Always CountDown diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipePushMetaRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipePushMetaRPCHandler.java index 774e914de367..3517bbeb93ea 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipePushMetaRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipePushMetaRPCHandler.java @@ -20,7 +20,7 @@ package org.apache.iotdb.confignode.client.async.handlers.rpc; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; -import org.apache.iotdb.confignode.client.DataNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; import org.apache.iotdb.mpp.rpc.thrift.TPushPipeMetaResp; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -31,11 +31,11 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; -public class PipePushMetaRPCHandler extends AbstractAsyncRPCHandler { +public class PipePushMetaRPCHandler extends DataNodeAsyncRequestRPCHandler { private static final Logger LOGGER = LoggerFactory.getLogger(PipePushMetaRPCHandler.class); public PipePushMetaRPCHandler( - DataNodeRequestType requestType, + CnToDnRequestType requestType, int requestId, TDataNodeLocation targetDataNode, Map dataNodeLocationMap, @@ -60,7 +60,7 @@ public void onComplete(TPushPipeMetaResp response) { } // Always remove to avoid retrying - dataNodeLocationMap.remove(requestId); + nodeLocationMap.remove(requestId); // Always CountDown countDownLatch.countDown(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/SchemaUpdateRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/SchemaUpdateRPCHandler.java index 3980e6e0c1b5..db8458a948ce 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/SchemaUpdateRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/SchemaUpdateRPCHandler.java @@ -21,7 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.confignode.client.DataNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -31,12 +31,12 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; -public class SchemaUpdateRPCHandler extends AsyncTSStatusRPCHandler { +public class SchemaUpdateRPCHandler extends DataNodeTSStatusRPCHandler { private static final Logger LOGGER = LoggerFactory.getLogger(SchemaUpdateRPCHandler.class); public SchemaUpdateRPCHandler( - DataNodeRequestType requestType, + CnToDnRequestType requestType, int requestId, TDataNodeLocation targetDataNode, Map dataNodeLocationMap, @@ -50,10 +50,10 @@ public void onComplete(TSStatus tsStatus) { responseMap.put(requestId, tsStatus); LOGGER.info("{} for {} receives: {}", requestType, requestId, tsStatus); if (tsStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - dataNodeLocationMap.remove(requestId); + nodeLocationMap.remove(requestId); LOGGER.info("Successfully {} on DataNode: {}", requestType, formattedTargetLocation); } else if (tsStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { - dataNodeLocationMap.remove(requestId); + nodeLocationMap.remove(requestId); LOGGER.warn( "Failed to {} on DataNode {}, {}", requestType, formattedTargetLocation, tsStatus); } else { @@ -68,9 +68,9 @@ public void onError(Exception e) { String errorMsg = requestType + " error on DataNode: {id=" - + targetDataNode.getDataNodeId() + + targetNode.getDataNodeId() + ", internalEndPoint=" - + targetDataNode.getInternalEndPoint() + + targetNode.getInternalEndPoint() + "}" + e.getMessage(); LOGGER.warn(errorMsg); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/SubmitTestConnectionTaskRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/SubmitTestConnectionTaskRPCHandler.java new file mode 100644 index 000000000000..4abf0a0eca11 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/SubmitTestConnectionTaskRPCHandler.java @@ -0,0 +1,68 @@ +/* + * 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.client.async.handlers.rpc; + +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResp; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +public class SubmitTestConnectionTaskRPCHandler + extends DataNodeAsyncRequestRPCHandler { + + private static final Logger LOGGER = + LoggerFactory.getLogger(SubmitTestConnectionTaskRPCHandler.class); + + public SubmitTestConnectionTaskRPCHandler( + CnToDnRequestType requestType, + int requestId, + TDataNodeLocation targetDataNode, + Map dataNodeLocationMap, + Map responseMap, + CountDownLatch countDownLatch) { + super(requestType, requestId, targetDataNode, dataNodeLocationMap, responseMap, countDownLatch); + } + + @Override + public void onComplete(TTestConnectionResp resp) { + responseMap.put(requestId, resp); + nodeLocationMap.remove(requestId); + countDownLatch.countDown(); + } + + @Override + public void onError(Exception e) { + responseMap.put( + requestId, + new TTestConnectionResp() + .setStatus( + new TSStatus(TSStatusCode.CAN_NOT_CONNECT_DATANODE.getStatusCode()) + .setMessage(e.getMessage()))); + nodeLocationMap.remove(requestId); + countDownLatch.countDown(); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/SubmitTestConnectionTaskToConfigNodeRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/SubmitTestConnectionTaskToConfigNodeRPCHandler.java new file mode 100644 index 000000000000..d26984fd0a70 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/SubmitTestConnectionTaskToConfigNodeRPCHandler.java @@ -0,0 +1,74 @@ +/* + * 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.client.async.handlers.rpc; + +import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResp; +import org.apache.iotdb.confignode.client.CnToCnNodeRequestType; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +public class SubmitTestConnectionTaskToConfigNodeRPCHandler + extends ConfigNodeAsyncRequestRPCHandler { + + private static final Logger LOGGER = + LoggerFactory.getLogger(SubmitTestConnectionTaskToConfigNodeRPCHandler.class); + + public SubmitTestConnectionTaskToConfigNodeRPCHandler( + CnToCnNodeRequestType requestType, + int requestId, + TConfigNodeLocation targetConfigNode, + Map configNodeLocationMap, + Map responseMap, + CountDownLatch countDownLatch) { + super( + requestType, + requestId, + targetConfigNode, + configNodeLocationMap, + responseMap, + countDownLatch); + } + + @Override + public void onComplete(TTestConnectionResp resp) { + responseMap.put(requestId, resp); + nodeLocationMap.remove(requestId); + countDownLatch.countDown(); + } + + @Override + public void onError(Exception e) { + responseMap.put( + requestId, + new TTestConnectionResp() + .setStatus( + new TSStatus(TSStatusCode.CAN_NOT_CONNECT_CONFIGNODE.getStatusCode()) + .setMessage(e.getMessage()))); + nodeLocationMap.remove(requestId); + countDownLatch.countDown(); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/TransferLeaderRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/TransferLeaderRPCHandler.java index d5107e315cf1..352cc0694e2b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/TransferLeaderRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/TransferLeaderRPCHandler.java @@ -20,7 +20,7 @@ package org.apache.iotdb.confignode.client.async.handlers.rpc; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; -import org.apache.iotdb.confignode.client.DataNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeResp; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -31,12 +31,13 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; -public class TransferLeaderRPCHandler extends AbstractAsyncRPCHandler { +public class TransferLeaderRPCHandler + extends DataNodeAsyncRequestRPCHandler { private static final Logger LOGGER = LoggerFactory.getLogger(TransferLeaderRPCHandler.class); public TransferLeaderRPCHandler( - DataNodeRequestType requestType, + CnToDnRequestType requestType, int requestId, TDataNodeLocation targetDataNode, Map dataNodeLocationMap, @@ -52,7 +53,7 @@ public void onComplete(TRegionLeaderChangeResp response) { if (response.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { // Remove only if success - dataNodeLocationMap.remove(requestId); + nodeLocationMap.remove(requestId); LOGGER.info("Successfully {} on DataNode: {}", requestType, formattedTargetLocation); } else { LOGGER.error( diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/subscription/CheckSchemaRegionUsingTemplateRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/subscription/CheckSchemaRegionUsingTemplateRPCHandler.java index ad6951629921..14898dcdc6c9 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/subscription/CheckSchemaRegionUsingTemplateRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/subscription/CheckSchemaRegionUsingTemplateRPCHandler.java @@ -21,8 +21,8 @@ import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.handlers.rpc.AbstractAsyncRPCHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.handlers.rpc.DataNodeAsyncRequestRPCHandler; import org.apache.iotdb.mpp.rpc.thrift.TCheckSchemaRegionUsingTemplateResp; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -34,13 +34,13 @@ import java.util.concurrent.CountDownLatch; public class CheckSchemaRegionUsingTemplateRPCHandler - extends AbstractAsyncRPCHandler { + extends DataNodeAsyncRequestRPCHandler { private static final Logger LOGGER = LoggerFactory.getLogger(CheckSchemaRegionUsingTemplateRPCHandler.class); public CheckSchemaRegionUsingTemplateRPCHandler( - DataNodeRequestType requestType, + CnToDnRequestType requestType, int requestId, TDataNodeLocation targetDataNode, Map dataNodeLocationMap, @@ -54,20 +54,15 @@ public void onComplete(TCheckSchemaRegionUsingTemplateResp response) { TSStatus tsStatus = response.getStatus(); responseMap.put(requestId, response); if (tsStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - dataNodeLocationMap.remove(requestId); - LOGGER.info( - "Successfully check schema region using template on DataNode: {}", targetDataNode); + nodeLocationMap.remove(requestId); + LOGGER.info("Successfully check schema region using template on DataNode: {}", targetNode); } else if (tsStatus.getCode() == TSStatusCode.MULTIPLE_ERROR.getStatusCode()) { - dataNodeLocationMap.remove(requestId); + nodeLocationMap.remove(requestId); LOGGER.error( - "Failed to check schema region using template on DataNode {}, {}", - targetDataNode, - tsStatus); + "Failed to check schema region using template on DataNode {}, {}", targetNode, tsStatus); } else { LOGGER.error( - "Failed to check schema region using template on DataNode {}, {}", - targetDataNode, - tsStatus); + "Failed to check schema region using template on DataNode {}, {}", targetNode, tsStatus); } countDownLatch.countDown(); } @@ -76,9 +71,9 @@ public void onComplete(TCheckSchemaRegionUsingTemplateResp response) { public void onError(Exception e) { String errorMsg = "Count paths using template error on DataNode: {id=" - + targetDataNode.getDataNodeId() + + targetNode.getDataNodeId() + ", internalEndPoint=" - + targetDataNode.getInternalEndPoint() + + targetNode.getInternalEndPoint() + "}" + e.getMessage(); LOGGER.error(errorMsg); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/subscription/ConsumerGroupPushMetaRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/subscription/ConsumerGroupPushMetaRPCHandler.java index 52f5063101f2..ee3c11eeb427 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/subscription/ConsumerGroupPushMetaRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/subscription/ConsumerGroupPushMetaRPCHandler.java @@ -20,8 +20,8 @@ package org.apache.iotdb.confignode.client.async.handlers.rpc.subscription; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.handlers.rpc.AbstractAsyncRPCHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.handlers.rpc.DataNodeAsyncRequestRPCHandler; import org.apache.iotdb.mpp.rpc.thrift.TPushConsumerGroupMetaResp; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -33,12 +33,12 @@ import java.util.concurrent.CountDownLatch; public class ConsumerGroupPushMetaRPCHandler - extends AbstractAsyncRPCHandler { + extends DataNodeAsyncRequestRPCHandler { private static final Logger LOGGER = LoggerFactory.getLogger(ConsumerGroupPushMetaRPCHandler.class); public ConsumerGroupPushMetaRPCHandler( - DataNodeRequestType requestType, + CnToDnRequestType requestType, int requestId, TDataNodeLocation targetDataNode, Map dataNodeLocationMap, @@ -63,7 +63,7 @@ public void onComplete(TPushConsumerGroupMetaResp response) { } // Always remove to avoid retrying - dataNodeLocationMap.remove(requestId); + nodeLocationMap.remove(requestId); // Always CountDown countDownLatch.countDown(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/subscription/TopicPushMetaRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/subscription/TopicPushMetaRPCHandler.java index 39cb9cd23e97..cf8451feaaca 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/subscription/TopicPushMetaRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/subscription/TopicPushMetaRPCHandler.java @@ -20,8 +20,8 @@ package org.apache.iotdb.confignode.client.async.handlers.rpc.subscription; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.handlers.rpc.AbstractAsyncRPCHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.handlers.rpc.DataNodeAsyncRequestRPCHandler; import org.apache.iotdb.mpp.rpc.thrift.TPushTopicMetaResp; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -32,12 +32,12 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; -public class TopicPushMetaRPCHandler extends AbstractAsyncRPCHandler { +public class TopicPushMetaRPCHandler extends DataNodeAsyncRequestRPCHandler { private static final Logger LOGGER = LoggerFactory.getLogger(TopicPushMetaRPCHandler.class); public TopicPushMetaRPCHandler( - DataNodeRequestType requestType, + CnToDnRequestType requestType, int requestId, TDataNodeLocation targetDataNode, Map dataNodeLocationMap, @@ -62,7 +62,7 @@ public void onComplete(TPushTopicMetaResp response) { } // Always remove to avoid retrying - dataNodeLocationMap.remove(requestId); + nodeLocationMap.remove(requestId); // Always CountDown countDownLatch.countDown(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncConfigNodeClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncConfigNodeClientPool.java index ab85a9badcb9..302e587af053 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncConfigNodeClientPool.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncConfigNodeClientPool.java @@ -21,13 +21,14 @@ import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.common.rpc.thrift.TNodeLocations; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TSetConfigurationReq; import org.apache.iotdb.commons.client.ClientPoolFactory; import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.exception.ClientManagerException; import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient; -import org.apache.iotdb.confignode.client.ConfigNodeRequestType; +import org.apache.iotdb.confignode.client.CnToCnNodeRequestType; import org.apache.iotdb.confignode.rpc.thrift.TAddConsensusGroupReq; import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeRegisterReq; import org.apache.iotdb.rpc.RpcUtils; @@ -66,7 +67,7 @@ private void updateConfigNodeLeader(TSStatus status) { } public Object sendSyncRequestToConfigNodeWithRetry( - TEndPoint endPoint, Object req, ConfigNodeRequestType requestType) { + TEndPoint endPoint, Object req, CnToCnNodeRequestType requestType) { Throwable lastException = null; for (int retry = 0; retry < MAX_RETRY_NUM; retry++) { @@ -91,6 +92,10 @@ public Object sendSyncRequestToConfigNodeWithRetry( return client.stopConfigNode((TConfigNodeLocation) req); case SET_CONFIGURATION: return client.setConfiguration((TSetConfigurationReq) req); + case SUBMIT_TEST_CONNECTION_TASK: + return client.submitTestConnectionTask((TNodeLocations) req); + case TEST_CONNECTION: + return client.testConnectionEmptyRPC(); default: return RpcUtils.getStatus( TSStatusCode.EXECUTE_STATEMENT_ERROR, "Unknown request type: " + requestType); @@ -145,7 +150,6 @@ private void doRetryWait(int retryNum) { } } - // TODO: Is the ClientPool must be a singleton? private static class SyncConfigNodeClientPoolHolder { private static final SyncConfigNodeClientPool INSTANCE = new SyncConfigNodeClientPool(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java index 9628bc805094..84e609ffb38c 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java @@ -27,7 +27,7 @@ import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.exception.ClientManagerException; import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient; -import org.apache.iotdb.confignode.client.DataNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; import org.apache.iotdb.mpp.rpc.thrift.TCreateDataRegionReq; import org.apache.iotdb.mpp.rpc.thrift.TCreatePeerReq; import org.apache.iotdb.mpp.rpc.thrift.TCreateSchemaRegionReq; @@ -65,7 +65,7 @@ private SyncDataNodeClientPool() { } public Object sendSyncRequestToDataNodeWithRetry( - TEndPoint endPoint, Object req, DataNodeRequestType requestType) { + TEndPoint endPoint, Object req, CnToDnRequestType requestType) { Throwable lastException = new TException(); for (int retry = 0; retry < DEFAULT_RETRY_NUM; retry++) { try (SyncDataNodeInternalServiceClient client = clientManager.borrowClient(endPoint)) { @@ -84,7 +84,7 @@ public Object sendSyncRequestToDataNodeWithRetry( } public Object sendSyncRequestToDataNodeWithGivenRetry( - TEndPoint endPoint, Object req, DataNodeRequestType requestType, int retryNum) { + TEndPoint endPoint, Object req, CnToDnRequestType requestType, int retryNum) { Throwable lastException = new TException(); for (int retry = 0; retry < retryNum; retry++) { try (SyncDataNodeInternalServiceClient client = clientManager.borrowClient(endPoint)) { @@ -103,7 +103,7 @@ public Object sendSyncRequestToDataNodeWithGivenRetry( } private Object executeSyncRequest( - DataNodeRequestType requestType, SyncDataNodeInternalServiceClient client, Object req) + CnToDnRequestType requestType, SyncDataNodeInternalServiceClient client, Object req) throws TException { switch (requestType) { case INVALIDATE_PARTITION_CACHE: @@ -189,7 +189,6 @@ public TRegionLeaderChangeResp changeRegionLeader( return new TRegionLeaderChangeResp(status, -1L); } - // TODO: Is the ClientPool must be a singleton? private static class ClientPoolHolder { private static final SyncDataNodeClientPool INSTANCE = new SyncDataNodeClientPool(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java index 99017b4f10c8..603892e36811 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java @@ -19,6 +19,7 @@ package org.apache.iotdb.confignode.conf; +import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.client.property.ClientPoolProperty.DefaultProperty; import org.apache.iotdb.commons.conf.IoTDBConstant; @@ -1196,4 +1197,18 @@ public void setDataRegionRatisPeriodicSnapshotInterval( long dataRegionRatisPeriodicSnapshotInterval) { this.dataRegionRatisPeriodicSnapshotInterval = dataRegionRatisPeriodicSnapshotInterval; } + + public TConfigNodeLocation generateLocalConfigNodeLocationWithSpecifiedNodeId(int configNodeId) { + return new TConfigNodeLocation( + configNodeId, + new TEndPoint(getInternalAddress(), getInternalPort()), + new TEndPoint(getInternalAddress(), getConsensusPort())); + } + + public TConfigNodeLocation generateLocalConfigNodeLocation() { + return new TConfigNodeLocation( + getConfigNodeId(), + new TEndPoint(getInternalAddress(), getInternalPort()), + new TEndPoint(getInternalAddress(), getConsensusPort())); + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeRemoveCheck.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeRemoveCheck.java index bdbbdc43c7a9..9c4c480b02ce 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeRemoveCheck.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeRemoveCheck.java @@ -25,7 +25,7 @@ import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.exception.BadNodeUrlException; import org.apache.iotdb.commons.utils.NodeUrlUtils; -import org.apache.iotdb.confignode.client.ConfigNodeRequestType; +import org.apache.iotdb.confignode.client.CnToCnNodeRequestType; import org.apache.iotdb.confignode.client.sync.SyncConfigNodeClientPool; import org.apache.iotdb.rpc.TSStatusCode; @@ -112,7 +112,7 @@ public void removeConfigNode(TConfigNodeLocation removedNode) .sendSyncRequestToConfigNodeWithRetry( configNodeLocation.getInternalEndPoint(), removedNode, - ConfigNodeRequestType.REMOVE_CONFIG_NODE); + CnToCnNodeRequestType.REMOVE_CONFIG_NODE); if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { break; } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterManager.java index 9781f77c0244..808112cff4cd 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterManager.java @@ -19,14 +19,41 @@ package org.apache.iotdb.confignode.manager; +import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration; +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.common.rpc.thrift.TNodeLocations; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.common.rpc.thrift.TSender; +import org.apache.iotdb.common.rpc.thrift.TServiceProvider; +import org.apache.iotdb.common.rpc.thrift.TServiceType; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResp; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResult; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.commons.client.request.Utils; +import org.apache.iotdb.confignode.client.CnToCnNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToCnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.ConfigNodeAsyncRequestContext; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; +import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.apache.iotdb.confignode.consensus.request.write.confignode.UpdateClusterIdPlan; import org.apache.iotdb.confignode.persistence.ClusterInfo; import org.apache.iotdb.consensus.exception.ConsensusException; +import org.apache.iotdb.rpc.TSStatusCode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; public class ClusterManager { @@ -79,4 +106,198 @@ private void generateClusterId() { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); } } + + // TODO: Parallel test ConfigNode and DataNode + public TTestConnectionResp submitTestConnectionTaskToEveryNode() { + TTestConnectionResp resp = new TTestConnectionResp(); + resp.resultList = new ArrayList<>(); + resp.setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + TNodeLocations nodeLocations = new TNodeLocations(); + nodeLocations.setConfigNodeLocations(configManager.getNodeManager().getRegisteredConfigNodes()); + nodeLocations.setDataNodeLocations( + configManager.getNodeManager().getRegisteredDataNodes().stream() + .map(TDataNodeConfiguration::getLocation) + .collect(Collectors.toList())); + // For ConfigNode + Map configNodeLocationMap = + configManager.getNodeManager().getRegisteredConfigNodes().stream() + .collect(Collectors.toMap(TConfigNodeLocation::getConfigNodeId, location -> location)); + ConfigNodeAsyncRequestContext + configNodeAsyncRequestContext = + new ConfigNodeAsyncRequestContext<>( + CnToCnNodeRequestType.SUBMIT_TEST_CONNECTION_TASK, + nodeLocations, + configNodeLocationMap); + CnToCnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequest(configNodeAsyncRequestContext); + Map anotherConfigNodeLocationMap = + configManager.getNodeManager().getRegisteredConfigNodes().stream() + .collect(Collectors.toMap(TConfigNodeLocation::getConfigNodeId, location -> location)); + configNodeAsyncRequestContext + .getResponseMap() + .forEach( + (nodeId, configNodeResp) -> { + if (configNodeResp.isSetResultList()) { + resp.getResultList().addAll(configNodeResp.getResultList()); + } else { + resp.getResultList() + .addAll( + badConfigNodeConnectionResult( + configNodeResp.getStatus(), + anotherConfigNodeLocationMap.get(nodeId), + nodeLocations)); + } + }); + // For DataNode + Map dataNodeLocationMap = + configManager.getNodeManager().getRegisteredDataNodes().stream() + .map(TDataNodeConfiguration::getLocation) + .collect(Collectors.toMap(TDataNodeLocation::getDataNodeId, location -> location)); + DataNodeAsyncRequestContext dataNodeAsyncRequestContext = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.SUBMIT_TEST_CONNECTION_TASK, nodeLocations, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequest(dataNodeAsyncRequestContext); + Map anotherDataNodeLocationMap = + configManager.getNodeManager().getRegisteredDataNodes().stream() + .map(TDataNodeConfiguration::getLocation) + .collect(Collectors.toMap(TDataNodeLocation::getDataNodeId, location -> location)); + dataNodeAsyncRequestContext + .getResponseMap() + .forEach( + (nodeId, dataNodeResp) -> { + if (dataNodeResp.isSetResultList()) { + resp.getResultList().addAll(dataNodeResp.getResultList()); + } else { + resp.getResultList() + .addAll( + badDataNodeConnectionResult( + dataNodeResp.getStatus(), + anotherDataNodeLocationMap.get(nodeId), + nodeLocations)); + } + }); + return resp; + } + + public TTestConnectionResp doConnectionTest(TNodeLocations nodeLocations) { + return new TTestConnectionResp( + new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()), + Stream.of( + testAllConfigNodeConnection(nodeLocations.getConfigNodeLocations()), + testAllDataNodeConnection(nodeLocations.getDataNodeLocations())) + .flatMap(Collection::stream) + .collect(Collectors.toList())); + } + + private List testAllConfigNodeConnection( + List configNodeLocations) { + final TSender sender = + new TSender() + .setConfigNodeLocation( + ConfigNodeDescriptor.getInstance().getConf().generateLocalConfigNodeLocation()); + return Utils.testConnectionsImpl( + configNodeLocations, + sender, + TConfigNodeLocation::getConfigNodeId, + TConfigNodeLocation::getInternalEndPoint, + TServiceType.ConfigNodeInternalService, + CnToCnNodeRequestType.TEST_CONNECTION, + (AsyncRequestContext + handler) -> + CnToCnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(handler)); + } + + private List badConfigNodeConnectionResult( + TSStatus badStatus, TConfigNodeLocation sourceConfigNode, TNodeLocations nodeLocations) { + final TSender sender = new TSender().setConfigNodeLocation(sourceConfigNode); + return badNodeConnectionResult(badStatus, nodeLocations, sender); + } + + private List testAllDataNodeConnection( + List dataNodeLocations) { + final TSender sender = + new TSender() + .setConfigNodeLocation( + ConfigNodeDescriptor.getInstance().getConf().generateLocalConfigNodeLocation()); + return Utils.testConnectionsImpl( + dataNodeLocations, + sender, + TDataNodeLocation::getDataNodeId, + TDataNodeLocation::getInternalEndPoint, + TServiceType.DataNodeInternalService, + CnToDnRequestType.TEST_CONNECTION, + (AsyncRequestContext handler) -> + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(handler)); + } + + private List badDataNodeConnectionResult( + TSStatus badStatus, TDataNodeLocation sourceDataNode, TNodeLocations nodeLocations) { + final TSender sender = new TSender().setDataNodeLocation(sourceDataNode); + return badNodeConnectionResult(badStatus, nodeLocations, sender); + } + + private List badNodeConnectionResult( + TSStatus badStatus, TNodeLocations nodeLocations, TSender sender) { + final String errorMessage = + "ConfigNode leader cannot connect to the sender: " + badStatus.getMessage(); + List results = new ArrayList<>(); + nodeLocations + .getConfigNodeLocations() + .forEach( + location -> { + TEndPoint endPoint = location.getInternalEndPoint(); + TServiceProvider serviceProvider = + new TServiceProvider(endPoint, TServiceType.ConfigNodeInternalService); + TTestConnectionResult result = + new TTestConnectionResult().setServiceProvider(serviceProvider).setSender(sender); + result.setSuccess(false).setReason(errorMessage); + results.add(result); + }); + nodeLocations + .getDataNodeLocations() + .forEach( + location -> { + TEndPoint endPoint = location.getInternalEndPoint(); + TServiceProvider serviceProvider = + new TServiceProvider(endPoint, TServiceType.DataNodeInternalService); + TTestConnectionResult result = + new TTestConnectionResult().setServiceProvider(serviceProvider).setSender(sender); + result.setSuccess(false).setReason(errorMessage); + results.add(result); + }); + if (sender.isSetDataNodeLocation()) { + nodeLocations + .getDataNodeLocations() + .forEach( + location -> { + TEndPoint endPoint = location.getMPPDataExchangeEndPoint(); + TServiceProvider serviceProvider = + new TServiceProvider(endPoint, TServiceType.DataNodeMPPService); + TTestConnectionResult result = + new TTestConnectionResult() + .setServiceProvider(serviceProvider) + .setSender(sender); + result.setSuccess(false).setReason(errorMessage); + results.add(result); + }); + nodeLocations + .getDataNodeLocations() + .forEach( + location -> { + TEndPoint endPoint = location.getClientRpcEndPoint(); + TServiceProvider serviceProvider = + new TServiceProvider(endPoint, TServiceType.DataNodeExternalService); + TTestConnectionResult result = + new TTestConnectionResult() + .setServiceProvider(serviceProvider) + .setSender(sender); + result.setSuccess(false).setReason(errorMessage); + results.add(result); + }); + } + return results; + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterQuotaManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterQuotaManager.java index 0c00005fc1b3..b0150b349d80 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterQuotaManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterQuotaManager.java @@ -26,9 +26,9 @@ import org.apache.iotdb.common.rpc.thrift.TSpaceQuota; import org.apache.iotdb.common.rpc.thrift.TThrottleQuota; import org.apache.iotdb.commons.conf.IoTDBConstant; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.write.quota.SetSpaceQuotaPlan; import org.apache.iotdb.confignode.consensus.request.write.quota.SetThrottleQuotaPlan; import org.apache.iotdb.confignode.manager.partition.PartitionManager; @@ -87,9 +87,11 @@ public TSStatus setSpaceQuota(TSetSpaceQuotaReq req) { if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { Map dataNodeLocationMap = configManager.getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.SET_SPACE_QUOTA, req, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.SET_SPACE_QUOTA, req, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(clientHandler); return RpcUtils.squashResponseStatusList(clientHandler.getResponseList()); } return response; @@ -192,10 +194,11 @@ public TSStatus setThrottleQuota(TSetThrottleQuotaReq req) { if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { Map dataNodeLocationMap = configManager.getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.SET_THROTTLE_QUOTA, req, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.SET_THROTTLE_QUOTA, req, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(clientHandler); return RpcUtils.squashResponseStatusList(clientHandler.getResponseList()); } return response; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java index 0b4bb73102f9..b1a96d029af2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java @@ -24,9 +24,9 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathDeserializeUtil; import org.apache.iotdb.commons.trigger.TriggerInformation; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.read.trigger.GetTransferringTriggersPlan; import org.apache.iotdb.confignode.consensus.request.read.trigger.GetTriggerJarPlan; import org.apache.iotdb.confignode.consensus.request.read.trigger.GetTriggerLocationPlan; @@ -248,10 +248,10 @@ public List updateTriggerLocation( final TUpdateTriggerLocationReq request = new TUpdateTriggerLocationReq(triggerName, dataNodeLocation); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.UPDATE_TRIGGER_LOCATION, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.UPDATE_TRIGGER_LOCATION, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/UDFManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/UDFManager.java index a84eb8359ead..ad86879d8280 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/UDFManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/UDFManager.java @@ -23,9 +23,9 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.udf.UDFInformation; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.apache.iotdb.confignode.consensus.request.read.function.GetFunctionTablePlan; import org.apache.iotdb.confignode.consensus.request.read.function.GetUDFJarPlan; @@ -127,9 +127,10 @@ private List createFunctionOnDataNodes(UDFInformation udfInformation, configManager.getNodeManager().getRegisteredDataNodeLocations(); final TCreateFunctionInstanceReq req = new TCreateFunctionInstanceReq(udfInformation.serialize()).setJarFile(jarFile); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.CREATE_FUNCTION, req, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.CREATE_FUNCTION, req, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } @@ -160,9 +161,10 @@ private List dropFunctionOnDataNodes(String functionName) { final TDropFunctionInstanceReq request = new TDropFunctionInstanceReq(functionName, false); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.DROP_FUNCTION, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.DROP_FUNCTION, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java index 9ccd02f12811..6125edc1ef7b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java @@ -24,7 +24,7 @@ import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient; import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.cq.TimeoutPolicy; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; import org.apache.iotdb.confignode.consensus.request.write.cq.UpdateCQLastExecTimePlan; import org.apache.iotdb.confignode.manager.ConfigManager; import org.apache.iotdb.confignode.persistence.cq.CQInfo; @@ -188,7 +188,8 @@ public void run() { new TExecuteCQ(queryBody, startTime, endTime, everyInterval, zoneId, cqId, username); try { AsyncDataNodeInternalServiceClient client = - AsyncDataNodeClientPool.getInstance().getAsyncClient(targetDataNode.get()); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .getAsyncClient(targetDataNode.get()); client.executeCQ(executeCQReq, new AsyncExecuteCQCallback(startTime, endTime)); } catch (Exception t) { LOGGER.warn("Execute CQ {} failed", cqId, t); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RouteBalancer.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RouteBalancer.java index 74a376a9170b..36f85afcbef6 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RouteBalancer.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RouteBalancer.java @@ -25,9 +25,9 @@ import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.cluster.NodeStatus; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.conf.ConfigNodeConfig; import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.apache.iotdb.confignode.manager.IManager; @@ -167,8 +167,8 @@ private void balanceRegionLeader( // Transfer leader to the optimal distribution long currentTime = System.nanoTime(); AtomicInteger requestId = new AtomicInteger(0); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.CHANGE_REGION_LEADER); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>(CnToDnRequestType.CHANGE_REGION_LEADER); Map successTransferMap = new TreeMap<>(); optimalLeaderMap.forEach( (regionGroupId, newLeaderId) -> { @@ -217,7 +217,7 @@ private void balanceRegionLeader( new TRegionLeaderChangeReq(regionGroupId, newLeader); int requestIndex = requestId.getAndIncrement(); clientHandler.putRequest(requestIndex, regionLeaderChangeReq); - clientHandler.putDataNodeLocation(requestIndex, newLeader); + clientHandler.putNodeLocation(requestIndex, newLeader); } break; } @@ -225,7 +225,7 @@ private void balanceRegionLeader( }); if (requestId.get() > 0) { // Don't retry ChangeLeader request - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNode(clientHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequest(clientHandler); for (int i = 0; i < requestId.get(); i++) { if (clientHandler.getResponseMap().get(i).getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { @@ -307,12 +307,12 @@ private void broadcastLatestRegionPriorityMap() { long broadcastTime = System.currentTimeMillis(); Map tmpPriorityMap = getRegionPriorityMap(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.UPDATE_REGION_ROUTE_MAP, + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.UPDATE_REGION_ROUTE_MAP, new TRegionRouteReq(broadcastTime, tmpPriorityMap), dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); } private void recordRegionPriorityMap( diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java index c85f0fbaba5a..0877acd336c2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java @@ -35,10 +35,10 @@ import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.consensus.ConsensusGroupId; import org.apache.iotdb.commons.service.metric.MetricService; -import org.apache.iotdb.confignode.client.ConfigNodeRequestType; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToCnNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.client.sync.SyncConfigNodeClientPool; import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool; import org.apache.iotdb.confignode.conf.ConfigNodeConfig; @@ -705,27 +705,27 @@ private TSStatus transferLeader( public List merge() { Map dataNodeLocationMap = configManager.getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.MERGE, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>(CnToDnRequestType.MERGE, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } public List flush(TFlushReq req) { Map dataNodeLocationMap = configManager.getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.FLUSH, req, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>(CnToDnRequestType.FLUSH, req, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } public List clearCache() { Map dataNodeLocationMap = configManager.getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.CLEAR_CACHE, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>(CnToDnRequestType.CLEAR_CACHE, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } @@ -743,9 +743,11 @@ public List setConfiguration(TSetConfigurationReq req) { targetDataNodes.putAll(dataNodeLocationMap); } if (!targetDataNodes.isEmpty()) { - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.SET_CONFIGURATION, req, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.SET_CONFIGURATION, req, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(clientHandler); responseList.addAll(clientHandler.getResponseList()); } @@ -766,7 +768,7 @@ public List setConfiguration(TSetConfigurationReq req) { .sendSyncRequestToConfigNodeWithRetry( configNode.getInternalEndPoint(), new TSetConfigurationReq(req.getConfigs(), configNode.getConfigNodeId()), - ConfigNodeRequestType.SET_CONFIGURATION); + CnToCnNodeRequestType.SET_CONFIGURATION); } catch (Exception e) { status = RpcUtils.getStatus( @@ -780,27 +782,28 @@ public List setConfiguration(TSetConfigurationReq req) { public List startRpairData() { Map dataNodeLocationMap = configManager.getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.START_REPAIR_DATA, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>(CnToDnRequestType.START_REPAIR_DATA, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } public List stopRepairData() { Map dataNodeLocationMap = configManager.getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.STOP_REPAIR_DATA, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>(CnToDnRequestType.STOP_REPAIR_DATA, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } public List loadConfiguration() { Map dataNodeLocationMap = configManager.getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.LOAD_CONFIGURATION, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.LOAD_CONFIGURATION, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } @@ -817,7 +820,7 @@ public TShowConfigurationResp showConfiguration(int nodeId) { .sendSyncRequestToDataNodeWithRetry( dataNodeLocation.getInternalEndPoint(), null, - DataNodeRequestType.SHOW_CONFIGURATION); + CnToDnRequestType.SHOW_CONFIGURATION); } // other config node @@ -831,7 +834,7 @@ public TShowConfigurationResp showConfiguration(int nodeId) { .sendSyncRequestToConfigNodeWithRetry( registeredConfigNode.getInternalEndPoint(), nodeId, - ConfigNodeRequestType.SHOW_CONFIGURATION); + CnToCnNodeRequestType.SHOW_CONFIGURATION); return resp; } return resp; @@ -840,10 +843,10 @@ public TShowConfigurationResp showConfiguration(int nodeId) { public List setSystemStatus(String status) { Map dataNodeLocationMap = configManager.getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.SET_SYSTEM_STATUS, status, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.SET_SYSTEM_STATUS, status, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } @@ -853,7 +856,7 @@ public TSStatus setDataNodeStatus(TSetDataNodeStatusReq setDataNodeStatusReq) { .sendSyncRequestToDataNodeWithRetry( setDataNodeStatusReq.getTargetDataNode().getInternalEndPoint(), setDataNodeStatusReq.getStatus(), - DataNodeRequestType.SET_SYSTEM_STATUS); + CnToDnRequestType.SET_SYSTEM_STATUS); } /** @@ -874,9 +877,10 @@ public TSStatus killQuery(String queryId, int dataNodeId) { private TSStatus killAllQueries() { Map dataNodeLocationMap = configManager.getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.KILL_QUERY_INSTANCE, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.KILL_QUERY_INSTANCE, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return RpcUtils.squashResponseStatusList(clientHandler.getResponseList()); } @@ -891,7 +895,7 @@ private TSStatus killSpecificQuery(String queryId, TDataNodeLocation dataNodeLoc .sendSyncRequestToDataNodeWithRetry( dataNodeLocation.getInternalEndPoint(), queryId, - DataNodeRequestType.KILL_QUERY_INSTANCE); + CnToDnRequestType.KILL_QUERY_INSTANCE); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionManager.java index b16314ff5ad3..c7b0c4bc19b9 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionManager.java @@ -34,9 +34,9 @@ import org.apache.iotdb.commons.partition.SchemaPartitionTable; import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor; import org.apache.iotdb.commons.path.PartialPath; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.conf.ConfigNodeConfig; import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; @@ -1255,10 +1255,10 @@ public void maintainRegionReplicas() { switch (selectedRegionMaintainTask.get(0).getRegionId().getType()) { case SchemaRegion: // create SchemaRegion - AsyncClientHandler + DataNodeAsyncRequestContext createSchemaRegionHandler = - new AsyncClientHandler<>( - DataNodeRequestType.CREATE_SCHEMA_REGION); + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.CREATE_SCHEMA_REGION); for (RegionMaintainTask regionMaintainTask : selectedRegionMaintainTask) { RegionCreateTask schemaRegionCreateTask = (RegionCreateTask) regionMaintainTask; @@ -1271,13 +1271,13 @@ public void maintainRegionReplicas() { new TCreateSchemaRegionReq( schemaRegionCreateTask.getRegionReplicaSet(), schemaRegionCreateTask.getStorageGroup())); - createSchemaRegionHandler.putDataNodeLocation( + createSchemaRegionHandler.putNodeLocation( schemaRegionCreateTask.getRegionId().getId(), schemaRegionCreateTask.getTargetDataNode()); } - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetry(createSchemaRegionHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(createSchemaRegionHandler); for (Map.Entry entry : createSchemaRegionHandler.getResponseMap().entrySet()) { @@ -1291,9 +1291,10 @@ public void maintainRegionReplicas() { break; case DataRegion: // Create DataRegion - AsyncClientHandler + DataNodeAsyncRequestContext createDataRegionHandler = - new AsyncClientHandler<>(DataNodeRequestType.CREATE_DATA_REGION); + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.CREATE_DATA_REGION); for (RegionMaintainTask regionMaintainTask : selectedRegionMaintainTask) { RegionCreateTask dataRegionCreateTask = (RegionCreateTask) regionMaintainTask; @@ -1306,13 +1307,13 @@ public void maintainRegionReplicas() { new TCreateDataRegionReq( dataRegionCreateTask.getRegionReplicaSet(), dataRegionCreateTask.getStorageGroup())); - createDataRegionHandler.putDataNodeLocation( + createDataRegionHandler.putNodeLocation( dataRegionCreateTask.getRegionId().getId(), dataRegionCreateTask.getTargetDataNode()); } - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetry(createDataRegionHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(createDataRegionHandler); for (Map.Entry entry : createDataRegionHandler.getResponseMap().entrySet()) { @@ -1328,8 +1329,8 @@ public void maintainRegionReplicas() { break; case DELETE: // delete region - AsyncClientHandler deleteRegionHandler = - new AsyncClientHandler<>(DataNodeRequestType.DELETE_REGION); + DataNodeAsyncRequestContext deleteRegionHandler = + new DataNodeAsyncRequestContext<>(CnToDnRequestType.DELETE_REGION); Map regionIdMap = new HashMap<>(); for (RegionMaintainTask regionMaintainTask : selectedRegionMaintainTask) { RegionDeleteTask regionDeleteTask = (RegionDeleteTask) regionMaintainTask; @@ -1339,7 +1340,7 @@ public void maintainRegionReplicas() { regionDeleteTask.getTargetDataNode()); deleteRegionHandler.putRequest( regionDeleteTask.getRegionId().getId(), regionDeleteTask.getRegionId()); - deleteRegionHandler.putDataNodeLocation( + deleteRegionHandler.putNodeLocation( regionDeleteTask.getRegionId().getId(), regionDeleteTask.getTargetDataNode()); regionIdMap.put( @@ -1347,8 +1348,8 @@ public void maintainRegionReplicas() { } long startTime = System.currentTimeMillis(); - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetry(deleteRegionHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(deleteRegionHandler); LOGGER.info( "Deleting regions costs {}ms", (System.currentTimeMillis() - startTime)); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatScheduler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatScheduler.java index ad691ea85657..b21fbc815f98 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatScheduler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatScheduler.java @@ -24,9 +24,9 @@ import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil; import org.apache.iotdb.commons.pipe.config.PipeConfig; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.apache.iotdb.confignode.manager.ConfigManager; import org.apache.iotdb.confignode.manager.pipe.agent.PipeConfigNodeAgent; @@ -94,10 +94,11 @@ private synchronized void heartbeat() { final TPipeHeartbeatReq request = new TPipeHeartbeatReq(System.currentTimeMillis()); LOGGER.info("Collecting pipe heartbeat {} from data nodes", request.heartbeatId); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.PIPE_HEARTBEAT, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetryAndTimeoutInMs( + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.PIPE_HEARTBEAT, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestToNodeWithRetryAndTimeoutInMs( clientHandler, PipeConfig.getInstance().getPipeHeartbeatIntervalSecondsForCollectingPipeMeta() * 1000L diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java index 181c132032bb..fd978ceb8335 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java @@ -32,9 +32,9 @@ import org.apache.iotdb.commons.utils.CommonDateTimeUtils; import org.apache.iotdb.commons.utils.PathUtils; import org.apache.iotdb.commons.utils.StatusUtils; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.conf.ConfigNodeConfig; import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.apache.iotdb.confignode.consensus.request.read.database.CountDatabasePlan; @@ -1039,10 +1039,10 @@ public synchronized TSStatus extendSchemaTemplate( Map dataNodeLocationMap = configManager.getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.UPDATE_TEMPLATE, updateTemplateReq, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.UPDATE_TEMPLATE, updateTemplateReq, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); Map statusMap = clientHandler.getResponseMap(); for (Map.Entry entry : statusMap.entrySet()) { if (entry.getValue().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java index 00d9569c6593..c2040a05328a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java @@ -32,10 +32,10 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.plugin.meta.PipePluginMeta; import org.apache.iotdb.commons.trigger.TriggerInformation; -import org.apache.iotdb.confignode.client.ConfigNodeRequestType; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToCnNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.client.sync.SyncConfigNodeClientPool; import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool; import org.apache.iotdb.confignode.consensus.request.write.confignode.RemoveConfigNodePlan; @@ -186,7 +186,7 @@ public boolean invalidateCache(String storageGroupName) throws IOException, TExc .sendSyncRequestToDataNodeWithRetry( dataNodeConfiguration.getLocation().getInternalEndPoint(), invalidateCacheReq, - DataNodeRequestType.INVALIDATE_PARTITION_CACHE); + CnToDnRequestType.INVALIDATE_PARTITION_CACHE); final TSStatus invalidateSchemaStatus = (TSStatus) @@ -194,7 +194,7 @@ public boolean invalidateCache(String storageGroupName) throws IOException, TExc .sendSyncRequestToDataNodeWithRetry( dataNodeConfiguration.getLocation().getInternalEndPoint(), invalidateCacheReq, - DataNodeRequestType.INVALIDATE_SCHEMA_CACHE); + CnToDnRequestType.INVALIDATE_SCHEMA_CACHE); if (!verifySucceed(invalidatePartitionStatus, invalidateSchemaStatus)) { LOG.error( @@ -251,7 +251,7 @@ public void addConsensusGroup(TConfigNodeLocation tConfigNodeLocation) .sendSyncRequestToConfigNodeWithRetry( tConfigNodeLocation.getInternalEndPoint(), new TAddConsensusGroupReq(configNodeLocations), - ConfigNodeRequestType.ADD_CONSENSUS_GROUP); + CnToCnNodeRequestType.ADD_CONSENSUS_GROUP); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { throw new AddConsensusGroupException(tConfigNodeLocation); } @@ -315,7 +315,7 @@ public void deleteConfigNodePeer(TConfigNodeLocation removedConfigNode) .sendSyncRequestToConfigNodeWithRetry( removedConfigNode.getInternalEndPoint(), removedConfigNode, - ConfigNodeRequestType.DELETE_CONFIG_NODE_PEER); + CnToCnNodeRequestType.DELETE_CONFIG_NODE_PEER); if (tsStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { throw new ProcedureException(tsStatus.getMessage()); } @@ -334,7 +334,7 @@ public void stopConfigNode(TConfigNodeLocation tConfigNodeLocation) throws Proce .sendSyncRequestToConfigNodeWithRetry( tConfigNodeLocation.getInternalEndPoint(), tConfigNodeLocation, - ConfigNodeRequestType.STOP_CONFIG_NODE); + CnToCnNodeRequestType.STOP_CONFIG_NODE); if (tsStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { throw new ProcedureException(tsStatus.getMessage()); @@ -364,7 +364,7 @@ public void notifyRegisterSuccess(TConfigNodeLocation configNodeLocation) { .sendSyncRequestToConfigNodeWithRetry( configNodeLocation.getInternalEndPoint(), null, - ConfigNodeRequestType.NOTIFY_REGISTER_SUCCESS); + CnToCnNodeRequestType.NOTIFY_REGISTER_SUCCESS); } /** @@ -389,14 +389,14 @@ public void markDataNodeAsRemovingAndBroadcast(TDataNodeLocation dataNodeLocatio .sendSyncRequestToDataNodeWithGivenRetry( dataNodeLocation.getInternalEndPoint(), NodeStatus.Removing.getStatus(), - DataNodeRequestType.SET_SYSTEM_STATUS, + CnToDnRequestType.SET_SYSTEM_STATUS, 1); } else { SyncDataNodeClientPool.getInstance() .sendSyncRequestToDataNodeWithRetry( dataNodeLocation.getInternalEndPoint(), NodeStatus.Removing.getStatus(), - DataNodeRequestType.SET_SYSTEM_STATUS); + CnToDnRequestType.SET_SYSTEM_STATUS); } long currentTime = System.nanoTime(); @@ -430,7 +430,7 @@ public Map doRegionCreation( TConsensusGroupType consensusGroupType, CreateRegionGroupsPlan createRegionGroupsPlan) { // Prepare clientHandler - AsyncClientHandler clientHandler; + DataNodeAsyncRequestContext clientHandler; switch (consensusGroupType) { case SchemaRegion: clientHandler = getCreateSchemaRegionClientHandler(createRegionGroupsPlan); @@ -445,7 +445,7 @@ public Map doRegionCreation( } // Send CreateRegion requests to DataNodes - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); // Filter RegionGroups that weren't created successfully int requestId = 0; @@ -469,10 +469,10 @@ public Map doRegionCreation( return failedRegions; } - private AsyncClientHandler getCreateSchemaRegionClientHandler( - CreateRegionGroupsPlan createRegionGroupsPlan) { - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.CREATE_SCHEMA_REGION); + private DataNodeAsyncRequestContext + getCreateSchemaRegionClientHandler(CreateRegionGroupsPlan createRegionGroupsPlan) { + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>(CnToDnRequestType.CREATE_SCHEMA_REGION); int requestId = 0; for (Map.Entry> sgRegionsEntry : @@ -483,7 +483,7 @@ private AsyncClientHandler getCreateSchemaRegi for (TDataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeLocations()) { clientHandler.putRequest( requestId, genCreateSchemaRegionReq(storageGroup, regionReplicaSet)); - clientHandler.putDataNodeLocation(requestId, dataNodeLocation); + clientHandler.putNodeLocation(requestId, dataNodeLocation); requestId += 1; } } @@ -492,10 +492,10 @@ private AsyncClientHandler getCreateSchemaRegi return clientHandler; } - private AsyncClientHandler getCreateDataRegionClientHandler( - CreateRegionGroupsPlan createRegionGroupsPlan) { - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.CREATE_DATA_REGION); + private DataNodeAsyncRequestContext + getCreateDataRegionClientHandler(CreateRegionGroupsPlan createRegionGroupsPlan) { + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>(CnToDnRequestType.CREATE_DATA_REGION); int requestId = 0; for (Map.Entry> sgRegionsEntry : @@ -506,7 +506,7 @@ private AsyncClientHandler getCreateDataRegionCl for (TDataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeLocations()) { clientHandler.putRequest( requestId, genCreateDataRegionReq(storageGroup, regionReplicaSet)); - clientHandler.putDataNodeLocation(requestId, dataNodeLocation); + clientHandler.putNodeLocation(requestId, dataNodeLocation); requestId += 1; } } @@ -588,10 +588,10 @@ public List createTriggerOnDataNodes( request.setJarFile(ByteBuffer.wrap(jarFile.getValues())); } - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.CREATE_TRIGGER_INSTANCE, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.CREATE_TRIGGER_INSTANCE, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } @@ -602,10 +602,10 @@ public List dropTriggerOnDataNodes(String triggerName, boolean needToD final TDropTriggerInstanceReq request = new TDropTriggerInstanceReq(triggerName, needToDeleteJarFile); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.DROP_TRIGGER_INSTANCE, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.DROP_TRIGGER_INSTANCE, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } @@ -615,10 +615,10 @@ public List activeTriggerOnDataNodes(String triggerName) { nodeManager.getRegisteredDataNodeLocations(); final TActiveTriggerInstanceReq request = new TActiveTriggerInstanceReq(triggerName); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.ACTIVE_TRIGGER_INSTANCE, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.ACTIVE_TRIGGER_INSTANCE, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } @@ -628,10 +628,10 @@ public List inactiveTriggerOnDataNodes(String triggerName) { nodeManager.getRegisteredDataNodeLocations(); final TInactiveTriggerInstanceReq request = new TInactiveTriggerInstanceReq(triggerName); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.INACTIVE_TRIGGER_INSTANCE, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.INACTIVE_TRIGGER_INSTANCE, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } @@ -642,10 +642,10 @@ public List createPipePluginOnDataNodes(PipePluginMeta pipePluginMeta, final TCreatePipePluginInstanceReq request = new TCreatePipePluginInstanceReq(pipePluginMeta.serialize(), ByteBuffer.wrap(jarFile)); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.CREATE_PIPE_PLUGIN, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.CREATE_PIPE_PLUGIN, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } @@ -656,10 +656,10 @@ public List dropPipePluginOnDataNodes( final TDropPipePluginInstanceReq request = new TDropPipePluginInstanceReq(pipePluginName, needToDeleteJarFile); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.DROP_PIPE_PLUGIN, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.DROP_PIPE_PLUGIN, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList(); } @@ -669,11 +669,11 @@ public Map pushAllPipeMetaToDataNodes( configManager.getNodeManager().getRegisteredDataNodeLocations(); final TPushPipeMetaReq request = new TPushPipeMetaReq().setPipeMetas(pipeMetaBinaryList); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.PIPE_PUSH_ALL_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetryAndTimeoutInMs( + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.PIPE_PUSH_ALL_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestToNodeWithRetryAndTimeoutInMs( clientHandler, PipeConfig.getInstance().getPipeMetaSyncerSyncIntervalMinutes() * 60 * 1000 * 2 / 3); return clientHandler.getResponseMap(); @@ -684,11 +684,11 @@ public Map pushSinglePipeMetaToDataNodes(ByteBuffer configManager.getNodeManager().getRegisteredDataNodeLocations(); final TPushSinglePipeMetaReq request = new TPushSinglePipeMetaReq().setPipeMeta(pipeMetaBinary); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.PIPE_PUSH_SINGLE_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetryAndTimeoutInMs( + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.PIPE_PUSH_SINGLE_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestToNodeWithRetryAndTimeoutInMs( clientHandler, PipeConfig.getInstance().getPipeMetaSyncerSyncIntervalMinutes() * 60 * 1000 * 2 / 3); return clientHandler.getResponseMap(); @@ -700,11 +700,11 @@ public Map dropSinglePipeOnDataNodes(String pipeName final TPushSinglePipeMetaReq request = new TPushSinglePipeMetaReq().setPipeNameToDrop(pipeNameToDrop); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.PIPE_PUSH_SINGLE_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetryAndTimeoutInMs( + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.PIPE_PUSH_SINGLE_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestToNodeWithRetryAndTimeoutInMs( clientHandler, PipeConfig.getInstance().getPipeMetaSyncerSyncIntervalMinutes() * 60 * 1000 * 2 / 3); return clientHandler.getResponseMap(); @@ -717,11 +717,11 @@ public Map pushMultiPipeMetaToDataNodes( final TPushMultiPipeMetaReq request = new TPushMultiPipeMetaReq().setPipeMetas(pipeMetaBinaryList); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.PIPE_PUSH_MULTI_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetryAndTimeoutInMs( + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.PIPE_PUSH_MULTI_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestToNodeWithRetryAndTimeoutInMs( clientHandler, PipeConfig.getInstance().getPipeMetaSyncerSyncIntervalMinutes() * 60 * 1000 * 2 / 3); return clientHandler.getResponseMap(); @@ -733,11 +733,11 @@ public Map dropMultiPipeOnDataNodes(List pip final TPushMultiPipeMetaReq request = new TPushMultiPipeMetaReq().setPipeNamesToDrop(pipeNamesToDrop); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.PIPE_PUSH_MULTI_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetryAndTimeoutInMs( + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.PIPE_PUSH_MULTI_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestToNodeWithRetryAndTimeoutInMs( clientHandler, PipeConfig.getInstance().getPipeMetaSyncerSyncIntervalMinutes() * 60 * 1000 * 2 / 3); return clientHandler.getResponseMap(); @@ -749,11 +749,11 @@ public Map pushAllTopicMetaToDataNodes( configManager.getNodeManager().getRegisteredDataNodeLocations(); final TPushTopicMetaReq request = new TPushTopicMetaReq().setTopicMetas(topicMetaBinaryList); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.TOPIC_PUSH_ALL_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetryAndTimeoutInMs( + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.TOPIC_PUSH_ALL_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestToNodeWithRetryAndTimeoutInMs( clientHandler, PipeConfig.getInstance().getPipeMetaSyncerSyncIntervalMinutes() * 60 * 1000 * 2 / 3); return clientHandler.getResponseMap(); @@ -764,10 +764,10 @@ public List pushSingleTopicOnDataNode(ByteBuffer topicMeta) { configManager.getNodeManager().getRegisteredDataNodeLocations(); final TPushSingleTopicMetaReq request = new TPushSingleTopicMetaReq().setTopicMeta(topicMeta); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.TOPIC_PUSH_SINGLE_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.TOPIC_PUSH_SINGLE_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList().stream() .map(TPushTopicMetaResp::getStatus) .collect(Collectors.toList()); @@ -779,10 +779,10 @@ public List dropSingleTopicOnDataNode(String topicNameToDrop) { final TPushSingleTopicMetaReq request = new TPushSingleTopicMetaReq().setTopicNameToDrop(topicNameToDrop); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.TOPIC_PUSH_SINGLE_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.TOPIC_PUSH_SINGLE_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList().stream() .map(TPushTopicMetaResp::getStatus) .collect(Collectors.toList()); @@ -795,11 +795,11 @@ public Map pushMultiTopicMetaToDataNodes( final TPushMultiTopicMetaReq request = new TPushMultiTopicMetaReq().setTopicMetas(topicMetaBinaryList); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.TOPIC_PUSH_MULTI_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetryAndTimeoutInMs( + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.TOPIC_PUSH_MULTI_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestToNodeWithRetryAndTimeoutInMs( clientHandler, PipeConfig.getInstance().getPipeMetaSyncerSyncIntervalMinutes() * 60 * 1000 * 2 / 3); return clientHandler.getResponseMap(); @@ -811,11 +811,11 @@ public Map dropMultiTopicOnDataNodes(List t final TPushMultiTopicMetaReq request = new TPushMultiTopicMetaReq().setTopicNamesToDrop(topicNamesToDrop); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.TOPIC_PUSH_MULTI_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetryAndTimeoutInMs( + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.TOPIC_PUSH_MULTI_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestToNodeWithRetryAndTimeoutInMs( clientHandler, PipeConfig.getInstance().getPipeMetaSyncerSyncIntervalMinutes() * 60 * 1000 * 2 / 3); return clientHandler.getResponseMap(); @@ -828,11 +828,12 @@ public Map pushAllConsumerGroupMetaToDataNo final TPushConsumerGroupMetaReq request = new TPushConsumerGroupMetaReq().setConsumerGroupMetas(consumerGroupMetaBinaryList); - final AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.CONSUMER_GROUP_PUSH_ALL_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetryAndTimeoutInMs( + final DataNodeAsyncRequestContext + clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.CONSUMER_GROUP_PUSH_ALL_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestToNodeWithRetryAndTimeoutInMs( clientHandler, PipeConfig.getInstance().getPipeMetaSyncerSyncIntervalMinutes() * 60 * 1000 * 2 / 3); return clientHandler.getResponseMap(); @@ -844,11 +845,11 @@ public List pushSingleConsumerGroupOnDataNode(ByteBuffer consumerGroup final TPushSingleConsumerGroupMetaReq request = new TPushSingleConsumerGroupMetaReq().setConsumerGroupMeta(consumerGroupMeta); - final AsyncClientHandler + final DataNodeAsyncRequestContext clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.CONSUMER_GROUP_PUSH_SINGLE_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.CONSUMER_GROUP_PUSH_SINGLE_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList().stream() .map(TPushConsumerGroupMetaResp::getStatus) .collect(Collectors.toList()); @@ -860,11 +861,11 @@ public List dropSingleConsumerGroupOnDataNode(String consumerGroupName final TPushSingleConsumerGroupMetaReq request = new TPushSingleConsumerGroupMetaReq().setConsumerGroupNameToDrop(consumerGroupNameToDrop); - final AsyncClientHandler + final DataNodeAsyncRequestContext clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.CONSUMER_GROUP_PUSH_SINGLE_META, request, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.CONSUMER_GROUP_PUSH_SINGLE_META, request, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseList().stream() .map(TPushConsumerGroupMetaResp::getStatus) .collect(Collectors.toList()); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/RegionMaintainHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/RegionMaintainHandler.java index 4f82d5c502fb..9bb5b860eb9a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/RegionMaintainHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/RegionMaintainHandler.java @@ -35,9 +35,9 @@ import org.apache.iotdb.commons.service.metric.MetricService; import org.apache.iotdb.commons.utils.CommonDateTimeUtils; import org.apache.iotdb.commons.utils.NodeUrlUtils; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool; import org.apache.iotdb.confignode.conf.ConfigNodeConfig; import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; @@ -145,7 +145,7 @@ public void broadcastDisableDataNode(TDataNodeLocation disabledDataNode) { .sendSyncRequestToDataNodeWithRetry( node.getLocation().getInternalEndPoint(), disableReq, - DataNodeRequestType.DISABLE_DATA_NODE); + CnToDnRequestType.DISABLE_DATA_NODE); if (!isSucceed(status)) { LOGGER.error( "{}, BroadcastDisableDataNode meets error, disabledDataNode: {}, error: {}", @@ -230,7 +230,7 @@ public TSStatus createNewRegionPeer(TConsensusGroupId regionId, TDataNodeLocatio .sendSyncRequestToDataNodeWithRetry( destDataNode.getInternalEndPoint(), req, - DataNodeRequestType.CREATE_NEW_REGION_PEER); + CnToDnRequestType.CREATE_NEW_REGION_PEER); if (isSucceed(status)) { LOGGER.info( @@ -276,7 +276,7 @@ public TSStatus submitAddRegionPeerTask( .sendSyncRequestToDataNodeWithRetry( coordinator.getInternalEndPoint(), maintainPeerReq, - DataNodeRequestType.ADD_REGION_PEER); + CnToDnRequestType.ADD_REGION_PEER); LOGGER.info( "{}, Send action addRegionPeer finished, regionId: {}, rpcDataNode: {}, destDataNode: {}, status: {}", REGION_MIGRATE_PROCESS, @@ -313,7 +313,7 @@ public TSStatus submitRemoveRegionPeerTask( .sendSyncRequestToDataNodeWithRetry( coordinator.getInternalEndPoint(), maintainPeerReq, - DataNodeRequestType.REMOVE_REGION_PEER); + CnToDnRequestType.REMOVE_REGION_PEER); LOGGER.info( "{}, Send action removeRegionPeer finished, regionId: {}, rpcDataNode: {}", REGION_MIGRATE_PROCESS, @@ -347,14 +347,14 @@ public TSStatus submitDeleteOldRegionPeerTask( .sendSyncRequestToDataNodeWithGivenRetry( originalDataNode.getInternalEndPoint(), maintainPeerReq, - DataNodeRequestType.DELETE_OLD_REGION_PEER, + CnToDnRequestType.DELETE_OLD_REGION_PEER, 1) : (TSStatus) SyncDataNodeClientPool.getInstance() .sendSyncRequestToDataNodeWithRetry( originalDataNode.getInternalEndPoint(), maintainPeerReq, - DataNodeRequestType.DELETE_OLD_REGION_PEER); + CnToDnRequestType.DELETE_OLD_REGION_PEER); LOGGER.info( "{}, Send action deleteOldRegionPeer finished, regionId: {}, dataNodeId: {}", REGION_MIGRATE_PROCESS, @@ -367,12 +367,12 @@ public Map resetPeerList( TConsensusGroupId regionId, List correctDataNodeLocations, Map dataNodeLocationMap) { - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.RESET_PEER_LIST, + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.RESET_PEER_LIST, new TResetPeerListReq(regionId, correctDataNodeLocations), dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); return clientHandler.getResponseMap(); } @@ -508,10 +508,7 @@ public void stopDataNode(TDataNodeLocation dataNode) { (TSStatus) SyncDataNodeClientPool.getInstance() .sendSyncRequestToDataNodeWithGivenRetry( - dataNode.getInternalEndPoint(), - dataNode, - DataNodeRequestType.STOP_DATA_NODE, - 2); + dataNode.getInternalEndPoint(), dataNode, CnToDnRequestType.STOP_DATA_NODE, 2); configManager.getLoadManager().removeNodeCache(dataNode.getDataNodeId()); LOGGER.info( "{}, Stop Data Node result: {}, stoppedDataNode: {}", diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterLogicalViewProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterLogicalViewProcedure.java index 66677129dab6..8c6eed065534 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterLogicalViewProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterLogicalViewProcedure.java @@ -30,9 +30,9 @@ import org.apache.iotdb.commons.path.PathDeserializeUtil; import org.apache.iotdb.commons.path.PathPatternTree; import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.exception.ProcedureSuspendedException; @@ -120,12 +120,12 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, AlterLogicalViewStat private void invalidateCache(ConfigNodeProcedureEnv env) { Map dataNodeLocationMap = env.getConfigManager().getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.INVALIDATE_MATCHED_SCHEMA_CACHE, + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.INVALIDATE_MATCHED_SCHEMA_CACHE, new TInvalidateMatchedSchemaCacheReq(patternTreeBytes), dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); Map statusMap = clientHandler.getResponseMap(); for (TSStatus status : statusMap.values()) { // all dataNodes must clear the related schemaengine cache @@ -155,7 +155,7 @@ private void alterLogicalView(ConfigNodeProcedureEnv env) throws ProcedureExcept "Alter view", env, targetSchemaRegionGroup, - DataNodeRequestType.ALTER_VIEW, + CnToDnRequestType.ALTER_VIEW, (dataNodeLocation, consensusGroupIdList) -> { TAlterViewReq req = new TAlterViewReq().setIsGeneratedByPipe(isGeneratedByPipe); req.setSchemaRegionIdList(consensusGroupIdList); @@ -323,7 +323,7 @@ private class AlterLogicalViewRegionTaskExecutor String taskName, ConfigNodeProcedureEnv env, Map targetSchemaRegionGroup, - DataNodeRequestType dataNodeRequestType, + CnToDnRequestType dataNodeRequestType, BiFunction, Q> dataNodeRequestGenerator) { super(env, targetSchemaRegionGroup, false, dataNodeRequestType, dataNodeRequestGenerator); this.taskName = taskName; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeRegionTaskExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeRegionTaskExecutor.java index f5071e95327d..61def5836ce2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeRegionTaskExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DataNodeRegionTaskExecutor.java @@ -22,9 +22,9 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.manager.ConfigManager; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; @@ -45,7 +45,7 @@ public abstract class DataNodeRegionTaskExecutor { protected final Map targetSchemaRegionGroup; protected final boolean executeOnAllReplicaset; - protected final DataNodeRequestType dataNodeRequestType; + protected final CnToDnRequestType dataNodeRequestType; protected final BiFunction, Q> dataNodeRequestGenerator; @@ -55,7 +55,7 @@ protected DataNodeRegionTaskExecutor( ConfigManager configManager, Map targetSchemaRegionGroup, boolean executeOnAllReplicaset, - DataNodeRequestType dataNodeRequestType, + CnToDnRequestType dataNodeRequestType, BiFunction, Q> dataNodeRequestGenerator) { this.configManager = configManager; this.targetSchemaRegionGroup = targetSchemaRegionGroup; @@ -68,7 +68,7 @@ protected DataNodeRegionTaskExecutor( ConfigNodeProcedureEnv env, Map targetSchemaRegionGroup, boolean executeOnAllReplicaset, - DataNodeRequestType dataNodeRequestType, + CnToDnRequestType dataNodeRequestType, BiFunction, Q> dataNodeRequestGenerator) { this.configManager = env.getConfigManager(); this.targetSchemaRegionGroup = targetSchemaRegionGroup; @@ -86,8 +86,10 @@ void execute() { : getLeaderDataNodeRegionGroupMap( configManager.getLoadManager().getRegionLeaderMap(), targetSchemaRegionGroup); while (!dataNodeConsensusGroupIdMap.isEmpty()) { - AsyncClientHandler clientHandler = prepareRequestHandler(dataNodeConsensusGroupIdMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + prepareRequestHandler(dataNodeConsensusGroupIdMap); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(clientHandler); Map> currentFailedDataNodeMap = checkDataNodeExecutionResult(clientHandler.getResponseMap(), dataNodeConsensusGroupIdMap); @@ -112,12 +114,13 @@ void execute() { } } - private AsyncClientHandler prepareRequestHandler( + private DataNodeAsyncRequestContext prepareRequestHandler( Map> dataNodeConsensusGroupIdMap) { - AsyncClientHandler clientHandler = new AsyncClientHandler<>(dataNodeRequestType); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>(dataNodeRequestType); for (Map.Entry> entry : dataNodeConsensusGroupIdMap.entrySet()) { - clientHandler.putDataNodeLocation(entry.getKey().getDataNodeId(), entry.getKey()); + clientHandler.putNodeLocation(entry.getKey().getDataNodeId(), entry.getKey()); clientHandler.putRequest( entry.getKey().getDataNodeId(), dataNodeRequestGenerator.apply(entry.getKey(), entry.getValue())); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java index 194e23d14886..f75814ef0f96 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java @@ -28,9 +28,9 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathDeserializeUtil; import org.apache.iotdb.commons.path.PathPatternTree; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeactivateTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; @@ -152,7 +152,7 @@ private long constructBlackList(ConfigNodeProcedureEnv env) { "construct schema black list", env, targetSchemaRegionGroup, - DataNodeRequestType.CONSTRUCT_SCHEMA_BLACK_LIST_WITH_TEMPLATE, + CnToDnRequestType.CONSTRUCT_SCHEMA_BLACK_LIST_WITH_TEMPLATE, ((dataNodeLocation, consensusGroupIdList) -> new TConstructSchemaBlackListWithTemplateReq( consensusGroupIdList, dataNodeRequest))) { @@ -198,12 +198,13 @@ private void invalidateCache(ConfigNodeProcedureEnv env) { if (!timeSeriesPatternTree.isEmpty()) { Map dataNodeLocationMap = env.getConfigManager().getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.INVALIDATE_MATCHED_SCHEMA_CACHE, + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.INVALIDATE_MATCHED_SCHEMA_CACHE, new TInvalidateMatchedSchemaCacheReq(timeSeriesPatternTreeBytes), dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(clientHandler); Map statusMap = clientHandler.getResponseMap(); for (TSStatus status : statusMap.values()) { // all dataNodes must clear the related schema cache @@ -232,7 +233,7 @@ private void deleteData(ConfigNodeProcedureEnv env) { env, relatedDataRegionGroup, true, - DataNodeRequestType.DELETE_DATA_FOR_DELETE_SCHEMA, + CnToDnRequestType.DELETE_DATA_FOR_DELETE_SCHEMA, ((dataNodeLocation, consensusGroupIdList) -> new TDeleteDataForDeleteSchemaReq( new ArrayList<>(consensusGroupIdList), timeSeriesPatternTreeBytes))); @@ -247,7 +248,7 @@ private void deactivateTemplate(ConfigNodeProcedureEnv env) { "deactivate template schema", env, env.getConfigManager().getRelatedSchemaRegionGroup(timeSeriesPatternTree), - DataNodeRequestType.DEACTIVATE_TEMPLATE, + CnToDnRequestType.DEACTIVATE_TEMPLATE, ((dataNodeLocation, consensusGroupIdList) -> new TDeactivateTemplateReq(consensusGroupIdList, dataNodeRequest) .setIsGeneratedByPipe(isGeneratedByPipe))); @@ -285,7 +286,7 @@ protected void rollbackState( "roll back schema black list", env, env.getConfigManager().getRelatedSchemaRegionGroup(timeSeriesPatternTree), - DataNodeRequestType.ROLLBACK_SCHEMA_BLACK_LIST_WITH_TEMPLATE, + CnToDnRequestType.ROLLBACK_SCHEMA_BLACK_LIST_WITH_TEMPLATE, ((dataNodeLocation, consensusGroupIdList) -> new TRollbackSchemaBlackListWithTemplateReq( consensusGroupIdList, dataNodeRequest))); @@ -438,7 +439,7 @@ private class DeactivateTemplateRegionTaskExecutor String taskName, ConfigNodeProcedureEnv env, Map targetSchemaRegionGroup, - DataNodeRequestType dataNodeRequestType, + CnToDnRequestType dataNodeRequestType, BiFunction, Q> dataNodeRequestGenerator) { super(env, targetSchemaRegionGroup, false, dataNodeRequestType, dataNodeRequestGenerator); this.taskName = taskName; @@ -449,7 +450,7 @@ private class DeactivateTemplateRegionTaskExecutor ConfigNodeProcedureEnv env, Map targetSchemaRegionGroup, boolean executeOnAllReplicaset, - DataNodeRequestType dataNodeRequestType, + CnToDnRequestType dataNodeRequestType, BiFunction, Q> dataNodeRequestGenerator) { super( env, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java index f03cc7619ef2..d47e575a3ad6 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java @@ -27,9 +27,9 @@ import org.apache.iotdb.commons.exception.runtime.ThriftSerDeException; import org.apache.iotdb.commons.service.metric.MetricService; import org.apache.iotdb.commons.utils.ThriftConfigNodeSerDeUtils; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.write.database.PreDeleteDatabasePlan; import org.apache.iotdb.confignode.consensus.request.write.region.OfferRegionMaintainTasksPlan; import org.apache.iotdb.confignode.manager.partition.PartitionMetrics; @@ -155,15 +155,15 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, DeleteStorageGroupSt MetricService.getInstance(), deleteDatabaseSchema.getName()); // try sync delete schemaengine region - AsyncClientHandler asyncClientHandler = - new AsyncClientHandler<>(DataNodeRequestType.DELETE_REGION); + DataNodeAsyncRequestContext asyncClientHandler = + new DataNodeAsyncRequestContext<>(CnToDnRequestType.DELETE_REGION); Map schemaRegionDeleteTaskMap = new HashMap<>(); int requestIndex = 0; for (TRegionReplicaSet schemaRegionReplicaSet : schemaRegionReplicaSets) { for (TDataNodeLocation dataNodeLocation : schemaRegionReplicaSet.getDataNodeLocations()) { asyncClientHandler.putRequest(requestIndex, schemaRegionReplicaSet.getRegionId()); - asyncClientHandler.putDataNodeLocation(requestIndex, dataNodeLocation); + asyncClientHandler.putNodeLocation(requestIndex, dataNodeLocation); schemaRegionDeleteTaskMap.put( requestIndex, new RegionDeleteTask(dataNodeLocation, schemaRegionReplicaSet.getRegionId())); @@ -171,8 +171,8 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, DeleteStorageGroupSt } } if (!schemaRegionDeleteTaskMap.isEmpty()) { - AsyncDataNodeClientPool.getInstance() - .sendAsyncRequestToDataNodeWithRetry(asyncClientHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(asyncClientHandler); for (Map.Entry entry : asyncClientHandler.getResponseMap().entrySet()) { if (entry.getValue().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java index 1a566ae44046..97583dfc03b0 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java @@ -26,9 +26,9 @@ import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathPatternTree; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; @@ -142,7 +142,7 @@ private long constructBlackList(ConfigNodeProcedureEnv env) { "construct view schemaengine black list", env, targetSchemaRegionGroup, - DataNodeRequestType.CONSTRUCT_VIEW_SCHEMA_BLACK_LIST, + CnToDnRequestType.CONSTRUCT_VIEW_SCHEMA_BLACK_LIST, ((dataNodeLocation, consensusGroupIdList) -> new TConstructViewSchemaBlackListReq(consensusGroupIdList, patternTreeBytes))) { @Override @@ -184,12 +184,12 @@ protected List processResponseOfOneDataNode( private void invalidateCache(ConfigNodeProcedureEnv env) { Map dataNodeLocationMap = env.getConfigManager().getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.INVALIDATE_MATCHED_SCHEMA_CACHE, + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.INVALIDATE_MATCHED_SCHEMA_CACHE, new TInvalidateMatchedSchemaCacheReq(patternTreeBytes), dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); Map statusMap = clientHandler.getResponseMap(); for (TSStatus status : statusMap.values()) { // all dataNodes must clear the related schemaengine cache @@ -211,7 +211,7 @@ private void deleteViewSchema(ConfigNodeProcedureEnv env) { "delete view schemaengine", env, env.getConfigManager().getRelatedSchemaRegionGroup(patternTree), - DataNodeRequestType.DELETE_VIEW, + CnToDnRequestType.DELETE_VIEW, ((dataNodeLocation, consensusGroupIdList) -> new TDeleteViewSchemaReq(consensusGroupIdList, patternTreeBytes) .setIsGeneratedByPipe(isGeneratedByPipe))); @@ -247,7 +247,7 @@ protected void rollbackState( "roll back view schemaengine black list", env, env.getConfigManager().getRelatedSchemaRegionGroup(patternTree), - DataNodeRequestType.ROLLBACK_VIEW_SCHEMA_BLACK_LIST, + CnToDnRequestType.ROLLBACK_VIEW_SCHEMA_BLACK_LIST, (dataNodeLocation, consensusGroupIdList) -> new TRollbackViewSchemaBlackListReq(consensusGroupIdList, patternTreeBytes)); rollbackStateTask.execute(); @@ -343,7 +343,7 @@ private class DeleteLogicalViewRegionTaskExecutor String taskName, ConfigNodeProcedureEnv env, Map targetSchemaRegionGroup, - DataNodeRequestType dataNodeRequestType, + CnToDnRequestType dataNodeRequestType, BiFunction, Q> dataNodeRequestGenerator) { super(env, targetSchemaRegionGroup, false, dataNodeRequestType, dataNodeRequestGenerator); this.taskName = taskName; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java index f49827f0895f..40b8c55d2374 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java @@ -26,9 +26,9 @@ import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathPatternTree; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; @@ -148,7 +148,7 @@ private long constructBlackList(ConfigNodeProcedureEnv env) { "construct schemaengine black list", env, targetSchemaRegionGroup, - DataNodeRequestType.CONSTRUCT_SCHEMA_BLACK_LIST, + CnToDnRequestType.CONSTRUCT_SCHEMA_BLACK_LIST, ((dataNodeLocation, consensusGroupIdList) -> new TConstructSchemaBlackListReq(consensusGroupIdList, patternTreeBytes))) { @Override @@ -190,12 +190,12 @@ protected List processResponseOfOneDataNode( private void invalidateCache(ConfigNodeProcedureEnv env) { Map dataNodeLocationMap = env.getConfigManager().getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.INVALIDATE_MATCHED_SCHEMA_CACHE, + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.INVALIDATE_MATCHED_SCHEMA_CACHE, new TInvalidateMatchedSchemaCacheReq(patternTreeBytes), dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); Map statusMap = clientHandler.getResponseMap(); for (TSStatus status : statusMap.values()) { // all dataNodes must clear the related schemaEngine cache @@ -237,7 +237,7 @@ private void executeDeleteData(ConfigNodeProcedureEnv env, PathPatternTree patte env, relatedDataRegionGroup, true, - DataNodeRequestType.DELETE_DATA_FOR_DELETE_SCHEMA, + CnToDnRequestType.DELETE_DATA_FOR_DELETE_SCHEMA, ((dataNodeLocation, consensusGroupIdList) -> new TDeleteDataForDeleteSchemaReq( new ArrayList<>(consensusGroupIdList), @@ -252,7 +252,7 @@ private void deleteTimeSeriesSchema(ConfigNodeProcedureEnv env) { "delete timeseries schemaengine", env, env.getConfigManager().getRelatedSchemaRegionGroup(patternTree), - DataNodeRequestType.DELETE_TIMESERIES, + CnToDnRequestType.DELETE_TIMESERIES, ((dataNodeLocation, consensusGroupIdList) -> new TDeleteTimeSeriesReq(consensusGroupIdList, patternTreeBytes) .setIsGeneratedByPipe(isGeneratedByPipe))); @@ -289,7 +289,7 @@ protected void rollbackState( "roll back schemaengine black list", env, env.getConfigManager().getRelatedSchemaRegionGroup(patternTree), - DataNodeRequestType.ROLLBACK_SCHEMA_BLACK_LIST, + CnToDnRequestType.ROLLBACK_SCHEMA_BLACK_LIST, (dataNodeLocation, consensusGroupIdList) -> new TRollbackSchemaBlackListReq(consensusGroupIdList, patternTreeBytes)); rollbackStateTask.execute(); @@ -386,7 +386,7 @@ private class DeleteTimeSeriesRegionTaskExecutor String taskName, ConfigNodeProcedureEnv env, Map targetSchemaRegionGroup, - DataNodeRequestType dataNodeRequestType, + CnToDnRequestType dataNodeRequestType, BiFunction, Q> dataNodeRequestGenerator) { super(env, targetSchemaRegionGroup, false, dataNodeRequestType, dataNodeRequestGenerator); this.taskName = taskName; @@ -397,7 +397,7 @@ private class DeleteTimeSeriesRegionTaskExecutor ConfigNodeProcedureEnv env, Map targetSchemaRegionGroup, boolean executeOnAllReplicaset, - DataNodeRequestType dataNodeRequestType, + CnToDnRequestType dataNodeRequestType, BiFunction, Q> dataNodeRequestGenerator) { super( env, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SchemaUtils.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SchemaUtils.java index 1dc978cc4040..86215919aa19 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SchemaUtils.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SchemaUtils.java @@ -26,7 +26,7 @@ import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathPatternTree; -import org.apache.iotdb.confignode.client.DataNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; import org.apache.iotdb.confignode.manager.ConfigManager; import org.apache.iotdb.db.exception.metadata.PathNotExistException; import org.apache.iotdb.db.schemaengine.template.Template; @@ -76,7 +76,7 @@ public static boolean checkDataNodeTemplateActivation( configManager, relatedSchemaRegionGroup, false, - DataNodeRequestType.COUNT_PATHS_USING_TEMPLATE, + CnToDnRequestType.COUNT_PATHS_USING_TEMPLATE, ((dataNodeLocation, consensusGroupIdList) -> new TCountPathsUsingTemplateReq( template.getId(), patternTreeBytes, consensusGroupIdList))) { @@ -155,7 +155,7 @@ public static void checkSchemaRegionUsingTemplate( configManager, relatedSchemaRegionGroup, false, - DataNodeRequestType.CHECK_SCHEMA_REGION_USING_TEMPLATE, + CnToDnRequestType.CHECK_SCHEMA_REGION_USING_TEMPLATE, ((dataNodeLocation, consensusGroupIdList) -> new TCheckSchemaRegionUsingTemplateReq(consensusGroupIdList))) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java index 4cd1b9fa37d0..ec9b003d7eee 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java @@ -24,9 +24,9 @@ import org.apache.iotdb.common.rpc.thrift.TSetTTLReq; import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.exception.MetadataException; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; @@ -108,15 +108,15 @@ private void setConfigNodeTTL(ConfigNodeProcedureEnv env) { private void updateDataNodeTTL(ConfigNodeProcedureEnv env) { Map dataNodeLocationMap = env.getConfigManager().getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.SET_TTL, + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.SET_TTL, new TSetTTLReq( Collections.singletonList(String.join(".", plan.getPathPattern())), plan.getTTL(), plan.isDataBase()), dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); Map statusMap = clientHandler.getResponseMap(); for (TSStatus status : statusMap.values()) { // all dataNodes must clear the related schemaengine cache diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTemplateProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTemplateProcedure.java index ab8ac167aee9..902221285e26 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTemplateProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTemplateProcedure.java @@ -28,9 +28,9 @@ import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathPatternTree; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.read.template.CheckTemplateSettablePlan; import org.apache.iotdb.confignode.consensus.request.read.template.GetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; @@ -210,9 +210,10 @@ private void preReleaseTemplate(ConfigNodeProcedureEnv env) { Map dataNodeLocationMap = env.getConfigManager().getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.UPDATE_TEMPLATE, req, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.UPDATE_TEMPLATE, req, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); Map statusMap = clientHandler.getResponseMap(); for (Map.Entry entry : statusMap.entrySet()) { if (entry.getValue().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { @@ -281,7 +282,7 @@ private void validateTimeSeriesExistence(ConfigNodeProcedureEnv env) { env, relatedSchemaRegionGroup, false, - DataNodeRequestType.CHECK_TIMESERIES_EXISTENCE, + CnToDnRequestType.CHECK_TIMESERIES_EXISTENCE, ((dataNodeLocation, consensusGroupIdList) -> new TCheckTimeSeriesExistenceReq(patternTreeBytes, consensusGroupIdList))) { @@ -382,9 +383,10 @@ private void commitReleaseTemplate(ConfigNodeProcedureEnv env) { Map dataNodeLocationMap = env.getConfigManager().getNodeManager().getRegisteredDataNodeLocations(); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>(DataNodeRequestType.UPDATE_TEMPLATE, req, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.UPDATE_TEMPLATE, req, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); Map statusMap = clientHandler.getResponseMap(); for (Map.Entry entry : statusMap.entrySet()) { if (entry.getValue().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { @@ -486,10 +488,10 @@ private void rollbackPreRelease(ConfigNodeProcedureEnv env) { TemplateInternalRPCUtil.generateInvalidateTemplateSetInfoBytes( template.getId(), templateSetPath)); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.UPDATE_TEMPLATE, invalidateTemplateSetInfoReq, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.UPDATE_TEMPLATE, invalidateTemplateSetInfoReq, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); Map statusMap = clientHandler.getResponseMap(); for (Map.Entry entry : statusMap.entrySet()) { // all dataNodes must clear the related template cache diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/UnsetTemplateProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/UnsetTemplateProcedure.java index e2d30f10bff7..ee70f3c7a7d1 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/UnsetTemplateProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/UnsetTemplateProcedure.java @@ -26,9 +26,9 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathDeserializeUtil; import org.apache.iotdb.commons.path.PathPatternTree; -import org.apache.iotdb.confignode.client.DataNodeRequestType; -import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool; -import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler; +import org.apache.iotdb.confignode.client.CnToDnRequestType; +import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.exception.ProcedureSuspendedException; @@ -155,10 +155,10 @@ private void executeInvalidateCache(ConfigNodeProcedureEnv env) throws Procedure invalidateTemplateSetInfoReq.setType( TemplateInternalRPCUpdateType.INVALIDATE_TEMPLATE_SET_INFO.toByte()); invalidateTemplateSetInfoReq.setTemplateInfo(getInvalidateTemplateSetInfo()); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.UPDATE_TEMPLATE, invalidateTemplateSetInfoReq, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.UPDATE_TEMPLATE, invalidateTemplateSetInfoReq, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); Map statusMap = clientHandler.getResponseMap(); for (TSStatus status : statusMap.values()) { // all dataNodes must clear the related template cache @@ -250,10 +250,10 @@ private void executeRollbackInvalidateCache(ConfigNodeProcedureEnv env) rollbackTemplateSetInfoReq.setType( TemplateInternalRPCUpdateType.ADD_TEMPLATE_SET_INFO.toByte()); rollbackTemplateSetInfoReq.setTemplateInfo(getAddTemplateSetInfo()); - AsyncClientHandler clientHandler = - new AsyncClientHandler<>( - DataNodeRequestType.UPDATE_TEMPLATE, rollbackTemplateSetInfoReq, dataNodeLocationMap); - AsyncDataNodeClientPool.getInstance().sendAsyncRequestToDataNodeWithRetry(clientHandler); + DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnRequestType.UPDATE_TEMPLATE, rollbackTemplateSetInfoReq, dataNodeLocationMap); + CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); Map statusMap = clientHandler.getResponseMap(); for (TSStatus status : statusMap.values()) { // all dataNodes must clear the related template cache diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/AuthOperationProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/AuthOperationProcedure.java index badc5d2aa96b..d8e999cd1878 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/AuthOperationProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/AuthOperationProcedure.java @@ -25,7 +25,7 @@ import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils; -import org.apache.iotdb.confignode.client.DataNodeRequestType; +import org.apache.iotdb.confignode.client.CnToDnRequestType; import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; import org.apache.iotdb.confignode.consensus.request.auth.AuthorPlan; @@ -112,7 +112,7 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, AuthOperationProcedu .sendSyncRequestToDataNodeWithRetry( pair.getLeft().getLocation().getInternalEndPoint(), req, - DataNodeRequestType.INVALIDATE_PERMISSION_CACHE); + CnToDnRequestType.INVALIDATE_PERMISSION_CACHE); if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { it.remove(); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java index eb33b0d70eba..3d537cdec61f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java @@ -38,7 +38,7 @@ import org.apache.iotdb.commons.service.metric.MetricService; import org.apache.iotdb.commons.service.metric.cpu.CpuUsageMetrics; import org.apache.iotdb.commons.utils.TestOnly; -import org.apache.iotdb.confignode.client.ConfigNodeRequestType; +import org.apache.iotdb.confignode.client.CnToCnNodeRequestType; import org.apache.iotdb.confignode.client.sync.SyncConfigNodeClientPool; import org.apache.iotdb.confignode.conf.ConfigNodeConfig; import org.apache.iotdb.confignode.conf.ConfigNodeConstant; @@ -186,7 +186,7 @@ public void active() { configManager .getNodeManager() .applyConfigNode( - generateConfigNodeLocation(SEED_CONFIG_NODE_ID), + CONF.generateLocalConfigNodeLocationWithSpecifiedNodeId(SEED_CONFIG_NODE_ID), new TNodeVersionInfo(IoTDBConstant.VERSION, IoTDBConstant.BUILD_INFO)); setUpMetricService(); // Notice: We always set up Seed-ConfigNode's RPC service lastly to ensure @@ -323,7 +323,7 @@ private void sendRegisterConfigNodeRequest() throws StartupException, IOExceptio TConfigNodeRegisterReq req = new TConfigNodeRegisterReq( configManager.getClusterParameters(), - generateConfigNodeLocation(INIT_NON_SEED_CONFIG_NODE_ID)); + CONF.generateLocalConfigNodeLocationWithSpecifiedNodeId(INIT_NON_SEED_CONFIG_NODE_ID)); req.setVersionInfo(new TNodeVersionInfo(IoTDBConstant.VERSION, IoTDBConstant.BUILD_INFO)); @@ -339,7 +339,7 @@ private void sendRegisterConfigNodeRequest() throws StartupException, IOExceptio Object obj = SyncConfigNodeClientPool.getInstance() .sendSyncRequestToConfigNodeWithRetry( - seedConfigNode, req, ConfigNodeRequestType.REGISTER_CONFIG_NODE); + seedConfigNode, req, CnToCnNodeRequestType.REGISTER_CONFIG_NODE); if (obj instanceof TConfigNodeRegisterResp) { resp = (TConfigNodeRegisterResp) obj; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNodeShutdownHook.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNodeShutdownHook.java index 369c6c96af84..5c3ec5af0632 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNodeShutdownHook.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNodeShutdownHook.java @@ -24,7 +24,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.cluster.NodeStatus; import org.apache.iotdb.commons.conf.CommonDescriptor; -import org.apache.iotdb.confignode.client.ConfigNodeRequestType; +import org.apache.iotdb.confignode.client.CnToCnNodeRequestType; import org.apache.iotdb.confignode.client.sync.SyncConfigNodeClientPool; import org.apache.iotdb.confignode.conf.ConfigNodeConfig; import org.apache.iotdb.confignode.conf.ConfigNodeConstant; @@ -71,7 +71,7 @@ public void run() { CONF.getConfigNodeId(), new TEndPoint(CONF.getInternalAddress(), CONF.getInternalPort()), new TEndPoint(CONF.getInternalAddress(), CONF.getConsensusPort())), - ConfigNodeRequestType.REPORT_CONFIG_NODE_SHUTDOWN); + CnToCnNodeRequestType.REPORT_CONFIG_NODE_SHUTDOWN); if (result.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { // Report success diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java index 2a465f34eaf1..0eeb3e9b8913 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java @@ -22,12 +22,14 @@ import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TFlushReq; +import org.apache.iotdb.common.rpc.thrift.TNodeLocations; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TSetConfigurationReq; import org.apache.iotdb.common.rpc.thrift.TSetSpaceQuotaReq; import org.apache.iotdb.common.rpc.thrift.TSetTTLReq; import org.apache.iotdb.common.rpc.thrift.TSetThrottleQuotaReq; import org.apache.iotdb.common.rpc.thrift.TShowConfigurationResp; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResp; import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.consensus.ConsensusGroupId; import org.apache.iotdb.commons.path.PartialPath; @@ -893,6 +895,25 @@ public TShowDatabaseResp showDatabase(TGetDatabaseReq req) { return configManager.showDatabase(req); } + /** Call by ConfigNode leader */ + @Override + public TTestConnectionResp submitTestConnectionTask(TNodeLocations nodeLocations) + throws TException { + return configManager.getClusterManager().doConnectionTest(nodeLocations); + } + + /** Call by client connected DataNode */ + @Override + public TTestConnectionResp submitTestConnectionTaskToLeader() throws TException { + return configManager.getClusterManager().submitTestConnectionTaskToEveryNode(); + } + + /** Call by every other nodes */ + @Override + public TSStatus testConnectionEmptyRPC() throws TException { + return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } + @Override public TSStatus createSchemaTemplate(TCreateSchemaTemplateReq req) { return configManager.createSchemaTemplate(req); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 421d8c5f0eda..9c20f6c32eca 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.conf; +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.client.property.ClientPoolProperty.DefaultProperty; import org.apache.iotdb.commons.conf.CommonDescriptor; @@ -3969,4 +3970,18 @@ public void setInnerCompactionTaskSelectionDiskRedundancy( double innerCompactionTaskSelectionDiskRedundancy) { this.innerCompactionTaskSelectionDiskRedundancy = innerCompactionTaskSelectionDiskRedundancy; } + + public TDataNodeLocation generateLocalDataNodeLocation() { + TDataNodeLocation result = new TDataNodeLocation(); + result.setDataNodeId(getDataNodeId()); + result.setClientRpcEndPoint(new TEndPoint(getInternalAddress(), getRpcPort())); + result.setInternalEndPoint(new TEndPoint(getInternalAddress(), getInternalPort())); + result.setMPPDataExchangeEndPoint( + new TEndPoint(getInternalAddress(), getMppDataExchangePort())); + result.setDataRegionConsensusEndPoint( + new TEndPoint(getInternalAddress(), getDataRegionConsensusPort())); + result.setSchemaRegionConsensusEndPoint( + new TEndPoint(getInternalAddress(), getSchemaRegionConsensusPort())); + return result; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java index cda4f597284c..980cb89b6e0c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java @@ -23,12 +23,14 @@ import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TFlushReq; +import org.apache.iotdb.common.rpc.thrift.TNodeLocations; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TSetConfigurationReq; import org.apache.iotdb.common.rpc.thrift.TSetSpaceQuotaReq; import org.apache.iotdb.common.rpc.thrift.TSetTTLReq; import org.apache.iotdb.common.rpc.thrift.TSetThrottleQuotaReq; import org.apache.iotdb.common.rpc.thrift.TShowConfigurationResp; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResp; import org.apache.iotdb.commons.client.ClientManager; import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.factory.ThriftClientFactory; @@ -240,7 +242,7 @@ private void tryToConnect() throws TException { try { connect(configLeader); return; - } catch (TException e) { + } catch (TException ignore) { logger.warn("The current node may have been down {},try next node", configLeader); configLeader = null; } @@ -248,7 +250,7 @@ private void tryToConnect() throws TException { try { // Wait to start the next try Thread.sleep(RETRY_INTERVAL_MS); - } catch (InterruptedException e) { + } catch (InterruptedException ignore) { Thread.currentThread().interrupt(); logger.warn("Unexpected interruption when waiting to try to connect to ConfigNode"); } @@ -265,7 +267,7 @@ private void tryToConnect() throws TException { try { connect(tryEndpoint); return; - } catch (TException e) { + } catch (TException ignore) { logger.warn("The current node may have been down {},try next node", tryEndpoint); } } @@ -726,6 +728,27 @@ public TShowDatabaseResp showDatabase(TGetDatabaseReq req) throws TException { () -> client.showDatabase(req), resp -> !updateConfigNodeLeader(resp.status)); } + @Override + public TTestConnectionResp submitTestConnectionTask(TNodeLocations nodeLocations) + throws TException { + return executeRemoteCallWithRetry( + () -> client.submitTestConnectionTask(nodeLocations), + resp -> !updateConfigNodeLeader(resp.getStatus())); + } + + @Override + public TTestConnectionResp submitTestConnectionTaskToLeader() throws TException { + return executeRemoteCallWithRetry( + () -> client.submitTestConnectionTaskToLeader(), + resp -> !updateConfigNodeLeader(resp.getStatus())); + } + + @Override + public TSStatus testConnectionEmptyRPC() throws TException { + return executeRemoteCallWithRetry( + () -> client.testConnectionEmptyRPC(), resp -> !updateConfigNodeLeader(resp)); + } + @Override public TRegionRouteMapResp getLatestRegionRouteMap() throws TException { return executeRemoteCallWithRetry( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/AsyncConfigNodeTSStatusRPCHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/AsyncConfigNodeTSStatusRPCHandler.java new file mode 100644 index 000000000000..fd1cbd539fbf --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/AsyncConfigNodeTSStatusRPCHandler.java @@ -0,0 +1,95 @@ +/* + * 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.protocol.client.cn; + +import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +/** General RPC handler for TSStatus response type. */ +public class AsyncConfigNodeTSStatusRPCHandler extends ConfigNodeAsyncRequestRPCHandler { + + private static final Logger LOGGER = + LoggerFactory.getLogger(AsyncConfigNodeTSStatusRPCHandler.class); + + public AsyncConfigNodeTSStatusRPCHandler( + DnToCnRequestType requestType, + int requestId, + TConfigNodeLocation targetConfigNode, + Map configNodeLocationMap, + Map responseMap, + CountDownLatch countDownLatch) { + super( + requestType, + requestId, + targetConfigNode, + configNodeLocationMap, + responseMap, + countDownLatch); + } + + @Override + public void onComplete(TSStatus response) { + // Put response + responseMap.put(requestId, response); + + if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + // Remove only if success + nodeLocationMap.remove(requestId); + LOGGER.info("Successfully {} on ConfigNode: {}", requestType, formattedTargetLocation); + } else { + LOGGER.error( + "Failed to {} on ConfigNode: {}, response: {}", + requestType, + formattedTargetLocation, + response); + } + + // Always CountDown + countDownLatch.countDown(); + } + + @Override + public void onError(Exception e) { + String errorMsg = + "Failed to " + + requestType + + " on ConfigNode: " + + formattedTargetLocation + + ", exception: " + + e.getMessage(); + LOGGER.error(errorMsg); + + responseMap.put( + requestId, + new TSStatus( + RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode(), errorMsg))); + + // Always CountDown + countDownLatch.countDown(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/ConfigNodeAsyncRequestRPCHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/ConfigNodeAsyncRequestRPCHandler.java new file mode 100644 index 000000000000..ece9a2fa5f0e --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/ConfigNodeAsyncRequestRPCHandler.java @@ -0,0 +1,79 @@ +/* + * 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.protocol.client.cn; + +import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; + +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +public abstract class ConfigNodeAsyncRequestRPCHandler + extends AsyncRequestRPCHandler { + + protected ConfigNodeAsyncRequestRPCHandler( + DnToCnRequestType configNodeRequestType, + int requestId, + TConfigNodeLocation targetNode, + Map integerTConfigNodeLocationMap, + Map integerResponseMap, + CountDownLatch countDownLatch) { + super( + configNodeRequestType, + requestId, + targetNode, + integerTConfigNodeLocationMap, + integerResponseMap, + countDownLatch); + } + + @Override + protected String generateFormattedTargetLocation(TConfigNodeLocation configNodeLocation) { + return "{id=" + + targetNode.getConfigNodeId() + + ", internalEndPoint=" + + targetNode.getInternalEndPoint() + + "}"; + } + + public static ConfigNodeAsyncRequestRPCHandler buildHandler( + AsyncRequestContext context, + int requestId, + TConfigNodeLocation targetConfigNode) { + DnToCnRequestType requestType = context.getRequestType(); + Map nodeLocationMap = context.getNodeLocationMap(); + Map responseMap = context.getResponseMap(); + CountDownLatch countDownLatch = context.getCountDownLatch(); + switch (requestType) { + case SUBMIT_TEST_CONNECTION_TASK: + case TEST_CONNECTION: + default: + return new AsyncConfigNodeTSStatusRPCHandler( + requestType, + requestId, + targetConfigNode, + nodeLocationMap, + (Map) responseMap, + countDownLatch); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/DnToCnInternalServiceAsyncRequestManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/DnToCnInternalServiceAsyncRequestManager.java new file mode 100644 index 000000000000..3f6763672409 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/DnToCnInternalServiceAsyncRequestManager.java @@ -0,0 +1,64 @@ +/* + * 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.protocol.client.cn; + +import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; +import org.apache.iotdb.commons.client.request.ConfigNodeInternalServiceAsyncRequestManager; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DnToCnInternalServiceAsyncRequestManager + extends ConfigNodeInternalServiceAsyncRequestManager { + + private static final Logger LOGGER = + LoggerFactory.getLogger(DnToCnInternalServiceAsyncRequestManager.class); + + @Override + protected void initActionMapBuilder() { + actionMapBuilder.put( + DnToCnRequestType.TEST_CONNECTION, + (req, client, handler) -> + client.testConnectionEmptyRPC((AsyncConfigNodeTSStatusRPCHandler) handler)); + } + + @Override + protected AsyncRequestRPCHandler buildHandler( + AsyncRequestContext requestContext, + int requestId, + TConfigNodeLocation targetNode) { + return ConfigNodeAsyncRequestRPCHandler.buildHandler(requestContext, requestId, targetNode); + } + + private static class ClientPoolHolder { + private static final DnToCnInternalServiceAsyncRequestManager INSTANCE = + new DnToCnInternalServiceAsyncRequestManager(); + + private ClientPoolHolder() { + // Empty constructor + } + } + + public static DnToCnInternalServiceAsyncRequestManager getInstance() { + return ClientPoolHolder.INSTANCE; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/DnToCnRequestType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/DnToCnRequestType.java new file mode 100644 index 000000000000..5708fec5bed2 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/DnToCnRequestType.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.protocol.client.cn; + +/** For DataNode async call ConfigNode */ +public enum DnToCnRequestType { + SUBMIT_TEST_CONNECTION_TASK, + TEST_CONNECTION, +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/AsyncTSStatusRPCHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/AsyncTSStatusRPCHandler.java similarity index 90% rename from iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/AsyncTSStatusRPCHandler.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/AsyncTSStatusRPCHandler.java index 1f61ee141a0b..9672d431c384 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/AsyncTSStatusRPCHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/AsyncTSStatusRPCHandler.java @@ -17,11 +17,10 @@ * under the License. */ -package org.apache.iotdb.confignode.client.async.handlers.rpc; +package org.apache.iotdb.db.protocol.client.dn; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.confignode.client.DataNodeRequestType; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -32,12 +31,12 @@ import java.util.concurrent.CountDownLatch; /** General RPC handler for TSStatus response type. */ -public class AsyncTSStatusRPCHandler extends AbstractAsyncRPCHandler { +public class AsyncTSStatusRPCHandler extends DataNodeAsyncRequestRPCHandler { private static final Logger LOGGER = LoggerFactory.getLogger(AsyncTSStatusRPCHandler.class); public AsyncTSStatusRPCHandler( - DataNodeRequestType requestType, + DnToDnRequestType requestType, int requestId, TDataNodeLocation targetDataNode, Map dataNodeLocationMap, @@ -53,7 +52,7 @@ public void onComplete(TSStatus response) { if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { // Remove only if success - dataNodeLocationMap.remove(requestId); + nodeLocationMap.remove(requestId); LOGGER.info("Successfully {} on DataNode: {}", requestType, formattedTargetLocation); } else { LOGGER.error( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeAsyncRequestRPCHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeAsyncRequestRPCHandler.java new file mode 100644 index 000000000000..76be9ac9d715 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeAsyncRequestRPCHandler.java @@ -0,0 +1,84 @@ +/* + * 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.protocol.client.dn; + +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +public abstract class DataNodeAsyncRequestRPCHandler + extends AsyncRequestRPCHandler { + private static final Logger LOGGER = + LoggerFactory.getLogger(DataNodeAsyncRequestRPCHandler.class); + + protected DataNodeAsyncRequestRPCHandler( + DnToDnRequestType dataNodeToDataNodeRequestType, + int requestId, + TDataNodeLocation targetNode, + Map dataNodeLocationMap, + Map integerResponseMap, + CountDownLatch countDownLatch) { + super( + dataNodeToDataNodeRequestType, + requestId, + targetNode, + dataNodeLocationMap, + integerResponseMap, + countDownLatch); + } + + @Override + protected String generateFormattedTargetLocation(TDataNodeLocation dataNodeLocation) { + return "{id=" + + targetNode.getDataNodeId() + + ", internalEndPoint=" + + targetNode.getInternalEndPoint() + + "}"; + } + + public static DataNodeAsyncRequestRPCHandler createAsyncRPCHandler( + AsyncRequestContext context, + int requestId, + TDataNodeLocation targetDataNode) { + DnToDnRequestType requestType = context.getRequestType(); + Map nodeLocationMap = context.getNodeLocationMap(); + Map responseMap = context.getResponseMap(); + CountDownLatch countDownLatch = context.getCountDownLatch(); + switch (requestType) { + case TEST_CONNECTION: + return new AsyncTSStatusRPCHandler( + requestType, + requestId, + targetDataNode, + nodeLocationMap, + (Map) responseMap, + countDownLatch); + default: + throw new UnsupportedOperationException("request type is not supported: " + requestType); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeExternalServiceAsyncRequestManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeExternalServiceAsyncRequestManager.java new file mode 100644 index 000000000000..dd56e1366c03 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeExternalServiceAsyncRequestManager.java @@ -0,0 +1,83 @@ +/* + * 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.protocol.client.dn; + +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.client.ClientPoolFactory; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.client.async.AsyncDataNodeExternalServiceClient; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.commons.client.request.AsyncRequestManager; +import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DataNodeExternalServiceAsyncRequestManager + extends AsyncRequestManager< + DnToDnRequestType, TDataNodeLocation, AsyncDataNodeExternalServiceClient> { + + private static final Logger LOGGER = + LoggerFactory.getLogger(DataNodeExternalServiceAsyncRequestManager.class); + + @Override + protected void initClientManager() { + clientManager = + new IClientManager.Factory() + .createClientManager( + new ClientPoolFactory.AsyncDataNodeExternalServiceClientPoolFactory()); + } + + @Override + protected void initActionMapBuilder() { + actionMapBuilder.put( + DnToDnRequestType.TEST_CONNECTION, + (req, client, handler) -> client.testConnectionEmptyRPC((AsyncTSStatusRPCHandler) handler)); + } + + @Override + protected TEndPoint nodeLocationToEndPoint(TDataNodeLocation dataNodeLocation) { + return dataNodeLocation.getClientRpcEndPoint(); + } + + @Override + protected AsyncRequestRPCHandler buildHandler( + AsyncRequestContext requestContext, + int requestId, + TDataNodeLocation targetNode) { + return DataNodeAsyncRequestRPCHandler.createAsyncRPCHandler( + requestContext, requestId, targetNode); + } + + private static class ClientPoolHolder { + + private static final DataNodeExternalServiceAsyncRequestManager INSTANCE = + new DataNodeExternalServiceAsyncRequestManager(); + + private ClientPoolHolder() { + // Empty constructor + } + } + + public static DataNodeExternalServiceAsyncRequestManager getInstance() { + return DataNodeExternalServiceAsyncRequestManager.ClientPoolHolder.INSTANCE; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeMPPServiceAsyncRequestManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeMPPServiceAsyncRequestManager.java new file mode 100644 index 000000000000..ab08d83f2645 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeMPPServiceAsyncRequestManager.java @@ -0,0 +1,84 @@ +/* + * 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.protocol.client.dn; + +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.client.ClientPoolFactory; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.client.async.AsyncDataNodeMPPDataExchangeServiceClient; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.commons.client.request.AsyncRequestManager; +import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DataNodeMPPServiceAsyncRequestManager + extends AsyncRequestManager< + DnToDnRequestType, TDataNodeLocation, AsyncDataNodeMPPDataExchangeServiceClient> { + private static final Logger LOGGER = + LoggerFactory.getLogger(DataNodeMPPServiceAsyncRequestManager.class); + + public DataNodeMPPServiceAsyncRequestManager() {} + + @Override + protected void initClientManager() { + clientManager = + new IClientManager.Factory() + .createClientManager( + new ClientPoolFactory.AsyncDataNodeMPPDataExchangeServiceClientPoolFactory()); + } + + @Override + protected void initActionMapBuilder() { + actionMapBuilder.put( + DnToDnRequestType.TEST_CONNECTION, + (req, client, handler) -> client.testConnectionEmptyRPC((AsyncTSStatusRPCHandler) handler)); + } + + @Override + protected TEndPoint nodeLocationToEndPoint(TDataNodeLocation dataNodeLocation) { + return dataNodeLocation.getMPPDataExchangeEndPoint(); + } + + @Override + protected AsyncRequestRPCHandler buildHandler( + AsyncRequestContext requestContext, + int requestId, + TDataNodeLocation targetNode) { + return DataNodeAsyncRequestRPCHandler.createAsyncRPCHandler( + requestContext, requestId, targetNode); + } + + private static class ClientPoolHolder { + + private static final DataNodeMPPServiceAsyncRequestManager INSTANCE = + new DataNodeMPPServiceAsyncRequestManager(); + + private ClientPoolHolder() { + // Empty constructor + } + } + + public static DataNodeMPPServiceAsyncRequestManager getInstance() { + return ClientPoolHolder.INSTANCE; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DnToDnInternalServiceAsyncRequestManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DnToDnInternalServiceAsyncRequestManager.java new file mode 100644 index 000000000000..88766458b649 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DnToDnInternalServiceAsyncRequestManager.java @@ -0,0 +1,64 @@ +/* + * 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.protocol.client.dn; + +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; +import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; +import org.apache.iotdb.commons.client.request.DataNodeInternalServiceRequestManager; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DnToDnInternalServiceAsyncRequestManager + extends DataNodeInternalServiceRequestManager { + private static final Logger LOGGER = + LoggerFactory.getLogger(DnToDnInternalServiceAsyncRequestManager.class); + + @Override + protected void initActionMapBuilder() { + actionMapBuilder.put( + DnToDnRequestType.TEST_CONNECTION, + (req, client, handler) -> client.testConnectionEmptyRPC((AsyncTSStatusRPCHandler) handler)); + } + + @Override + protected AsyncRequestRPCHandler buildHandler( + AsyncRequestContext requestContext, + int requestId, + TDataNodeLocation targetNode) { + return DataNodeAsyncRequestRPCHandler.createAsyncRPCHandler( + requestContext, requestId, targetNode); + } + + private static class ClientPoolHolder { + + private static final DnToDnInternalServiceAsyncRequestManager INSTANCE = + new DnToDnInternalServiceAsyncRequestManager(); + + private ClientPoolHolder() { + // Empty constructor + } + } + + public static DnToDnInternalServiceAsyncRequestManager getInstance() { + return ClientPoolHolder.INSTANCE; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DnToDnRequestType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DnToDnRequestType.java new file mode 100644 index 000000000000..35f5b8d93a9b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DnToDnRequestType.java @@ -0,0 +1,25 @@ +/* + * 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.protocol.client.dn; + +/** For DataNode async call DataNode */ +public enum DnToDnRequestType { + TEST_CONNECTION, +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java index f7a2d3ffcd5f..78f7d03fd2ff 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java @@ -2670,6 +2670,11 @@ public TSConnectionInfoResp fetchAllConnectionsInfo() { return SESSION_MANAGER.getAllConnectionInfo(); } + @Override + public TSStatus testConnectionEmptyRPC() throws TException { + return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } + @Override public TSStatus insertStringRecord(final TSInsertStringRecordReq req) { final long t1 = System.nanoTime(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java index 10155e26c478..0db6eff2e712 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -19,17 +19,24 @@ package org.apache.iotdb.db.protocol.thrift.impl; +import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TFlushReq; +import org.apache.iotdb.common.rpc.thrift.TNodeLocations; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.common.rpc.thrift.TSender; +import org.apache.iotdb.common.rpc.thrift.TServiceType; import org.apache.iotdb.common.rpc.thrift.TSetConfigurationReq; import org.apache.iotdb.common.rpc.thrift.TSetSpaceQuotaReq; import org.apache.iotdb.common.rpc.thrift.TSetTTLReq; import org.apache.iotdb.common.rpc.thrift.TSetThrottleQuotaReq; import org.apache.iotdb.common.rpc.thrift.TSettleReq; import org.apache.iotdb.common.rpc.thrift.TShowConfigurationResp; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResp; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResult; +import org.apache.iotdb.commons.client.request.AsyncRequestContext; import org.apache.iotdb.commons.cluster.NodeStatus; import org.apache.iotdb.commons.conf.CommonConfig; import org.apache.iotdb.commons.conf.CommonDescriptor; @@ -69,6 +76,12 @@ import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; +import org.apache.iotdb.db.protocol.client.cn.DnToCnInternalServiceAsyncRequestManager; +import org.apache.iotdb.db.protocol.client.cn.DnToCnRequestType; +import org.apache.iotdb.db.protocol.client.dn.DataNodeExternalServiceAsyncRequestManager; +import org.apache.iotdb.db.protocol.client.dn.DataNodeMPPServiceAsyncRequestManager; +import org.apache.iotdb.db.protocol.client.dn.DnToDnInternalServiceAsyncRequestManager; +import org.apache.iotdb.db.protocol.client.dn.DnToDnRequestType; import org.apache.iotdb.db.protocol.session.IClientSession; import org.apache.iotdb.db.protocol.session.InternalClientSession; import org.apache.iotdb.db.protocol.session.SessionManager; @@ -252,6 +265,7 @@ import java.time.ZoneId; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -262,9 +276,12 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReadWriteLock; +import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.Stream; +import static org.apache.iotdb.commons.client.request.Utils.testConnectionsImpl; import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD; import static org.apache.iotdb.db.service.RegionMigrateService.REGION_MIGRATE_PROCESS; import static org.apache.iotdb.db.utils.ErrorHandlingUtils.onQueryException; @@ -1411,6 +1428,91 @@ public TFetchFragmentInstanceStatisticsResp fetchFragmentInstanceStatistics( return resp; } + @Override + public TTestConnectionResp submitTestConnectionTask(TNodeLocations nodeLocations) + throws TException { + return new TTestConnectionResp( + new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()), + Stream.of( + testAllConfigNodeConnection(nodeLocations.getConfigNodeLocations()), + testAllDataNodeInternalServiceConnection(nodeLocations.getDataNodeLocations()), + testAllDataNodeMPPServiceConnection(nodeLocations.getDataNodeLocations()), + testAllDataNodeExternalServiceConnection(nodeLocations.getDataNodeLocations())) + .flatMap(Collection::stream) + .collect(Collectors.toList())); + } + + private static List testConnections( + List nodeLocations, + Function getId, + Function getEndPoint, + TServiceType serviceType, + RequestType requestType, + Consumer> sendRequest) { + TSender sender = + new TSender() + .setDataNodeLocation( + IoTDBDescriptor.getInstance().getConfig().generateLocalDataNodeLocation()); + return testConnectionsImpl( + nodeLocations, sender, getId, getEndPoint, serviceType, requestType, sendRequest); + } + + private List testAllConfigNodeConnection( + List configNodeLocations) { + return testConnections( + configNodeLocations, + TConfigNodeLocation::getConfigNodeId, + TConfigNodeLocation::getInternalEndPoint, + TServiceType.ConfigNodeInternalService, + DnToCnRequestType.TEST_CONNECTION, + (AsyncRequestContext handler) -> + DnToCnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(handler)); + } + + private List testAllDataNodeInternalServiceConnection( + List dataNodeLocations) { + return testConnections( + dataNodeLocations, + TDataNodeLocation::getDataNodeId, + TDataNodeLocation::getInternalEndPoint, + TServiceType.DataNodeInternalService, + DnToDnRequestType.TEST_CONNECTION, + (AsyncRequestContext handler) -> + DnToDnInternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(handler)); + } + + private List testAllDataNodeMPPServiceConnection( + List dataNodeLocations) { + return testConnections( + dataNodeLocations, + TDataNodeLocation::getDataNodeId, + TDataNodeLocation::getMPPDataExchangeEndPoint, + TServiceType.DataNodeMPPService, + DnToDnRequestType.TEST_CONNECTION, + (AsyncRequestContext handler) -> + DataNodeMPPServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(handler)); + } + + private List testAllDataNodeExternalServiceConnection( + List dataNodeLocations) { + return testConnections( + dataNodeLocations, + TDataNodeLocation::getDataNodeId, + TDataNodeLocation::getClientRpcEndPoint, + TServiceType.DataNodeExternalService, + DnToDnRequestType.TEST_CONNECTION, + (AsyncRequestContext handler) -> + DataNodeExternalServiceAsyncRequestManager.getInstance() + .sendAsyncRequestWithRetry(handler)); + } + + @Override + public TSStatus testConnectionEmptyRPC() throws TException { + return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } + private PathPatternTree filterPathPatternTree(PathPatternTree patternTree, String storageGroup) { PathPatternTree filteredPatternTree = new PathPatternTree(); try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/header/ColumnHeaderConstant.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/header/ColumnHeaderConstant.java index cd78981d21c0..e7e93d4fb3b1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/header/ColumnHeaderConstant.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/header/ColumnHeaderConstant.java @@ -94,6 +94,11 @@ private ColumnHeaderConstant() { // column names for show clusterId statement public static final String CLUSTER_ID = "ClusterId"; + // column names for verify connection statement + public static final String SERVICE_PROVIDER = "ServiceProvider"; + public static final String SENDER = "Sender"; + public static final String CONNECTION = "Connection"; + // column names for show functions statement public static final String FUNCTION_NAME = "FunctionName"; public static final String FUNCTION_TYPE = "FunctionType"; @@ -370,6 +375,12 @@ private ColumnHeaderConstant() { public static final List showClusterIdColumnHeaders = ImmutableList.of(new ColumnHeader(CLUSTER_ID, TSDataType.TEXT)); + public static final List testConnectionColumnHeaders = + ImmutableList.of( + new ColumnHeader(SERVICE_PROVIDER, TSDataType.TEXT), + new ColumnHeader(SENDER, TSDataType.TEXT), + new ColumnHeader(CONNECTION, TSDataType.TEXT)); + public static final List showVariablesColumnHeaders = ImmutableList.of( new ColumnHeader(VARIABLE, TSDataType.TEXT), new ColumnHeader(VALUE, TSDataType.TEXT)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/header/DatasetHeaderFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/header/DatasetHeaderFactory.java index 78250b05bcb7..9c0f89b19a57 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/header/DatasetHeaderFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/header/DatasetHeaderFactory.java @@ -99,6 +99,10 @@ public static DatasetHeader getShowClusterIdHeader() { return new DatasetHeader(ColumnHeaderConstant.showClusterIdColumnHeaders, true); } + public static DatasetHeader getTestConnectionHeader() { + return new DatasetHeader(ColumnHeaderConstant.testConnectionColumnHeaders, true); + } + public static DatasetHeader getShowFunctionsHeader() { return new DatasetHeader(ColumnHeaderConstant.showFunctionsColumnHeaders, true); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/MPPDataExchangeManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/MPPDataExchangeManager.java index 2b59d72cd070..a38ef923bbb3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/MPPDataExchangeManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/MPPDataExchangeManager.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.execution.exchange; import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.sync.SyncDataNodeMPPDataExchangeServiceClient; import org.apache.iotdb.db.queryengine.exception.exchange.GetTsBlockFromClosedOrAbortedChannelException; @@ -49,6 +50,7 @@ import org.apache.iotdb.mpp.rpc.thrift.TGetDataBlockRequest; import org.apache.iotdb.mpp.rpc.thrift.TGetDataBlockResponse; import org.apache.iotdb.mpp.rpc.thrift.TNewDataBlockEvent; +import org.apache.iotdb.rpc.TSStatusCode; import org.apache.commons.lang3.Validate; import org.apache.thrift.TException; @@ -283,6 +285,11 @@ public void onEndOfDataBlockEvent(TEndOfDataBlockEvent e) throws TException { sourceHandle.setNoMoreTsBlocks(e.getLastSequenceId()); } } + + @Override + public TSStatus testConnectionEmptyRPC() throws TException { + return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } } // endregion diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/ConfigTaskVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/ConfigTaskVisitor.java index 0786d2fb8938..e9b36c80c914 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/ConfigTaskVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/ConfigTaskVisitor.java @@ -74,6 +74,7 @@ import org.apache.iotdb.db.queryengine.plan.execution.config.sys.SetSystemStatusTask; import org.apache.iotdb.db.queryengine.plan.execution.config.sys.StartRepairDataTask; import org.apache.iotdb.db.queryengine.plan.execution.config.sys.StopRepairDataTask; +import org.apache.iotdb.db.queryengine.plan.execution.config.sys.TestConnectionTask; import org.apache.iotdb.db.queryengine.plan.execution.config.sys.pipe.AlterPipeTask; import org.apache.iotdb.db.queryengine.plan.execution.config.sys.pipe.CreatePipeTask; import org.apache.iotdb.db.queryengine.plan.execution.config.sys.pipe.DropPipeTask; @@ -154,6 +155,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.sys.SetSystemStatusStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.StartRepairDataStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.StopRepairDataStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.TestConnectionStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.quota.SetSpaceQuotaStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.quota.SetThrottleQuotaStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.quota.ShowSpaceQuotaStatement; @@ -240,6 +242,12 @@ public IConfigTask visitShowClusterId( return new ShowClusterIdTask(); } + @Override + public IConfigTask visitTestConnection( + TestConnectionStatement testConnectionStatement, MPPQueryContext context) { + return new TestConnectionTask(testConnectionStatement.needDetails()); + } + @Override public IConfigTask visitAuthor(AuthorStatement statement, MPPQueryContext context) { return new AuthorizerTask(statement); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java index 9852e202e7c2..538337ffa164 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java @@ -26,6 +26,7 @@ import org.apache.iotdb.common.rpc.thrift.TSetTTLReq; import org.apache.iotdb.common.rpc.thrift.TSetThrottleQuotaReq; import org.apache.iotdb.common.rpc.thrift.TSpaceQuota; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResp; import org.apache.iotdb.common.rpc.thrift.TThrottleQuota; import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.exception.ClientManagerException; @@ -149,6 +150,7 @@ import org.apache.iotdb.db.queryengine.plan.execution.config.metadata.template.ShowNodesInSchemaTemplateTask; import org.apache.iotdb.db.queryengine.plan.execution.config.metadata.template.ShowPathSetTemplateTask; import org.apache.iotdb.db.queryengine.plan.execution.config.metadata.template.ShowSchemaTemplateTask; +import org.apache.iotdb.db.queryengine.plan.execution.config.sys.TestConnectionTask; import org.apache.iotdb.db.queryengine.plan.execution.config.sys.pipe.ShowPipeTask; import org.apache.iotdb.db.queryengine.plan.execution.config.sys.quota.ShowSpaceQuotaTask; import org.apache.iotdb.db.queryengine.plan.execution.config.sys.quota.ShowThrottleQuotaTask; @@ -1288,6 +1290,24 @@ public SettableFuture showClusterId() { return future; } + @Override + public SettableFuture testConnection(boolean needDetails) { + SettableFuture future = SettableFuture.create(); + try (ConfigNodeClient client = + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + TTestConnectionResp result = client.submitTestConnectionTaskToLeader(); + int configNodeNum = 0, dataNodeNum = 0; + if (!needDetails) { + configNodeNum = client.showConfigNodes().getConfigNodesInfoListSize(); + dataNodeNum = client.showDataNodes().getDataNodesInfoListSize(); + } + TestConnectionTask.buildTSBlock(result, configNodeNum, dataNodeNum, needDetails, future); + } catch (Exception e) { + future.setException(e); + } + return future; + } + @Override public SettableFuture showTTL(ShowTTLStatement showTTLStatement) { SettableFuture future = SettableFuture.create(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/IConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/IConfigTaskExecutor.java index 5adb94a56ce9..05ad88bfc5e9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/IConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/IConfigTaskExecutor.java @@ -135,6 +135,8 @@ public interface IConfigTaskExecutor { SettableFuture showClusterId(); + SettableFuture testConnection(boolean needDetails); + SettableFuture showTTL(ShowTTLStatement showTTLStatement); SettableFuture showRegion(ShowRegionStatement showRegionStatement); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/sys/TestConnectionTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/sys/TestConnectionTask.java new file mode 100644 index 000000000000..ce37a7ad44c9 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/sys/TestConnectionTask.java @@ -0,0 +1,218 @@ +/* + * 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.queryengine.plan.execution.config.sys; + +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.common.rpc.thrift.TSender; +import org.apache.iotdb.common.rpc.thrift.TServiceProvider; +import org.apache.iotdb.common.rpc.thrift.TServiceType; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResp; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResult; +import org.apache.iotdb.commons.service.ThriftService; +import org.apache.iotdb.db.queryengine.common.header.ColumnHeader; +import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant; +import org.apache.iotdb.db.queryengine.common.header.DatasetHeaderFactory; +import org.apache.iotdb.db.queryengine.plan.execution.config.ConfigTaskResult; +import org.apache.iotdb.db.queryengine.plan.execution.config.IConfigTask; +import org.apache.iotdb.db.queryengine.plan.execution.config.executor.IConfigTaskExecutor; +import org.apache.iotdb.rpc.TSStatusCode; + +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.utils.Binary; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class TestConnectionTask implements IConfigTask { + + private final boolean needDetails; + + public TestConnectionTask(boolean needDetails) { + this.needDetails = needDetails; + } + + @Override + public ListenableFuture execute(IConfigTaskExecutor configTaskExecutor) + throws InterruptedException { + return configTaskExecutor.testConnection(needDetails); + } + + public static void buildTSBlock( + TTestConnectionResp resp, + int configNodeNum, + int dataNodeNum, + boolean needDetails, + SettableFuture future) { + sortTestConnectionResp(resp); + if (!needDetails) { + Map expectedNumMap = + calculateExpectedResultNum(configNodeNum, dataNodeNum); + List newResultList = new ArrayList<>(); + for (int i = 0; i < resp.getResultListSize(); ) { + TTestConnectionResult result = resp.getResultList().get(i); + final int expectNum = expectedNumMap.get(result.getServiceProvider().getServiceType()); + final boolean allSameServiceProviderAllUp = + resp.getResultList().stream() + .skip(i) + .limit(expectNum) + .allMatch( + result1 -> + result.getServiceProvider().equals(result1.serviceProvider) + && result1.isSuccess()); + if (allSameServiceProviderAllUp) { + TTestConnectionResult allUpResult = new TTestConnectionResult(result); + allUpResult.setSender(new TSender()); + newResultList.add(allUpResult); + } else { + newResultList.addAll( + resp.getResultList().stream() + .skip(i) + .limit(expectNum) + .filter(result1 -> !result1.isSuccess()) + .collect(Collectors.toList())); + } + i += expectNum; + } + resp.setResultList(newResultList); + } + List outputDataTypes = + ColumnHeaderConstant.testConnectionColumnHeaders.stream() + .map(ColumnHeader::getColumnType) + .collect(Collectors.toList()); + TsBlockBuilder builder = new TsBlockBuilder(outputDataTypes); + int serviceProviderMaxLen = calculateServiceProviderMaxLen(resp); + int connectionMaxLen = calculateConnectionMaxLen(resp); + for (TTestConnectionResult result : resp.getResultList()) { + // ServiceProvider column + builder.getTimeColumnBuilder().writeLong(0); + StringBuilder serviceStr = + new StringBuilder(serviceProviderToString(result.getServiceProvider())); + while (serviceStr.length() < serviceProviderMaxLen) { + serviceStr.append(" "); + } + builder + .getColumnBuilder(0) + .writeBinary(new Binary(serviceStr.toString(), TSFileConfig.STRING_CHARSET)); + // Sender column + String senderStr; + if (result.getSender().isSetConfigNodeLocation()) { + senderStr = + endPointToString(result.getSender().getConfigNodeLocation().getInternalEndPoint()); + senderStr += " (ConfigNode)"; + } else if (result.getSender().isSetDataNodeLocation()) { + senderStr = + endPointToString(result.getSender().getDataNodeLocation().getInternalEndPoint()); + senderStr += " (DataNode)"; + } else { + senderStr = "All"; + } + builder.getColumnBuilder(1).writeBinary(new Binary(senderStr, TSFileConfig.STRING_CHARSET)); + // Connection column + StringBuilder connectionStatus = new StringBuilder(connectionResultToString(result)); + while (connectionStatus.length() < connectionMaxLen) { + connectionStatus.append(" "); + } + builder + .getColumnBuilder(2) + .writeBinary(new Binary(connectionStatus.toString(), TSFileConfig.STRING_CHARSET)); + builder.declarePosition(); + } + + future.set( + new ConfigTaskResult( + TSStatusCode.SUCCESS_STATUS, + builder.build(), + DatasetHeaderFactory.getTestConnectionHeader())); + } + + private static Map calculateExpectedResultNum( + int configNodeNum, int dataNodeNum) { + Map result = new HashMap<>(); + result.put(TServiceType.ConfigNodeInternalService, configNodeNum + dataNodeNum); + result.put(TServiceType.DataNodeInternalService, configNodeNum + dataNodeNum); + result.put(TServiceType.DataNodeMPPService, dataNodeNum); + result.put(TServiceType.DataNodeExternalService, dataNodeNum); + return result; + } + + private static String serviceProviderToString(TServiceProvider provider) { + String serviceStr = endPointToString(provider.getEndPoint()); + serviceStr += " (" + provider.getServiceType() + ")"; + return serviceStr; + } + + private static String connectionResultToString(TTestConnectionResult result) { + if (result.isSuccess()) { + return ThriftService.STATUS_UP; + } + return ThriftService.STATUS_DOWN + " (" + result.getReason() + ")"; + } + + private static String endPointToString(TEndPoint endPoint) { + return endPoint.getIp() + ":" + endPoint.getPort(); + } + + private static void sortTestConnectionResp(TTestConnectionResp origin) { + origin + .getResultList() + .sort( + (o1, o2) -> { + { + String serviceIp1 = o1.getServiceProvider().getEndPoint().getIp(); + String serviceIp2 = o2.getServiceProvider().getEndPoint().getIp(); + if (!serviceIp1.equals(serviceIp2)) { + return serviceIp1.compareTo(serviceIp2); + } + } + { + int servicePort1 = o1.getServiceProvider().getEndPoint().getPort(); + int servicePort2 = o2.getServiceProvider().getEndPoint().getPort(); + if (servicePort1 != servicePort2) { + return Integer.compare(servicePort1, servicePort2); + } + } + return 0; + }); + } + + private static int calculateServiceProviderMaxLen(TTestConnectionResp resp) { + return resp.getResultList().stream() + .map(TTestConnectionResult::getServiceProvider) + .map(TestConnectionTask::serviceProviderToString) + .mapToInt(String::length) + .max() + .getAsInt(); + } + + private static int calculateConnectionMaxLen(TTestConnectionResp resp) { + return resp.getResultList().stream() + .map(TestConnectionTask::connectionResultToString) + .mapToInt(String::length) + .max() + .getAsInt(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java index 68f865d362c0..1dbec56c3dbc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java @@ -202,6 +202,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowVersionStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.StartRepairDataStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.StopRepairDataStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.TestConnectionStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.quota.SetSpaceQuotaStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.quota.SetThrottleQuotaStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.quota.ShowSpaceQuotaStatement; @@ -4013,6 +4014,11 @@ public Statement visitMigrateRegion(IoTDBSqlParser.MigrateRegionContext ctx) { Integer.parseInt(ctx.toId.getText())); } + @Override + public Statement visitVerifyConnection(IoTDBSqlParser.VerifyConnectionContext ctx) { + return new TestConnectionStatement(ctx.DETAILS() != null); + } + // Quota @Override public Statement visitSetSpaceQuota(IoTDBSqlParser.SetSpaceQuotaContext ctx) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java index 248cf881de12..d8a0a42016d0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java @@ -118,6 +118,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowVersionStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.StartRepairDataStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.StopRepairDataStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.TestConnectionStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.quota.SetSpaceQuotaStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.quota.SetThrottleQuotaStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.quota.ShowSpaceQuotaStatement; @@ -214,6 +215,10 @@ public R visitShowClusterId(ShowClusterIdStatement showClusterIdStatement, C con return visitStatement(showClusterIdStatement, context); } + public R visitTestConnection(TestConnectionStatement testConnectionStatement, C context) { + return visitStatement(testConnectionStatement, context); + } + // UDF public R visitCreateFunction(CreateFunctionStatement createFunctionStatement, C context) { return visitStatement(createFunctionStatement, context); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/sys/TestConnectionStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/sys/TestConnectionStatement.java new file mode 100644 index 000000000000..2e01ea151dc2 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/sys/TestConnectionStatement.java @@ -0,0 +1,62 @@ +/* + * 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.queryengine.plan.statement.sys; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.queryengine.plan.analyze.QueryType; +import org.apache.iotdb.db.queryengine.plan.statement.IConfigStatement; +import org.apache.iotdb.db.queryengine.plan.statement.Statement; +import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor; + +import java.util.Collections; +import java.util.List; + +public class TestConnectionStatement extends Statement implements IConfigStatement { + private boolean needDetails; + + public TestConnectionStatement(boolean needDetails) { + this.needDetails = needDetails; + } + + public boolean needDetails() { + return needDetails; + } + + @Override + public QueryType getQueryType() { + return QueryType.READ; + } + + @Override + public List getPaths() { + return Collections.emptyList(); + } + + @Override + public R accept(StatementVisitor visitor, C context) { + return visitor.visitTestConnection(this, context); + } + + @Override + public TSStatus checkPermissionBeforeProcess(String userName) { + return super.checkPermissionBeforeProcess(userName); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java index a157a9eaf63a..ee550d09a74d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java @@ -20,7 +20,8 @@ package org.apache.iotdb.commons.client; import org.apache.iotdb.common.rpc.thrift.TEndPoint; -import org.apache.iotdb.commons.client.async.AsyncConfigNodeIServiceClient; +import org.apache.iotdb.commons.client.async.AsyncConfigNodeInternalServiceClient; +import org.apache.iotdb.commons.client.async.AsyncDataNodeExternalServiceClient; import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient; import org.apache.iotdb.commons.client.async.AsyncDataNodeMPPDataExchangeServiceClient; import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; @@ -67,15 +68,15 @@ public KeyedObjectPool createClientPool } } - public static class AsyncConfigNodeIServiceClientPoolFactory - implements IClientPoolFactory { + public static class AsyncConfigNodeInternalServiceClientPoolFactory + implements IClientPoolFactory { @Override - public KeyedObjectPool createClientPool( - ClientManager manager) { - GenericKeyedObjectPool clientPool = + public KeyedObjectPool createClientPool( + ClientManager manager) { + GenericKeyedObjectPool clientPool = new GenericKeyedObjectPool<>( - new AsyncConfigNodeIServiceClient.Factory( + new AsyncConfigNodeInternalServiceClient.Factory( manager, new ThriftClientProperty.Builder() .setConnectionTimeoutMs(conf.getConnectionTimeoutInMS()) @@ -83,7 +84,9 @@ public KeyedObjectPool createClientPoo .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager()) .build(), ThreadName.ASYNC_CONFIGNODE_CLIENT_POOL.getName()), - new ClientPoolProperty.Builder().build().getConfig()); + new ClientPoolProperty.Builder() + .build() + .getConfig()); ClientManagerMetrics.getInstance() .registerClientManager(this.getClass().getSimpleName(), clientPool); return clientPool; @@ -138,16 +141,41 @@ public KeyedObjectPool createClie } } + public static class AsyncDataNodeExternalServiceClientPoolFactory + implements IClientPoolFactory { + + @Override + public KeyedObjectPool createClientPool( + ClientManager manager) { + GenericKeyedObjectPool clientPool = + new GenericKeyedObjectPool<>( + new AsyncDataNodeExternalServiceClient.Factory( + manager, + new ThriftClientProperty.Builder() + .setConnectionTimeoutMs(conf.getConnectionTimeoutInMS()) + .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled()) + .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager()) + .build(), + ThreadName.ASYNC_DATANODE_CLIENT_POOL.getName()), + new ClientPoolProperty.Builder() + .build() + .getConfig()); + ClientManagerMetrics.getInstance() + .registerClientManager(this.getClass().getSimpleName(), clientPool); + return clientPool; + } + } + public static class AsyncConfigNodeHeartbeatServiceClientPoolFactory - implements IClientPoolFactory { + implements IClientPoolFactory { @Override - public KeyedObjectPool createClientPool( - ClientManager manager) { + public KeyedObjectPool createClientPool( + ClientManager manager) { - GenericKeyedObjectPool clientPool = + GenericKeyedObjectPool clientPool = new GenericKeyedObjectPool<>( - new AsyncConfigNodeIServiceClient.Factory( + new AsyncConfigNodeInternalServiceClient.Factory( manager, new ThriftClientProperty.Builder() .setConnectionTimeoutMs(conf.getConnectionTimeoutInMS()) @@ -156,7 +184,9 @@ public KeyedObjectPool createClientPoo .setPrintLogWhenEncounterException(false) .build(), ThreadName.ASYNC_CONFIGNODE_HEARTBEAT_CLIENT_POOL.getName()), - new ClientPoolProperty.Builder().build().getConfig()); + new ClientPoolProperty.Builder() + .build() + .getConfig()); ClientManagerMetrics.getInstance() .registerClientManager(this.getClass().getSimpleName(), clientPool); return clientPool; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncConfigNodeIServiceClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncConfigNodeInternalServiceClient.java similarity index 83% rename from iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncConfigNodeIServiceClient.java rename to iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncConfigNodeInternalServiceClient.java index 36d273c7b782..b5adc9611360 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncConfigNodeIServiceClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncConfigNodeInternalServiceClient.java @@ -36,20 +36,21 @@ import java.io.IOException; -public class AsyncConfigNodeIServiceClient extends IConfigNodeRPCService.AsyncClient +public class AsyncConfigNodeInternalServiceClient extends IConfigNodeRPCService.AsyncClient implements ThriftClient { - private static final Logger logger = LoggerFactory.getLogger(AsyncConfigNodeIServiceClient.class); + private static final Logger logger = + LoggerFactory.getLogger(AsyncConfigNodeInternalServiceClient.class); private final boolean printLogWhenEncounterException; private final TEndPoint endpoint; - private final ClientManager clientManager; + private final ClientManager clientManager; - public AsyncConfigNodeIServiceClient( + public AsyncConfigNodeInternalServiceClient( ThriftClientProperty property, TEndPoint endpoint, TAsyncClientManager tClientManager, - ClientManager clientManager) + ClientManager clientManager) throws IOException { super( property.getProtocolFactory(), @@ -126,10 +127,10 @@ public String toString() { } public static class Factory - extends AsyncThriftClientFactory { + extends AsyncThriftClientFactory { public Factory( - ClientManager clientManager, + ClientManager clientManager, ThriftClientProperty thriftClientProperty, String threadName) { super(clientManager, thriftClientProperty, threadName); @@ -137,15 +138,15 @@ public Factory( @Override public void destroyObject( - TEndPoint endPoint, PooledObject pooledObject) { + TEndPoint endPoint, PooledObject pooledObject) { pooledObject.getObject().close(); } @Override - public PooledObject makeObject(TEndPoint endPoint) + public PooledObject makeObject(TEndPoint endPoint) throws Exception { return new DefaultPooledObject<>( - new AsyncConfigNodeIServiceClient( + new AsyncConfigNodeInternalServiceClient( thriftClientProperty, endPoint, tManagers[clientCnt.incrementAndGet() % tManagers.length], @@ -154,7 +155,7 @@ public PooledObject makeObject(TEndPoint endPoint @Override public boolean validateObject( - TEndPoint endPoint, PooledObject pooledObject) { + TEndPoint endPoint, PooledObject pooledObject) { return pooledObject.getObject().isReady(); } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncDataNodeExternalServiceClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncDataNodeExternalServiceClient.java new file mode 100644 index 000000000000..5de58b8eef14 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncDataNodeExternalServiceClient.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.client.async; + +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.client.ClientManager; +import org.apache.iotdb.commons.client.ThriftClient; +import org.apache.iotdb.commons.client.factory.AsyncThriftClientFactory; +import org.apache.iotdb.commons.client.property.ThriftClientProperty; +import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.mpp.rpc.thrift.IDataNodeRPCService; +import org.apache.iotdb.rpc.TNonblockingSocketWrapper; + +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.commons.pool2.PooledObject; +import org.apache.commons.pool2.impl.DefaultPooledObject; +import org.apache.thrift.async.TAsyncClientManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public class AsyncDataNodeExternalServiceClient extends IDataNodeRPCService.AsyncClient + implements ThriftClient { + + private static final Logger logger = + LoggerFactory.getLogger(AsyncDataNodeExternalServiceClient.class); + + private final boolean printLogWhenEncounterException; + + private final TEndPoint endpoint; + private final ClientManager clientManager; + + public AsyncDataNodeExternalServiceClient( + ThriftClientProperty property, + TEndPoint endpoint, + TAsyncClientManager tClientManager, + ClientManager clientManager) + throws IOException { + super( + property.getProtocolFactory(), + tClientManager, + TNonblockingSocketWrapper.wrap( + endpoint.getIp(), endpoint.getPort(), property.getConnectionTimeoutMs())); + setTimeout(property.getConnectionTimeoutMs()); + this.printLogWhenEncounterException = property.isPrintLogWhenEncounterException(); + this.endpoint = endpoint; + this.clientManager = clientManager; + } + + @TestOnly + public TEndPoint getTEndpoint() { + return endpoint; + } + + @TestOnly + public ClientManager getClientManager() { + return clientManager; + } + + @Override + public void onComplete() { + super.onComplete(); + returnSelf(); + } + + @Override + public void onError(Exception e) { + super.onError(e); + ThriftClient.resolveException(e, this); + returnSelf(); + } + + @Override + public void invalidate() { + if (!hasError()) { + super.onError(new Exception("This client has been invalidated")); + } + } + + @Override + public void invalidateAll() { + clientManager.clear(endpoint); + } + + @Override + public boolean printLogWhenEncounterException() { + return printLogWhenEncounterException; + } + + /** + * return self, the method doesn't need to be called by the user and will be triggered after the + * RPC is finished. + */ + private void returnSelf() { + clientManager.returnClient(endpoint, this); + } + + private void close() { + ___transport.close(); + ___currentMethod = null; + } + + public boolean isReady() { + try { + checkReady(); + return true; + } catch (Exception e) { + if (printLogWhenEncounterException) { + logger.error( + "Unexpected exception occurs in {}, error msg is {}", + this, + ExceptionUtils.getRootCause(e).toString()); + } + return false; + } + } + + @Override + public String toString() { + return String.format("AsyncDataNodeInternalServiceClient{%s}", endpoint); + } + + public static class Factory + extends AsyncThriftClientFactory { + + public Factory( + ClientManager clientManager, + ThriftClientProperty thriftClientProperty, + String threadName) { + super(clientManager, thriftClientProperty, threadName); + } + + @Override + public void destroyObject( + TEndPoint endPoint, PooledObject pooledObject) { + pooledObject.getObject().close(); + } + + @Override + public PooledObject makeObject(TEndPoint endPoint) + throws Exception { + return new DefaultPooledObject<>( + new AsyncDataNodeExternalServiceClient( + thriftClientProperty, + endPoint, + tManagers[clientCnt.incrementAndGet() % tManagers.length], + clientManager)); + } + + @Override + public boolean validateObject( + TEndPoint endPoint, PooledObject pooledObject) { + return pooledObject.getObject().isReady(); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestContext.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestContext.java new file mode 100644 index 000000000000..8b54072a4c40 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestContext.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.client.request; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; + +/** + * Asynchronous Client handler. + * + * @param ClassName of RPC request + * @param ClassName of RPC response + */ +public class AsyncRequestContext { + + // Type of RPC request + protected final RequestType requestType; + + /** + * Map key: The indices of asynchronous RPC requests. + * + *

Map value: The corresponding RPC request + */ + private final Map requestMap; + + /** + * Map key: The indices of asynchronous RPC requests. + * + *

Map value: The target Nodes of corresponding indices + * + *

All kinds of AsyncHandler will remove its targetNode from the nodeLocationMap only if its + * corresponding RPC request success + */ + private final Map nodeLocationMap; + + /** + * Map key: The indices(targetNode's ID) of asynchronous RPC requests. + * + *

Map value: The response of corresponding indices + * + *

All kinds of AsyncHandler will add response to the responseMap after its corresponding RPC + * request finished + */ + private final Map responseMap; + + private CountDownLatch countDownLatch; + + /** Custom constructor. */ + public AsyncRequestContext(RequestType requestType) { + this.requestType = requestType; + this.requestMap = new ConcurrentHashMap<>(); + this.nodeLocationMap = new ConcurrentHashMap<>(); + this.responseMap = new ConcurrentHashMap<>(); + } + + public void putRequest(int requestId, Request request) { + requestMap.put(requestId, request); + } + + public void putNodeLocation(int requestId, NodeLocation nodeLocation) { + nodeLocationMap.put(requestId, nodeLocation); + } + + /** Constructor for null requests. */ + public AsyncRequestContext(RequestType requestType, Map nodeLocationMap) { + this.requestType = requestType; + this.nodeLocationMap = nodeLocationMap; + this.requestMap = new ConcurrentHashMap<>(); + this.responseMap = new ConcurrentHashMap<>(); + } + + /** Constructor for unique request. */ + public AsyncRequestContext( + RequestType requestType, Request request, Map nodeLocationMap) { + this.requestType = requestType; + this.nodeLocationMap = nodeLocationMap; + this.requestMap = new ConcurrentHashMap<>(); + this.nodeLocationMap.keySet().forEach(nodeId -> this.requestMap.put(nodeId, request)); + this.responseMap = new ConcurrentHashMap<>(); + } + + public RequestType getRequestType() { + return requestType; + } + + public List getRequestIndices() { + return new ArrayList<>(nodeLocationMap.keySet()); + } + + public Request getRequest(int requestId) { + return requestMap.get(requestId); + } + + public NodeLocation getNodeLocation(int requestId) { + return nodeLocationMap.get(requestId); + } + + public Map getNodeLocationMap() { + return nodeLocationMap; + } + + public List getResponseList() { + return new ArrayList<>(responseMap.values()); + } + + public Map getResponseMap() { + return responseMap; + } + + /** Always reset CountDownLatch before retry. */ + public void resetCountDownLatch() { + countDownLatch = new CountDownLatch(nodeLocationMap.size()); + } + + public CountDownLatch getCountDownLatch() { + return countDownLatch; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestManager.java new file mode 100644 index 000000000000..ed3feb6598b8 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestManager.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.client.request; + +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.client.exception.ClientManagerException; +import org.apache.iotdb.commons.utils.function.CheckedTriConsumer; + +import com.google.common.collect.ImmutableMap; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; +import java.util.concurrent.TimeUnit; + +/** Asynchronously send RPC requests to Nodes. See queryengine.thrift for more details. */ +public abstract class AsyncRequestManager { + + private static final Logger LOGGER = LoggerFactory.getLogger(AsyncRequestManager.class); + + protected IClientManager clientManager; + + protected ImmutableMap< + RequestType, + CheckedTriConsumer< + Object, Client, AsyncRequestRPCHandler, TException>> + actionMap; + + protected ImmutableMap.Builder< + RequestType, + CheckedTriConsumer< + Object, Client, AsyncRequestRPCHandler, TException>> + actionMapBuilder; + + private static final int MAX_RETRY_NUM = 6; + + protected AsyncRequestManager() { + initClientManager(); + actionMapBuilder = ImmutableMap.builder(); + initActionMapBuilder(); + this.actionMap = this.actionMapBuilder.build(); + } + + protected abstract void initClientManager(); + + protected abstract void initActionMapBuilder(); + + /** + * Send asynchronous requests to the specified Nodes with default retry num + * + *

Notice: The Nodes that failed to receive the requests will be reconnected + * + * @param requestContext which will also contain the result + * @param timeoutInMs timeout in milliseconds + */ + public void sendAsyncRequestToNodeWithRetryAndTimeoutInMs( + AsyncRequestContext requestContext, long timeoutInMs) { + sendAsyncRequest(requestContext, MAX_RETRY_NUM, timeoutInMs); + } + + /** + * Send asynchronous requests to the specified Nodes with default retry num + * + *

Notice: The Nodes that failed to receive the requests will be reconnected + * + * @param requestContext which will also contain the result + */ + public final void sendAsyncRequestWithRetry( + AsyncRequestContext requestContext) { + sendAsyncRequest(requestContext, MAX_RETRY_NUM, null); + } + + public final void sendAsyncRequest( + AsyncRequestContext requestContext) { + sendAsyncRequest(requestContext, 1, null); + } + + private void sendAsyncRequest( + AsyncRequestContext requestContext, + int retryNum, + Long timeoutInMs) { + if (requestContext.getRequestIndices().isEmpty()) { + return; + } + + RequestType requestType = requestContext.getRequestType(); + for (int retry = 0; retry < retryNum; retry++) { + // Always Reset CountDownLatch first + requestContext.resetCountDownLatch(); + + // Send requests to all targetNodes + for (int requestId : requestContext.getRequestIndices()) { + NodeLocation targetNode = requestContext.getNodeLocation(requestId); + sendAsyncRequest(requestContext, requestId, targetNode, retry); + } + + // Wait for this batch of asynchronous RPC requests finish + try { + if (timeoutInMs == null) { + requestContext.getCountDownLatch().await(); + } else { + if (!requestContext.getCountDownLatch().await(timeoutInMs, TimeUnit.MILLISECONDS)) { + LOGGER.warn( + "Timeout during {} on ConfigNode. Retry: {}/{}", requestType, retry, retryNum); + } + } + } catch (InterruptedException e) { + LOGGER.error( + "Interrupted during {} on ConfigNode. Retry: {}/{}", requestType, retry, retryNum); + Thread.currentThread().interrupt(); + } + + // Check if there is a Node that fails to execute the request, and retry if there exists + if (requestContext.getRequestIndices().isEmpty()) { + return; + } + } + + if (!requestContext.getRequestIndices().isEmpty()) { + LOGGER.warn( + "Failed to {} on ConfigNode after {} retries, requestIndices: {}", + requestType, + retryNum, + requestContext.getRequestIndices()); + } + } + + protected void sendAsyncRequest( + AsyncRequestContext requestContext, + int requestId, + NodeLocation targetNode, + int retryCount) { + try { + if (!actionMap.containsKey(requestContext.getRequestType())) { + throw new UnsupportedOperationException( + "unsupported request type: " + requestContext.getRequestType()); + } + Client client = clientManager.borrowClient(nodeLocationToEndPoint(targetNode)); + Object req = requestContext.getRequest(requestId); + AsyncRequestRPCHandler handler = + buildHandler(requestContext, requestId, targetNode); + Objects.requireNonNull(actionMap.get(requestContext.getRequestType())) + .accept(req, client, handler); + } catch (Exception e) { + LOGGER.warn( + "{} failed on Node {}, because {}, retrying {}...", + requestContext.getRequestType(), + nodeLocationToEndPoint(targetNode), + e.getMessage(), + retryCount); + } + } + + protected abstract TEndPoint nodeLocationToEndPoint(NodeLocation location); + + protected abstract AsyncRequestRPCHandler buildHandler( + AsyncRequestContext requestContext, + int requestId, + NodeLocation targetNode); + + /** + * Always call this interface when a Node is restarted or removed. + * + * @param endPoint The specific Node + */ + public void resetClient(TEndPoint endPoint) { + clientManager.clear(endPoint); + } + + public Client getAsyncClient(NodeLocation targetNode) throws ClientManagerException { + return clientManager.borrowClient(nodeLocationToEndPoint(targetNode)); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/AbstractAsyncRPCHandler.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestRPCHandler.java similarity index 56% rename from iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/AbstractAsyncRPCHandler.java rename to iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestRPCHandler.java index 387359d7ee29..136f4e126f22 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/AbstractAsyncRPCHandler.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestRPCHandler.java @@ -17,69 +17,64 @@ * under the License. */ -package org.apache.iotdb.confignode.client.async.handlers.rpc; - -import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; -import org.apache.iotdb.confignode.client.DataNodeRequestType; +package org.apache.iotdb.commons.client.request; import org.apache.thrift.async.AsyncMethodCallback; import java.util.Map; import java.util.concurrent.CountDownLatch; -public abstract class AbstractAsyncRPCHandler implements AsyncMethodCallback { +public abstract class AsyncRequestRPCHandler + implements AsyncMethodCallback { // Type of RPC request - protected final DataNodeRequestType requestType; + protected final RequestType requestType; // Index of request protected final int requestId; - // Target DataNode - protected final TDataNodeLocation targetDataNode; + // Target Node + protected final NodeLocation targetNode; /** * Map key: The indices of asynchronous RPC requests. * - *

Map value: The target DataNodes of corresponding indices + *

Map value: The target Nodes of corresponding indices * - *

All kinds of AsyncHandler will remove its targetDataNode from the dataNodeLocationMap only - * if its corresponding RPC request success + *

All kinds of AsyncHandler will remove its targetNode from the nodeLocationMap only if its + * corresponding RPC request success */ - protected final Map dataNodeLocationMap; + protected final Map nodeLocationMap; /** - * Map key: The indices(targetDataNode's ID) of asynchronous RPC requests. + * Map key: The indices(targetNode's ID) of asynchronous RPC requests. * *

Map value: The response of corresponding indices * *

All kinds of AsyncHandler will add response to the responseMap after its corresponding RPC * request finished */ - protected final Map responseMap; + protected final Map responseMap; // All kinds of AsyncHandler will invoke countDown after its corresponding RPC request finished protected final CountDownLatch countDownLatch; protected final String formattedTargetLocation; - protected AbstractAsyncRPCHandler( - DataNodeRequestType requestType, + protected AsyncRequestRPCHandler( + RequestType requestType, int requestId, - TDataNodeLocation targetDataNode, - Map dataNodeLocationMap, - Map responseMap, + NodeLocation targetNode, + Map nodeLocationMap, + Map responseMap, CountDownLatch countDownLatch) { this.requestType = requestType; this.requestId = requestId; - this.targetDataNode = targetDataNode; - this.formattedTargetLocation = - "{id=" - + targetDataNode.getDataNodeId() - + ", internalEndPoint=" - + targetDataNode.getInternalEndPoint() - + "}"; + this.targetNode = targetNode; + this.formattedTargetLocation = generateFormattedTargetLocation(targetNode); - this.dataNodeLocationMap = dataNodeLocationMap; + this.nodeLocationMap = nodeLocationMap; this.responseMap = responseMap; this.countDownLatch = countDownLatch; } + + protected abstract String generateFormattedTargetLocation(NodeLocation location); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/ConfigNodeInternalServiceAsyncRequestManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/ConfigNodeInternalServiceAsyncRequestManager.java new file mode 100644 index 000000000000..791a1e5df0e9 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/ConfigNodeInternalServiceAsyncRequestManager.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.client.request; + +import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.client.ClientPoolFactory; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.client.async.AsyncConfigNodeInternalServiceClient; + +public abstract class ConfigNodeInternalServiceAsyncRequestManager + extends AsyncRequestManager< + RequestType, TConfigNodeLocation, AsyncConfigNodeInternalServiceClient> { + @Override + protected void initClientManager() { + clientManager = + new IClientManager.Factory() + .createClientManager( + new ClientPoolFactory.AsyncConfigNodeInternalServiceClientPoolFactory()); + } + + @Override + protected TEndPoint nodeLocationToEndPoint(TConfigNodeLocation configNodeLocation) { + return configNodeLocation.getInternalEndPoint(); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/DataNodeInternalServiceRequestManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/DataNodeInternalServiceRequestManager.java new file mode 100644 index 000000000000..fcb1b01857df --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/DataNodeInternalServiceRequestManager.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.client.request; + +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.client.ClientPoolFactory; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient; + +public abstract class DataNodeInternalServiceRequestManager + extends AsyncRequestManager< + RequestType, TDataNodeLocation, AsyncDataNodeInternalServiceClient> { + @Override + protected void initClientManager() { + clientManager = + new IClientManager.Factory() + .createClientManager( + new ClientPoolFactory.AsyncDataNodeInternalServiceClientPoolFactory()); + } + + @Override + protected TEndPoint nodeLocationToEndPoint(TDataNodeLocation dataNodeLocation) { + return dataNodeLocation.getInternalEndPoint(); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/Utils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/Utils.java new file mode 100644 index 000000000000..6d4704f059b8 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/Utils.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.client.request; + +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.common.rpc.thrift.TSender; +import org.apache.iotdb.common.rpc.thrift.TServiceProvider; +import org.apache.iotdb.common.rpc.thrift.TServiceType; +import org.apache.iotdb.common.rpc.thrift.TTestConnectionResult; +import org.apache.iotdb.rpc.TSStatusCode; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class Utils { + public static + List testConnectionsImpl( + List nodeLocations, + TSender sender, + Function getId, + Function getEndPoint, + TServiceType serviceType, + RequestType requestType, + Consumer> + sendRequest) { + // prepare request context + Map nodeLocationMap = + nodeLocations.stream().collect(Collectors.toMap(getId, location -> location)); + AsyncRequestContext requestContext = + new AsyncRequestContext<>(requestType, new Object(), nodeLocationMap); + // do the test + sendRequest.accept(requestContext); + // collect result + Map anotherNodeLocationMap = + nodeLocations.stream().collect(Collectors.toMap(getId, location -> location)); + List results = new ArrayList<>(); + requestContext + .getResponseMap() + .forEach( + (nodeId, status) -> { + TEndPoint endPoint = getEndPoint.apply(anotherNodeLocationMap.get(nodeId)); + TServiceProvider serviceProvider = new TServiceProvider(endPoint, serviceType); + TTestConnectionResult result = new TTestConnectionResult(); + result.setSender(sender); + result.setServiceProvider(serviceProvider); + if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + result.setSuccess(true); + } else { + result.setSuccess(false); + result.setReason(status.getMessage()); + } + results.add(result); + }); + return results; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/function/CheckedTriConsumer.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/function/CheckedTriConsumer.java new file mode 100644 index 000000000000..bb9a5ebaaab1 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/function/CheckedTriConsumer.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.utils.function; + +@FunctionalInterface +public interface CheckedTriConsumer { + void accept(INPUT1 var1, INPUT2 var2, INPUT3 var3) throws THROWABLE; +} diff --git a/iotdb-protocol/thrift-commons/src/main/thrift/common.thrift b/iotdb-protocol/thrift-commons/src/main/thrift/common.thrift index 49dc50d94c47..495a7fc6a67b 100644 --- a/iotdb-protocol/thrift-commons/src/main/thrift/common.thrift +++ b/iotdb-protocol/thrift-commons/src/main/thrift/common.thrift @@ -191,6 +191,40 @@ struct TLicense { 9: required i16 mlNodeNumLimit } +enum TServiceType { + ConfigNodeInternalService, + DataNodeInternalService, + DataNodeMPPService, + DataNodeExternalService, +} + +struct TServiceProvider { + 1: required TEndPoint endPoint + 2: required TServiceType serviceType +} + +struct TSender { + 1: optional TDataNodeLocation dataNodeLocation + 2: optional TConfigNodeLocation configNodeLocation +} + +struct TTestConnectionResult { + 1: required TServiceProvider serviceProvider + 2: required TSender sender + 3: required bool success + 4: optional string reason +} + +struct TTestConnectionResp { + 1: required TSStatus status + 2: required list resultList +} + +struct TNodeLocations { + 1: optional list configNodeLocations + 2: optional list dataNodeLocations +} + enum TAggregationType { COUNT, AVG, diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index ad75ba17c1d3..e11cf7540fa8 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -1376,6 +1376,14 @@ service IConfigNodeRPCService { /** Show cluster Databases' information */ TShowDatabaseResp showDatabase(TGetDatabaseReq req) + /** Test connection of every node in the cluster */ + common.TTestConnectionResp submitTestConnectionTask(common.TNodeLocations nodeLocations) + + common.TTestConnectionResp submitTestConnectionTaskToLeader() + + /** Empty rpc, only for connection test */ + common.TSStatus testConnectionEmptyRPC() + /** * Show the matched cluster Regions' information * See https://apache-iotdb.feishu.cn/docx/doxcnOzmIlaE2MX5tKjmYWuMSRg for detailed matching rules diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift index 6069199bd64c..917a9a579002 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift @@ -659,4 +659,7 @@ service IClientRPCService { TSBackupConfigurationResp getBackupConfiguration(); TSConnectionInfoResp fetchAllConnectionsInfo(); + + /** For other node's call */ + common.TSStatus testConnectionEmptyRPC() } \ No newline at end of file diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift index d4ef59eee2b7..60e605a6a34a 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift @@ -999,6 +999,11 @@ service IDataNodeRPCService { * Fetch fragment instance statistics for EXPLAIN ANALYZE */ TFetchFragmentInstanceStatisticsResp fetchFragmentInstanceStatistics(TFetchFragmentInstanceStatisticsReq req) + + common.TTestConnectionResp submitTestConnectionTask(common.TNodeLocations nodeLocations) + + /** Empty rpc, only for connection test */ + common.TSStatus testConnectionEmptyRPC() } service MPPDataExchangeService { @@ -1011,4 +1016,7 @@ service MPPDataExchangeService { void onNewDataBlockEvent(TNewDataBlockEvent e); void onEndOfDataBlockEvent(TEndOfDataBlockEvent e); + + /** Empty rpc, only for connection test */ + common.TSStatus testConnectionEmptyRPC() }