From c790c03bd8f71ada53ab893956944efaa5d3b601 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <78788603+Pengzna@users.noreply.github.com> Date: Fri, 31 May 2024 09:12:17 -0500 Subject: [PATCH] PipeConsensus: complete consensus prodedure and pipe components with new thrift service (#12355) Co-authored-by: yschengzi <yscysc99@126.com> Co-authored-by: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Co-authored-by: Steve Yurong Su <rong@apache.org> Co-authored-by: OneSizeFitQuorum <tanxinyu@apache.org> --- .../iotdb/it/env/cluster/ClusterConstant.java | 3 + .../apache/iotdb/it/env/cluster/EnvUtils.java | 18 + .../autocreate/IoTDBPipeAutoConflictIT.java | 33 + .../org/apache/iotdb/rpc/TSStatusCode.java | 8 + .../conf/ConfigNodeStartupCheck.java | 12 + .../manager/load/balancer/RouteBalancer.java | 9 +- .../persistence/pipe/PipeTaskInfo.java | 5 + .../impl/pipe/task/CreatePipeProcedureV2.java | 79 +- iotdb-core/consensus/pom.xml | 9 +- .../iotdb/consensus/ConsensusFactory.java | 22 + .../consensus/config/ConsensusConfig.java | 19 +- .../consensus/config/PipeConsensusConfig.java | 358 +++++ .../iotdb/consensus/iot/IoTConsensus.java | 1 + .../AsyncIoTConsensusServiceClient.java | 2 +- .../iotdb/consensus/pipe/PipeConsensus.java | 465 ++++++ .../pipe/PipeConsensusPeerManager.java | 154 ++ .../pipe/PipeConsensusServerImpl.java | 543 +++++++ .../ConsensusPipeDispatcher.java | 41 + .../consensuspipe/ConsensusPipeGuardian.java | 26 + .../consensuspipe/ConsensusPipeManager.java | 108 ++ .../pipe/consensuspipe/ConsensusPipeName.java | 98 ++ .../consensuspipe/ConsensusPipeReceiver.java | 29 + .../consensuspipe/ConsensusPipeSelector.java | 28 + .../consensuspipe/ProgressIndexManager.java | 31 + .../pipe/service/PipeConsensusRPCService.java | 97 ++ .../PipeConsensusRPCServiceHandler.java | 51 + .../service/PipeConsensusRPCServiceMBean.java | 22 + .../PipeConsensusRPCServiceProcessor.java | 210 +++ iotdb-core/datanode/pom.xml | 5 + .../org/apache/iotdb/db/conf/IoTDBConfig.java | 35 + .../apache/iotdb/db/conf/IoTDBDescriptor.java | 24 + .../db/consensus/DataRegionConsensusImpl.java | 38 + .../PipeDataRegionConnectorConstructor.java | 7 + .../PipeDataRegionProcessorConstructor.java | 4 + .../receiver/PipeDataNodeReceiverAgent.java | 13 + .../runtime/PipeDataNodeRuntimeAgent.java | 32 +- .../agent/task/PipeDataNodeTaskAgent.java | 47 + .../PipeConsensusAsyncConnector.java | 524 +++++++ .../PipeConsensusSyncConnector.java | 455 ++++++ .../PipeConsensusTabletBatchEventHandler.java | 128 ++ ...ConsensusTabletInsertNodeEventHandler.java | 45 + ...eConsensusTabletInsertionEventHandler.java | 109 ++ .../PipeConsensusTabletRawEventHandler.java | 45 + ...eConsensusTsFileInsertionEventHandler.java | 279 ++++ .../PipeConsensusAsyncBatchReqBuilder.java | 37 + .../PipeConsensusSyncBatchReqBuilder.java | 45 + .../PipeConsensusTransferBatchReqBuilder.java | 218 +++ .../request/PipeConsensusTabletBatchReq.java | 83 ++ .../request/PipeConsensusTabletBinaryReq.java | 126 ++ .../PipeConsensusTabletInsertNodeReq.java | 154 ++ .../request/PipeConsensusTsFilePieceReq.java | 79 ++ .../PipeConsensusTsFilePieceWithModReq.java | 80 ++ .../request/PipeConsensusTsFileSealReq.java | 74 + .../PipeConsensusTsFileSealWithModReq.java | 85 ++ .../ConsensusPipeDataNodeDispatcher.java | 121 ++ ...ensusPipeDataNodeRuntimeAgentGuardian.java | 48 + .../PipeConsensusSubtaskExecutor.java | 32 + .../ProgressIndexDataNodeManager.java | 128 ++ .../execution/PipeSubtaskExecutorManager.java | 7 + .../pipeconsensus/PipeConsensusProcessor.java | 103 ++ ....java => SimpleProgressIndexAssigner.java} | 11 +- .../pipeconsensus/PipeConsensusReceiver.java | 1244 +++++++++++++++++ .../PipeConsensusReceiverAgent.java | 169 +++ .../task/builder/PipeDataNodeTaskBuilder.java | 2 + .../task/connection/PipeEventCollector.java | 4 + iotdb-core/node-commons/pom.xml | 5 + .../resources/conf/iotdb-system.properties | 21 + .../resources/sbin/destroy-datanode.bat | 34 +- .../resources/sbin/destroy-datanode.sh | 2 + .../commons/client/ClientPoolFactory.java | 72 + .../AsyncPipeConsensusServiceClient.java | 171 +++ .../PipeConsensusClientMgrContainer.java | 91 ++ .../property/PipeConsensusClientProperty.java | 99 ++ .../sync/SyncPipeConsensusServiceClient.java | 139 ++ .../iotdb/commons/concurrent/ThreadName.java | 14 + .../commons/consensus/ConsensusGroupId.java | 29 + .../index/impl/HybridProgressIndex.java | 4 + .../index/impl/RecoverProgressIndex.java | 4 + .../constant/PipeConnectorConstant.java | 2 + .../constant/PipeExtractorConstant.java | 7 + .../request/PipeConsensusRequestType.java | 67 + .../request/PipeConsensusRequestVersion.java | 35 + .../PipeConsensusTransferFilePieceReq.java | 139 ++ .../PipeConsensusTransferFileSealReq.java | 138 ++ ...peConsensusTransferFileSealWithModReq.java | 177 +++ .../PipeConsensusTransferFilePieceResp.java | 110 ++ .../commons/pipe/event/EnrichedEvent.java | 27 + .../plugin/builtin/BuiltinPipePlugin.java | 7 + .../PipeConsensusAsyncConnector.java | 30 + .../pipeconsensus/PipeConsensusProcessor.java | 30 + .../pipe/task/meta/PipeStaticMeta.java | 1 + .../commons/pipe/task/meta/PipeType.java | 3 + .../iotdb/commons/service/ServiceType.java | 1 + .../src/main/thrift/pipeconsensus.thrift | 109 ++ 94 files changed, 8635 insertions(+), 54 deletions(-) create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/PipeConsensusConfig.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensus.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusPeerManager.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusServerImpl.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeDispatcher.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeGuardian.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeName.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeReceiver.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeSelector.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ProgressIndexManager.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCService.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCServiceHandler.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCServiceMBean.java create mode 100644 iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCServiceProcessor.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletBatchEventHandler.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertNodeEventHandler.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertionEventHandler.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletRawEventHandler.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTsFileInsertionEventHandler.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusAsyncBatchReqBuilder.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusSyncBatchReqBuilder.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusTransferBatchReqBuilder.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletBatchReq.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletBinaryReq.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletInsertNodeReq.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFilePieceReq.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFilePieceWithModReq.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFileSealReq.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFileSealWithModReq.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeDispatcher.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeRuntimeAgentGuardian.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusSubtaskExecutor.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/pipeconsensus/PipeConsensusProcessor.java rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/progress/{SimpleConsensusProgressIndexAssigner.java => SimpleProgressIndexAssigner.java} (91%) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiverAgent.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeConsensusServiceClient.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/container/PipeConsensusClientMgrContainer.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/property/PipeConsensusClientProperty.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/sync/SyncPipeConsensusServiceClient.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusRequestType.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusRequestVersion.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusTransferFilePieceReq.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusTransferFileSealReq.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusTransferFileSealWithModReq.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/response/PipeConsensusTransferFilePieceResp.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/connector/iotdb/consensus/PipeConsensusAsyncConnector.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/pipeconsensus/PipeConsensusProcessor.java create mode 100644 iotdb-protocol/thrift-consensus/src/main/thrift/pipeconsensus.thrift diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/ClusterConstant.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/ClusterConstant.java index b1eff829299e..d688f2169d5d 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/ClusterConstant.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/ClusterConstant.java @@ -210,6 +210,9 @@ public class ClusterConstant { public static final String SIMPLE_CONSENSUS_STR = "Simple"; public static final String RATIS_CONSENSUS_STR = "Ratis"; public static final String IOT_CONSENSUS_STR = "IoT"; + public static final String PIPE_CONSENSUS_STR = "Pipe"; + public static final String STREAM_CONSENSUS_STR = "Stream"; + public static final String BATCH_CONSENSUS_STR = "Batch"; public static final String JAVA_CMD = System.getProperty("java.home") diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/EnvUtils.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/EnvUtils.java index 8b08edc5bfeb..6ecb87c3f042 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/EnvUtils.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/EnvUtils.java @@ -30,10 +30,14 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.apache.iotdb.consensus.ConsensusFactory.FAST_IOT_CONSENSUS; +import static org.apache.iotdb.consensus.ConsensusFactory.IOTV2_CONSENSUS; import static org.apache.iotdb.consensus.ConsensusFactory.IOT_CONSENSUS; import static org.apache.iotdb.consensus.ConsensusFactory.RATIS_CONSENSUS; +import static org.apache.iotdb.consensus.ConsensusFactory.REAL_PIPE_CONSENSUS; import static org.apache.iotdb.consensus.ConsensusFactory.SIMPLE_CONSENSUS; import static org.apache.iotdb.db.utils.DateTimeUtils.convertLongToDate; +import static org.apache.iotdb.it.env.cluster.ClusterConstant.BATCH_CONSENSUS_STR; import static org.apache.iotdb.it.env.cluster.ClusterConstant.CLUSTER_CONFIGURATIONS; import static org.apache.iotdb.it.env.cluster.ClusterConstant.DEFAULT_CONFIG_NODE_NUM; import static org.apache.iotdb.it.env.cluster.ClusterConstant.DEFAULT_DATA_NODE_NUM; @@ -47,11 +51,13 @@ import static org.apache.iotdb.it.env.cluster.ClusterConstant.LIGHT_WEIGHT_STANDALONE_MODE_CONFIG_NODE_NUM; import static org.apache.iotdb.it.env.cluster.ClusterConstant.LIGHT_WEIGHT_STANDALONE_MODE_DATA_NODE_NUM; import static org.apache.iotdb.it.env.cluster.ClusterConstant.LOCK_FILE_PATH; +import static org.apache.iotdb.it.env.cluster.ClusterConstant.PIPE_CONSENSUS_STR; import static org.apache.iotdb.it.env.cluster.ClusterConstant.RATIS_CONSENSUS_STR; import static org.apache.iotdb.it.env.cluster.ClusterConstant.SCALABLE_SINGLE_NODE_MODE; import static org.apache.iotdb.it.env.cluster.ClusterConstant.SCALABLE_SINGLE_NODE_MODE_CONFIG_NODE_NUM; import static org.apache.iotdb.it.env.cluster.ClusterConstant.SCALABLE_SINGLE_NODE_MODE_DATA_NODE_NUM; import static org.apache.iotdb.it.env.cluster.ClusterConstant.SIMPLE_CONSENSUS_STR; +import static org.apache.iotdb.it.env.cluster.ClusterConstant.STREAM_CONSENSUS_STR; import static org.apache.iotdb.it.env.cluster.ClusterConstant.STRONG_CONSISTENCY_CLUSTER_MODE; import static org.apache.iotdb.it.env.cluster.ClusterConstant.STRONG_CONSISTENCY_CLUSTER_MODE_CONFIG_NODE_NUM; import static org.apache.iotdb.it.env.cluster.ClusterConstant.STRONG_CONSISTENCY_CLUSTER_MODE_DATA_NODE_NUM; @@ -216,6 +222,12 @@ public static String fromConsensusFullNameToAbbr(String consensus) { return RATIS_CONSENSUS_STR; case IOT_CONSENSUS: return IOT_CONSENSUS_STR; + case REAL_PIPE_CONSENSUS: + return PIPE_CONSENSUS_STR; + case IOTV2_CONSENSUS: + return STREAM_CONSENSUS_STR; + case FAST_IOT_CONSENSUS: + return BATCH_CONSENSUS_STR; default: throw new IllegalArgumentException("Unknown consensus type: " + consensus); } @@ -229,6 +241,12 @@ public static String fromConsensusAbbrToFullName(String consensus) { return RATIS_CONSENSUS; case IOT_CONSENSUS_STR: return IOT_CONSENSUS; + case PIPE_CONSENSUS_STR: + return REAL_PIPE_CONSENSUS; + case STREAM_CONSENSUS_STR: + return IOTV2_CONSENSUS; + case BATCH_CONSENSUS_STR: + return FAST_IOT_CONSENSUS; default: throw new IllegalArgumentException("Unknown consensus type: " + consensus); } diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoConflictIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoConflictIT.java index 1b2a5d5ff37a..1c021b074215 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoConflictIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoConflictIT.java @@ -22,13 +22,16 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient; import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq; +import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.db.it.utils.TestUtils; +import org.apache.iotdb.it.env.MultiEnvFactory; import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper; import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.MultiClusterIT2AutoCreateSchema; import org.apache.iotdb.rpc.TSStatusCode; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -42,6 +45,36 @@ @RunWith(IoTDBTestRunner.class) @Category({MultiClusterIT2AutoCreateSchema.class}) public class IoTDBPipeAutoConflictIT extends AbstractPipeDualAutoIT { + @Before + public void setUp() { + MultiEnvFactory.createEnv(2); + senderEnv = MultiEnvFactory.getEnv(0); + receiverEnv = MultiEnvFactory.getEnv(1); + + // TODO: delete ratis configurations + senderEnv + .getConfig() + .getCommonConfig() + .setAutoCreateSchemaEnabled(true) + .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS); + receiverEnv + .getConfig() + .getCommonConfig() + .setAutoCreateSchemaEnabled(true) + .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS); + + // 10 min, assert that the operations will not time out + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + + senderEnv.initClusterEnvironment(); + receiverEnv.initClusterEnvironment(); + } + @Test public void testDoubleLivingAutoConflict() throws Exception { // Double living is two clusters each with a pipe connecting to the other. 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 87673ebf9b03..41ac7bf47f8f 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 @@ -254,6 +254,14 @@ public enum TSStatusCode { DROP_CONSUMER_ERROR(2101), ALTER_CONSUMER_ERROR(2102), CONSUMER_PUSH_META_ERROR(2103), + + // Pipe Consensus + PIPE_CONSENSUS_CONNECTOR_RESTART_ERROR(2200), + PIPE_CONSENSUS_VERSION_ERROR(2201), + PIPE_CONSENSUS_DEPRECATED_REQUEST(2202), + PIPE_CONSENSUS_TRANSFER_FILE_OFFSET_RESET(2203), + PIPE_CONSENSUS_TRANSFER_FILE_ERROR(2204), + PIPE_CONSENSUS_TYPE_ERROR(2205), ; private final int statusCode; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeStartupCheck.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeStartupCheck.java index 3e82e0bb4a6f..88be42eb417f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeStartupCheck.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeStartupCheck.java @@ -143,6 +143,18 @@ private void checkGlobalConfig() throws ConfigurationException { "the SchemaRegion doesn't support org.apache.iotdb.consensus.iot.IoTConsensus"); } + // When the schemaengine region consensus protocol is set to PipeConsensus, + // we should report an error + if (CONF.getSchemaRegionConsensusProtocolClass().equals(ConsensusFactory.FAST_IOT_CONSENSUS) + || CONF.getSchemaRegionConsensusProtocolClass().equals(ConsensusFactory.IOTV2_CONSENSUS)) { + throw new ConfigurationException( + "schema_region_consensus_protocol_class", + String.valueOf(CONF.getSchemaRegionConsensusProtocolClass()), + String.format( + "%s or %s", ConsensusFactory.SIMPLE_CONSENSUS, ConsensusFactory.RATIS_CONSENSUS), + "the SchemaRegion doesn't support org.apache.iotdb.consensus.iot.FastIoTConsensus"); + } + // The leader distribution policy is limited if (!AbstractLeaderBalancer.GREEDY_POLICY.equals(CONF.getLeaderDistributionPolicy()) && !AbstractLeaderBalancer.CFD_POLICY.equals(CONF.getLeaderDistributionPolicy())) { 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 c97ceb34ce26..5640721fba1a 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 @@ -80,6 +80,10 @@ public class RouteBalancer implements IClusterStatusSubscriber { && ConsensusFactory.RATIS_CONSENSUS.equals(DATA_REGION_CONSENSUS_PROTOCOL_CLASS)) || (CONF.isEnableAutoLeaderBalanceForIoTConsensus() && ConsensusFactory.IOT_CONSENSUS.equals(DATA_REGION_CONSENSUS_PROTOCOL_CLASS)) + || (CONF.isEnableAutoLeaderBalanceForIoTConsensus() + && ConsensusFactory.FAST_IOT_CONSENSUS.equals(DATA_REGION_CONSENSUS_PROTOCOL_CLASS)) + || (CONF.isEnableAutoLeaderBalanceForIoTConsensus() + && ConsensusFactory.IOTV2_CONSENSUS.equals(DATA_REGION_CONSENSUS_PROTOCOL_CLASS)) // The simple consensus protocol will always automatically designate itself as the leader || ConsensusFactory.SIMPLE_CONSENSUS.equals(DATA_REGION_CONSENSUS_PROTOCOL_CLASS); private static final boolean IS_ENABLE_AUTO_LEADER_BALANCE_FOR_SCHEMA_REGION = @@ -180,9 +184,12 @@ private void balanceRegionLeader( regionGroupId, newLeaderId); switch (consensusProtocolClass) { + case ConsensusFactory.FAST_IOT_CONSENSUS: + case ConsensusFactory.IOTV2_CONSENSUS: case ConsensusFactory.IOT_CONSENSUS: case ConsensusFactory.SIMPLE_CONSENSUS: - // For IoTConsensus or SimpleConsensus protocol, change RegionRouteMap is enough + // For IoTConsensus or SimpleConsensus or PipeConsensus protocol, change + // RegionRouteMap is enough successTransferMap.put( regionGroupId, new ConsensusGroupHeartbeatSample(currentTime, newLeaderId)); break; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java index 5d3a4cc2245d..843bea6d8e36 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java @@ -30,6 +30,7 @@ import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeTemporaryMeta; +import org.apache.iotdb.commons.pipe.task.meta.PipeType; import org.apache.iotdb.commons.snapshot.SnapshotProcessor; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.runtime.PipeHandleLeaderChangePlan; @@ -498,6 +499,10 @@ private TSStatus handleLeaderChangeInternal(final PipeHandleLeaderChangePlan pla .getPipeMetaList() .forEach( pipeMeta -> { + if (PipeType.CONSENSUS.equals(pipeMeta.getStaticMeta().getPipeType())) { + return; // pipe consensus pipe task will not change + } + final Map<Integer, PipeTaskMeta> consensusGroupIdToTaskMetaMap = pipeMeta.getRuntimeMeta().getConsensusGroupId2TaskMetaMap(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java index 3bd1c130a49e..bbc9d7b2298f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java @@ -19,12 +19,17 @@ package org.apache.iotdb.confignode.procedure.impl.pipe.task; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.consensus.ConsensusGroupId; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeStaticMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; +import org.apache.iotdb.commons.pipe.task.meta.PipeType; import org.apache.iotdb.commons.schema.SchemaConstant; import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.apache.iotdb.confignode.consensus.request.write.pipe.task.CreatePipePlanV2; @@ -54,6 +59,9 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_CONSENSUS_GROUP_ID_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_CONSENSUS_SENDER_DATANODE_ID_KEY; + public class CreatePipeProcedureV2 extends AbstractOperatePipeProcedureV2 { private static final Logger LOGGER = LoggerFactory.getLogger(CreatePipeProcedureV2.class); @@ -137,31 +145,52 @@ public void executeFromCalculateInfoForTask(ConfigNodeProcedureEnv env) { final ConcurrentMap<Integer, PipeTaskMeta> consensusGroupIdToTaskMetaMap = new ConcurrentHashMap<>(); - // data regions & schema regions - env.getConfigManager() - .getLoadManager() - .getRegionLeaderMap() - .forEach( - (regionGroupId, regionLeaderNodeId) -> { - final String databaseName = - env.getConfigManager().getPartitionManager().getRegionStorageGroup(regionGroupId); - if (databaseName != null && !databaseName.equals(SchemaConstant.SYSTEM_DATABASE)) { - // Pipe only collect user's data, filter out metric database here. - consensusGroupIdToTaskMetaMap.put( - regionGroupId.getId(), - new PipeTaskMeta(MinimumProgressIndex.INSTANCE, regionLeaderNodeId)); - } - }); - - // config region - consensusGroupIdToTaskMetaMap.put( - // 0 is the consensus group id of the config region, but data region id and schema region id - // also start from 0, so we use Integer.MIN_VALUE to represent the config region - Integer.MIN_VALUE, - new PipeTaskMeta( - MinimumProgressIndex.INSTANCE, - // The leader of the config region is the config node itself - ConfigNodeDescriptor.getInstance().getConf().getConfigNodeId())); + if (PipeType.CONSENSUS.equals(pipeStaticMeta.getPipeType())) { + final TConsensusGroupId groupId = + ConsensusGroupId.Factory.createFromString( + createPipeRequest.getExtractorAttributes().get(EXTRACTOR_CONSENSUS_GROUP_ID_KEY)) + .convertToTConsensusGroupId(); + + final int senderDataNodeId = + Integer.parseInt( + createPipeRequest + .getExtractorAttributes() + .get(EXTRACTOR_CONSENSUS_SENDER_DATANODE_ID_KEY)); + consensusGroupIdToTaskMetaMap.put( + groupId.getId(), + new PipeTaskMeta( + new RecoverProgressIndex(senderDataNodeId, new SimpleProgressIndex(0, 0)), + senderDataNodeId)); + } else { + // data regions & schema regions + env.getConfigManager() + .getLoadManager() + .getRegionLeaderMap() + .forEach( + (regionGroupId, regionLeaderNodeId) -> { + final String databaseName = + env.getConfigManager() + .getPartitionManager() + .getRegionStorageGroup(regionGroupId); + if (databaseName != null && !databaseName.equals(SchemaConstant.SYSTEM_DATABASE)) { + // Pipe only collect user's data, filter out metric database here. + consensusGroupIdToTaskMetaMap.put( + regionGroupId.getId(), + new PipeTaskMeta(MinimumProgressIndex.INSTANCE, regionLeaderNodeId)); + } + }); + + // config region + consensusGroupIdToTaskMetaMap.put( + // 0 is the consensus group id of the config region, but data region id and schema region + // id + // also start from 0, so we use Integer.MIN_VALUE to represent the config region + Integer.MIN_VALUE, + new PipeTaskMeta( + MinimumProgressIndex.INSTANCE, + // The leader of the config region is the config node itself + ConfigNodeDescriptor.getInstance().getConf().getConfigNodeId())); + } pipeRuntimeMeta = new PipeRuntimeMeta(consensusGroupIdToTaskMetaMap); pipeRuntimeMeta.getStatus().set(PipeStatus.RUNNING); diff --git a/iotdb-core/consensus/pom.xml b/iotdb-core/consensus/pom.xml index 981f71156945..f6622cfd3673 100644 --- a/iotdb-core/consensus/pom.xml +++ b/iotdb-core/consensus/pom.xml @@ -48,12 +48,12 @@ </dependency> <dependency> <groupId>org.apache.iotdb</groupId> - <artifactId>iotdb-thrift-consensus</artifactId> + <artifactId>iotdb-thrift-commons</artifactId> <version>1.3.2-SNAPSHOT</version> </dependency> <dependency> <groupId>org.apache.iotdb</groupId> - <artifactId>iotdb-thrift-commons</artifactId> + <artifactId>iotdb-thrift-consensus</artifactId> <version>1.3.2-SNAPSHOT</version> </dependency> <dependency> @@ -61,6 +61,11 @@ <artifactId>service-rpc</artifactId> <version>1.3.2-SNAPSHOT</version> </dependency> + <dependency> + <groupId>org.apache.iotdb</groupId> + <artifactId>pipe-api</artifactId> + <version>1.3.2-SNAPSHOT</version> + </dependency> <dependency> <groupId>org.apache.ratis</groupId> <artifactId>ratis-server</artifactId> diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java index a32ee0201cdf..7742baf3bdcc 100644 --- a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java @@ -19,13 +19,17 @@ package org.apache.iotdb.consensus; +import org.apache.iotdb.commons.client.container.PipeConsensusClientMgrContainer; import org.apache.iotdb.consensus.config.ConsensusConfig; +import org.apache.iotdb.consensus.config.PipeConsensusConfig.ReplicateMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; public class ConsensusFactory { @@ -35,6 +39,17 @@ public class ConsensusFactory { public static final String SIMPLE_CONSENSUS = "org.apache.iotdb.consensus.simple.SimpleConsensus"; public static final String RATIS_CONSENSUS = "org.apache.iotdb.consensus.ratis.RatisConsensus"; public static final String IOT_CONSENSUS = "org.apache.iotdb.consensus.iot.IoTConsensus"; + public static final String REAL_PIPE_CONSENSUS = "org.apache.iotdb.consensus.pipe.PipeConsensus"; + // Corresponding to streamConsensus + public static final String IOTV2_CONSENSUS = "org.apache.iotdb.consensus.iot.IoTV2Consensus"; + // Corresponding to batchConsensus + public static final String FAST_IOT_CONSENSUS = "org.apache.iotdb.consensus.iot.FastIoTConsensus"; + private static final Map<String, ReplicateMode> PIPE_CONSENSUS_MODE_MAP = new HashMap<>(); + + static { + PIPE_CONSENSUS_MODE_MAP.put(IOTV2_CONSENSUS, ReplicateMode.STREAM); + PIPE_CONSENSUS_MODE_MAP.put(FAST_IOT_CONSENSUS, ReplicateMode.BATCH); + } private static final Logger logger = LoggerFactory.getLogger(ConsensusFactory.class); @@ -45,6 +60,13 @@ private ConsensusFactory() { public static Optional<IConsensus> getConsensusImpl( String className, ConsensusConfig config, IStateMachine.Registry registry) { try { + // special judge for PipeConsensus + if (className.equals(IOTV2_CONSENSUS) || className.equals(FAST_IOT_CONSENSUS)) { + config.getPipeConsensusConfig().setReplicateMode(PIPE_CONSENSUS_MODE_MAP.get(className)); + className = REAL_PIPE_CONSENSUS; + // initialize pipeConsensus' thrift component + PipeConsensusClientMgrContainer.build(); + } Class<?> executor = Class.forName(className); Constructor<?> executorConstructor = executor.getDeclaredConstructor(ConsensusConfig.class, IStateMachine.Registry.class); diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java index 74600440b7be..dba53214abd7 100644 --- a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java @@ -32,6 +32,7 @@ public class ConsensusConfig { private final TConsensusGroupType consensusGroupType; private final RatisConfig ratisConfig; private final IoTConsensusConfig iotConsensusConfig; + private final PipeConsensusConfig pipeConsensusConfig; private ConsensusConfig( TEndPoint thisNode, @@ -39,13 +40,15 @@ private ConsensusConfig( String storageDir, TConsensusGroupType consensusGroupType, RatisConfig ratisConfig, - IoTConsensusConfig iotConsensusConfig) { + IoTConsensusConfig iotConsensusConfig, + PipeConsensusConfig pipeConsensusConfig) { this.thisNodeEndPoint = thisNode; this.thisNodeId = thisNodeId; this.storageDir = storageDir; this.consensusGroupType = consensusGroupType; this.ratisConfig = ratisConfig; this.iotConsensusConfig = iotConsensusConfig; + this.pipeConsensusConfig = pipeConsensusConfig; } public TEndPoint getThisNodeEndPoint() { @@ -72,6 +75,10 @@ public IoTConsensusConfig getIotConsensusConfig() { return iotConsensusConfig; } + public PipeConsensusConfig getPipeConsensusConfig() { + return pipeConsensusConfig; + } + public static ConsensusConfig.Builder newBuilder() { return new ConsensusConfig.Builder(); } @@ -84,6 +91,7 @@ public static class Builder { private TConsensusGroupType consensusGroupType; private RatisConfig ratisConfig; private IoTConsensusConfig iotConsensusConfig; + private PipeConsensusConfig pipeConsensusConfig; public ConsensusConfig build() { return new ConsensusConfig( @@ -93,7 +101,9 @@ public ConsensusConfig build() { consensusGroupType, Optional.ofNullable(ratisConfig).orElseGet(() -> RatisConfig.newBuilder().build()), Optional.ofNullable(iotConsensusConfig) - .orElseGet(() -> IoTConsensusConfig.newBuilder().build())); + .orElseGet(() -> IoTConsensusConfig.newBuilder().build()), + Optional.ofNullable(pipeConsensusConfig) + .orElseGet(() -> PipeConsensusConfig.newBuilder().build())); } public Builder setThisNode(TEndPoint thisNode) { @@ -125,5 +135,10 @@ public Builder setIoTConsensusConfig(IoTConsensusConfig iotConsensusConfig) { this.iotConsensusConfig = iotConsensusConfig; return this; } + + public Builder setPipeConsensusConfig(PipeConsensusConfig pipeConsensusConfig) { + this.pipeConsensusConfig = pipeConsensusConfig; + return this; + } } } diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/PipeConsensusConfig.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/PipeConsensusConfig.java new file mode 100644 index 000000000000..bec2c7d8e1e0 --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/PipeConsensusConfig.java @@ -0,0 +1,358 @@ +/* + * 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.consensus.config; + +import org.apache.iotdb.commons.pipe.plugin.builtin.BuiltinPipePlugin; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeDispatcher; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeGuardian; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeReceiver; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeSelector; +import org.apache.iotdb.consensus.pipe.consensuspipe.ProgressIndexManager; + +import java.util.concurrent.TimeUnit; + +public class PipeConsensusConfig { + private final RPC rpc; + private final Pipe pipe; + // Use stream mode by default. User can configure it + private ReplicateMode replicateMode = ReplicateMode.STREAM; + + public PipeConsensusConfig(RPC rpc, Pipe pipe) { + this.rpc = rpc; + this.pipe = pipe; + } + + public void setReplicateMode(ReplicateMode replicateMode) { + this.replicateMode = replicateMode; + } + + public ReplicateMode getReplicateMode() { + return replicateMode; + } + + public RPC getRpc() { + return rpc; + } + + public Pipe getPipe() { + return pipe; + } + + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + private RPC rpc; + private Pipe pipe; + + public Builder setPipe(Pipe pipe) { + this.pipe = pipe; + return this; + } + + public Builder setRPC(RPC rpc) { + this.rpc = rpc; + return this; + } + + public PipeConsensusConfig build() { + return new PipeConsensusConfig(rpc, pipe); + } + } + + public static class RPC { + private final int rpcSelectorThreadNum; + private final int rpcMinConcurrentClientNum; + private final int rpcMaxConcurrentClientNum; + private final int thriftServerAwaitTimeForStopService; + private final boolean isRpcThriftCompressionEnabled; + private final int connectionTimeoutInMs; + private final int thriftMaxFrameSize; + + public RPC( + int rpcSelectorThreadNum, + int rpcMinConcurrentClientNum, + int rpcMaxConcurrentClientNum, + int thriftServerAwaitTimeForStopService, + boolean isRpcThriftCompressionEnabled, + int connectionTimeoutInMs, + int thriftMaxFrameSize) { + this.rpcSelectorThreadNum = rpcSelectorThreadNum; + this.rpcMinConcurrentClientNum = rpcMinConcurrentClientNum; + this.rpcMaxConcurrentClientNum = rpcMaxConcurrentClientNum; + this.thriftServerAwaitTimeForStopService = thriftServerAwaitTimeForStopService; + this.isRpcThriftCompressionEnabled = isRpcThriftCompressionEnabled; + this.connectionTimeoutInMs = connectionTimeoutInMs; + this.thriftMaxFrameSize = thriftMaxFrameSize; + } + + public int getRpcSelectorThreadNum() { + return rpcSelectorThreadNum; + } + + public int getRpcMinConcurrentClientNum() { + return rpcMinConcurrentClientNum; + } + + public int getRpcMaxConcurrentClientNum() { + return rpcMaxConcurrentClientNum; + } + + public int getThriftServerAwaitTimeForStopService() { + return thriftServerAwaitTimeForStopService; + } + + public boolean isRpcThriftCompressionEnabled() { + return isRpcThriftCompressionEnabled; + } + + public int getConnectionTimeoutInMs() { + return connectionTimeoutInMs; + } + + public int getThriftMaxFrameSize() { + return thriftMaxFrameSize; + } + + public static RPC.Builder newBuilder() { + return new RPC.Builder(); + } + + public static class Builder { + private int rpcSelectorThreadNum = 1; + private int rpcMinConcurrentClientNum = Runtime.getRuntime().availableProcessors(); + private int rpcMaxConcurrentClientNum = 65535; + private int thriftServerAwaitTimeForStopService = 60; + private boolean isRpcThriftCompressionEnabled = false; + private int connectionTimeoutInMs = (int) TimeUnit.SECONDS.toMillis(60); + private int thriftMaxFrameSize = 536870912; + + public RPC.Builder setRpcSelectorThreadNum(int rpcSelectorThreadNum) { + this.rpcSelectorThreadNum = rpcSelectorThreadNum; + return this; + } + + public RPC.Builder setRpcMinConcurrentClientNum(int rpcMinConcurrentClientNum) { + this.rpcMinConcurrentClientNum = rpcMinConcurrentClientNum; + return this; + } + + public RPC.Builder setRpcMaxConcurrentClientNum(int rpcMaxConcurrentClientNum) { + this.rpcMaxConcurrentClientNum = rpcMaxConcurrentClientNum; + return this; + } + + public RPC.Builder setThriftServerAwaitTimeForStopService( + int thriftServerAwaitTimeForStopService) { + this.thriftServerAwaitTimeForStopService = thriftServerAwaitTimeForStopService; + return this; + } + + public RPC.Builder setIsRpcThriftCompressionEnabled(boolean isRpcThriftCompressionEnabled) { + this.isRpcThriftCompressionEnabled = isRpcThriftCompressionEnabled; + return this; + } + + public RPC.Builder setConnectionTimeoutInMs(int connectionTimeoutInMs) { + this.connectionTimeoutInMs = connectionTimeoutInMs; + return this; + } + + public RPC.Builder setThriftMaxFrameSize(int thriftMaxFrameSize) { + this.thriftMaxFrameSize = thriftMaxFrameSize; + return this; + } + + public RPC build() { + return new RPC( + rpcSelectorThreadNum, + rpcMinConcurrentClientNum, + rpcMaxConcurrentClientNum, + thriftServerAwaitTimeForStopService, + isRpcThriftCompressionEnabled, + connectionTimeoutInMs, + thriftMaxFrameSize); + } + } + } + + public static class Pipe { + private final String extractorPluginName; + private final String processorPluginName; + private final String connectorPluginName; + private final ConsensusPipeDispatcher consensusPipeDispatcher; + private final ConsensusPipeGuardian consensusPipeGuardian; + private final ConsensusPipeSelector consensusPipeSelector; + private final ProgressIndexManager progressIndexManager; + private final ConsensusPipeReceiver consensusPipeReceiver; + private final long consensusPipeGuardJobIntervalInSeconds; + + public Pipe( + String extractorPluginName, + String processorPluginName, + String connectorPluginName, + ConsensusPipeDispatcher consensusPipeDispatcher, + ConsensusPipeGuardian consensusPipeGuardian, + ConsensusPipeSelector consensusPipeSelector, + ProgressIndexManager progressIndexManager, + ConsensusPipeReceiver consensusPipeReceiver, + long consensusPipeGuardJobIntervalInSeconds) { + this.extractorPluginName = extractorPluginName; + this.processorPluginName = processorPluginName; + this.connectorPluginName = connectorPluginName; + this.consensusPipeDispatcher = consensusPipeDispatcher; + this.consensusPipeGuardian = consensusPipeGuardian; + this.consensusPipeSelector = consensusPipeSelector; + this.progressIndexManager = progressIndexManager; + this.consensusPipeReceiver = consensusPipeReceiver; + this.consensusPipeGuardJobIntervalInSeconds = consensusPipeGuardJobIntervalInSeconds; + } + + public String getExtractorPluginName() { + return extractorPluginName; + } + + public String getProcessorPluginName() { + return processorPluginName; + } + + public String getConnectorPluginName() { + return connectorPluginName; + } + + public ConsensusPipeDispatcher getConsensusPipeDispatcher() { + return consensusPipeDispatcher; + } + + public ConsensusPipeGuardian getConsensusPipeGuardian() { + return consensusPipeGuardian; + } + + public ConsensusPipeSelector getConsensusPipeSelector() { + return consensusPipeSelector; + } + + public ConsensusPipeReceiver getConsensusPipeReceiver() { + return consensusPipeReceiver; + } + + public ProgressIndexManager getProgressIndexManager() { + return progressIndexManager; + } + + public long getConsensusPipeGuardJobIntervalInSeconds() { + return consensusPipeGuardJobIntervalInSeconds; + } + + public static Pipe.Builder newBuilder() { + return new Pipe.Builder(); + } + + public static class Builder { + private String extractorPluginName = BuiltinPipePlugin.IOTDB_EXTRACTOR.getPipePluginName(); + private String processorPluginName = + BuiltinPipePlugin.PIPE_CONSENSUS_PROCESSOR.getPipePluginName(); + private String connectorPluginName = + BuiltinPipePlugin.PIPE_CONSENSUS_ASYNC_CONNECTOR.getPipePluginName(); + private ConsensusPipeDispatcher consensusPipeDispatcher = null; + private ConsensusPipeGuardian consensusPipeGuardian = null; + private ConsensusPipeSelector consensusPipeSelector = null; + private ProgressIndexManager progressIndexManager = null; + private ConsensusPipeReceiver consensusPipeReceiver = null; + private long consensusPipeGuardJobIntervalInSeconds = 180L; + + public Pipe.Builder setExtractorPluginName(String extractorPluginName) { + this.extractorPluginName = extractorPluginName; + return this; + } + + public Pipe.Builder setProcessorPluginName(String processorPluginName) { + this.processorPluginName = processorPluginName; + return this; + } + + public Pipe.Builder setConnectorPluginName(String connectorPluginName) { + this.connectorPluginName = connectorPluginName; + return this; + } + + public Pipe.Builder setConsensusPipeDispatcher( + ConsensusPipeDispatcher consensusPipeDispatcher) { + this.consensusPipeDispatcher = consensusPipeDispatcher; + return this; + } + + public Pipe.Builder setConsensusPipeGuardian(ConsensusPipeGuardian consensusPipeGuardian) { + this.consensusPipeGuardian = consensusPipeGuardian; + return this; + } + + public Pipe.Builder setConsensusPipeSelector(ConsensusPipeSelector consensusPipeSelector) { + this.consensusPipeSelector = consensusPipeSelector; + return this; + } + + public Pipe.Builder setConsensusPipeReceiver(ConsensusPipeReceiver consensusPipeReceiver) { + this.consensusPipeReceiver = consensusPipeReceiver; + return this; + } + + public Pipe.Builder setProgressIndexManager(ProgressIndexManager progressIndexManager) { + this.progressIndexManager = progressIndexManager; + return this; + } + + public Pipe.Builder setConsensusPipeGuardJobIntervalInSeconds( + long consensusPipeGuardJobIntervalInSeconds) { + this.consensusPipeGuardJobIntervalInSeconds = consensusPipeGuardJobIntervalInSeconds; + return this; + } + + public Pipe build() { + return new Pipe( + extractorPluginName, + processorPluginName, + connectorPluginName, + consensusPipeDispatcher, + consensusPipeGuardian, + consensusPipeSelector, + progressIndexManager, + consensusPipeReceiver, + consensusPipeGuardJobIntervalInSeconds); + } + } + } + + public enum ReplicateMode { + STREAM("stream"), + BATCH("batch"); + + private final String value; + + ReplicateMode(String value) { + this.value = value; + } + + public String getValue() { + return value; + } + } +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensus.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensus.java index d220d7cac274..0526729033be 100644 --- a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensus.java +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensus.java @@ -473,6 +473,7 @@ public void reloadConsensusConfig(ConsensusConfig consensusConfig) { .init(config.getReplication().getRegionMigrationSpeedLimitBytesPerSecond()); } + @Override public void resetPeerList(ConsensusGroupId groupId, List<Peer> correctPeers) throws ConsensusException { IoTConsensusServerImpl impl = diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/client/AsyncIoTConsensusServiceClient.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/client/AsyncIoTConsensusServiceClient.java index f2b77a499ec6..cb635c8e6dd7 100644 --- a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/client/AsyncIoTConsensusServiceClient.java +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/client/AsyncIoTConsensusServiceClient.java @@ -121,7 +121,7 @@ public boolean isReady() { @Override public String toString() { - return String.format("AsyncConfigNodeIServiceClient{%s}", endpoint); + return String.format("AsyncIoTConsensusServiceClient{%s}", endpoint); } public static class Factory diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensus.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensus.java new file mode 100644 index 000000000000..fccbef50c679 --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensus.java @@ -0,0 +1,465 @@ +/* + * 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.consensus.pipe; + +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.async.AsyncPipeConsensusServiceClient; +import org.apache.iotdb.commons.client.container.PipeConsensusClientMgrContainer; +import org.apache.iotdb.commons.client.sync.SyncPipeConsensusServiceClient; +import org.apache.iotdb.commons.consensus.ConsensusGroupId; +import org.apache.iotdb.commons.exception.StartupException; +import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; +import org.apache.iotdb.commons.service.RegisterManager; +import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.commons.utils.StatusUtils; +import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.consensus.IConsensus; +import org.apache.iotdb.consensus.IStateMachine; +import org.apache.iotdb.consensus.common.DataSet; +import org.apache.iotdb.consensus.common.Peer; +import org.apache.iotdb.consensus.common.request.IConsensusRequest; +import org.apache.iotdb.consensus.config.ConsensusConfig; +import org.apache.iotdb.consensus.config.PipeConsensusConfig; +import org.apache.iotdb.consensus.exception.ConsensusException; +import org.apache.iotdb.consensus.exception.ConsensusGroupAlreadyExistException; +import org.apache.iotdb.consensus.exception.ConsensusGroupModifyPeerException; +import org.apache.iotdb.consensus.exception.ConsensusGroupNotExistException; +import org.apache.iotdb.consensus.exception.IllegalPeerEndpointException; +import org.apache.iotdb.consensus.exception.IllegalPeerNumException; +import org.apache.iotdb.consensus.exception.PeerAlreadyInConsensusGroupException; +import org.apache.iotdb.consensus.exception.PeerNotInConsensusGroupException; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeGuardian; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeManager; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeName; +import org.apache.iotdb.consensus.pipe.service.PipeConsensusRPCService; +import org.apache.iotdb.consensus.pipe.service.PipeConsensusRPCServiceProcessor; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.TSStatusCode; + +import com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; + +import static org.apache.iotdb.consensus.iot.IoTConsensus.getConsensusGroupIdsFromDir; + +// TODO: support syncLag +public class PipeConsensus implements IConsensus { + private static final String CONSENSUS_PIPE_GUARDIAN_TASK_ID = "consensus_pipe_guardian"; + private static final String CLASS_NAME = PipeConsensus.class.getSimpleName(); + private static final Logger LOGGER = LoggerFactory.getLogger(PipeConsensus.class); + + private final TEndPoint thisNode; + private final int thisNodeId; + private final File storageDir; + private final IStateMachine.Registry registry; + private final Map<ConsensusGroupId, PipeConsensusServerImpl> stateMachineMap = + new ConcurrentHashMap<>(); + private final PipeConsensusRPCService rpcService; + private final RegisterManager registerManager = new RegisterManager(); + private final ReentrantLock stateMachineMapLock = new ReentrantLock(); + private final PipeConsensusConfig config; + private final ConsensusPipeManager consensusPipeManager; + private final ConsensusPipeGuardian consensusPipeGuardian; + private final IClientManager<TEndPoint, AsyncPipeConsensusServiceClient> asyncClientManager; + private final IClientManager<TEndPoint, SyncPipeConsensusServiceClient> syncClientManager; + + public PipeConsensus(ConsensusConfig config, IStateMachine.Registry registry) { + this.thisNode = config.getThisNodeEndPoint(); + this.thisNodeId = config.getThisNodeId(); + this.storageDir = new File(config.getStorageDir()); + this.config = config.getPipeConsensusConfig(); + this.registry = registry; + this.rpcService = new PipeConsensusRPCService(thisNode, config.getPipeConsensusConfig()); + this.consensusPipeManager = + new ConsensusPipeManager( + config.getPipeConsensusConfig().getPipe(), + config.getPipeConsensusConfig().getReplicateMode()); + this.consensusPipeGuardian = + config.getPipeConsensusConfig().getPipe().getConsensusPipeGuardian(); + this.asyncClientManager = PipeConsensusClientMgrContainer.getInstance().getAsyncClientManager(); + this.syncClientManager = PipeConsensusClientMgrContainer.getInstance().getSyncClientManager(); + } + + @Override + public synchronized void start() throws IOException { + initAndRecover(); + + rpcService.initAsyncedServiceImpl(new PipeConsensusRPCServiceProcessor(this, config.getPipe())); + try { + registerManager.register(rpcService); + } catch (StartupException e) { + throw new IOException(e); + } + + consensusPipeGuardian.start( + CONSENSUS_PIPE_GUARDIAN_TASK_ID, + this::checkAllConsensusPipe, + config.getPipe().getConsensusPipeGuardJobIntervalInSeconds()); + } + + private void initAndRecover() throws IOException { + if (!storageDir.exists()) { + if (!storageDir.mkdirs()) { + LOGGER.warn("Unable to create consensus dir at {}", storageDir); + throw new IOException(String.format("Unable to create consensus dir at %s", storageDir)); + } + } else { + try (DirectoryStream<Path> stream = Files.newDirectoryStream(storageDir.toPath())) { + for (Path path : stream) { + ConsensusGroupId consensusGroupId = parsePeerFileName(path.getFileName().toString()); + PipeConsensusServerImpl consensus = + new PipeConsensusServerImpl( + new Peer(consensusGroupId, thisNodeId, thisNode), + registry.apply(consensusGroupId), + path.toString(), + new ArrayList<>(), + config, + consensusPipeManager, + syncClientManager); + stateMachineMap.put(consensusGroupId, consensus); + consensus.start(true); + } + } + } + } + + @Override + public synchronized void stop() { + asyncClientManager.close(); + syncClientManager.close(); + registerManager.deregisterAll(); + consensusPipeGuardian.stop(); + stateMachineMap.values().parallelStream().forEach(PipeConsensusServerImpl::stop); + } + + private void checkAllConsensusPipe() { + final Map<ConsensusGroupId, Map<ConsensusPipeName, PipeStatus>> existedPipes = + consensusPipeManager.getAllConsensusPipe().entrySet().stream() + .filter(entry -> entry.getKey().getSenderDataNodeId() == thisNodeId) + .collect( + Collectors.groupingBy( + entry -> entry.getKey().getConsensusGroupId(), + Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))); + try { + stateMachineMapLock.lock(); + stateMachineMap.forEach( + (key, value) -> + value.checkConsensusPipe(existedPipes.getOrDefault(key, ImmutableMap.of()))); + existedPipes.entrySet().stream() + .filter(entry -> !stateMachineMap.containsKey(entry.getKey())) + .flatMap(entry -> entry.getValue().keySet().stream()) + .forEach( + consensusPipeName -> { + try { + LOGGER.warn( + "{} drop consensus pipe [{}]", + consensusPipeName.getConsensusGroupId(), + consensusPipeName); + consensusPipeManager.updateConsensusPipe(consensusPipeName, PipeStatus.DROPPED); + } catch (Exception e) { + LOGGER.warn( + "{} cannot drop consensus pipe [{}]", + consensusPipeName.getConsensusGroupId(), + consensusPipeName, + e); + } + }); + } finally { + stateMachineMapLock.unlock(); + } + } + + @Override + public TSStatus write(ConsensusGroupId groupId, IConsensusRequest request) + throws ConsensusException { + final PipeConsensusServerImpl impl = + Optional.ofNullable(stateMachineMap.get(groupId)) + .orElseThrow(() -> new ConsensusGroupNotExistException(groupId)); + if (impl.isReadOnly()) { + return StatusUtils.getStatus(TSStatusCode.SYSTEM_READ_ONLY); + } else if (!impl.isActive()) { + return RpcUtils.getStatus( + TSStatusCode.WRITE_PROCESS_REJECT, + "peer is inactive and not ready to receive sync log request."); + } else { + return impl.write(request); + } + } + + @Override + public DataSet read(ConsensusGroupId groupId, IConsensusRequest request) + throws ConsensusException { + return Optional.ofNullable(stateMachineMap.get(groupId)) + .orElseThrow(() -> new ConsensusGroupNotExistException(groupId)) + .read(request); + } + + private String getPeerDir(ConsensusGroupId groupId) { + return storageDir + File.separator + groupId.getType().getValue() + "_" + groupId.getId(); + } + + private ConsensusGroupId parsePeerFileName(String fileName) { + String[] items = fileName.split("_"); + return ConsensusGroupId.Factory.create(Integer.parseInt(items[0]), Integer.parseInt(items[1])); + } + + @Override + public void createLocalPeer(ConsensusGroupId groupId, List<Peer> peers) + throws ConsensusException { + final int consensusGroupSize = peers.size(); + if (consensusGroupSize == 0) { + throw new IllegalPeerNumException(consensusGroupSize); + } + if (!peers.contains(new Peer(groupId, thisNodeId, thisNode))) { + throw new IllegalPeerEndpointException(thisNode, peers); + } + if (stateMachineMap.containsKey(groupId)) { + throw new ConsensusGroupAlreadyExistException(groupId); + } + + try { + stateMachineMapLock.lock(); + + final String path = getPeerDir(groupId); + if (!new File(path).mkdirs()) { + LOGGER.warn("Unable to create consensus dir for group {} at {}", groupId, path); + throw new ConsensusException( + String.format("Unable to create consensus dir for group %s", groupId)); + } + + PipeConsensusServerImpl consensus = + new PipeConsensusServerImpl( + new Peer(groupId, thisNodeId, thisNode), + registry.apply(groupId), + path, + peers, + config, + consensusPipeManager, + syncClientManager); + stateMachineMap.put(groupId, consensus); + consensus.start(false); // pipe will start after creating + } catch (IOException e) { + LOGGER.warn("Cannot create local peer for group {} with peers {}", groupId, peers, e); + throw new ConsensusException(e); + } finally { + stateMachineMapLock.unlock(); + } + } + + @Override + public void deleteLocalPeer(ConsensusGroupId groupId) throws ConsensusException { + if (!stateMachineMap.containsKey(groupId)) { + throw new ConsensusGroupNotExistException(groupId); + } + + try { + stateMachineMapLock.lock(); + + final PipeConsensusServerImpl consensus = stateMachineMap.get(groupId); + consensus.clear(); + + FileUtils.deleteFileOrDirectory(new File(getPeerDir(groupId))); + } catch (IOException e) { + LOGGER.warn("Cannot delete local peer for group {}", groupId, e); + throw new ConsensusException(e); + } finally { + stateMachineMapLock.unlock(); + } + } + + @Override + public void addRemotePeer(ConsensusGroupId groupId, Peer peer) throws ConsensusException { + PipeConsensusServerImpl impl = + Optional.ofNullable(stateMachineMap.get(groupId)) + .orElseThrow(() -> new ConsensusGroupNotExistException(groupId)); + if (impl.containsPeer(peer)) { + throw new PeerAlreadyInConsensusGroupException(groupId, peer); + } + try { + // step 1: inactive new Peer to prepare for following steps + LOGGER.info("[{}] inactivate new peer: {}", CLASS_NAME, peer); + impl.setRemotePeerActive(peer, false); + + // step 2: notify all the other Peers to create consensus pipes to newPeer + LOGGER.info("[{}] notify current peers to create consensus pipes...", CLASS_NAME); + impl.notifyPeersToCreateConsensusPipes(peer); + + // step 3: wait until all the other Peers finish transferring + LOGGER.info("[{}] wait until all the other peers finish transferring...", CLASS_NAME); + impl.waitPeersToTargetPeerTransmissionCompleted(peer); + + // step 4: active new Peer + LOGGER.info("[{}] activate new peer...", CLASS_NAME); + impl.setRemotePeerActive(peer, true); + } catch (ConsensusGroupModifyPeerException e) { + try { + LOGGER.info("[{}] add remote peer failed, automatic cleanup side effects...", CLASS_NAME); + + // roll back + impl.notifyPeersToDropConsensusPipe(peer); + + } catch (ConsensusGroupModifyPeerException mpe) { + LOGGER.error( + "[{}] failed to cleanup side effects after failed to add remote peer", CLASS_NAME, mpe); + } + throw new ConsensusException(e); + } + } + + @Override + public void removeRemotePeer(ConsensusGroupId groupId, Peer peer) throws ConsensusException { + PipeConsensusServerImpl impl = + Optional.ofNullable(stateMachineMap.get(groupId)) + .orElseThrow(() -> new ConsensusGroupNotExistException(groupId)); + if (!impl.containsPeer(peer)) { + throw new PeerNotInConsensusGroupException(groupId, peer.toString()); + } + + try { + // let other peers remove the consensus pipe to target peer + impl.notifyPeersToDropConsensusPipe(peer); + // let target peer reject new write + impl.setRemotePeerActive(peer, false); + // wait its consensus pipes to complete + impl.waitTargetPeerToPeersTransmissionCompleted(peer); + } catch (ConsensusGroupModifyPeerException e) { + throw new ConsensusException(e.getMessage()); + } + } + + @Override + public void resetPeerList(ConsensusGroupId groupId, List<Peer> correctPeers) + throws ConsensusException { + PipeConsensusServerImpl impl = + Optional.ofNullable(stateMachineMap.get(groupId)) + .orElseThrow(() -> new ConsensusGroupNotExistException(groupId)); + if (!correctPeers.contains(new Peer(groupId, thisNodeId, thisNode))) { + LOGGER.warn( + "[RESET PEER LIST] Local peer is not in the correct configuration, delete local peer {}", + groupId); + deleteLocalPeer(groupId); + return; + } + String previousPeerListStr = impl.getPeers().toString(); + for (Peer peer : impl.getPeers()) { + if (!correctPeers.contains(peer)) { + try { + impl.dropConsensusPipeToTargetPeer(peer); + } catch (ConsensusGroupModifyPeerException e) { + LOGGER.error( + "[RESET PEER LIST] Failed to remove peer {}'s consensus pipe from group {}", + peer, + groupId, + e); + } + } + } + LOGGER.info( + "[RESET PEER LIST] Local peer list has been reset: {} -> {}", + previousPeerListStr, + impl.getPeers()); + for (Peer peer : correctPeers) { + if (!impl.containsPeer(peer)) { + LOGGER.warn("[RESET PEER LIST] \"Correct peer\" {} is not in local peer list", peer); + } + } + } + + @Override + public void transferLeader(ConsensusGroupId groupId, Peer newLeader) throws ConsensusException { + throw new ConsensusException(String.format("%s does not support leader transfer", CLASS_NAME)); + } + + @Override + public void triggerSnapshot(ConsensusGroupId groupId, boolean force) throws ConsensusException { + Optional.ofNullable(stateMachineMap.get(groupId)) + .orElseThrow(() -> new ConsensusGroupNotExistException(groupId)); + // Do nothing here because we do not need to transfer snapshot when there are new peers + } + + @Override + public boolean isLeader(ConsensusGroupId groupId) { + return true; + } + + @Override + public long getLogicalClock(ConsensusGroupId groupId) { + // TODO: check logical clock + return 0; + } + + @Override + public boolean isLeaderReady(ConsensusGroupId groupId) { + return true; + } + + @Override + public Peer getLeader(ConsensusGroupId groupId) { + if (!stateMachineMap.containsKey(groupId)) { + return null; + } + return new Peer(groupId, thisNodeId, thisNode); + } + + @Override + public List<ConsensusGroupId> getAllConsensusGroupIds() { + return new ArrayList<>(stateMachineMap.keySet()); + } + + @Override + public List<ConsensusGroupId> getAllConsensusGroupIdsWithoutStarting() { + return getConsensusGroupIdsFromDir(storageDir, LOGGER); + } + + @Override + public String getRegionDirFromConsensusGroupId(ConsensusGroupId groupId) { + return getPeerDir(groupId); + } + + @Override + public void reloadConsensusConfig(ConsensusConfig consensusConfig) { + // TODO: impl for hot config loading + } + + public PipeConsensusServerImpl getImpl(ConsensusGroupId groupId) { + return stateMachineMap.get(groupId); + } + + //////////////////////////// APIs provided for Test //////////////////////////// + @TestOnly + public int getPipeCount() { + return this.consensusPipeManager.getAllConsensusPipe().size(); + } +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusPeerManager.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusPeerManager.java new file mode 100644 index 000000000000..6376097d15f7 --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusPeerManager.java @@ -0,0 +1,154 @@ +/* + * 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.consensus.pipe; + +import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.consensus.common.Peer; + +import com.google.common.collect.ImmutableList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Stream; + +public class PipeConsensusPeerManager { + + private static final String CONFIGURATION_FILE_NAME = "configuration.dat"; + private static final Logger LOGGER = LoggerFactory.getLogger(PipeConsensusPeerManager.class); + + private final String storageDir; + private final Set<Peer> peers; + + public PipeConsensusPeerManager(String storageDir, List<Peer> peers) { + this.storageDir = storageDir; + this.peers = Collections.newSetFromMap(new ConcurrentHashMap<>()); + + this.peers.addAll(peers); + if (this.peers.size() != peers.size()) { + LOGGER.warn("Duplicate peers in the input list, ignore the duplicates."); + } + } + + public void recover() throws IOException { + try (Stream<Path> pathStream = Files.walk(Paths.get(storageDir), 1)) { + Path[] configurationPaths = + pathStream + .filter(Files::isRegularFile) + .filter(path -> path.getFileName().toString().endsWith(CONFIGURATION_FILE_NAME)) + .toArray(Path[]::new); + ByteBuffer readBuffer; + for (Path path : configurationPaths) { + readBuffer = ByteBuffer.wrap(Files.readAllBytes(path)); + peers.add(Peer.deserialize(readBuffer)); + } + } + } + + private void persist(Peer peer) throws IOException { + File configurationFile = new File(storageDir, generateConfigurationFileName(peer)); + if (configurationFile.exists()) { + LOGGER.warn("Configuration file {} already exists, delete it.", configurationFile); + FileUtils.deleteFileOrDirectory(configurationFile); + } + + try (FileOutputStream fileOutputStream = new FileOutputStream(configurationFile)) { + try (DataOutputStream dataOutputStream = new DataOutputStream(fileOutputStream)) { + peer.serialize(dataOutputStream); + } finally { + try { + fileOutputStream.flush(); + fileOutputStream.getFD().sync(); + } catch (IOException ignore) { + // ignore sync exception + } + } + } + } + + private String generateConfigurationFileName(Peer peer) { + return peer.getNodeId() + "_" + CONFIGURATION_FILE_NAME; + } + + public void persistAll() throws IOException { + for (Peer peer : peers) { + persist(peer); + } + } + + public boolean contains(Peer peer) { + return peers.contains(peer); + } + + public void addAndPersist(Peer peer) throws IOException { + peers.add(peer); + persist(peer); + } + + public void removeAndPersist(Peer peer) throws IOException { + Files.deleteIfExists(Paths.get(storageDir, generateConfigurationFileName(peer))); + peers.remove(peer); + } + + public List<Peer> getOtherPeers(Peer thisNode) { + return peers.stream() + .filter(peer -> !peer.equals(thisNode)) + .collect(ImmutableList.toImmutableList()); + } + + public List<Peer> getPeers() { + return ImmutableList.copyOf(peers); + } + + public void deleteAllFiles() throws IOException { + IOException exception = null; + for (Peer peer : peers) { + try { + Files.deleteIfExists(Paths.get(storageDir, generateConfigurationFileName(peer))); + } catch (IOException e) { + LOGGER.error("Failed to delete configuration file for peer {}", peer, e); + if (exception == null) { + exception = e; + } else { + exception.addSuppressed(e); + } + } + } + if (exception != null) { + throw exception; + } + } + + public void clear() throws IOException { + deleteAllFiles(); + peers.clear(); + } +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusServerImpl.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusServerImpl.java new file mode 100644 index 000000000000..f4a64fc691bf --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusServerImpl.java @@ -0,0 +1,543 @@ +/* + * 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.consensus.pipe; + +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.exception.ClientManagerException; +import org.apache.iotdb.commons.client.sync.SyncPipeConsensusServiceClient; +import org.apache.iotdb.commons.consensus.index.ComparableConsensusRequest; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; +import org.apache.iotdb.consensus.IStateMachine; +import org.apache.iotdb.consensus.common.DataSet; +import org.apache.iotdb.consensus.common.Peer; +import org.apache.iotdb.consensus.common.request.IConsensusRequest; +import org.apache.iotdb.consensus.config.PipeConsensusConfig; +import org.apache.iotdb.consensus.exception.ConsensusGroupModifyPeerException; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeManager; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeName; +import org.apache.iotdb.consensus.pipe.consensuspipe.ProgressIndexManager; +import org.apache.iotdb.consensus.pipe.thrift.TCheckConsensusPipeCompletedReq; +import org.apache.iotdb.consensus.pipe.thrift.TCheckConsensusPipeCompletedResp; +import org.apache.iotdb.consensus.pipe.thrift.TNotifyPeerToCreateConsensusPipeReq; +import org.apache.iotdb.consensus.pipe.thrift.TNotifyPeerToCreateConsensusPipeResp; +import org.apache.iotdb.consensus.pipe.thrift.TNotifyPeerToDropConsensusPipeReq; +import org.apache.iotdb.consensus.pipe.thrift.TNotifyPeerToDropConsensusPipeResp; +import org.apache.iotdb.consensus.pipe.thrift.TSetActiveReq; +import org.apache.iotdb.consensus.pipe.thrift.TSetActiveResp; +import org.apache.iotdb.pipe.api.exception.PipeException; +import org.apache.iotdb.rpc.RpcUtils; + +import com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; + +/** PipeConsensusServerImpl is a consensus server implementation for pipe consensus. */ +public class PipeConsensusServerImpl { + private static final Logger LOGGER = LoggerFactory.getLogger(PipeConsensusServerImpl.class); + private static final long CHECK_TRANSMISSION_COMPLETION_INTERVAL_IN_MILLISECONDS = 2_000L; + + private final Peer thisNode; + private final IStateMachine stateMachine; + private final Lock stateMachineLock = new ReentrantLock(); + private final PipeConsensusPeerManager peerManager; + private final AtomicBoolean active; + private final AtomicBoolean isStarted; + private final String consensusGroupId; + private final ConsensusPipeManager consensusPipeManager; + private final ProgressIndexManager progressIndexManager; + private final IClientManager<TEndPoint, SyncPipeConsensusServiceClient> syncClientManager; + + private ProgressIndex cachedProgressIndex = MinimumProgressIndex.INSTANCE; + + public PipeConsensusServerImpl( + Peer thisNode, + IStateMachine stateMachine, + String storageDir, + List<Peer> configuration, + PipeConsensusConfig config, + ConsensusPipeManager consensusPipeManager, + IClientManager<TEndPoint, SyncPipeConsensusServiceClient> syncClientManager) + throws IOException { + this.thisNode = thisNode; + this.stateMachine = stateMachine; + this.peerManager = new PipeConsensusPeerManager(storageDir, configuration); + this.active = new AtomicBoolean(true); + this.isStarted = new AtomicBoolean(false); + this.consensusGroupId = thisNode.getGroupId().toString(); + this.consensusPipeManager = consensusPipeManager; + this.progressIndexManager = config.getPipe().getProgressIndexManager(); + this.syncClientManager = syncClientManager; + + if (configuration.isEmpty()) { + peerManager.recover(); + } else { + // create consensus pipes + List<Peer> deepCopyPeersWithoutSelf = + configuration.stream() + .filter(peer -> !peer.equals(thisNode)) + .collect(Collectors.toList()); + final List<Peer> successfulPipes = createConsensusPipes(deepCopyPeersWithoutSelf); + if (successfulPipes.size() < deepCopyPeersWithoutSelf.size()) { + // roll back + updateConsensusPipesStatus(successfulPipes, PipeStatus.DROPPED); + throw new IOException(String.format("%s cannot create all consensus pipes", thisNode)); + } + + // persist peers' info + try { + peerManager.persistAll(); + } catch (Exception e) { + // roll back + LOGGER.warn("{} cannot persist all peers", thisNode, e); + peerManager.deleteAllFiles(); + updateConsensusPipesStatus(successfulPipes, PipeStatus.DROPPED); + throw e; + } + } + } + + public synchronized void start(boolean startConsensusPipes) throws IOException { + stateMachine.start(); + + if (startConsensusPipes) { + // start all consensus pipes + final List<Peer> otherPeers = peerManager.getOtherPeers(thisNode); + final List<Peer> successfulPipes = + updateConsensusPipesStatus(new ArrayList<>(otherPeers), PipeStatus.RUNNING); + if (successfulPipes.size() < otherPeers.size()) { + // roll back + updateConsensusPipesStatus(successfulPipes, PipeStatus.STOPPED); + throw new IOException(String.format("%s cannot start all consensus pipes", thisNode)); + } + } + isStarted.set(true); + } + + public synchronized void stop() { + // stop all consensus pipes + final List<Peer> otherPeers = peerManager.getOtherPeers(thisNode); + final List<Peer> successfulPipes = + updateConsensusPipesStatus(new ArrayList<>(otherPeers), PipeStatus.STOPPED); + if (successfulPipes.size() < otherPeers.size()) { + // do not roll back, because it will stop anyway + LOGGER.warn("{} cannot stop all consensus pipes", thisNode); + } + + stateMachine.stop(); + isStarted.set(false); + } + + public synchronized void clear() throws IOException { + final List<Peer> otherPeers = peerManager.getOtherPeers(thisNode); + final List<Peer> successfulPipes = + updateConsensusPipesStatus(new ArrayList<>(otherPeers), PipeStatus.DROPPED); + if (successfulPipes.size() < otherPeers.size()) { + // do not roll back, because it will clear anyway + LOGGER.warn("{} cannot drop all consensus pipes", thisNode); + } + + peerManager.clear(); + stateMachine.stop(); + isStarted.set(false); + active.set(false); + } + + private List<Peer> createConsensusPipes(List<Peer> peers) { + return peers.stream() + .filter( + peer -> { + try { + if (!peers.equals(thisNode)) { + consensusPipeManager.createConsensusPipe(thisNode, peer); + } + return true; + } catch (Exception e) { + LOGGER.warn("{} cannot create consensus pipe between {} and {}", thisNode, peer, e); + return false; + } + }) + .collect(Collectors.toList()); + } + + private List<Peer> updateConsensusPipesStatus(List<Peer> peers, PipeStatus status) { + return peers.stream() + .filter( + peer -> { + try { + if (!peer.equals(thisNode)) { + consensusPipeManager.updateConsensusPipe( + new ConsensusPipeName(thisNode, peer), status); + } + return true; + } catch (Exception e) { + LOGGER.warn( + "{} cannot update consensus pipe between {} and {} to status {}", + thisNode, + peer, + status, + e); + return false; + } + }) + .collect(Collectors.toList()); + } + + public synchronized void checkConsensusPipe(Map<ConsensusPipeName, PipeStatus> existedPipes) { + final PipeStatus expectedStatus = isStarted.get() ? PipeStatus.RUNNING : PipeStatus.STOPPED; + final Map<ConsensusPipeName, Peer> expectedPipes = + peerManager.getOtherPeers(thisNode).stream() + .collect( + ImmutableMap.toImmutableMap( + peer -> new ConsensusPipeName(thisNode, peer), peer -> peer)); + + existedPipes.forEach( + (existedName, existedStatus) -> { + if (!expectedPipes.containsKey(existedName)) { + try { + LOGGER.warn("{} drop consensus pipe [{}]", consensusGroupId, existedName); + consensusPipeManager.updateConsensusPipe(existedName, PipeStatus.DROPPED); + } catch (Exception e) { + LOGGER.warn("{} cannot drop consensus pipe [{}]", consensusGroupId, existedName, e); + } + } else if (!expectedStatus.equals(existedStatus)) { + try { + LOGGER.warn( + "{} update consensus pipe [{}] to status {}", + consensusGroupId, + existedName, + expectedStatus); + consensusPipeManager.updateConsensusPipe(existedName, expectedStatus); + } catch (Exception e) { + LOGGER.warn( + "{} cannot update consensus pipe [{}] to status {}", + consensusGroupId, + existedName, + expectedStatus, + e); + } + } + }); + + expectedPipes.forEach( + (expectedName, expectedPeer) -> { + if (!existedPipes.containsKey(expectedName)) { + try { + LOGGER.warn( + "{} create and update consensus pipe [{}] to status {}", + consensusGroupId, + expectedName, + expectedStatus); + consensusPipeManager.createConsensusPipe(thisNode, expectedPeer); + consensusPipeManager.updateConsensusPipe(expectedName, expectedStatus); + } catch (Exception e) { + LOGGER.warn( + "{} cannot create and update consensus pipe [{}] to status {}", + consensusGroupId, + expectedName, + expectedStatus, + e); + } + } + }); + } + + public TSStatus write(IConsensusRequest request) { + try { + stateMachineLock.lock(); + if (request instanceof ComparableConsensusRequest) { + ((ComparableConsensusRequest) request) + .setProgressIndex(progressIndexManager.assignProgressIndex(thisNode.getGroupId())); + } + return stateMachine.write(request); + } finally { + stateMachineLock.unlock(); + } + } + + public TSStatus writeOnFollowerReplica(IConsensusRequest request) { + try { + stateMachineLock.lock(); + return stateMachine.write(request); + } finally { + stateMachineLock.unlock(); + } + } + + public DataSet read(IConsensusRequest request) { + return stateMachine.read(request); + } + + public void setRemotePeerActive(Peer peer, boolean isActive) + throws ConsensusGroupModifyPeerException { + try (SyncPipeConsensusServiceClient client = + syncClientManager.borrowClient(peer.getEndpoint())) { + try { + TSetActiveResp res = + client.setActive( + new TSetActiveReq(peer.getGroupId().convertToTConsensusGroupId(), isActive)); + if (!RpcUtils.SUCCESS_STATUS.equals(res.getStatus())) { + throw new ConsensusGroupModifyPeerException( + String.format( + "error when set peer %s to active %s. result status: %s", + peer, isActive, res.getStatus())); + } + } catch (Exception e) { + throw new ConsensusGroupModifyPeerException( + String.format("error when set peer %s to active %s", peer, isActive), e); + } + } catch (ClientManagerException e) { + throw new ConsensusGroupModifyPeerException(e); + } + } + + public void notifyPeersToCreateConsensusPipes(Peer targetPeer) + throws ConsensusGroupModifyPeerException { + final List<Peer> otherPeers = peerManager.getOtherPeers(thisNode); + Exception exception = null; + for (Peer peer : otherPeers) { + try (SyncPipeConsensusServiceClient client = + syncClientManager.borrowClient(peer.getEndpoint())) { + TNotifyPeerToCreateConsensusPipeResp resp = + client.notifyPeerToCreateConsensusPipe( + new TNotifyPeerToCreateConsensusPipeReq( + targetPeer.getGroupId().convertToTConsensusGroupId(), + targetPeer.getEndpoint(), + targetPeer.getNodeId())); + if (!RpcUtils.SUCCESS_STATUS.equals(resp.getStatus())) { + throw new ConsensusGroupModifyPeerException( + String.format("error when notify peer %s to create consensus pipe", peer)); + } + } catch (Exception e) { + exception = e; + LOGGER.warn("{} cannot notify peer {} to create consensus pipe", thisNode, peer, e); + } + } + + createConsensusPipeToTargetPeer(targetPeer); + if (exception != null) { + throw new ConsensusGroupModifyPeerException(exception); + } + } + + public synchronized void createConsensusPipeToTargetPeer(Peer targetPeer) + throws ConsensusGroupModifyPeerException { + try { + consensusPipeManager.createConsensusPipe(thisNode, targetPeer); + peerManager.addAndPersist(targetPeer); + } catch (IOException e) { + LOGGER.warn("{} cannot persist peer {}", thisNode, targetPeer, e); + throw new ConsensusGroupModifyPeerException( + String.format("%s cannot persist peer %s", thisNode, targetPeer), e); + } catch (Exception e) { + LOGGER.warn("{} cannot create consensus pipe to {}", thisNode, targetPeer, e); + throw new ConsensusGroupModifyPeerException( + String.format("%s cannot create consensus pipe to %s", thisNode, targetPeer), e); + } + } + + public void notifyPeersToDropConsensusPipe(Peer targetPeer) + throws ConsensusGroupModifyPeerException { + final List<Peer> otherPeers = peerManager.getOtherPeers(thisNode); + Exception exception = null; + for (Peer peer : otherPeers) { + try (SyncPipeConsensusServiceClient client = + syncClientManager.borrowClient(peer.getEndpoint())) { + TNotifyPeerToDropConsensusPipeResp resp = + client.notifyPeerToDropConsensusPipe( + new TNotifyPeerToDropConsensusPipeReq( + targetPeer.getGroupId().convertToTConsensusGroupId(), + targetPeer.getEndpoint(), + targetPeer.getNodeId())); + if (!RpcUtils.SUCCESS_STATUS.equals(resp.getStatus())) { + throw new ConsensusGroupModifyPeerException( + String.format("error when notify peer %s to drop consensus pipe", peer)); + } + } catch (Exception e) { + exception = e; + LOGGER.warn("{} cannot notify peer {} to drop consensus pipe", thisNode, peer, e); + } + } + + dropConsensusPipeToTargetPeer(targetPeer); + if (exception != null) { + throw new ConsensusGroupModifyPeerException(exception); + } + } + + public synchronized void dropConsensusPipeToTargetPeer(Peer targetPeer) + throws ConsensusGroupModifyPeerException { + try { + consensusPipeManager.dropConsensusPipe(thisNode, targetPeer); + peerManager.removeAndPersist(targetPeer); + } catch (IOException e) { + LOGGER.warn("{} cannot persist peer {}", thisNode, targetPeer, e); + throw new ConsensusGroupModifyPeerException( + String.format("%s cannot persist peer %s", thisNode, targetPeer), e); + } catch (Exception e) { + LOGGER.warn("{} cannot drop consensus pipe to {}", thisNode, targetPeer, e); + throw new ConsensusGroupModifyPeerException( + String.format("%s cannot drop consensus pipe to %s", thisNode, targetPeer), e); + } + } + + public void waitPeersToTargetPeerTransmissionCompleted(Peer targetPeer) + throws ConsensusGroupModifyPeerException { + boolean isTransmissionCompleted = false; + boolean isFirstCheck = true; + + try { + while (!isTransmissionCompleted) { + Thread.sleep(CHECK_TRANSMISSION_COMPLETION_INTERVAL_IN_MILLISECONDS); + + if (isConsensusPipesTransmissionCompleted( + Collections.singletonList(new ConsensusPipeName(thisNode, targetPeer).toString()), + isFirstCheck)) { + final List<Peer> otherPeers = peerManager.getOtherPeers(thisNode); + + isTransmissionCompleted = true; + for (Peer peer : otherPeers) { + isTransmissionCompleted &= + isRemotePeerConsensusPipesTransmissionCompleted( + peer, + Collections.singletonList(new ConsensusPipeName(peer, targetPeer).toString()), + isFirstCheck); + } + } + + isFirstCheck = false; + } + } catch (InterruptedException e) { + LOGGER.warn("{} is interrupted when waiting for transfer completed", thisNode, e); + Thread.currentThread().interrupt(); + throw new ConsensusGroupModifyPeerException( + String.format("%s is interrupted when waiting for transfer completed", thisNode), e); + } + } + + public void waitTargetPeerToPeersTransmissionCompleted(Peer targetPeer) + throws ConsensusGroupModifyPeerException { + boolean isTransmissionCompleted = false; + boolean isFirstCheck = true; + + try { + while (!isTransmissionCompleted) { + Thread.sleep(CHECK_TRANSMISSION_COMPLETION_INTERVAL_IN_MILLISECONDS); + + final List<String> consensusPipeNames = + peerManager.getPeers().stream() + .map(peer -> new ConsensusPipeName(targetPeer, peer).toString()) + .collect(Collectors.toList()); + isTransmissionCompleted = + isRemotePeerConsensusPipesTransmissionCompleted( + targetPeer, consensusPipeNames, isFirstCheck); + + isFirstCheck = false; + } + } catch (InterruptedException e) { + LOGGER.warn("{} is interrupted when waiting for transfer completed", thisNode, e); + Thread.currentThread().interrupt(); + throw new ConsensusGroupModifyPeerException( + String.format("%s is interrupted when waiting for transfer completed", thisNode), e); + } + } + + private boolean isRemotePeerConsensusPipesTransmissionCompleted( + Peer targetPeer, List<String> consensusPipeNames, boolean refreshCachedProgressIndex) + throws ConsensusGroupModifyPeerException { + try (SyncPipeConsensusServiceClient client = + syncClientManager.borrowClient(targetPeer.getEndpoint())) { + TCheckConsensusPipeCompletedResp resp = + client.checkConsensusPipeCompleted( + new TCheckConsensusPipeCompletedReq( + thisNode.getGroupId().convertToTConsensusGroupId(), + consensusPipeNames, + refreshCachedProgressIndex)); + if (!RpcUtils.SUCCESS_STATUS.equals(resp.getStatus())) { + LOGGER.warn( + "{} cannot check consensus pipes transmission completed to peer {}", + thisNode, + targetPeer); + throw new ConsensusGroupModifyPeerException( + String.format( + "error when check consensus pipes transmission completed to peer %s", targetPeer)); + } + return resp.isCompleted; + } catch (Exception e) { + LOGGER.warn("{} cannot check consensus pipes transmission completed", thisNode, e); + throw new ConsensusGroupModifyPeerException( + String.format("%s cannot check consensus pipes transmission completed", thisNode), e); + } + } + + public synchronized boolean isConsensusPipesTransmissionCompleted( + List<String> consensusPipeNames, boolean refreshCachedProgressIndex) { + if (refreshCachedProgressIndex) { + cachedProgressIndex = + cachedProgressIndex.updateToMinimumEqualOrIsAfterProgressIndex( + progressIndexManager.getMaxAssignedProgressIndex(thisNode.getGroupId())); + } + + try { + return consensusPipeNames.stream() + .noneMatch( + name -> + cachedProgressIndex.isAfter( + progressIndexManager.getProgressIndex(new ConsensusPipeName(name)))); + } catch (PipeException e) { + LOGGER.info(e.getMessage()); + return false; + } + } + + public boolean isReadOnly() { + return stateMachine.isReadOnly(); + } + + public boolean isActive() { + return active.get(); + } + + public void setActive(boolean active) { + LOGGER.info("set {} active status to {}", this.thisNode, active); + this.active.set(active); + } + + public boolean containsPeer(Peer peer) { + return peerManager.contains(peer); + } + + public List<Peer> getPeers() { + return peerManager.getPeers(); + } +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeDispatcher.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeDispatcher.java new file mode 100644 index 000000000000..ac7045a55147 --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeDispatcher.java @@ -0,0 +1,41 @@ +/* + * 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.consensus.pipe.consensuspipe; + +import java.util.Map; + +public interface ConsensusPipeDispatcher { + void createPipe( + String pipeName, + Map<String, String> extractorAttributes, + Map<String, String> processorAttributes, + Map<String, String> connectorAttributes) + throws Exception; + + void startPipe(String pipeName) throws Exception; + + void stopPipe(String pipeName) throws Exception; + + /** + * Use ConsensusPipeName instead of String to provide information for receiverAgent to release + * corresponding resource + */ + void dropPipe(ConsensusPipeName pipeName) throws Exception; +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeGuardian.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeGuardian.java new file mode 100644 index 000000000000..6c1e9cc1eb13 --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeGuardian.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.consensus.pipe.consensuspipe; + +public interface ConsensusPipeGuardian { + void start(String id, Runnable guardJob, long intervalInSeconds); + + void stop(); +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java new file mode 100644 index 000000000000..bb1bf2a17694 --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java @@ -0,0 +1,108 @@ +/* + * 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.consensus.pipe.consensuspipe; + +import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; +import org.apache.iotdb.consensus.common.Peer; +import org.apache.iotdb.consensus.config.PipeConsensusConfig; +import org.apache.iotdb.consensus.config.PipeConsensusConfig.ReplicateMode; + +import com.google.common.collect.ImmutableMap; + +import java.util.Map; + +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_CONSENSUS_GROUP_ID_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_IP_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_PARALLEL_TASKS_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_PORT_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_CONSENSUS_GROUP_ID_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_CONSENSUS_RECEIVER_DATANODE_ID_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_CONSENSUS_SENDER_DATANODE_ID_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_REALTIME_MODE_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_KEY; + +public class ConsensusPipeManager { + private final PipeConsensusConfig.Pipe config; + private final ReplicateMode replicateMode; + private final ConsensusPipeDispatcher dispatcher; + private final ConsensusPipeSelector selector; + + public ConsensusPipeManager(PipeConsensusConfig.Pipe config, ReplicateMode replicateMode) { + this.config = config; + this.replicateMode = replicateMode; + this.dispatcher = config.getConsensusPipeDispatcher(); + this.selector = config.getConsensusPipeSelector(); + } + + public void createConsensusPipe(Peer senderPeer, Peer receiverPeer) throws Exception { + ConsensusPipeName consensusPipeName = new ConsensusPipeName(senderPeer, receiverPeer); + dispatcher.createPipe( + consensusPipeName.toString(), + ImmutableMap.<String, String>builder() + .put(EXTRACTOR_KEY, config.getExtractorPluginName()) + .put( + EXTRACTOR_CONSENSUS_GROUP_ID_KEY, + consensusPipeName.getConsensusGroupId().toString()) + .put( + EXTRACTOR_CONSENSUS_SENDER_DATANODE_ID_KEY, + String.valueOf(consensusPipeName.getSenderDataNodeId())) + .put( + EXTRACTOR_CONSENSUS_RECEIVER_DATANODE_ID_KEY, + String.valueOf(consensusPipeName.getReceiverDataNodeId())) + .put(EXTRACTOR_REALTIME_MODE_KEY, replicateMode.getValue()) + .build(), + ImmutableMap.<String, String>builder() + .put(PROCESSOR_KEY, config.getProcessorPluginName()) + .build(), + ImmutableMap.<String, String>builder() + .put(CONNECTOR_KEY, config.getConnectorPluginName()) + .put( + CONNECTOR_CONSENSUS_GROUP_ID_KEY, + String.valueOf(consensusPipeName.getConsensusGroupId().getId())) + .put(CONNECTOR_IOTDB_IP_KEY, receiverPeer.getEndpoint().ip) + .put(CONNECTOR_IOTDB_PORT_KEY, String.valueOf(receiverPeer.getEndpoint().port)) + .put(CONNECTOR_IOTDB_PARALLEL_TASKS_KEY, String.valueOf(1)) + .build()); + } + + public void dropConsensusPipe(Peer senderPeer, Peer receiverPeer) throws Exception { + ConsensusPipeName consensusPipeName = new ConsensusPipeName(senderPeer, receiverPeer); + dispatcher.dropPipe(consensusPipeName); + } + + public void updateConsensusPipe(ConsensusPipeName consensusPipeName, PipeStatus pipeStatus) + throws Exception { + if (PipeStatus.RUNNING.equals(pipeStatus)) { + dispatcher.startPipe(consensusPipeName.toString()); + } else if (PipeStatus.STOPPED.equals(pipeStatus)) { + dispatcher.stopPipe(consensusPipeName.toString()); + } else if (PipeStatus.DROPPED.equals(pipeStatus)) { + dispatcher.dropPipe(consensusPipeName); + } else { + throw new IllegalArgumentException("Unsupported pipe status: " + pipeStatus); + } + } + + public Map<ConsensusPipeName, PipeStatus> getAllConsensusPipe() { + return selector.getAllConsensusPipe(); + } +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeName.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeName.java new file mode 100644 index 000000000000..c653b0fee37b --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeName.java @@ -0,0 +1,98 @@ +/* + * 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.consensus.pipe.consensuspipe; + +import org.apache.iotdb.commons.consensus.ConsensusGroupId; +import org.apache.iotdb.commons.pipe.task.meta.PipeStaticMeta; +import org.apache.iotdb.consensus.common.Peer; + +import java.util.Objects; + +public class ConsensusPipeName { + private static final String CONSENSUS_PIPE_NAME_SPLITTER_CHAR = "_"; + private final ConsensusGroupId consensusGroupId; + private final int senderDataNodeId; + private final int receiverDataNodeId; + + public ConsensusPipeName(Peer senderPeer, Peer receiverPeer) { + this.consensusGroupId = senderPeer.getGroupId(); + this.senderDataNodeId = senderPeer.getNodeId(); + this.receiverDataNodeId = receiverPeer.getNodeId(); + } + + public ConsensusPipeName( + ConsensusGroupId consensusGroupId, int senderDataNodeId, int receiverDataNodeId) { + this.consensusGroupId = consensusGroupId; + this.senderDataNodeId = senderDataNodeId; + this.receiverDataNodeId = receiverDataNodeId; + } + + public ConsensusPipeName(String pipeName) throws IllegalArgumentException { + if (!pipeName.startsWith(PipeStaticMeta.CONSENSUS_PIPE_PREFIX)) { + throw new IllegalArgumentException("Invalid pipe name: " + pipeName); + } + String[] pipeNameParts = + pipeName + .substring(PipeStaticMeta.CONSENSUS_PIPE_PREFIX.length()) + .split(CONSENSUS_PIPE_NAME_SPLITTER_CHAR); + if (pipeNameParts.length != 3) { + throw new IllegalArgumentException("Invalid pipe name: " + pipeName); + } + this.consensusGroupId = ConsensusGroupId.Factory.createFromString(pipeNameParts[0]); + this.senderDataNodeId = Integer.parseInt(pipeNameParts[1]); + this.receiverDataNodeId = Integer.parseInt(pipeNameParts[2]); + } + + public ConsensusGroupId getConsensusGroupId() { + return consensusGroupId; + } + + public int getSenderDataNodeId() { + return senderDataNodeId; + } + + public int getReceiverDataNodeId() { + return receiverDataNodeId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ConsensusPipeName that = (ConsensusPipeName) o; + return Objects.equals(consensusGroupId, that.consensusGroupId) + && Objects.equals(senderDataNodeId, that.senderDataNodeId) + && Objects.equals(receiverDataNodeId, that.receiverDataNodeId); + } + + @Override + public int hashCode() { + return Objects.hash(consensusGroupId, senderDataNodeId, receiverDataNodeId); + } + + @Override + public String toString() { + return String.join( + CONSENSUS_PIPE_NAME_SPLITTER_CHAR, + PipeStaticMeta.CONSENSUS_PIPE_PREFIX + consensusGroupId, + String.valueOf(senderDataNodeId), + String.valueOf(receiverDataNodeId)); + } +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeReceiver.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeReceiver.java new file mode 100644 index 000000000000..ba0c89fbc6fd --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeReceiver.java @@ -0,0 +1,29 @@ +/* + * 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.consensus.pipe.consensuspipe; + +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; + +public interface ConsensusPipeReceiver { + TPipeConsensusTransferResp receive(TPipeConsensusTransferReq req); + + void handleDropPipeConsensusTask(ConsensusPipeName pipeName); +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeSelector.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeSelector.java new file mode 100644 index 000000000000..1b130a0ea95c --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeSelector.java @@ -0,0 +1,28 @@ +/* + * 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.consensus.pipe.consensuspipe; + +import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; + +import java.util.Map; + +public interface ConsensusPipeSelector { + Map<ConsensusPipeName, PipeStatus> getAllConsensusPipe(); +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ProgressIndexManager.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ProgressIndexManager.java new file mode 100644 index 000000000000..e5dc2e922ab6 --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ProgressIndexManager.java @@ -0,0 +1,31 @@ +/* + * 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.consensus.pipe.consensuspipe; + +import org.apache.iotdb.commons.consensus.ConsensusGroupId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; + +public interface ProgressIndexManager { + ProgressIndex getProgressIndex(ConsensusPipeName consensusPipeName); + + ProgressIndex assignProgressIndex(ConsensusGroupId consensusGroupId); + + ProgressIndex getMaxAssignedProgressIndex(ConsensusGroupId consensusGroupId); +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCService.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCService.java new file mode 100644 index 000000000000..83c80574e329 --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCService.java @@ -0,0 +1,97 @@ +/* + * 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.consensus.pipe.service; + +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.concurrent.ThreadName; +import org.apache.iotdb.commons.exception.runtime.RPCServiceException; +import org.apache.iotdb.commons.service.ServiceType; +import org.apache.iotdb.commons.service.ThriftService; +import org.apache.iotdb.commons.service.ThriftServiceThread; +import org.apache.iotdb.consensus.config.PipeConsensusConfig; +import org.apache.iotdb.consensus.pipe.thrift.PipeConsensusIService; +import org.apache.iotdb.rpc.ZeroCopyRpcTransportFactory; + +import org.apache.thrift.TBaseAsyncProcessor; + +public class PipeConsensusRPCService extends ThriftService implements PipeConsensusRPCServiceMBean { + + private final TEndPoint thisNode; + private final PipeConsensusConfig config; + private PipeConsensusRPCServiceProcessor pipeConsensusRPCServiceProcessor; + + public PipeConsensusRPCService(TEndPoint thisNode, PipeConsensusConfig config) { + this.thisNode = thisNode; + this.config = config; + } + + @Override + public ServiceType getID() { + return ServiceType.PIPE_CONSENSUS_SERVICE; + } + + @Override + public void initAsyncedServiceImpl(Object pipeConsensusRPCServiceProcessor) { + this.pipeConsensusRPCServiceProcessor = + (PipeConsensusRPCServiceProcessor) pipeConsensusRPCServiceProcessor; + super.initAsyncedServiceImpl(this.pipeConsensusRPCServiceProcessor); + } + + @Override + public void initTProcessor() { + processor = new PipeConsensusIService.AsyncProcessor<>(pipeConsensusRPCServiceProcessor); + } + + @Override + public void initThriftServiceThread() throws IllegalAccessException { + try { + thriftServiceThread = + new ThriftServiceThread( + (TBaseAsyncProcessor<?>) processor, + getID().getName(), + ThreadName.PIPE_CONSENSUS_RPC_PROCESSOR.getName(), + getBindIP(), + getBindPort(), + config.getRpc().getRpcSelectorThreadNum(), + config.getRpc().getRpcMinConcurrentClientNum(), + config.getRpc().getRpcMaxConcurrentClientNum(), + config.getRpc().getThriftServerAwaitTimeForStopService(), + new PipeConsensusRPCServiceHandler(pipeConsensusRPCServiceProcessor), + config.getRpc().isRpcThriftCompressionEnabled(), + config.getRpc().getConnectionTimeoutInMs(), + config.getRpc().getThriftMaxFrameSize(), + ThriftServiceThread.ServerType.SELECTOR, + ZeroCopyRpcTransportFactory.INSTANCE); + } catch (RPCServiceException e) { + throw new IllegalAccessException(e.getMessage()); + } + thriftServiceThread.setName(ThreadName.PIPE_CONSENSUS_RPC_SERVICE.getName()); + } + + @Override + public String getBindIP() { + return thisNode.getIp(); + } + + @Override + public int getBindPort() { + return thisNode.getPort(); + } +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCServiceHandler.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCServiceHandler.java new file mode 100644 index 000000000000..fd6a64269796 --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCServiceHandler.java @@ -0,0 +1,51 @@ +/* + * 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.consensus.pipe.service; + +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.server.ServerContext; +import org.apache.thrift.server.TServerEventHandler; +import org.apache.thrift.transport.TTransport; + +public class PipeConsensusRPCServiceHandler implements TServerEventHandler { + + private final PipeConsensusRPCServiceProcessor processor; + + public PipeConsensusRPCServiceHandler(PipeConsensusRPCServiceProcessor processor) { + this.processor = processor; + } + + @Override + public void preServe() {} + + @Override + public ServerContext createContext(TProtocol input, TProtocol output) { + return null; + } + + @Override + public void deleteContext(ServerContext serverContext, TProtocol input, TProtocol output) { + processor.handleExit(); + } + + @Override + public void processContext( + ServerContext serverContext, TTransport inputTransport, TTransport outputTransport) {} +} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCServiceMBean.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCServiceMBean.java new file mode 100644 index 000000000000..25f479bbdb7e --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCServiceMBean.java @@ -0,0 +1,22 @@ +/* + * 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.consensus.pipe.service; + +public interface PipeConsensusRPCServiceMBean {} diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCServiceProcessor.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCServiceProcessor.java new file mode 100644 index 000000000000..e41fffb58e00 --- /dev/null +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCServiceProcessor.java @@ -0,0 +1,210 @@ +/* + * 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.consensus.pipe.service; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.consensus.ConsensusGroupId; +import org.apache.iotdb.consensus.common.Peer; +import org.apache.iotdb.consensus.config.PipeConsensusConfig; +import org.apache.iotdb.consensus.exception.ConsensusGroupModifyPeerException; +import org.apache.iotdb.consensus.pipe.PipeConsensus; +import org.apache.iotdb.consensus.pipe.PipeConsensusServerImpl; +import org.apache.iotdb.consensus.pipe.thrift.PipeConsensusIService; +import org.apache.iotdb.consensus.pipe.thrift.TCheckConsensusPipeCompletedReq; +import org.apache.iotdb.consensus.pipe.thrift.TCheckConsensusPipeCompletedResp; +import org.apache.iotdb.consensus.pipe.thrift.TNotifyPeerToCreateConsensusPipeReq; +import org.apache.iotdb.consensus.pipe.thrift.TNotifyPeerToCreateConsensusPipeResp; +import org.apache.iotdb.consensus.pipe.thrift.TNotifyPeerToDropConsensusPipeReq; +import org.apache.iotdb.consensus.pipe.thrift.TNotifyPeerToDropConsensusPipeResp; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusBatchTransferReq; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusBatchTransferResp; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; +import org.apache.iotdb.consensus.pipe.thrift.TSetActiveReq; +import org.apache.iotdb.consensus.pipe.thrift.TSetActiveResp; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PipeConsensusRPCServiceProcessor implements PipeConsensusIService.AsyncIface { + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeConsensusRPCServiceProcessor.class); + private final PipeConsensus pipeConsensus; + + private final PipeConsensusConfig.Pipe config; + + public PipeConsensusRPCServiceProcessor( + PipeConsensus pipeConsensus, PipeConsensusConfig.Pipe config) { + this.pipeConsensus = pipeConsensus; + this.config = config; + } + + @Override + public void pipeConsensusTransfer( + TPipeConsensusTransferReq req, + AsyncMethodCallback<TPipeConsensusTransferResp> resultHandler) { + try { + TPipeConsensusTransferResp resp = config.getConsensusPipeReceiver().receive(req); + // we need to call onComplete by hand + resultHandler.onComplete(resp); + } catch (Exception e) { + resultHandler.onError(e); + } + } + + // TODO: consider batch transfer + @Override + public void pipeConsensusBatchTransfer( + TPipeConsensusBatchTransferReq req, + AsyncMethodCallback<TPipeConsensusBatchTransferResp> resultHandler) + throws TException {} + + @Override + public void setActive(TSetActiveReq req, AsyncMethodCallback<TSetActiveResp> resultHandler) + throws TException { + ConsensusGroupId groupId = + ConsensusGroupId.Factory.createFromTConsensusGroupId(req.consensusGroupId); + PipeConsensusServerImpl impl = pipeConsensus.getImpl(groupId); + if (impl == null) { + String message = + String.format("unexpected consensusGroupId %s for set active request %s", groupId, req); + LOGGER.error(message); + TSStatus status = new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + status.setMessage(message); + resultHandler.onComplete(new TSetActiveResp(status)); + return; + } + impl.setActive(req.isActive); + resultHandler.onComplete(new TSetActiveResp(RpcUtils.SUCCESS_STATUS)); + } + + @Override + public void notifyPeerToCreateConsensusPipe( + TNotifyPeerToCreateConsensusPipeReq req, + AsyncMethodCallback<TNotifyPeerToCreateConsensusPipeResp> resultHandler) + throws TException { + ConsensusGroupId groupId = + ConsensusGroupId.Factory.createFromTConsensusGroupId(req.targetPeerConsensusGroupId); + PipeConsensusServerImpl impl = pipeConsensus.getImpl(groupId); + if (impl == null) { + String message = + String.format( + "unexpected consensusGroupId %s for create consensus pipe request %s", groupId, req); + LOGGER.error(message); + TSStatus status = new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + status.setMessage(message); + resultHandler.onComplete(new TNotifyPeerToCreateConsensusPipeResp(status)); + return; + } + TSStatus responseStatus; + try { + impl.createConsensusPipeToTargetPeer( + new Peer( + ConsensusGroupId.Factory.createFromTConsensusGroupId(req.targetPeerConsensusGroupId), + req.targetPeerNodeId, + req.targetPeerEndPoint)); + responseStatus = new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } catch (ConsensusGroupModifyPeerException e) { + responseStatus = new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + responseStatus.setMessage(e.getMessage()); + LOGGER.warn("Failed to create consensus pipe to target peer with req {}", req, e); + } + resultHandler.onComplete(new TNotifyPeerToCreateConsensusPipeResp(responseStatus)); + } + + @Override + public void notifyPeerToDropConsensusPipe( + TNotifyPeerToDropConsensusPipeReq req, + AsyncMethodCallback<TNotifyPeerToDropConsensusPipeResp> resultHandler) + throws TException { + ConsensusGroupId groupId = + ConsensusGroupId.Factory.createFromTConsensusGroupId(req.targetPeerConsensusGroupId); + PipeConsensusServerImpl impl = pipeConsensus.getImpl(groupId); + if (impl == null) { + String message = + String.format( + "unexpected consensusGroupId %s for drop consensus pipe request %s", groupId, req); + LOGGER.error(message); + TSStatus status = new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + status.setMessage(message); + resultHandler.onComplete(new TNotifyPeerToDropConsensusPipeResp(status)); + return; + } + TSStatus responseStatus; + try { + impl.dropConsensusPipeToTargetPeer( + new Peer( + ConsensusGroupId.Factory.createFromTConsensusGroupId(req.targetPeerConsensusGroupId), + req.targetPeerNodeId, + req.targetPeerEndPoint)); + responseStatus = new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } catch (ConsensusGroupModifyPeerException e) { + responseStatus = new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + responseStatus.setMessage(e.getMessage()); + LOGGER.warn("Failed to drop consensus pipe to target peer with req {}", req, e); + } + resultHandler.onComplete(new TNotifyPeerToDropConsensusPipeResp(responseStatus)); + } + + @Override + public void checkConsensusPipeCompleted( + TCheckConsensusPipeCompletedReq req, + AsyncMethodCallback<TCheckConsensusPipeCompletedResp> resultHandler) + throws TException { + ConsensusGroupId groupId = + ConsensusGroupId.Factory.createFromTConsensusGroupId(req.consensusGroupId); + PipeConsensusServerImpl impl = pipeConsensus.getImpl(groupId); + if (impl == null) { + String message = + String.format( + "unexpected consensusGroupId %s for check transfer completed request %s", + groupId, req); + LOGGER.error(message); + TSStatus status = new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + status.setMessage(message); + resultHandler.onComplete(new TCheckConsensusPipeCompletedResp(status, true)); + return; + } + TSStatus responseStatus; + boolean isCompleted; + try { + isCompleted = + impl.isConsensusPipesTransmissionCompleted( + req.consensusPipeNames, req.refreshCachedProgressIndex); + responseStatus = new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } catch (Exception e) { + responseStatus = new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + responseStatus.setMessage(e.getMessage()); + isCompleted = true; + LOGGER.warn( + "Failed to check consensus pipe completed with req {}, set is completed to {}", + req, + true, + e); + } + resultHandler.onComplete(new TCheckConsensusPipeCompletedResp(responseStatus, isCompleted)); + } + + public void handleExit() {} +} diff --git a/iotdb-core/datanode/pom.xml b/iotdb-core/datanode/pom.xml index 67ee544008f7..69af0b3522fc 100644 --- a/iotdb-core/datanode/pom.xml +++ b/iotdb-core/datanode/pom.xml @@ -84,6 +84,11 @@ <artifactId>iotdb-thrift-commons</artifactId> <version>1.3.2-SNAPSHOT</version> </dependency> + <dependency> + <groupId>org.apache.iotdb</groupId> + <artifactId>iotdb-thrift-consensus</artifactId> + <version>1.3.2-SNAPSHOT</version> + </dependency> <dependency> <groupId>org.apache.iotdb</groupId> <artifactId>udf-api</artifactId> 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 6b238c69809f..4a763640eb9a 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 @@ -1102,6 +1102,9 @@ public class IoTDBConfig { private double maxMemoryRatioForQueue = 0.6; private long regionMigrationSpeedLimitBytesPerSecond = 32 * 1024 * 1024L; + // PipeConsensus Config + private int pipeConsensusPipelineSize = 5; + /** Load related */ private double maxAllocateMemoryRatioForLoad = 0.8; @@ -1120,6 +1123,8 @@ public class IoTDBConfig { /** initialized as empty, updated based on the latest `systemDir` during querying */ private String[] pipeReceiverFileDirs = new String[0]; + private String[] pipeConsensusReceiverFileDirs = new String[0]; + /** Resource control */ private boolean quotaEnable = false; @@ -1161,6 +1166,14 @@ public void setRegionMigrationSpeedLimitBytesPerSecond( this.regionMigrationSpeedLimitBytesPerSecond = regionMigrationSpeedLimitBytesPerSecond; } + public int getPipeConsensusPipelineSize() { + return pipeConsensusPipelineSize; + } + + public void setPipeConsensusPipelineSize(int pipeConsensusPipelineSize) { + this.pipeConsensusPipelineSize = pipeConsensusPipelineSize; + } + public void setMaxSizePerBatch(int maxSizePerBatch) { this.maxSizePerBatch = maxSizePerBatch; } @@ -1279,6 +1292,9 @@ private void formulateFolders() { for (int i = 0; i < pipeReceiverFileDirs.length; i++) { pipeReceiverFileDirs[i] = addDataHomeDir(pipeReceiverFileDirs[i]); } + for (int i = 0; i < pipeConsensusReceiverFileDirs.length; i++) { + pipeConsensusReceiverFileDirs[i] = addDataHomeDir(pipeConsensusReceiverFileDirs[i]); + } mqttDir = addDataHomeDir(mqttDir); extPipeDir = addDataHomeDir(extPipeDir); queryDir = addDataHomeDir(queryDir); @@ -3855,6 +3871,25 @@ public String[] getPipeReceiverFileDirs() { : this.pipeReceiverFileDirs; } + public void setPipeConsensusReceiverFileDirs(String[] pipeConsensusReceiverFileDirs) { + this.pipeConsensusReceiverFileDirs = pipeConsensusReceiverFileDirs; + } + + public String[] getPipeConsensusReceiverFileDirs() { + return (Objects.isNull(this.pipeConsensusReceiverFileDirs) + || this.pipeConsensusReceiverFileDirs.length == 0) + ? new String[] { + systemDir + + File.separator + + "pipe" + + File.separator + + "consensus" + + File.separator + + "receiver" + } + : this.pipeConsensusReceiverFileDirs; + } + public boolean isQuotaEnable() { return quotaEnable; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index ce48fc6c8eaa..bbcbc0ed7947 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -1056,10 +1056,12 @@ public void loadProperties(Properties properties) throws BadNodeUrlException, IO "datanode_schema_cache_eviction_policy", conf.getDataNodeSchemaCacheEvictionPolicy())); loadIoTConsensusProps(properties); + loadPipeConsensusProps(properties); } private void reloadConsensusProps(Properties properties) { loadIoTConsensusProps(properties); + loadPipeConsensusProps(properties); DataRegionConsensusImpl.reloadConsensusConfig(); } @@ -1100,6 +1102,17 @@ private void loadIoTConsensusProps(Properties properties) { .trim())); } + private void loadPipeConsensusProps(Properties properties) { + conf.setPipeConsensusPipelineSize( + Integer.parseInt( + properties.getProperty( + "fast_iot_consensus_pipeline_size", + Integer.toString(conf.getPipeConsensusPipelineSize())))); + if (conf.getPipeConsensusPipelineSize() <= 0) { + conf.setPipeConsensusPipelineSize(5); + } + } + private void loadAuthorCache(Properties properties) { conf.setAuthorCacheSize( Integer.parseInt( @@ -2184,6 +2197,17 @@ private void loadPipeProps(Properties properties) { .split(",")) .filter(dir -> !dir.isEmpty()) .toArray(String[]::new)); + + conf.setPipeConsensusReceiverFileDirs( + Arrays.stream( + properties + .getProperty( + "pipe_consensus_receiver_file_dirs", + String.join(",", conf.getPipeConsensusReceiverFileDirs())) + .trim() + .split(",")) + .filter(dir -> !dir.isEmpty()) + .toArray(String[]::new)); } private void loadCQProps(Properties properties) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java index 5a2caee03893..5873a0ff628e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java @@ -23,17 +23,23 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.consensus.ConsensusGroupId; import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.pipe.plugin.builtin.BuiltinPipePlugin; import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.consensus.IConsensus; import org.apache.iotdb.consensus.config.ConsensusConfig; import org.apache.iotdb.consensus.config.IoTConsensusConfig; import org.apache.iotdb.consensus.config.IoTConsensusConfig.RPC; +import org.apache.iotdb.consensus.config.PipeConsensusConfig; import org.apache.iotdb.consensus.config.RatisConfig; import org.apache.iotdb.consensus.config.RatisConfig.Snapshot; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.consensus.statemachine.dataregion.DataRegionStateMachine; import org.apache.iotdb.db.consensus.statemachine.dataregion.IoTConsensusDataRegionStateMachine; +import org.apache.iotdb.db.pipe.agent.PipeAgent; +import org.apache.iotdb.db.pipe.consensus.ConsensusPipeDataNodeDispatcher; +import org.apache.iotdb.db.pipe.consensus.ConsensusPipeDataNodeRuntimeAgentGuardian; +import org.apache.iotdb.db.pipe.consensus.ProgressIndexDataNodeManager; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; @@ -74,6 +80,7 @@ private static class DataRegionConsensusImplHolder { // Make sure both statics are initialized. static { reinitializeStatics(); + PipeAgent.receiver().pipeConsensus().initConsensusInRuntime(); } private static void reinitializeStatics() { @@ -132,6 +139,37 @@ private static ConsensusConfig buildConsensusConfig() { CONF.getRegionMigrationSpeedLimitBytesPerSecond()) .build()) .build()) + .setPipeConsensusConfig( + PipeConsensusConfig.newBuilder() + .setRPC( + PipeConsensusConfig.RPC + .newBuilder() + .setConnectionTimeoutInMs(CONF.getConnectionTimeoutInMS()) + .setRpcSelectorThreadNum(CONF.getRpcSelectorThreadCount()) + .setRpcMinConcurrentClientNum(CONF.getRpcMinConcurrentClientNum()) + .setRpcMaxConcurrentClientNum(CONF.getRpcMaxConcurrentClientNum()) + .setIsRpcThriftCompressionEnabled(CONF.isRpcThriftCompressionEnable()) + .setThriftServerAwaitTimeForStopService( + CONF.getThriftServerAwaitTimeForStopService()) + .setThriftMaxFrameSize(CONF.getThriftMaxFrameSize()) + .build()) + .setPipe( + PipeConsensusConfig.Pipe.newBuilder() + .setExtractorPluginName( + BuiltinPipePlugin.IOTDB_EXTRACTOR.getPipePluginName()) + .setProcessorPluginName( + BuiltinPipePlugin.PIPE_CONSENSUS_PROCESSOR.getPipePluginName()) + .setConnectorPluginName( + BuiltinPipePlugin.PIPE_CONSENSUS_ASYNC_CONNECTOR.getPipePluginName()) + // name + .setConsensusPipeDispatcher(new ConsensusPipeDataNodeDispatcher()) + .setConsensusPipeGuardian(new ConsensusPipeDataNodeRuntimeAgentGuardian()) + .setConsensusPipeSelector(() -> PipeAgent.task().getAllConsensusPipe()) + .setConsensusPipeReceiver(PipeAgent.receiver().pipeConsensus()) + .setProgressIndexManager(new ProgressIndexDataNodeManager()) + .setConsensusPipeGuardJobIntervalInSeconds(300) // TODO: move to config + .build()) + .build()) .setRatisConfig( RatisConfig.newBuilder() // An empty log is committed after each restart, even if no data is diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionConnectorConstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionConnectorConstructor.java index bc2da860edf7..c9b073a01db8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionConnectorConstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionConnectorConstructor.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.pipe.connector.protocol.airgap.IoTDBDataRegionAirGapConnector; import org.apache.iotdb.db.pipe.connector.protocol.legacy.IoTDBLegacyPipeConnector; import org.apache.iotdb.db.pipe.connector.protocol.opcua.OpcUaConnector; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.PipeConsensusAsyncConnector; import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector; import org.apache.iotdb.db.pipe.connector.protocol.thrift.sync.IoTDBDataRegionSyncConnector; import org.apache.iotdb.db.pipe.connector.protocol.websocket.WebSocketConnector; @@ -51,6 +52,9 @@ protected void initConstructors() { pluginConstructors.put( BuiltinPipePlugin.IOTDB_THRIFT_ASYNC_CONNECTOR.getPipePluginName(), IoTDBDataRegionAsyncConnector::new); + pluginConstructors.put( + BuiltinPipePlugin.PIPE_CONSENSUS_ASYNC_CONNECTOR.getPipePluginName(), + PipeConsensusAsyncConnector::new); pluginConstructors.put( BuiltinPipePlugin.IOTDB_LEGACY_PIPE_CONNECTOR.getPipePluginName(), IoTDBLegacyPipeConnector::new); @@ -93,5 +97,8 @@ protected void initConstructors() { BuiltinPipePlugin.WRITE_BACK_SINK.getPipePluginName(), WriteBackConnector::new); pluginConstructors.put( BuiltinPipePlugin.SUBSCRIPTION_SINK.getPipePluginName(), DoNothingConnector::new); + pluginConstructors.put( + BuiltinPipePlugin.PIPE_CONSENSUS_ASYNC_SINK.getPipePluginName(), + PipeConsensusAsyncConnector::new); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionProcessorConstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionProcessorConstructor.java index 6d747d156a2b..6f172ae5fb83 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionProcessorConstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionProcessorConstructor.java @@ -30,6 +30,7 @@ import org.apache.iotdb.db.pipe.processor.downsampling.changing.ChangingValueSamplingProcessor; import org.apache.iotdb.db.pipe.processor.downsampling.sdt.SwingingDoorTrendingSamplingProcessor; import org.apache.iotdb.db.pipe.processor.downsampling.tumbling.TumblingTimeSamplingProcessor; +import org.apache.iotdb.db.pipe.processor.pipeconsensus.PipeConsensusProcessor; import org.apache.iotdb.db.pipe.processor.twostage.plugin.TwoStageCountProcessor; class PipeDataRegionProcessorConstructor extends PipeProcessorConstructor { @@ -64,5 +65,8 @@ protected void initConstructors() { TumblingWindowingProcessor::new); pluginConstructors.put( BuiltinPipePlugin.COUNT_POINT_PROCESSOR.getPipePluginName(), TwoStageCountProcessor::new); + pluginConstructors.put( + BuiltinPipePlugin.PIPE_CONSENSUS_PROCESSOR.getPipePluginName(), + PipeConsensusProcessor::new); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/receiver/PipeDataNodeReceiverAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/receiver/PipeDataNodeReceiverAgent.java index 31d38d00f6fe..7886ab22569c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/receiver/PipeDataNodeReceiverAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/receiver/PipeDataNodeReceiverAgent.java @@ -23,6 +23,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.receiver.protocol.airgap.IoTDBAirGapReceiverAgent; import org.apache.iotdb.db.pipe.receiver.protocol.legacy.IoTDBLegacyPipeReceiverAgent; +import org.apache.iotdb.db.pipe.receiver.protocol.pipeconsensus.PipeConsensusReceiverAgent; import org.apache.iotdb.db.pipe.receiver.protocol.thrift.IoTDBDataNodeReceiverAgent; import java.io.File; @@ -34,11 +35,13 @@ public class PipeDataNodeReceiverAgent { private final IoTDBDataNodeReceiverAgent thriftAgent; private final IoTDBAirGapReceiverAgent airGapAgent; private final IoTDBLegacyPipeReceiverAgent legacyAgent; + private final PipeConsensusReceiverAgent pipeConsensusAgent; public PipeDataNodeReceiverAgent() { thriftAgent = new IoTDBDataNodeReceiverAgent(); airGapAgent = new IoTDBAirGapReceiverAgent(); legacyAgent = new IoTDBLegacyPipeReceiverAgent(); + pipeConsensusAgent = new PipeConsensusReceiverAgent(); } public IoTDBDataNodeReceiverAgent thrift() { @@ -53,11 +56,21 @@ public IoTDBLegacyPipeReceiverAgent legacy() { return legacyAgent; } + public PipeConsensusReceiverAgent pipeConsensus() { + return pipeConsensusAgent; + } + public void cleanPipeReceiverDirs() { String[] pipeReceiverFileDirs = IoTDBDescriptor.getInstance().getConfig().getPipeReceiverFileDirs(); Arrays.stream(pipeReceiverFileDirs) .map(File::new) .forEach(IoTDBReceiverAgent::cleanPipeReceiverDir); + // consensus + String[] pipeConsensusReceiverFileDirs = + IoTDBDescriptor.getInstance().getConfig().getPipeConsensusReceiverFileDirs(); + Arrays.stream(pipeConsensusReceiverFileDirs) + .map(File::new) + .forEach(IoTDBReceiverAgent::cleanPipeReceiverDir); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java index c7dc2d9c39d7..447fcbc3b6af 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.agent.runtime; import org.apache.iotdb.commons.consensus.SchemaRegionId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; import org.apache.iotdb.commons.exception.StartupException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; @@ -33,7 +34,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.pipe.extractor.schemaregion.SchemaRegionListeningQueue; -import org.apache.iotdb.db.pipe.progress.SimpleConsensusProgressIndexAssigner; +import org.apache.iotdb.db.pipe.progress.SimpleProgressIndexAssigner; import org.apache.iotdb.db.pipe.resource.PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; import org.apache.iotdb.db.service.ResourcesInformationHolder; @@ -55,8 +56,8 @@ public class PipeDataNodeRuntimeAgent implements IService { private final PipeSchemaRegionListenerManager regionListenerManager = new PipeSchemaRegionListenerManager(); - private final SimpleConsensusProgressIndexAssigner simpleConsensusProgressIndexAssigner = - new SimpleConsensusProgressIndexAssigner(); + private final SimpleProgressIndexAssigner simpleProgressIndexAssigner = + new SimpleProgressIndexAssigner(); private final PipePeriodicalJobExecutor pipePeriodicalJobExecutor = new PipePeriodicalJobExecutor(); @@ -72,7 +73,7 @@ public synchronized void preparePipeResources( PipeAgent.receiver().cleanPipeReceiverDirs(); PipeAgentLauncher.launchPipePluginAgent(resourcesInformationHolder); - simpleConsensusProgressIndexAssigner.start(); + simpleProgressIndexAssigner.start(); } @Override @@ -142,7 +143,14 @@ public boolean isSchemaLeaderReady(SchemaRegionId schemaRegionId) { ////////////////////// SimpleConsensus ProgressIndex Assigner ////////////////////// public void assignSimpleProgressIndexIfNeeded(InsertNode insertNode) { - simpleConsensusProgressIndexAssigner.assignIfNeeded(insertNode); + simpleProgressIndexAssigner.assignIfNeeded(insertNode); + } + + ////////////////////// PipeConsensus ProgressIndex Assigner ////////////////////// + + public ProgressIndex assignProgressIndexForPipeConsensus() { + return new RecoverProgressIndex( + DATA_NODE_ID, simpleProgressIndexAssigner.getSimpleProgressIndex()); } ////////////////////// Load ProgressIndex Assigner ////////////////////// @@ -153,9 +161,14 @@ public void assignProgressIndexForTsFileLoad(TsFileResource tsFileResource) { } public RecoverProgressIndex getNextProgressIndexForTsFileLoad() { + // The data node id must be negative. Because in the consensus layer Pipe, the processor of the + // pipe task will filter the progress index of the event. Leaving the recover progress index + // containing the datanode id of the datanode where the pipe task is located. The tsfile loaded + // in cannot be filtered by the processor of the datanode where the pipe task is located, + // because the load directly sends the tsfile to all replicas. Therefore, it is necessary to + // ensure that the datanode id generated by the load is negative. return new RecoverProgressIndex( - DATA_NODE_ID, - simpleConsensusProgressIndexAssigner.getSimpleProgressIndexForTsFileRecovery()); + -DATA_NODE_ID, simpleProgressIndexAssigner.getSimpleProgressIndex()); } ////////////////////// Recover ProgressIndex Assigner ////////////////////// @@ -163,14 +176,13 @@ public RecoverProgressIndex getNextProgressIndexForTsFileLoad() { public void assignProgressIndexForTsFileRecovery(TsFileResource tsFileResource) { tsFileResource.updateProgressIndex( new RecoverProgressIndex( - DATA_NODE_ID, - simpleConsensusProgressIndexAssigner.getSimpleProgressIndexForTsFileRecovery())); + DATA_NODE_ID, simpleProgressIndexAssigner.getSimpleProgressIndex())); } ////////////////////// Provided for Subscription Agent ////////////////////// public int getRebootTimes() { - return simpleConsensusProgressIndexAssigner.getRebootTimes(); + return simpleProgressIndexAssigner.getRebootTimes(); } //////////////////////////// Runtime Exception Handlers //////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 098c5b68ceea..37c04e26f508 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -32,9 +32,11 @@ import org.apache.iotdb.commons.pipe.task.meta.PipeStaticMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; +import org.apache.iotdb.commons.pipe.task.meta.PipeType; import org.apache.iotdb.commons.service.metric.MetricService; import org.apache.iotdb.commons.service.metric.enums.Tag; import org.apache.iotdb.consensus.exception.ConsensusException; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeName; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.consensus.SchemaRegionConsensusImpl; @@ -63,6 +65,7 @@ import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.pipe.api.exception.PipeException; +import com.google.common.collect.ImmutableMap; import org.apache.thrift.TException; import org.apache.tsfile.utils.Pair; import org.slf4j.Logger; @@ -81,6 +84,7 @@ import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; public class PipeDataNodeTaskAgent extends PipeTaskAgent { @@ -605,4 +609,47 @@ public Set<Integer> getPipeTaskRegionIdSet(final String pipeName, final long cre ? Collections.emptySet() : pipeMeta.getRuntimeMeta().getConsensusGroupId2TaskMetaMap().keySet(); } + + ///////////////////////// Pipe Consensus ///////////////////////// + + public ProgressIndex getPipeTaskProgressIndex(String pipeName, int consensusGroupId) { + if (!tryReadLockWithTimeOut(10)) { + throw new PipeException( + String.format( + "Failed to get pipe task progress index with pipe name: %s, consensus group id %s.", + pipeName, consensusGroupId)); + } + + try { + if (!pipeMetaKeeper.containsPipeMeta(pipeName)) { + throw new PipeException("Pipe meta not found: " + pipeName); + } + + return pipeMetaKeeper + .getPipeMeta(pipeName) + .getRuntimeMeta() + .getConsensusGroupId2TaskMetaMap() + .get(consensusGroupId) + .getProgressIndex(); + } finally { + releaseReadLock(); + } + } + + public Map<ConsensusPipeName, PipeStatus> getAllConsensusPipe() { + if (!tryReadLockWithTimeOut(10)) { + throw new PipeException("Failed to get all consensus pipe."); + } + + try { + return StreamSupport.stream(pipeMetaKeeper.getPipeMetaList().spliterator(), false) + .filter(pipeMeta -> PipeType.CONSENSUS.equals(pipeMeta.getStaticMeta().getPipeType())) + .collect( + ImmutableMap.toImmutableMap( + pipeMeta -> new ConsensusPipeName(pipeMeta.getStaticMeta().getPipeName()), + pipeMeta -> pipeMeta.getRuntimeMeta().getStatus().get())); + } finally { + releaseReadLock(); + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java new file mode 100644 index 000000000000..50da4d56839f --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java @@ -0,0 +1,524 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; +import org.apache.iotdb.commons.client.container.PipeConsensusClientMgrContainer; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant; +import org.apache.iotdb.commons.pipe.connector.protocol.IoTDBConnector; +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.handler.PipeConsensusTabletBatchEventHandler; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.handler.PipeConsensusTabletInsertNodeEventHandler; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.handler.PipeConsensusTsFileInsertionEventHandler; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.builder.PipeConsensusAsyncBatchReqBuilder; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletBinaryReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletInsertNodeReq; +import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; +import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeConnectorRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; +import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; +import org.apache.iotdb.pipe.api.exception.PipeException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Iterator; +import java.util.Objects; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +// TODO: Optimize the network and disk io for TsFile onComplete +// TODO: support Tablet Batch +public class PipeConsensusAsyncConnector extends IoTDBConnector { + + private static final Logger LOGGER = LoggerFactory.getLogger(PipeConsensusAsyncConnector.class); + + private static final String CUSTOMIZE_EXCEPTION_MSG = + "Failed to customize pipeConsensusAsyncConnector because there isn't consensusGroupId passed in. Please check your construct parameters."; + + private static final String ENQUEUE_EXCEPTION_MSG = + "Timeout: PipeConsensusConnector offers an event into transferBuffer failed, because transferBuffer is full."; + + private static final String THRIFT_ERROR_FORMATTER_WITHOUT_ENDPOINT = + "Failed to borrow client from client pool or exception occurred " + + "when sending to receiver."; + + private static final String THRIFT_ERROR_FORMATTER_WITH_ENDPOINT = + "Failed to borrow client from client pool or exception occurred " + + "when sending to receiver %s:%s."; + + private static final IoTDBConfig IOTDB_CONFIG = IoTDBDescriptor.getInstance().getConfig(); + + private static final long PIPE_CONSENSUS_EVENT_ENQUEUE_TIMEOUT_IN_MS = + IOTDB_CONFIG.getConnectionTimeoutInMS() / 6; + + private final BlockingQueue<Event> retryEventQueue = new LinkedBlockingQueue<>(); + + // We use enrichedEvent here to make use of EnrichedEvent.equalsInPipeConsensus + private final BlockingQueue<EnrichedEvent> transferBuffer = + new LinkedBlockingDeque<>(IOTDB_CONFIG.getPipeConsensusPipelineSize()); + + private final AtomicBoolean isClosed = new AtomicBoolean(false); + + private final int thisDataNodeId = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + + private int consensusGroupId; + + private PipeConsensusSyncConnector retryConnector; + + private IClientManager<TEndPoint, AsyncPipeConsensusServiceClient> asyncTransferClientManager; + + private PipeConsensusAsyncBatchReqBuilder tabletBatchBuilder; + + @Override + public void customize(PipeParameters parameters, PipeConnectorRuntimeConfiguration configuration) + throws Exception { + super.customize(parameters, configuration); + + // Get consensusGroupId from parameters passed by PipeConsensusImpl + if (!parameters.hasAttribute(PipeConnectorConstant.CONNECTOR_CONSENSUS_GROUP_ID_KEY)) { + throw new PipeException(CUSTOMIZE_EXCEPTION_MSG); + } + consensusGroupId = parameters.getInt(PipeConnectorConstant.CONNECTOR_CONSENSUS_GROUP_ID_KEY); + + // In PipeConsensus, one pipeConsensusTask corresponds to a pipeConsensusConnector. Thus, + // `nodeUrls` here actually is a singletonList that contains one peer's TEndPoint. But here we + // retain the implementation of list to cope with possible future expansion + retryConnector = new PipeConsensusSyncConnector(nodeUrls, consensusGroupId, thisDataNodeId); + retryConnector.customize(parameters, configuration); + asyncTransferClientManager = + PipeConsensusClientMgrContainer.getInstance().getAsyncClientManager(); + + if (isTabletBatchModeEnabled) { + tabletBatchBuilder = + new PipeConsensusAsyncBatchReqBuilder( + parameters, + new TConsensusGroupId(TConsensusGroupType.DataRegion, consensusGroupId), + thisDataNodeId); + } + + // currently, tablet batch is false by default in PipeConsensus; + isTabletBatchModeEnabled = false; + } + + /** Add an event to transferBuffer, whose events will be asynchronizedly transfer to receiver. */ + private boolean addEvent2Buffer(EnrichedEvent event) { + try { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-ConsensusGroup-{}: no.{} event-{} added to connector buffer", + consensusGroupId, + event.getCommitId(), + event); + } + boolean result = + transferBuffer.offer( + event, PIPE_CONSENSUS_EVENT_ENQUEUE_TIMEOUT_IN_MS, TimeUnit.MILLISECONDS); + // add reference + if (result) { + event.increaseReferenceCount(PipeConsensusAsyncConnector.class.getName()); + } + return result; + } catch (InterruptedException e) { + LOGGER.info("PipeConsensusConnector transferBuffer queue offer is interrupted.", e); + Thread.currentThread().interrupt(); + return false; + } + } + + /** + * if one event is successfully processed by receiver in PipeConsensus, we will remove this event + * from transferBuffer in order to transfer other event. + */ + public synchronized void removeEventFromBuffer(EnrichedEvent event) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-ConsensusGroup-{}: one event-{} successfully received by the follower, will be removed from queue, queue size = {}, limit size = {}", + consensusGroupId, + event, + transferBuffer.size(), + IOTDB_CONFIG.getPipeConsensusPipelineSize()); + } + Iterator<EnrichedEvent> iterator = transferBuffer.iterator(); + EnrichedEvent current = iterator.next(); + while (!current.equalsInPipeConsensus(event) && iterator.hasNext()) { + current = iterator.next(); + } + iterator.remove(); + // decrease reference count + event.decreaseReferenceCount(PipeConsensusAsyncConnector.class.getName(), true); + } + + @Override + public void handshake() throws Exception { + // do nothing + // PipeConsensus doesn't need to do handshake, since nodes in same consensusGroup/cluster + // usually have same configuration. + } + + @Override + public void heartbeat() throws Exception { + // do nothing + } + + @Override + public void transfer(TabletInsertionEvent tabletInsertionEvent) throws Exception { + boolean enqueueResult = addEvent2Buffer((EnrichedEvent) tabletInsertionEvent); + if (!enqueueResult) { + throw new PipeException(ENQUEUE_EXCEPTION_MSG); + } + + syncTransferQueuedEventsIfNecessary(); + + // batch transfer tablets. + if (isTabletBatchModeEnabled) { + if (tabletBatchBuilder.onEvent(tabletInsertionEvent)) { + final PipeConsensusTabletBatchEventHandler pipeConsensusTabletBatchEventHandler = + new PipeConsensusTabletBatchEventHandler(tabletBatchBuilder, this); + + transfer(pipeConsensusTabletBatchEventHandler); + + tabletBatchBuilder.onSuccess(); + } + } else { + TCommitId tCommitId; + TConsensusGroupId tConsensusGroupId = + new TConsensusGroupId(TConsensusGroupType.DataRegion, consensusGroupId); + // tabletInsertionEvent instanceof PipeInsertNodeTabletInsertionEvent) + final PipeInsertNodeTabletInsertionEvent pipeInsertNodeTabletInsertionEvent = + (PipeInsertNodeTabletInsertionEvent) tabletInsertionEvent; + tCommitId = + new TCommitId( + pipeInsertNodeTabletInsertionEvent.getCommitId(), + pipeInsertNodeTabletInsertionEvent.getRebootTimes()); + + // We increase the reference count for this event to determine if the event may be released. + if (!pipeInsertNodeTabletInsertionEvent.increaseReferenceCount( + PipeConsensusAsyncConnector.class.getName())) { + pipeInsertNodeTabletInsertionEvent.decreaseReferenceCount( + PipeConsensusAsyncConnector.class.getName(), false); + return; + } + + final InsertNode insertNode = + pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); + final ProgressIndex progressIndex = pipeInsertNodeTabletInsertionEvent.getProgressIndex(); + final TPipeConsensusTransferReq pipeConsensusTransferReq = + Objects.isNull(insertNode) + ? PipeConsensusTabletBinaryReq.toTPipeConsensusTransferReq( + pipeInsertNodeTabletInsertionEvent.getByteBuffer(), + tCommitId, + tConsensusGroupId, + progressIndex, + thisDataNodeId) + : PipeConsensusTabletInsertNodeReq.toTPipeConsensusTransferReq( + insertNode, tCommitId, tConsensusGroupId, progressIndex, thisDataNodeId); + final PipeConsensusTabletInsertNodeEventHandler pipeConsensusInsertNodeReqHandler = + new PipeConsensusTabletInsertNodeEventHandler( + pipeInsertNodeTabletInsertionEvent, pipeConsensusTransferReq, this); + + transfer(pipeConsensusInsertNodeReqHandler); + } + } + + private void transfer( + final PipeConsensusTabletBatchEventHandler pipeConsensusTabletBatchEventHandler) { + AsyncPipeConsensusServiceClient client = null; + try { + client = asyncTransferClientManager.borrowClient(getFollowerUrl()); + pipeConsensusTabletBatchEventHandler.transfer(client); + } catch (final Exception ex) { + logOnClientException(client, ex); + pipeConsensusTabletBatchEventHandler.onError(ex); + } + } + + private void transfer( + final PipeConsensusTabletInsertNodeEventHandler pipeConsensusInsertNodeReqHandler) { + AsyncPipeConsensusServiceClient client = null; + try { + client = asyncTransferClientManager.borrowClient(getFollowerUrl()); + pipeConsensusInsertNodeReqHandler.transfer(client); + } catch (final Exception ex) { + logOnClientException(client, ex); + pipeConsensusInsertNodeReqHandler.onError(ex); + } + } + + @Override + public void transfer(TsFileInsertionEvent tsFileInsertionEvent) throws Exception { + boolean enqueueResult = addEvent2Buffer((EnrichedEvent) tsFileInsertionEvent); + if (!enqueueResult) { + throw new PipeException(ENQUEUE_EXCEPTION_MSG); + } + + syncTransferQueuedEventsIfNecessary(); + transferBatchedEventsIfNecessary(); + + if (!(tsFileInsertionEvent instanceof PipeTsFileInsertionEvent)) { + LOGGER.warn( + "PipeConsensusAsyncConnector only support PipeTsFileInsertionEvent. Current event: {}.", + tsFileInsertionEvent); + return; + } + + final PipeTsFileInsertionEvent pipeTsFileInsertionEvent = + (PipeTsFileInsertionEvent) tsFileInsertionEvent; + TCommitId tCommitId = + new TCommitId( + pipeTsFileInsertionEvent.getCommitId(), pipeTsFileInsertionEvent.getRebootTimes()); + TConsensusGroupId tConsensusGroupId = + new TConsensusGroupId(TConsensusGroupType.DataRegion, consensusGroupId); + // We increase the reference count for this event to determine if the event may be released. + if (!pipeTsFileInsertionEvent.increaseReferenceCount( + PipeConsensusAsyncConnector.class.getName())) { + pipeTsFileInsertionEvent.decreaseReferenceCount( + PipeConsensusAsyncConnector.class.getName(), false); + return; + } + + try { + // Just in case. To avoid the case that exception occurred when constructing the handler. + if (!pipeTsFileInsertionEvent.getTsFile().exists()) { + throw new FileNotFoundException(pipeTsFileInsertionEvent.getTsFile().getAbsolutePath()); + } + + final PipeConsensusTsFileInsertionEventHandler pipeConsensusTsFileInsertionEventHandler = + new PipeConsensusTsFileInsertionEventHandler( + pipeTsFileInsertionEvent, this, tCommitId, tConsensusGroupId, thisDataNodeId); + + transfer(pipeConsensusTsFileInsertionEventHandler); + } catch (Exception e) { + // Just in case. To avoid the case that exception occurred when constructing the handler. + pipeTsFileInsertionEvent.decreaseReferenceCount( + PipeConsensusAsyncConnector.class.getName(), false); + throw e; + } + } + + private void transfer( + final PipeConsensusTsFileInsertionEventHandler pipeConsensusTsFileInsertionEventHandler) { + AsyncPipeConsensusServiceClient client = null; + try { + client = asyncTransferClientManager.borrowClient(getFollowerUrl()); + pipeConsensusTsFileInsertionEventHandler.transfer(client); + } catch (final Exception ex) { + logOnClientException(client, ex); + pipeConsensusTsFileInsertionEventHandler.onError(ex); + } + } + + /** + * PipeConsensus only need transfer heartbeat event here. And heartbeat event doesn't need to be + * added to transferBuffer. + */ + @Override + public void transfer(Event event) throws Exception { + syncTransferQueuedEventsIfNecessary(); + transferBatchedEventsIfNecessary(); + + if (!(event instanceof PipeHeartbeatEvent)) { + LOGGER.warn( + "PipeConsensusAsyncConnector does not support transferring generic event: {}.", event); + return; + } + + retryConnector.transfer(event); + } + + /** Try its best to commit data in order. Flush can also be a trigger to transfer batched data. */ + private void transferBatchedEventsIfNecessary() throws IOException { + if (!isTabletBatchModeEnabled || tabletBatchBuilder.isEmpty()) { + return; + } + + transfer(new PipeConsensusTabletBatchEventHandler(tabletBatchBuilder, this)); + + tabletBatchBuilder.onSuccess(); + } + + /** + * Transfer queued {@link Event}s which are waiting for retry. + * + * @throws Exception if an error occurs. The error will be handled by pipe framework, which will + * retry the {@link Event} and mark the {@link Event} as failure and stop the pipe if the + * retry times exceeds the threshold. + */ + private synchronized void syncTransferQueuedEventsIfNecessary() throws Exception { + while (!retryEventQueue.isEmpty()) { + synchronized (this) { + if (isClosed.get() || retryEventQueue.isEmpty()) { + return; + } + + final Event peekedEvent = retryEventQueue.peek(); + // do transfer + if (peekedEvent instanceof PipeInsertNodeTabletInsertionEvent) { + retryConnector.transfer((PipeInsertNodeTabletInsertionEvent) peekedEvent); + } else if (peekedEvent instanceof PipeRawTabletInsertionEvent) { + retryConnector.transfer((PipeRawTabletInsertionEvent) peekedEvent); + } else if (peekedEvent instanceof PipeTsFileInsertionEvent) { + retryConnector.transfer((PipeTsFileInsertionEvent) peekedEvent); + } else { + if (LOGGER.isWarnEnabled()) { + LOGGER.warn( + "PipeConsensusAsyncConnector does not support transfer generic event: {}.", + peekedEvent); + } + } + // release resource + if (peekedEvent instanceof EnrichedEvent) { + ((EnrichedEvent) peekedEvent) + .decreaseReferenceCount(PipeConsensusAsyncConnector.class.getName(), true); + } + + final Event polledEvent = retryEventQueue.poll(); + if (polledEvent != peekedEvent) { + if (LOGGER.isErrorEnabled()) { + LOGGER.error( + "The event polled from the queue is not the same as the event peeked from the queue. " + + "Peeked event: {}, polled event: {}.", + peekedEvent, + polledEvent); + } + } + if (polledEvent != null && LOGGER.isDebugEnabled()) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Polled event {} from retry queue.", polledEvent); + } + // poll it from transferBuffer + removeEventFromBuffer((EnrichedEvent) polledEvent); + } + } + } + } + + /** + * Add failure event to retry queue. + * + * @param event event to retry + */ + public void addFailureEventToRetryQueue(final Event event) { + if (isClosed.get()) { + if (event instanceof EnrichedEvent) { + ((EnrichedEvent) event).clearReferenceCount(PipeConsensusAsyncConnector.class.getName()); + } + return; + } + + retryEventQueue.offer(event); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-ConsensusGroup-{}: Event {} transfer failed, will be added to retry queue.", + consensusGroupId, + event); + } + + if (isClosed.get()) { + if (event instanceof EnrichedEvent) { + ((EnrichedEvent) event).clearReferenceCount(PipeConsensusAsyncConnector.class.getName()); + } + } + } + + /** + * Add failure events to retry queue. + * + * @param events events to retry + */ + public void addFailureEventsToRetryQueue(final Iterable<Event> events) { + for (final Event event : events) { + addFailureEventToRetryQueue(event); + } + } + + public synchronized void clearRetryEventsReferenceCount() { + while (!retryEventQueue.isEmpty()) { + final Event event = retryEventQueue.poll(); + if (event instanceof EnrichedEvent) { + ((EnrichedEvent) event).clearReferenceCount(PipeConsensusAsyncConnector.class.getName()); + } + } + } + + private void logOnClientException( + final AsyncPipeConsensusServiceClient client, final Exception e) { + if (client == null) { + LOGGER.warn(THRIFT_ERROR_FORMATTER_WITHOUT_ENDPOINT, e); + } else { + LOGGER.warn( + String.format( + THRIFT_ERROR_FORMATTER_WITH_ENDPOINT, + client.getTEndpoint().getIp(), + client.getTEndpoint().getPort()), + e); + } + } + + private TEndPoint getFollowerUrl() { + // In current pipeConsensus design, one connector corresponds to one follower, so the peers is + // actually a singleton list + return nodeUrls.get(0); + } + + // synchronized to avoid close connector when transfer event + @Override + public synchronized void close() { + super.close(); + isClosed.set(true); + + retryConnector.close(); + clearRetryEventsReferenceCount(); + + if (tabletBatchBuilder != null) { + tabletBatchBuilder.close(); + } + } + + //////////////////////////// TODO: APIs provided for metric framework //////////////////////////// + + public int getTransferBufferSize() { + return transferBuffer.size(); + } + + public int getRetryBufferSize() { + return retryEventQueue.size(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java new file mode 100644 index 000000000000..e29d5f49398c --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java @@ -0,0 +1,455 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +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.container.PipeConsensusClientMgrContainer; +import org.apache.iotdb.commons.client.sync.SyncPipeConsensusServiceClient; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.response.PipeConsensusTransferFilePieceResp; +import org.apache.iotdb.commons.pipe.connector.protocol.IoTDBConnector; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusBatchTransferResp; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.builder.PipeConsensusSyncBatchReqBuilder; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletBinaryReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletInsertNodeReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceWithModReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealWithModReq; +import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; +import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeConnectorRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; +import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; +import org.apache.iotdb.pipe.api.exception.PipeConnectionException; +import org.apache.iotdb.pipe.api.exception.PipeException; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** This connector is used for PipeConsensus to transfer queued event. */ +public class PipeConsensusSyncConnector extends IoTDBConnector { + + private static final Logger LOGGER = LoggerFactory.getLogger(PipeConsensusSyncConnector.class); + + private static final String PIPE_CONSENSUS_SYNC_CONNECTION_FAILED_FORMAT = + "PipeConsensus: syncClient connection to %s:%s failed when %s, because: %s"; + + private static final String TABLET_INSERTION_NODE_SCENARIO = "transfer insertionNode tablet"; + + private static final String TSFILE_SCENARIO = "transfer tsfile"; + + private static final String TABLET_BATCH_SCENARIO = "transfer tablet batch"; + + private final IClientManager<TEndPoint, SyncPipeConsensusServiceClient> syncRetryClientManager; + + private final List<TEndPoint> peers; + + private final int thisDataNodeId; + + private final int consensusGroupId; + + private PipeConsensusSyncBatchReqBuilder tabletBatchBuilder; + + public PipeConsensusSyncConnector( + List<TEndPoint> peers, int consensusGroupId, int thisDataNodeId) { + // In PipeConsensus, one pipeConsensusTask corresponds to a pipeConsensusConnector. Thus, + // `peers` here actually is a singletonList that contains one peer's TEndPoint. But here we + // retain the implementation of list to cope with possible future expansion + this.peers = peers; + this.consensusGroupId = consensusGroupId; + this.thisDataNodeId = thisDataNodeId; + this.syncRetryClientManager = + PipeConsensusClientMgrContainer.getInstance().getSyncClientManager(); + } + + @Override + public void customize(PipeParameters parameters, PipeConnectorRuntimeConfiguration configuration) + throws Exception { + super.customize(parameters, configuration); + if (isTabletBatchModeEnabled) { + tabletBatchBuilder = + new PipeConsensusSyncBatchReqBuilder( + parameters, + new TConsensusGroupId(TConsensusGroupType.DataRegion, consensusGroupId), + thisDataNodeId); + } + // Currently, tablet batch is false by default in PipeConsensus; + isTabletBatchModeEnabled = false; + } + + @Override + public void handshake() throws Exception { + // Do nothing + // PipeConsensus doesn't need to do handshake, since nodes in same consensusGroup/cluster + // usually have same configuration. + } + + @Override + public void heartbeat() throws Exception { + // Do nothing + } + + @Override + public void transfer(TabletInsertionEvent tabletInsertionEvent) throws Exception { + // Note: here we don't need to do type judgment here, because PipeConsensus uses + // PIPE_CONSENSUS_PROCESSOR and will not change the event type like + // org.apache.iotdb.db.pipe.connector.protocol.thrift.sync.IoTDBDataRegionSyncConnector + try { + if (isTabletBatchModeEnabled) { + if (tabletBatchBuilder.onEvent(tabletInsertionEvent)) { + doTransfer(); + } + } else { + doTransferWrapper((PipeInsertNodeTabletInsertionEvent) tabletInsertionEvent); + } + } catch (Exception e) { + throw new PipeConnectionException( + String.format( + "Failed to transfer tablet insertion event %s, because %s.", + tabletInsertionEvent, e.getMessage()), + e); + } + } + + @Override + public void transfer(TsFileInsertionEvent tsFileInsertionEvent) throws Exception { + // Note: here we don't need to do type judgment here, because PipeConsensus uses DO_NOTHING + // processor and will not change the event type like + // org.apache.iotdb.db.pipe.connector.protocol.thrift.sync.IoTDBDataRegionSyncConnector + try { + // In order to commit in order + if (isTabletBatchModeEnabled && !tabletBatchBuilder.isEmpty()) { + doTransfer(); + } + + doTransfer((PipeTsFileInsertionEvent) tsFileInsertionEvent); + } catch (Exception e) { + throw new PipeConnectionException( + String.format( + "Failed to transfer tsfile insertion event %s, because %s.", + tsFileInsertionEvent, e.getMessage()), + e); + } + } + + @Override + public void transfer(Event event) throws Exception { + // in order to commit in order + if (isTabletBatchModeEnabled && !tabletBatchBuilder.isEmpty()) { + doTransfer(); + } + + if (!(event instanceof PipeHeartbeatEvent)) { + LOGGER.warn( + "PipeConsensusSyncConnector does not support transferring generic event: {}.", event); + } + } + + private void doTransfer() { + try (final SyncPipeConsensusServiceClient syncPipeConsensusServiceClient = + syncRetryClientManager.borrowClient(getFollowerUrl())) { + final TPipeConsensusBatchTransferResp resp; + resp = + syncPipeConsensusServiceClient.pipeConsensusBatchTransfer( + tabletBatchBuilder.toTPipeConsensusBatchTransferReq()); + + final List<TSStatus> statusList = + resp.getBatchResps().stream() + .map(TPipeConsensusTransferResp::getStatus) + .collect(Collectors.toList()); + + // TODO(support batch): handle retry logic + // Only handle the failed statuses to avoid string format performance overhead + // if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() + // && status.getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { + // receiverStatusHandler.handle( + // resp.getStatus(), + // String.format( + // "Transfer PipeConsensusTransferTabletBatchReq error, result status %s", + // resp.status), + // tabletBatchBuilder.deepCopyEvents().toString()); + // } + + tabletBatchBuilder.onSuccess(); + } catch (Exception e) { + throw new PipeConnectionException( + String.format( + PIPE_CONSENSUS_SYNC_CONNECTION_FAILED_FORMAT, + getFollowerUrl().getIp(), + getFollowerUrl().getPort(), + TABLET_BATCH_SCENARIO, + e.getMessage()), + e); + } + } + + private void doTransferWrapper( + final PipeInsertNodeTabletInsertionEvent pipeInsertNodeTabletInsertionEvent) + throws PipeException { + try { + // We increase the reference count for this event to determine if the event may be released. + if (!pipeInsertNodeTabletInsertionEvent.increaseReferenceCount( + PipeConsensusSyncConnector.class.getName())) { + return; + } + doTransfer(pipeInsertNodeTabletInsertionEvent); + } finally { + pipeInsertNodeTabletInsertionEvent.decreaseReferenceCount( + PipeConsensusSyncConnector.class.getName(), false); + } + } + + private void doTransfer(PipeInsertNodeTabletInsertionEvent pipeInsertNodeTabletInsertionEvent) + throws PipeException { + final InsertNode insertNode; + final ProgressIndex progressIndex; + final TPipeConsensusTransferResp resp; + TCommitId tCommitId = + new TCommitId( + pipeInsertNodeTabletInsertionEvent.getCommitId(), + pipeInsertNodeTabletInsertionEvent.getRebootTimes()); + TConsensusGroupId tConsensusGroupId = + new TConsensusGroupId(TConsensusGroupType.DataRegion, consensusGroupId); + + try (final SyncPipeConsensusServiceClient syncPipeConsensusServiceClient = + syncRetryClientManager.borrowClient(getFollowerUrl())) { + insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); + progressIndex = pipeInsertNodeTabletInsertionEvent.getProgressIndex(); + + if (insertNode != null) { + resp = + syncPipeConsensusServiceClient.pipeConsensusTransfer( + PipeConsensusTabletInsertNodeReq.toTPipeConsensusTransferReq( + insertNode, tCommitId, tConsensusGroupId, progressIndex, thisDataNodeId)); + } else { + resp = + syncPipeConsensusServiceClient.pipeConsensusTransfer( + PipeConsensusTabletBinaryReq.toTPipeConsensusTransferReq( + pipeInsertNodeTabletInsertionEvent.getByteBuffer(), + tCommitId, + tConsensusGroupId, + progressIndex, + thisDataNodeId)); + } + } catch (Exception e) { + throw new PipeConnectionException( + String.format( + PIPE_CONSENSUS_SYNC_CONNECTION_FAILED_FORMAT, + getFollowerUrl().getIp(), + getFollowerUrl().getPort(), + TABLET_INSERTION_NODE_SCENARIO, + e.getMessage()), + e); + } + + final TSStatus status = resp.getStatus(); + // Only handle the failed statuses to avoid string format performance overhead + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() + && status.getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { + receiverStatusHandler.handle( + status, + String.format( + "PipeConsensus transfer PipeInsertNodeTabletInsertionEvent %s error, result status %s", + pipeInsertNodeTabletInsertionEvent, status), + pipeInsertNodeTabletInsertionEvent.toString()); + } + } + + private void doTransfer(PipeTsFileInsertionEvent pipeTsFileInsertionEvent) throws PipeException { + final File tsFile = pipeTsFileInsertionEvent.getTsFile(); + final File modFile = pipeTsFileInsertionEvent.getModFile(); + final TPipeConsensusTransferResp resp; + + try (final SyncPipeConsensusServiceClient syncPipeConsensusServiceClient = + syncRetryClientManager.borrowClient(getFollowerUrl())) { + final TCommitId tCommitId = + new TCommitId( + pipeTsFileInsertionEvent.getCommitId(), pipeTsFileInsertionEvent.getRebootTimes()); + final TConsensusGroupId tConsensusGroupId = + new TConsensusGroupId(TConsensusGroupType.DataRegion, consensusGroupId); + + // 1. Transfer tsFile, and mod file if exists + if (pipeTsFileInsertionEvent.isWithMod()) { + transferFilePieces( + modFile, syncPipeConsensusServiceClient, true, tCommitId, tConsensusGroupId); + transferFilePieces( + tsFile, syncPipeConsensusServiceClient, true, tCommitId, tConsensusGroupId); + // 2. Transfer file seal signal with mod, which means the file is transferred completely + resp = + syncPipeConsensusServiceClient.pipeConsensusTransfer( + PipeConsensusTsFileSealWithModReq.toTPipeConsensusTransferReq( + modFile.getName(), + modFile.length(), + tsFile.getName(), + tsFile.length(), + tCommitId, + tConsensusGroupId, + pipeTsFileInsertionEvent.getProgressIndex(), + thisDataNodeId)); + } else { + transferFilePieces( + tsFile, syncPipeConsensusServiceClient, false, tCommitId, tConsensusGroupId); + // 2. Transfer file seal signal without mod, which means the file is transferred completely + resp = + syncPipeConsensusServiceClient.pipeConsensusTransfer( + PipeConsensusTsFileSealReq.toTPipeConsensusTransferReq( + tsFile.getName(), + tsFile.length(), + tCommitId, + tConsensusGroupId, + pipeTsFileInsertionEvent.getProgressIndex(), + thisDataNodeId)); + } + } catch (Exception e) { + throw new PipeConnectionException( + String.format( + PIPE_CONSENSUS_SYNC_CONNECTION_FAILED_FORMAT, + getFollowerUrl().getIp(), + getFollowerUrl().getPort(), + TSFILE_SCENARIO, + e.getMessage()), + e); + } + + final TSStatus status = resp.getStatus(); + // Only handle the failed statuses to avoid string format performance overhead + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() + && status.getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { + receiverStatusHandler.handle( + resp.getStatus(), + String.format("Seal file %s error, result status %s.", tsFile, resp.getStatus()), + tsFile.getName()); + } + + LOGGER.info("Successfully transferred file {}.", tsFile); + } + + protected void transferFilePieces( + File file, + SyncPipeConsensusServiceClient syncPipeConsensusServiceClient, + boolean isMultiFile, + TCommitId tCommitId, + TConsensusGroupId tConsensusGroupId) + throws PipeException, IOException { + final int readFileBufferSize = PipeConfig.getInstance().getPipeConnectorReadFileBufferSize(); + final byte[] readBuffer = new byte[readFileBufferSize]; + long position = 0; + try (final RandomAccessFile reader = new RandomAccessFile(file, "r")) { + while (true) { + final int readLength = reader.read(readBuffer); + if (readLength == -1) { + break; + } + + final byte[] payLoad = + readLength == readFileBufferSize + ? readBuffer + : Arrays.copyOfRange(readBuffer, 0, readLength); + final PipeConsensusTransferFilePieceResp resp; + try { + resp = + PipeConsensusTransferFilePieceResp.fromTPipeConsensusTransferResp( + syncPipeConsensusServiceClient.pipeConsensusTransfer( + isMultiFile + ? PipeConsensusTsFilePieceWithModReq.toTPipeConsensusTransferReq( + file.getName(), + position, + payLoad, + tCommitId, + tConsensusGroupId, + thisDataNodeId) + : PipeConsensusTsFilePieceReq.toTPipeConsensusTransferReq( + file.getName(), + position, + payLoad, + tCommitId, + tConsensusGroupId, + thisDataNodeId))); + } catch (Exception e) { + throw new PipeConnectionException( + String.format( + "Network error when transfer file %s, because %s.", file, e.getMessage()), + e); + } + + position += readLength; + + final TSStatus status = resp.getStatus(); + // This case only happens when the connection is broken, and the connector is reconnected + // to the receiver, then the receiver will redirect the file position to the last position + if (status.getCode() + == TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_OFFSET_RESET.getStatusCode()) { + position = resp.getEndWritingOffset(); + reader.seek(position); + LOGGER.info("Redirect file position to {}.", position); + continue; + } + + // Only handle the failed statuses to avoid string format performance overhead + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() + && status.getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { + receiverStatusHandler.handle( + resp.getStatus(), + String.format("Transfer file %s error, result status %s.", file, resp.getStatus()), + file.getName()); + } + } + } + } + + private TEndPoint getFollowerUrl() { + // In current pipeConsensus design, one connector corresponds to one follower, so the peers is + // actually a singleton list + return peers.get(0); + } + + // synchronized to avoid close connector when transfer event + @Override + public synchronized void close() { + super.close(); + if (syncRetryClientManager != null) { + syncRetryClientManager.close(); + } + + if (tabletBatchBuilder != null) { + tabletBatchBuilder.close(); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletBatchEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletBatchEventHandler.java new file mode 100644 index 000000000000..25816e68fa6a --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletBatchEventHandler.java @@ -0,0 +1,128 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus.handler; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusBatchTransferReq; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusBatchTransferResp; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.PipeConsensusAsyncConnector; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.builder.PipeConsensusAsyncBatchReqBuilder; +import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.pipe.api.exception.PipeException; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; + +public class PipeConsensusTabletBatchEventHandler + implements AsyncMethodCallback<TPipeConsensusBatchTransferResp> { + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeConsensusTabletBatchEventHandler.class); + private final List<Long> requestCommitIds; + private final List<Event> events; + private final TPipeConsensusBatchTransferReq req; + private final PipeConsensusAsyncConnector connector; + + public PipeConsensusTabletBatchEventHandler( + final PipeConsensusAsyncBatchReqBuilder batchBuilder, + final PipeConsensusAsyncConnector connector) + throws IOException { + // Deep copy to keep Ids' and events' reference + requestCommitIds = batchBuilder.deepCopyRequestCommitIds(); + events = batchBuilder.deepCopyEvents(); + req = batchBuilder.toTPipeConsensusBatchTransferReq(); + + this.connector = connector; + } + + public void transfer(final AsyncPipeConsensusServiceClient client) throws TException { + client.pipeConsensusBatchTransfer(req, this); + } + + @Override + public void onComplete(final TPipeConsensusBatchTransferResp response) { + // Just in case + if (response == null) { + onError(new PipeException("TPipeConsensusBatchTransferResp is null")); + return; + } + + try { + final List<TSStatus> status = + response.getBatchResps().stream() + .map(TPipeConsensusTransferResp::getStatus) + .collect(Collectors.toList()); + + if (status.stream() + .anyMatch( + tsStatus -> tsStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode())) { + status.stream() + .filter(tsStatus -> tsStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) + .forEach( + tsStatus -> { + connector + .statusHandler() + .handle(tsStatus, tsStatus.getMessage(), events.toString()); + }); + // if any events failed, we will resend it all. + connector.addFailureEventsToRetryQueue(events); + } + // if all events success, remove them from transferBuffer + else { + events.forEach(event -> connector.removeEventFromBuffer((EnrichedEvent) event)); + } + + for (final Event event : events) { + if (event instanceof EnrichedEvent) { + ((EnrichedEvent) event) + .decreaseReferenceCount(PipeConsensusTabletBatchEventHandler.class.getName(), true); + } + } + } catch (final Exception e) { + onError(e); + } + } + + @Override + public void onError(final Exception exception) { + LOGGER.warn( + "PipeConsensus: Failed to transfer TabletInsertionEvent batch {} (request commit ids={}).", + events.stream() + .map( + event -> + event instanceof EnrichedEvent + ? ((EnrichedEvent) event).coreReportMessage() + : event.toString()) + .collect(Collectors.toList()), + requestCommitIds, + exception); + + connector.addFailureEventsToRetryQueue(events); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertNodeEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertNodeEventHandler.java new file mode 100644 index 000000000000..7b05fe297f27 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertNodeEventHandler.java @@ -0,0 +1,45 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus.handler; + +import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.PipeConsensusAsyncConnector; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; + +import org.apache.thrift.TException; + +public class PipeConsensusTabletInsertNodeEventHandler + extends PipeConsensusTabletInsertionEventHandler<TPipeConsensusTransferResp> { + + public PipeConsensusTabletInsertNodeEventHandler( + PipeInsertNodeTabletInsertionEvent event, + TPipeConsensusTransferReq req, + PipeConsensusAsyncConnector connector) { + super(event, req, connector); + } + + @Override + protected void doTransfer(AsyncPipeConsensusServiceClient client, TPipeConsensusTransferReq req) + throws TException { + client.pipeConsensusTransfer(req, this); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertionEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertionEventHandler.java new file mode 100644 index 000000000000..3e12b23f4633 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertionEventHandler.java @@ -0,0 +1,109 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus.handler; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.PipeConsensusAsyncConnector; +import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.handler.PipeTransferTabletInsertionEventHandler; +import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; +import org.apache.iotdb.pipe.api.exception.PipeException; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class PipeConsensusTabletInsertionEventHandler<E extends TPipeConsensusTransferResp> + implements AsyncMethodCallback<E> { + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeConsensusTabletInsertionEventHandler.class); + + protected final TabletInsertionEvent event; + + protected final TPipeConsensusTransferReq req; + + protected final PipeConsensusAsyncConnector connector; + + protected PipeConsensusTabletInsertionEventHandler( + TabletInsertionEvent event, + TPipeConsensusTransferReq req, + PipeConsensusAsyncConnector connector) { + this.event = event; + this.req = req; + this.connector = connector; + } + + public void transfer(AsyncPipeConsensusServiceClient client) throws TException { + doTransfer(client, req); + } + + protected abstract void doTransfer( + AsyncPipeConsensusServiceClient client, TPipeConsensusTransferReq req) throws TException; + + @Override + public void onComplete(TPipeConsensusTransferResp response) { + // Just in case + if (response == null) { + onError(new PipeException("TPipeConsensusTransferResp is null")); + return; + } + + final TSStatus status = response.getStatus(); + try { + // Only handle the failed statuses to avoid string format performance overhead + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() + && status.getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { + connector.statusHandler().handle(status, status.getMessage(), event.toString()); + } + if (event instanceof EnrichedEvent) { + ((EnrichedEvent) event) + .decreaseReferenceCount(PipeTransferTabletInsertionEventHandler.class.getName(), true); + } + + if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + LOGGER.info( + "Debug only: no.{} event successfully processed!", + ((EnrichedEvent) event).getCommitId()); + connector.removeEventFromBuffer((EnrichedEvent) event); + } + } catch (Exception e) { + onError(e); + } + } + + @Override + public void onError(Exception exception) { + LOGGER.warn( + "Failed to transfer TabletInsertionEvent {} (committer key={}, commit id={}).", + event instanceof EnrichedEvent + ? ((EnrichedEvent) event).coreReportMessage() + : event.toString(), + event instanceof EnrichedEvent ? ((EnrichedEvent) event).getCommitterKey() : null, + event instanceof EnrichedEvent ? ((EnrichedEvent) event).getCommitId() : null, + exception); + + connector.addFailureEventToRetryQueue(event); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletRawEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletRawEventHandler.java new file mode 100644 index 000000000000..35dcae79efe9 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletRawEventHandler.java @@ -0,0 +1,45 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus.handler; + +import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.PipeConsensusAsyncConnector; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; + +import org.apache.thrift.TException; + +public class PipeConsensusTabletRawEventHandler + extends PipeConsensusTabletInsertionEventHandler<TPipeConsensusTransferResp> { + + public PipeConsensusTabletRawEventHandler( + PipeRawTabletInsertionEvent event, + TPipeConsensusTransferReq req, + PipeConsensusAsyncConnector connector) { + super(event, req, connector); + } + + @Override + protected void doTransfer(AsyncPipeConsensusServiceClient client, TPipeConsensusTransferReq req) + throws TException { + client.pipeConsensusTransfer(req, this); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTsFileInsertionEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTsFileInsertionEventHandler.java new file mode 100644 index 000000000000..4ed42afc63c4 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTsFileInsertionEventHandler.java @@ -0,0 +1,279 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus.handler; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; +import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.response.PipeConsensusTransferFilePieceResp; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.PipeConsensusAsyncConnector; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceWithModReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealWithModReq; +import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.util.Arrays; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicBoolean; + +public class PipeConsensusTsFileInsertionEventHandler + implements AsyncMethodCallback<TPipeConsensusTransferResp> { + + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeConsensusTsFileInsertionEventHandler.class); + + private final PipeTsFileInsertionEvent event; + private final PipeConsensusAsyncConnector connector; + private final TCommitId commitId; + private final TConsensusGroupId consensusGroupId; + private final int thisDataNodeId; + private final File tsFile; + private final File modFile; + private File currentFile; + + private final boolean transferMod; + + private final int readFileBufferSize; + private final byte[] readBuffer; + private long position; + + private RandomAccessFile reader; + + private final AtomicBoolean isSealSignalSent; + + private AsyncPipeConsensusServiceClient client; + + public PipeConsensusTsFileInsertionEventHandler( + final PipeTsFileInsertionEvent event, + final PipeConsensusAsyncConnector connector, + final TCommitId commitId, + final TConsensusGroupId consensusGroupId, + final int thisDataNodeId) + throws FileNotFoundException { + this.event = event; + this.connector = connector; + this.commitId = commitId; + this.consensusGroupId = consensusGroupId; + this.thisDataNodeId = thisDataNodeId; + + tsFile = event.getTsFile(); + modFile = event.getModFile(); + transferMod = event.isWithMod(); + currentFile = transferMod ? modFile : tsFile; + + readFileBufferSize = PipeConfig.getInstance().getPipeConnectorReadFileBufferSize(); + readBuffer = new byte[readFileBufferSize]; + position = 0; + + reader = + Objects.nonNull(modFile) + ? new RandomAccessFile(modFile, "r") + : new RandomAccessFile(tsFile, "r"); + + isSealSignalSent = new AtomicBoolean(false); + } + + public void transfer(final AsyncPipeConsensusServiceClient client) + throws TException, IOException { + this.client = client; + client.setShouldReturnSelf(false); + + final int readLength = reader.read(readBuffer); + + if (readLength == -1) { + if (currentFile == modFile) { + currentFile = tsFile; + position = 0; + try { + reader.close(); + } catch (final IOException e) { + LOGGER.warn("Failed to close file reader when successfully transferred mod file.", e); + } + reader = new RandomAccessFile(tsFile, "r"); + transfer(client); + } else if (currentFile == tsFile) { + isSealSignalSent.set(true); + client.pipeConsensusTransfer( + transferMod + ? PipeConsensusTsFileSealWithModReq.toTPipeConsensusTransferReq( + modFile.getName(), + modFile.length(), + tsFile.getName(), + tsFile.length(), + commitId, + consensusGroupId, + event.getProgressIndex(), + thisDataNodeId) + : PipeConsensusTsFileSealReq.toTPipeConsensusTransferReq( + tsFile.getName(), + tsFile.length(), + commitId, + consensusGroupId, + event.getProgressIndex(), + thisDataNodeId), + this); + } + return; + } + + // for save some mem + final byte[] payload = + readLength == readFileBufferSize + ? readBuffer + : Arrays.copyOfRange(readBuffer, 0, readLength); + client.pipeConsensusTransfer( + transferMod + ? PipeConsensusTsFilePieceWithModReq.toTPipeConsensusTransferReq( + currentFile.getName(), + position, + payload, + commitId, + consensusGroupId, + thisDataNodeId) + : PipeConsensusTsFilePieceReq.toTPipeConsensusTransferReq( + currentFile.getName(), + position, + payload, + commitId, + consensusGroupId, + thisDataNodeId), + this); + position += readLength; + } + + @Override + public void onComplete(final TPipeConsensusTransferResp response) { + if (isSealSignalSent.get()) { + try { + final TSStatus status = response.getStatus(); + // Only handle the failed statuses to avoid string format performance overhead + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() + && status.getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { + connector + .statusHandler() + .handle( + status, + String.format( + "Seal file %s error, result status %s.", tsFile, response.getStatus()), + tsFile.getName()); + } + + if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + connector.removeEventFromBuffer(event); + } + } catch (final Exception e) { + onError(e); + return; + } + + try { + if (reader != null) { + reader.close(); + } + } catch (final IOException e) { + LOGGER.warn("Failed to close file reader when successfully transferred file.", e); + } finally { + event.decreaseReferenceCount( + PipeConsensusTsFileInsertionEventHandler.class.getName(), true); + + LOGGER.info( + "Successfully transferred file {} (committer key={}, commit id={}).", + tsFile, + event.getCommitterKey(), + event.getCommitId()); + + if (client != null) { + client.setShouldReturnSelf(true); + client.returnSelf(); + } + } + return; + } + + // If the isSealSignalSent is false, then the response must be a + // PipeConsensusTransferFilePieceResp + try { + final PipeConsensusTransferFilePieceResp resp = + PipeConsensusTransferFilePieceResp.fromTPipeConsensusTransferResp(response); + + // This case only happens when the connection is broken, and the connector is reconnected + // to the receiver, then the receiver will redirect the file position to the last position + final long code = resp.getStatus().getCode(); + + if (code == TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_OFFSET_RESET.getStatusCode()) { + position = resp.getEndWritingOffset(); + reader.seek(position); + LOGGER.info("Redirect file position to {}.", position); + } else { + final TSStatus status = response.getStatus(); + // Only handle the failed statuses to avoid string format performance overhead + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() + && status.getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { + connector + .statusHandler() + .handle(status, response.getStatus().getMessage(), tsFile.getName()); + } + } + + transfer(client); + } catch (final Exception e) { + onError(e); + } + } + + @Override + public void onError(final Exception exception) { + LOGGER.warn( + "Failed to transfer TsFileInsertionEvent {} (committer key {}, commit id {}).", + tsFile, + event.getCommitterKey(), + event.getCommitId(), + exception); + + try { + if (reader != null) { + reader.close(); + } + } catch (final IOException e) { + LOGGER.warn("Failed to close file reader when failed to transfer file.", e); + } finally { + connector.addFailureEventToRetryQueue(event); + + if (client != null) { + client.setShouldReturnSelf(true); + client.returnSelf(); + } + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusAsyncBatchReqBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusAsyncBatchReqBuilder.java new file mode 100644 index 000000000000..b2205df5c1a5 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusAsyncBatchReqBuilder.java @@ -0,0 +1,37 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus.payload.builder; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; + +import java.util.ArrayList; +import java.util.List; + +public class PipeConsensusAsyncBatchReqBuilder extends PipeConsensusTransferBatchReqBuilder { + public PipeConsensusAsyncBatchReqBuilder( + PipeParameters parameters, TConsensusGroupId consensusGroupId, int thisDataNodeId) { + super(parameters, consensusGroupId, thisDataNodeId); + } + + public List<Long> deepCopyRequestCommitIds() { + return new ArrayList<>(requestCommitIds); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusSyncBatchReqBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusSyncBatchReqBuilder.java new file mode 100644 index 000000000000..07f5af5079ee --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusSyncBatchReqBuilder.java @@ -0,0 +1,45 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus.payload.builder; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.pipe.api.event.Event; + +public class PipeConsensusSyncBatchReqBuilder extends PipeConsensusTransferBatchReqBuilder { + + public PipeConsensusSyncBatchReqBuilder( + PipeParameters parameters, TConsensusGroupId consensusGroupId, int thisDataNodeId) { + super(parameters, consensusGroupId, thisDataNodeId); + } + + @Override + public synchronized void onSuccess() { + for (final Event event : events) { + if (event instanceof EnrichedEvent) { + ((EnrichedEvent) event) + .decreaseReferenceCount(PipeConsensusSyncBatchReqBuilder.class.getName(), true); + } + } + + super.onSuccess(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusTransferBatchReqBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusTransferBatchReqBuilder.java new file mode 100644 index 000000000000..daa70051d595 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusTransferBatchReqBuilder.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.pipe.connector.protocol.pipeconsensus.payload.builder; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletBatchReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletBinaryReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletInsertNodeReq; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; +import org.apache.iotdb.db.pipe.resource.PipeResourceManager; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; +import org.apache.iotdb.db.storageengine.dataregion.wal.exception.WALPipeException; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; + +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_BATCH_DELAY_DEFAULT_VALUE; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_BATCH_DELAY_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_BATCH_SIZE_DEFAULT_VALUE; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_BATCH_SIZE_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_BATCH_DELAY_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_BATCH_SIZE_KEY; + +public abstract class PipeConsensusTransferBatchReqBuilder implements AutoCloseable { + + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeConsensusTransferBatchReqBuilder.class); + + protected final List<Event> events = new ArrayList<>(); + protected final List<Long> requestCommitIds = new ArrayList<>(); + protected final List<TPipeConsensusTransferReq> batchReqs = new ArrayList<>(); + // limit in delayed time + protected final int maxDelayInMs; + protected final TConsensusGroupId consensusGroupId; + protected final int thisDataNodeId; + protected long firstEventProcessingTime = Long.MIN_VALUE; + + // limit in buffer size + protected final PipeMemoryBlock allocatedMemoryBlock; + protected long totalBufferSize = 0; + + protected PipeConsensusTransferBatchReqBuilder( + PipeParameters parameters, TConsensusGroupId consensusGroupId, int thisDataNodeId) { + maxDelayInMs = + parameters.getIntOrDefault( + Arrays.asList(CONNECTOR_IOTDB_BATCH_DELAY_KEY, SINK_IOTDB_BATCH_DELAY_KEY), + CONNECTOR_IOTDB_BATCH_DELAY_DEFAULT_VALUE) + * 1000; + + this.consensusGroupId = consensusGroupId; + this.thisDataNodeId = thisDataNodeId; + + final long requestMaxBatchSizeInBytes = + parameters.getLongOrDefault( + Arrays.asList(CONNECTOR_IOTDB_BATCH_SIZE_KEY, SINK_IOTDB_BATCH_SIZE_KEY), + CONNECTOR_IOTDB_BATCH_SIZE_DEFAULT_VALUE); + + allocatedMemoryBlock = + PipeResourceManager.memory() + .tryAllocate(requestMaxBatchSizeInBytes) + .setShrinkMethod(oldMemory -> Math.max(oldMemory / 2, 0)) + .setShrinkCallback( + (oldMemory, newMemory) -> + LOGGER.info( + "The batch size limit has shrunk from {} to {}.", oldMemory, newMemory)) + .setExpandMethod( + oldMemory -> Math.min(Math.max(oldMemory, 1) * 2, requestMaxBatchSizeInBytes)) + .setExpandCallback( + (oldMemory, newMemory) -> + LOGGER.info( + "The batch size limit has expanded from {} to {}.", oldMemory, newMemory)); + + if (getMaxBatchSizeInBytes() != requestMaxBatchSizeInBytes) { + LOGGER.info( + "PipeConsensusTransferBatchReqBuilder: the max batch size is adjusted from {} to {} due to the " + + "memory restriction", + requestMaxBatchSizeInBytes, + getMaxBatchSizeInBytes()); + } + } + + /** + * Try offer {@link Event} into cache if the given {@link Event} is not duplicated. + * + * @param event the given {@link Event} + * @return {@link true} if the batch can be transferred + */ + public synchronized boolean onEvent(TabletInsertionEvent event) + throws IOException, WALPipeException { + if (!(event instanceof EnrichedEvent)) { + return false; + } + + final long requestCommitId = ((EnrichedEvent) event).getCommitId(); + + // The deduplication logic here is to avoid the accumulation of the same event in a batch when + // retrying. + if ((events.isEmpty() || !events.get(events.size() - 1).equals(event))) { + events.add(event); + requestCommitIds.add(requestCommitId); + final int bufferSize = buildTabletInsertionBuffer(event); + + ((EnrichedEvent) event) + .increaseReferenceCount(PipeConsensusTransferBatchReqBuilder.class.getName()); + + if (firstEventProcessingTime == Long.MIN_VALUE) { + firstEventProcessingTime = System.currentTimeMillis(); + } + + totalBufferSize += bufferSize; + } + + return totalBufferSize >= getMaxBatchSizeInBytes() + || System.currentTimeMillis() - firstEventProcessingTime >= maxDelayInMs; + } + + public synchronized void onSuccess() { + batchReqs.clear(); + + events.clear(); + requestCommitIds.clear(); + + firstEventProcessingTime = Long.MIN_VALUE; + + totalBufferSize = 0; + } + + public PipeConsensusTabletBatchReq toTPipeConsensusBatchTransferReq() throws IOException { + return PipeConsensusTabletBatchReq.toTPipeConsensusBatchTransferReq(batchReqs); + } + + protected long getMaxBatchSizeInBytes() { + return allocatedMemoryBlock.getMemoryUsageInBytes(); + } + + public boolean isEmpty() { + return batchReqs.isEmpty(); + } + + public List<Event> deepCopyEvents() { + return new ArrayList<>(events); + } + + protected int buildTabletInsertionBuffer(TabletInsertionEvent event) throws WALPipeException { + final ByteBuffer buffer; + final TCommitId commitId; + + // event instanceof PipeInsertNodeTabletInsertionEvent) + final PipeInsertNodeTabletInsertionEvent pipeInsertNodeTabletInsertionEvent = + (PipeInsertNodeTabletInsertionEvent) event; + commitId = + new TCommitId( + pipeInsertNodeTabletInsertionEvent.getCommitId(), + pipeInsertNodeTabletInsertionEvent.getRebootTimes()); + + // Read the bytebuffer from the wal file and transfer it directly without serializing or + // deserializing if possible + final InsertNode insertNode = + pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); + // PipeConsensus will transfer binary data to TPipeConsensusTransferReq + final ProgressIndex progressIndex = pipeInsertNodeTabletInsertionEvent.getProgressIndex(); + if (Objects.isNull(insertNode)) { + buffer = pipeInsertNodeTabletInsertionEvent.getByteBuffer(); + batchReqs.add( + PipeConsensusTabletBinaryReq.toTPipeConsensusTransferReq( + buffer, commitId, consensusGroupId, progressIndex, thisDataNodeId)); + } else { + buffer = insertNode.serializeToByteBuffer(); + batchReqs.add( + PipeConsensusTabletInsertNodeReq.toTPipeConsensusTransferReq( + insertNode, commitId, consensusGroupId, progressIndex, thisDataNodeId)); + } + + return buffer.limit(); + } + + @Override + public synchronized void close() { + for (final Event event : events) { + if (event instanceof EnrichedEvent) { + ((EnrichedEvent) event).clearReferenceCount(this.getClass().getName()); + } + } + allocatedMemoryBlock.close(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletBatchReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletBatchReq.java new file mode 100644 index 000000000000..1401b19adaf7 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletBatchReq.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.pipe.connector.protocol.pipeconsensus.payload.request; + +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusBatchTransferReq; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +public class PipeConsensusTabletBatchReq extends TPipeConsensusBatchTransferReq { + private final transient List<PipeConsensusTabletBinaryReq> binaryReqs = new ArrayList<>(); + private final transient List<PipeConsensusTabletInsertNodeReq> insertNodeReqs = new ArrayList<>(); + + private PipeConsensusTabletBatchReq() { + // do nothing + } + + /////////////////////////////// Thrift /////////////////////////////// + public static PipeConsensusTabletBatchReq toTPipeConsensusBatchTransferReq( + List<TPipeConsensusTransferReq> reqs) throws IOException { + final PipeConsensusTabletBatchReq batchReq = new PipeConsensusTabletBatchReq(); + + batchReq.batchReqs = reqs; + + return batchReq; + } + + public static PipeConsensusTabletBatchReq fromTPipeConsensusBatchTransferReq( + TPipeConsensusBatchTransferReq transferReq) { + final PipeConsensusTabletBatchReq batchReq = new PipeConsensusTabletBatchReq(); + + for (final TPipeConsensusTransferReq req : transferReq.getBatchReqs()) { + if (req instanceof PipeConsensusTabletBinaryReq) { + batchReq.binaryReqs.add((PipeConsensusTabletBinaryReq) req); + } else { + batchReq.insertNodeReqs.add((PipeConsensusTabletInsertNodeReq) req); + } + } + + return batchReq; + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + PipeConsensusTabletBatchReq that = (PipeConsensusTabletBatchReq) obj; + return binaryReqs.equals(that.binaryReqs) + && insertNodeReqs.equals(that.insertNodeReqs) + && Objects.equals(batchReqs, that.batchReqs); + } + + @Override + public int hashCode() { + return Objects.hash(binaryReqs, insertNodeReqs, batchReqs); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletBinaryReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletBinaryReq.java new file mode 100644 index 000000000000..88ac59e30f57 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletBinaryReq.java @@ -0,0 +1,126 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus.payload.request; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestType; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestVersion; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; +import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntry; + +import org.apache.tsfile.utils.PublicBAOS; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +public class PipeConsensusTabletBinaryReq extends TPipeConsensusTransferReq { + private static final Logger LOGGER = LoggerFactory.getLogger(PipeConsensusTabletBinaryReq.class); + private transient ByteBuffer byteBuffer; + + private PipeConsensusTabletBinaryReq() { + // Do nothing + } + + public InsertNode convertToInsertNode() { + final PlanNode node = WALEntry.deserializeForConsensus(byteBuffer); + return node instanceof InsertNode ? (InsertNode) node : null; + } + + /////////////////////////////// Thrift /////////////////////////////// + + public static PipeConsensusTabletBinaryReq toTPipeConsensusTransferReq( + ByteBuffer byteBuffer, + TCommitId commitId, + TConsensusGroupId consensusGroupId, + ProgressIndex progressIndex, + int thisDataNodeId) { + final PipeConsensusTabletBinaryReq req = new PipeConsensusTabletBinaryReq(); + req.byteBuffer = byteBuffer; + + req.commitId = commitId; + req.consensusGroupId = consensusGroupId; + req.dataNodeId = thisDataNodeId; + req.version = PipeConsensusRequestVersion.VERSION_1.getVersion(); + req.type = PipeConsensusRequestType.TRANSFER_TABLET_BINARY.getType(); + req.body = byteBuffer; + + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + progressIndex.serialize(outputStream); + req.progressIndex = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } catch (IOException e) { + LOGGER.warn("Failed to serialize progress index {}", progressIndex, e); + } + + return req; + } + + public static PipeConsensusTabletBinaryReq fromTPipeConsensusTransferReq( + TPipeConsensusTransferReq transferReq) { + final PipeConsensusTabletBinaryReq binaryReq = new PipeConsensusTabletBinaryReq(); + binaryReq.byteBuffer = transferReq.body; + + binaryReq.version = transferReq.version; + binaryReq.type = transferReq.type; + binaryReq.body = transferReq.body; + binaryReq.commitId = transferReq.commitId; + binaryReq.dataNodeId = transferReq.dataNodeId; + binaryReq.consensusGroupId = transferReq.consensusGroupId; + binaryReq.progressIndex = transferReq.progressIndex; + + return binaryReq; + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + PipeConsensusTabletBinaryReq that = (PipeConsensusTabletBinaryReq) obj; + return byteBuffer.equals(that.byteBuffer) + && version == that.version + && type == that.type + && body.equals(that.body) + && Objects.equals(commitId, that.commitId) + && Objects.equals(consensusGroupId, that.consensusGroupId) + && Objects.equals(progressIndex, that.progressIndex) + && Objects.equals(dataNodeId, that.dataNodeId); + } + + @Override + public int hashCode() { + return Objects.hash( + byteBuffer, version, type, body, commitId, consensusGroupId, dataNodeId, progressIndex); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletInsertNodeReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletInsertNodeReq.java new file mode 100644 index 000000000000..e4fcf7ec457d --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletInsertNodeReq.java @@ -0,0 +1,154 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus.payload.request; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestType; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestVersion; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; + +import org.apache.tsfile.utils.PublicBAOS; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +public class PipeConsensusTabletInsertNodeReq extends TPipeConsensusTransferReq { + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeConsensusTabletInsertNodeReq.class); + private transient InsertNode insertNode; + + private PipeConsensusTabletInsertNodeReq() { + // Do nothing + } + + public InsertNode getInsertNode() { + return insertNode; + } + + /////////////////////////////// WriteBack & Batch /////////////////////////////// + + public static PipeConsensusTabletInsertNodeReq toTPipeConsensusTransferRawReq( + InsertNode insertNode, + TCommitId commitId, + TConsensusGroupId consensusGroupId, + ProgressIndex progressIndex, + int thisDataNodeId) { + final PipeConsensusTabletInsertNodeReq req = new PipeConsensusTabletInsertNodeReq(); + + req.insertNode = insertNode; + req.commitId = commitId; + req.consensusGroupId = consensusGroupId; + req.dataNodeId = thisDataNodeId; + + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + progressIndex.serialize(outputStream); + req.progressIndex = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } catch (IOException e) { + LOGGER.warn("Failed to serialize progress index {}", progressIndex, e); + } + + return req; + } + + /////////////////////////////// Thrift /////////////////////////////// + + public static PipeConsensusTabletInsertNodeReq toTPipeConsensusTransferReq( + InsertNode insertNode, + TCommitId commitId, + TConsensusGroupId consensusGroupId, + ProgressIndex progressIndex, + int thisDataNodeId) { + final PipeConsensusTabletInsertNodeReq req = new PipeConsensusTabletInsertNodeReq(); + + req.insertNode = insertNode; + + req.commitId = commitId; + req.consensusGroupId = consensusGroupId; + req.dataNodeId = thisDataNodeId; + req.version = PipeConsensusRequestVersion.VERSION_1.getVersion(); + req.type = PipeConsensusRequestType.TRANSFER_TABLET_INSERT_NODE.getType(); + req.body = insertNode.serializeToByteBuffer(); + + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + progressIndex.serialize(outputStream); + req.progressIndex = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } catch (IOException e) { + LOGGER.warn("Failed to serialize progress index {}", progressIndex, e); + } + + return req; + } + + public static PipeConsensusTabletInsertNodeReq fromTPipeConsensusTransferReq( + TPipeConsensusTransferReq transferReq) { + final PipeConsensusTabletInsertNodeReq insertNodeReq = new PipeConsensusTabletInsertNodeReq(); + + insertNodeReq.insertNode = (InsertNode) PlanNodeType.deserialize(transferReq.body); + + insertNodeReq.version = transferReq.version; + insertNodeReq.type = transferReq.type; + insertNodeReq.body = transferReq.body; + insertNodeReq.commitId = transferReq.commitId; + insertNodeReq.dataNodeId = transferReq.dataNodeId; + insertNodeReq.consensusGroupId = transferReq.consensusGroupId; + insertNodeReq.progressIndex = transferReq.progressIndex; + + return insertNodeReq; + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + PipeConsensusTabletInsertNodeReq that = (PipeConsensusTabletInsertNodeReq) obj; + return Objects.equals(insertNode, that.insertNode) + && version == that.version + && type == that.type + && Objects.equals(body, that.body) + && Objects.equals(commitId, that.commitId) + && Objects.equals(consensusGroupId, that.consensusGroupId) + && Objects.equals(progressIndex, that.progressIndex) + && Objects.equals(dataNodeId, that.dataNodeId); + } + + @Override + public int hashCode() { + return Objects.hash( + insertNode, version, type, body, commitId, consensusGroupId, dataNodeId, progressIndex); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFilePieceReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFilePieceReq.java new file mode 100644 index 000000000000..3afc373e2aa8 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFilePieceReq.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.pipe.connector.protocol.pipeconsensus.payload.request; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestType; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusTransferFilePieceReq; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; + +import java.io.IOException; + +public class PipeConsensusTsFilePieceReq extends PipeConsensusTransferFilePieceReq { + + private PipeConsensusTsFilePieceReq() { + // Empty constructor + } + + @Override + protected PipeConsensusRequestType getPlanType() { + return PipeConsensusRequestType.TRANSFER_TS_FILE_PIECE; + } + + /////////////////////////////// Thrift /////////////////////////////// + + public static PipeConsensusTsFilePieceReq toTPipeConsensusTransferReq( + String fileName, + long startWritingOffset, + byte[] filePiece, + TCommitId commitId, + TConsensusGroupId consensusGroupId, + int thisDataNodeId) + throws IOException { + return (PipeConsensusTsFilePieceReq) + new PipeConsensusTsFilePieceReq() + .convertToTPipeConsensusTransferReq( + fileName, + startWritingOffset, + filePiece, + commitId, + consensusGroupId, + thisDataNodeId); + } + + public static PipeConsensusTsFilePieceReq fromTPipeConsensusTransferReq( + TPipeConsensusTransferReq transferReq) { + return (PipeConsensusTsFilePieceReq) + new PipeConsensusTsFilePieceReq().translateFromTPipeConsensusTransferReq(transferReq); + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + return obj instanceof PipeConsensusTsFilePieceReq && super.equals(obj); + } + + @Override + public int hashCode() { + return super.hashCode(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFilePieceWithModReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFilePieceWithModReq.java new file mode 100644 index 000000000000..fd16625b81a7 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFilePieceWithModReq.java @@ -0,0 +1,80 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus.payload.request; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestType; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusTransferFilePieceReq; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; + +import java.io.IOException; + +public class PipeConsensusTsFilePieceWithModReq extends PipeConsensusTransferFilePieceReq { + + private PipeConsensusTsFilePieceWithModReq() { + // Empty constructor + } + + @Override + protected PipeConsensusRequestType getPlanType() { + return PipeConsensusRequestType.TRANSFER_TS_FILE_PIECE_WITH_MOD; + } + + /////////////////////////////// Thrift /////////////////////////////// + + public static PipeConsensusTsFilePieceWithModReq toTPipeConsensusTransferReq( + String fileName, + long startWritingOffset, + byte[] filePiece, + TCommitId commitId, + TConsensusGroupId consensusGroupId, + int thisDataNodeId) + throws IOException { + return (PipeConsensusTsFilePieceWithModReq) + new PipeConsensusTsFilePieceWithModReq() + .convertToTPipeConsensusTransferReq( + fileName, + startWritingOffset, + filePiece, + commitId, + consensusGroupId, + thisDataNodeId); + } + + public static PipeConsensusTsFilePieceWithModReq fromTPipeConsensusTransferReq( + TPipeConsensusTransferReq transferReq) { + return (PipeConsensusTsFilePieceWithModReq) + new PipeConsensusTsFilePieceWithModReq() + .translateFromTPipeConsensusTransferReq(transferReq); + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + return obj instanceof PipeConsensusTsFilePieceWithModReq && super.equals(obj); + } + + @Override + public int hashCode() { + return super.hashCode(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFileSealReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFileSealReq.java new file mode 100644 index 000000000000..0caf783d6957 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFileSealReq.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.db.pipe.connector.protocol.pipeconsensus.payload.request; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestType; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusTransferFileSealReq; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; + +import java.io.IOException; + +public class PipeConsensusTsFileSealReq extends PipeConsensusTransferFileSealReq { + private PipeConsensusTsFileSealReq() { + // Empty constructor + } + + @Override + protected PipeConsensusRequestType getPlanType() { + return PipeConsensusRequestType.TRANSFER_TS_FILE_SEAL; + } + + /////////////////////////////// Thrift /////////////////////////////// + + public static PipeConsensusTsFileSealReq toTPipeConsensusTransferReq( + String fileName, + long fileLength, + TCommitId commitId, + TConsensusGroupId consensusGroupId, + ProgressIndex progressIndex, + int thisDataNodeId) + throws IOException { + return (PipeConsensusTsFileSealReq) + new PipeConsensusTsFileSealReq() + .convertToTPipeConsensusTransferReq( + fileName, fileLength, commitId, consensusGroupId, progressIndex, thisDataNodeId); + } + + public static PipeConsensusTsFileSealReq fromTPipeConsensusTransferReq( + TPipeConsensusTransferReq req) { + return (PipeConsensusTsFileSealReq) + new PipeConsensusTsFileSealReq().translateFromTPipeConsensusTransferReq(req); + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + return obj instanceof PipeConsensusTsFileSealReq && super.equals(obj); + } + + @Override + public int hashCode() { + return super.hashCode(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFileSealWithModReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFileSealWithModReq.java new file mode 100644 index 000000000000..64b037174f7f --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTsFileSealWithModReq.java @@ -0,0 +1,85 @@ +/* + * 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.pipe.connector.protocol.pipeconsensus.payload.request; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestType; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusTransferFileSealWithModReq; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; + +public class PipeConsensusTsFileSealWithModReq extends PipeConsensusTransferFileSealWithModReq { + + private PipeConsensusTsFileSealWithModReq() { + // Empty constructor + } + + @Override + protected PipeConsensusRequestType getPlanType() { + return PipeConsensusRequestType.TRANSFER_TS_FILE_SEAL_WITH_MOD; + } + + /////////////////////////////// Thrift /////////////////////////////// + + public static PipeConsensusTsFileSealWithModReq toTPipeConsensusTransferReq( + String modFileName, + long modFileLength, + String tsFileName, + long tsFileLength, + TCommitId commitId, + TConsensusGroupId consensusGroupId, + ProgressIndex progressIndex, + int thisDataNodeId) + throws IOException { + return (PipeConsensusTsFileSealWithModReq) + new PipeConsensusTsFileSealWithModReq() + .convertToTPipeConsensusTransferReq( + Arrays.asList(modFileName, tsFileName), + Arrays.asList(modFileLength, tsFileLength), + new HashMap<>(), + commitId, + consensusGroupId, + progressIndex, + thisDataNodeId); + } + + public static PipeConsensusTsFileSealWithModReq fromTPipeConsensusTransferReq( + TPipeConsensusTransferReq req) { + return (PipeConsensusTsFileSealWithModReq) + new PipeConsensusTsFileSealWithModReq().translateFromTPipeConsensusTransferReq(req); + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + return obj instanceof PipeConsensusTsFileSealWithModReq && super.equals(obj); + } + + @Override + public int hashCode() { + return super.hashCode(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeDispatcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeDispatcher.java new file mode 100644 index 000000000000..82a6231cdd16 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeDispatcher.java @@ -0,0 +1,121 @@ +/* + * 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.pipe.consensus; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.consensus.ConfigRegionId; +import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeDispatcher; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeName; +import org.apache.iotdb.db.pipe.agent.PipeAgent; +import org.apache.iotdb.db.protocol.client.ConfigNodeClient; +import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; +import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; +import org.apache.iotdb.pipe.api.exception.PipeException; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +public class ConsensusPipeDataNodeDispatcher implements ConsensusPipeDispatcher { + private static final Logger LOGGER = + LoggerFactory.getLogger(ConsensusPipeDataNodeDispatcher.class); + + private static final IClientManager<ConfigRegionId, ConfigNodeClient> CONFIG_NODE_CLIENT_MANAGER = + ConfigNodeClientManager.getInstance(); + + @Override + public void createPipe( + String pipeName, + Map<String, String> extractorAttributes, + Map<String, String> processorAttributes, + Map<String, String> connectorAttributes) + throws Exception { + try (ConfigNodeClient configNodeClient = + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + TCreatePipeReq req = + new TCreatePipeReq() + .setPipeName(pipeName) + .setExtractorAttributes(extractorAttributes) + .setProcessorAttributes(processorAttributes) + .setConnectorAttributes(connectorAttributes); + TSStatus status = configNodeClient.createPipe(req); + if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != status.getCode()) { + LOGGER.warn("Failed to create consensus pipe-{}, status: {}", pipeName, status); + throw new PipeException(status.getMessage()); + } + } catch (Exception e) { + LOGGER.warn("Failed to create consensus pipe-{}", pipeName); + throw new PipeException("Failed to create consensus pipe", e); + } + } + + @Override + public void startPipe(String pipeName) throws Exception { + try (ConfigNodeClient configNodeClient = + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + TSStatus status = configNodeClient.startPipe(pipeName); + if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != status.getCode()) { + LOGGER.warn("Failed to start consensus pipe-{}, status: {}", pipeName, status); + throw new PipeException(status.getMessage()); + } + } catch (Exception e) { + LOGGER.warn("Failed to start consensus pipe-{}", pipeName); + throw new PipeException("Failed to start consensus pipe", e); + } + } + + @Override + public void stopPipe(String pipeName) throws Exception { + try (ConfigNodeClient configNodeClient = + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + final TSStatus status = configNodeClient.stopPipe(pipeName); + if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != status.getCode()) { + LOGGER.warn("Failed to stop consensus pipe-{}, status: {}", pipeName, status); + throw new PipeException(status.getMessage()); + } + } catch (Exception e) { + LOGGER.warn("Failed to stop consensus pipe-{}", pipeName); + throw new PipeException("Failed to stop consensus pipe", e); + } + } + + // Use ConsensusPipeName instead of String to provide information for receiverAgent to release + // corresponding resource + @Override + public void dropPipe(ConsensusPipeName pipeName) throws Exception { + try (ConfigNodeClient configNodeClient = + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + final TSStatus status = configNodeClient.dropPipe(pipeName.toString()); + if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != status.getCode()) { + LOGGER.warn("Failed to drop consensus pipe-{}, status: {}", pipeName, status); + throw new PipeException(status.getMessage()); + } + } catch (Exception e) { + LOGGER.warn("Failed to drop consensus pipe-{}", pipeName); + throw new PipeException("Failed to drop consensus pipe", e); + } + // Release corresponding receiver's resource + PipeAgent.receiver().pipeConsensus().handleDropPipeConsensusTask(pipeName); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeRuntimeAgentGuardian.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeRuntimeAgentGuardian.java new file mode 100644 index 000000000000..096cfdc6985e --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeRuntimeAgentGuardian.java @@ -0,0 +1,48 @@ +/* + * 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.pipe.consensus; + +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeGuardian; +import org.apache.iotdb.db.pipe.agent.PipeAgent; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ConsensusPipeDataNodeRuntimeAgentGuardian implements ConsensusPipeGuardian { + private static final Logger LOGGER = + LoggerFactory.getLogger(ConsensusPipeDataNodeRuntimeAgentGuardian.class); + private boolean registered = false; + + @Override + public synchronized void start(String id, Runnable guardJob, long intervalInSeconds) { + if (!registered) { + LOGGER.info( + "Registering periodical job {} with interval in seconds {}.", id, intervalInSeconds); + + this.registered = true; + PipeAgent.runtime().registerPeriodicalJob(id, guardJob, intervalInSeconds); + } + } + + @Override + public synchronized void stop() { + // Do nothing because PipePeriodicalJobExecutor currently has no deregister logic + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusSubtaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusSubtaskExecutor.java new file mode 100644 index 000000000000..5d99c5038fc6 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusSubtaskExecutor.java @@ -0,0 +1,32 @@ +/* + * 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.pipe.consensus; + +import org.apache.iotdb.commons.concurrent.ThreadName; +import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.commons.pipe.execution.executor.PipeSubtaskExecutor; + +public class PipeConsensusSubtaskExecutor extends PipeSubtaskExecutor { + + public PipeConsensusSubtaskExecutor() { + super( + PipeConfig.getInstance().getPipeSubtaskExecutorMaxThreadNum(), + ThreadName.PIPE_CONSENSUS_EXECUTOR_POOL); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java new file mode 100644 index 000000000000..3f7cf9239db6 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java @@ -0,0 +1,128 @@ +/* + * 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.pipe.consensus; + +import org.apache.iotdb.commons.consensus.ConsensusGroupId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.ProgressIndexType; +import org.apache.iotdb.commons.consensus.index.impl.HybridProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeName; +import org.apache.iotdb.consensus.pipe.consensuspipe.ProgressIndexManager; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.pipe.agent.PipeAgent; +import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +public class ProgressIndexDataNodeManager implements ProgressIndexManager { + private final Map<ConsensusGroupId, ProgressIndex> groupId2MaxProgressIndex; + private static final int DATA_NODE_ID = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + + public ProgressIndexDataNodeManager() { + this.groupId2MaxProgressIndex = new ConcurrentHashMap<>(); + + recoverMaxProgressIndexFromDataRegion(); + } + + private void recoverMaxProgressIndexFromDataRegion() { + StorageEngine.getInstance() + .getAllDataRegionIds() + .forEach( + dataRegionId -> { + final TsFileManager tsFileManager = + StorageEngine.getInstance().getDataRegion(dataRegionId).getTsFileManager(); + + final List<ProgressIndex> allProgressIndex = new ArrayList<>(); + allProgressIndex.addAll( + tsFileManager.getTsFileList(true).stream() + .map(TsFileResource::getMaxProgressIndex) + .collect(Collectors.toList())); + allProgressIndex.addAll( + tsFileManager.getTsFileList(false).stream() + .map(TsFileResource::getMaxProgressIndex) + .collect(Collectors.toList())); + + ProgressIndex maxProgressIndex = MinimumProgressIndex.INSTANCE; + for (ProgressIndex progressIndex : allProgressIndex) { + maxProgressIndex = + maxProgressIndex.updateToMinimumEqualOrIsAfterProgressIndex( + extractLocalSimpleProgressIndex(progressIndex)); + } + groupId2MaxProgressIndex + .computeIfAbsent(dataRegionId, o -> MinimumProgressIndex.INSTANCE) + .updateToMinimumEqualOrIsAfterProgressIndex(maxProgressIndex); + }); + + // TODO: update deletion progress index + } + + private ProgressIndex extractLocalSimpleProgressIndex(ProgressIndex progressIndex) { + if (progressIndex instanceof RecoverProgressIndex) { + final Map<Integer, SimpleProgressIndex> dataNodeId2LocalIndex = + ((RecoverProgressIndex) progressIndex).getDataNodeId2LocalIndex(); + return dataNodeId2LocalIndex.containsKey(DATA_NODE_ID) + ? dataNodeId2LocalIndex.get(DATA_NODE_ID) + : MinimumProgressIndex.INSTANCE; + } else if (progressIndex instanceof HybridProgressIndex) { + final Map<Short, ProgressIndex> type2Index = + ((HybridProgressIndex) progressIndex).getType2Index(); + if (!type2Index.containsKey(ProgressIndexType.RECOVER_PROGRESS_INDEX.getType())) { + return MinimumProgressIndex.INSTANCE; + } + final Map<Integer, SimpleProgressIndex> dataNodeId2LocalIndex = + ((RecoverProgressIndex) + type2Index.get(ProgressIndexType.RECOVER_PROGRESS_INDEX.getType())) + .getDataNodeId2LocalIndex(); + return dataNodeId2LocalIndex.containsKey(DATA_NODE_ID) + ? dataNodeId2LocalIndex.get(DATA_NODE_ID) + : MinimumProgressIndex.INSTANCE; + } + return MinimumProgressIndex.INSTANCE; + } + + @Override + public ProgressIndex getProgressIndex(ConsensusPipeName consensusPipeName) { + return PipeAgent.task() + .getPipeTaskProgressIndex( + consensusPipeName.toString(), consensusPipeName.getConsensusGroupId().getId()); + } + + @Override + public ProgressIndex assignProgressIndex(ConsensusGroupId consensusGroupId) { + return groupId2MaxProgressIndex + .computeIfAbsent(consensusGroupId, o -> MinimumProgressIndex.INSTANCE) + .updateToMinimumEqualOrIsAfterProgressIndex( + PipeAgent.runtime().assignProgressIndexForPipeConsensus()); + } + + @Override + public ProgressIndex getMaxAssignedProgressIndex(ConsensusGroupId consensusGroupId) { + return groupId2MaxProgressIndex.getOrDefault(consensusGroupId, MinimumProgressIndex.INSTANCE); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/execution/PipeSubtaskExecutorManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/execution/PipeSubtaskExecutorManager.java index 1792a1015d6d..78db89315229 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/execution/PipeSubtaskExecutorManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/execution/PipeSubtaskExecutorManager.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.pipe.execution; +import org.apache.iotdb.db.pipe.consensus.PipeConsensusSubtaskExecutor; import org.apache.iotdb.db.subscription.execution.executor.SubscriptionSubtaskExecutor; /** @@ -29,6 +30,7 @@ public class PipeSubtaskExecutorManager { private final PipeProcessorSubtaskExecutor processorExecutor; private final PipeConnectorSubtaskExecutor connectorExecutor; private final SubscriptionSubtaskExecutor subscriptionExecutor; + private final PipeConsensusSubtaskExecutor consensusExecutor; public PipeProcessorSubtaskExecutor getProcessorExecutor() { return processorExecutor; @@ -42,12 +44,17 @@ public SubscriptionSubtaskExecutor getSubscriptionExecutor() { return subscriptionExecutor; } + public PipeConsensusSubtaskExecutor getConsensusExecutor() { + return consensusExecutor; + } + ///////////////////////// Singleton Instance Holder ///////////////////////// private PipeSubtaskExecutorManager() { processorExecutor = new PipeProcessorSubtaskExecutor(); connectorExecutor = new PipeConnectorSubtaskExecutor(); subscriptionExecutor = new SubscriptionSubtaskExecutor(); + consensusExecutor = new PipeConsensusSubtaskExecutor(); } private static class PipeTaskExecutorHolder { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/pipeconsensus/PipeConsensusProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/pipeconsensus/PipeConsensusProcessor.java new file mode 100644 index 000000000000..ee2bfc5ce5de --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/pipeconsensus/PipeConsensusProcessor.java @@ -0,0 +1,103 @@ +/* + * 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.pipe.processor.pipeconsensus; + +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.ProgressIndexType; +import org.apache.iotdb.commons.consensus.index.impl.HybridProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.pipe.api.PipeProcessor; +import org.apache.iotdb.pipe.api.collector.EventCollector; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeProcessorRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; +import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; + +import java.util.Map; + +public class PipeConsensusProcessor implements PipeProcessor { + private static final int DATA_NODE_ID = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + + @Override + public void validate(PipeParameterValidator validator) throws Exception {} + + @Override + public void customize(PipeParameters parameters, PipeProcessorRuntimeConfiguration configuration) + throws Exception {} + + private boolean isContainLocalData(EnrichedEvent enrichedEvent) { + final ProgressIndex progressIndex = enrichedEvent.getProgressIndex(); + if (progressIndex instanceof RecoverProgressIndex) { + return ((RecoverProgressIndex) progressIndex) + .getDataNodeId2LocalIndex() + .containsKey(DATA_NODE_ID); + } else if (progressIndex instanceof HybridProgressIndex) { + final Map<Short, ProgressIndex> type2Index = + ((HybridProgressIndex) progressIndex).getType2Index(); + if (!type2Index.containsKey(ProgressIndexType.RECOVER_PROGRESS_INDEX.getType())) { + return false; + } + return ((RecoverProgressIndex) + type2Index.get(ProgressIndexType.RECOVER_PROGRESS_INDEX.getType())) + .getDataNodeId2LocalIndex() + .containsKey(DATA_NODE_ID); + } + return false; + } + + @Override + public void process(TsFileInsertionEvent tsFileInsertionEvent, EventCollector eventCollector) + throws Exception { + if (tsFileInsertionEvent instanceof EnrichedEvent) { + final EnrichedEvent enrichedEvent = (EnrichedEvent) tsFileInsertionEvent; + if (isContainLocalData(enrichedEvent)) { + eventCollector.collect(tsFileInsertionEvent); + } + } + } + + @Override + public void process(TabletInsertionEvent tabletInsertionEvent, EventCollector eventCollector) + throws Exception { + if (tabletInsertionEvent instanceof EnrichedEvent) { + final EnrichedEvent enrichedEvent = (EnrichedEvent) tabletInsertionEvent; + if (isContainLocalData(enrichedEvent)) { + eventCollector.collect(tabletInsertionEvent); + } + } + } + + @Override + public void process(Event event, EventCollector eventCollector) throws Exception { + if (event instanceof EnrichedEvent) { + final EnrichedEvent enrichedEvent = (EnrichedEvent) event; + if (isContainLocalData(enrichedEvent)) { + eventCollector.collect(event); + } + } + } + + @Override + public void close() throws Exception {} +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/progress/SimpleConsensusProgressIndexAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/progress/SimpleProgressIndexAssigner.java similarity index 91% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/progress/SimpleConsensusProgressIndexAssigner.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/progress/SimpleProgressIndexAssigner.java index 0105acd434c9..766945a934b6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/progress/SimpleConsensusProgressIndexAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/progress/SimpleProgressIndexAssigner.java @@ -37,10 +37,9 @@ import static org.apache.iotdb.consensus.ConsensusFactory.SIMPLE_CONSENSUS; -public class SimpleConsensusProgressIndexAssigner { +public class SimpleProgressIndexAssigner { - private static final Logger LOGGER = - LoggerFactory.getLogger(SimpleConsensusProgressIndexAssigner.class); + private static final Logger LOGGER = LoggerFactory.getLogger(SimpleProgressIndexAssigner.class); private static final IoTDBConfig IOTDB_CONFIG = IoTDBDescriptor.getInstance().getConfig(); @@ -54,12 +53,12 @@ public class SimpleConsensusProgressIndexAssigner { private boolean isSimpleConsensusEnable = false; private int rebootTimes = 0; - private final AtomicLong insertionRequestId = new AtomicLong(0); + private final AtomicLong insertionRequestId = new AtomicLong(1); public void start() throws StartupException { isSimpleConsensusEnable = IOTDB_CONFIG.getDataRegionConsensusProtocolClass().equals(SIMPLE_CONSENSUS); - LOGGER.info("Start SimpleConsensusProgressIndexAssigner ..."); + LOGGER.info("Start SimpleProgressIndexAssigner ..."); try { makeDirIfNecessary(); @@ -107,7 +106,7 @@ public void assignIfNeeded(InsertNode insertNode) { new SimpleProgressIndex(rebootTimes, insertionRequestId.getAndIncrement())); } - public SimpleProgressIndex getSimpleProgressIndexForTsFileRecovery() { + public SimpleProgressIndex getSimpleProgressIndex() { return new SimpleProgressIndex(rebootTimes, insertionRequestId.getAndIncrement()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java new file mode 100644 index 000000000000..60a0b88beb90 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java @@ -0,0 +1,1244 @@ +/* + * 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.pipe.receiver.protocol.pipeconsensus; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.consensus.ConsensusGroupId; +import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.ProgressIndexType; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestType; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestVersion; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusTransferFilePieceReq; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.response.PipeConsensusTransferFilePieceResp; +import org.apache.iotdb.commons.pipe.connector.payload.thrift.response.PipeTransferFilePieceResp; +import org.apache.iotdb.commons.pipe.receiver.IoTDBReceiverAgent; +import org.apache.iotdb.consensus.exception.ConsensusGroupNotExistException; +import org.apache.iotdb.consensus.pipe.PipeConsensus; +import org.apache.iotdb.consensus.pipe.PipeConsensusServerImpl; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeName; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.exception.DiskSpaceInsufficientException; +import org.apache.iotdb.db.exception.LoadFileException; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletBinaryReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletInsertNodeReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealWithModReq; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; +import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus; +import org.apache.iotdb.db.storageengine.dataregion.utils.TsFileResourceUtils; +import org.apache.iotdb.db.storageengine.rescon.disk.FolderManager; +import org.apache.iotdb.db.storageengine.rescon.disk.strategy.DirectoryStrategyType; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.commons.io.FileUtils; +import org.apache.tsfile.common.constant.TsFileConstant; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.TreeSet; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; + +public class PipeConsensusReceiver { + private static final Logger LOGGER = LoggerFactory.getLogger(PipeConsensusReceiver.class); + private static final IoTDBConfig IOTDB_CONFIG = IoTDBDescriptor.getInstance().getConfig(); + private static final long PIPE_CONSENSUS_RECEIVER_MAX_WAITING_TIME_IN_MS = + (long) IOTDB_CONFIG.getConnectionTimeoutInMS() + / 6 + * IOTDB_CONFIG.getPipeConsensusPipelineSize(); + private static final long CLOSE_TSFILE_WRITER_MAX_WAIT_TIME_IN_MS = 5000; + private static final long RETRY_WAIT_TIME = 500; + private final RequestExecutor requestExecutor = new RequestExecutor(); + private final PipeConsensus pipeConsensus; + private final ConsensusGroupId consensusGroupId; + // Used to buffer TsFile when transfer TsFile asynchronously. + private final List<String> receiverBaseDirsName; + private final PipeConsensusTsFileWriterPool pipeConsensusTsFileWriterPool = + new PipeConsensusTsFileWriterPool(); + private final AtomicReference<File> receiverFileDirWithIdSuffix = new AtomicReference<>(); + private FolderManager folderManager; + + public PipeConsensusReceiver( + PipeConsensus pipeConsensus, + ConsensusGroupId consensusGroupId, + ConsensusPipeName consensusPipeName) { + this.pipeConsensus = pipeConsensus; + this.consensusGroupId = consensusGroupId; + + // Each pipeConsensusReceiver has its own base directories. for example, a default dir path is + // data/datanode/system/pipe/consensus/receiver/__consensus{consensusGroupId}_{leaderDataNodeId}_{followerDataNodeId} + receiverBaseDirsName = + Arrays.stream(IoTDBDescriptor.getInstance().getConfig().getPipeConsensusReceiverFileDirs()) + .map(s -> s + File.separator + consensusPipeName) + .collect(Collectors.toList()); + + try { + this.folderManager = + new FolderManager(receiverBaseDirsName, DirectoryStrategyType.SEQUENCE_STRATEGY); + initiateTsFileBufferFolder(); + } catch (Exception e) { + LOGGER.error( + "Fail to create pipeConsensus receiver file folders allocation strategy because all disks of folders are full.", + e); + throw new RuntimeException(e); + } + } + + /** + * This method cannot be set to synchronize. Receive events can be concurrent since reqBuffer but + * load event must be synchronized. + */ + public TPipeConsensusTransferResp receive(final TPipeConsensusTransferReq req) { + // PreCheck: if there are these cases: read-only; null impl; inactive impl, etc. The receiver + // will reject synchronization. + TPipeConsensusTransferResp resp = preCheckForReceiver(req); + if (resp != null) { + return resp; + } + + final short rawRequestType = req.getType(); + if (PipeConsensusRequestType.isValidatedRequestType(rawRequestType)) { + switch (PipeConsensusRequestType.valueOf(rawRequestType)) { + case TRANSFER_TS_FILE_PIECE: + case TRANSFER_TS_FILE_PIECE_WITH_MOD: + // Just take a place in requestExecutor's buffer, the further seal request will remove + // its place from buffer. + requestExecutor.onRequest(req, true); + return loadEvent(req); + case TRANSFER_TS_FILE_SEAL: + case TRANSFER_TS_FILE_SEAL_WITH_MOD: + // TODO: check memory when logging wal(in further version) + case TRANSFER_TABLET_BINARY: + case TRANSFER_TABLET_INSERT_NODE: + // TODO: support batch transfer(in further version) + case TRANSFER_TABLET_BATCH: + default: + return requestExecutor.onRequest(req, false); + } + } + // Unknown request type, which means the request can not be handled by this receiver, + // maybe the version of the receiver is not compatible with the sender + final TSStatus status = + RpcUtils.getStatus( + TSStatusCode.PIPE_TYPE_ERROR, + String.format("PipeConsensus Unknown PipeRequestType %s.", rawRequestType)); + if (LOGGER.isWarnEnabled()) { + LOGGER.warn("PipeConsensus Unknown PipeRequestType, response status = {}.", status); + } + return new TPipeConsensusTransferResp(status); + } + + private TPipeConsensusTransferResp preCheckForReceiver(final TPipeConsensusTransferReq req) { + ConsensusGroupId groupId = + ConsensusGroupId.Factory.createFromTConsensusGroupId(req.getConsensusGroupId()); + PipeConsensusServerImpl impl = pipeConsensus.getImpl(groupId); + + if (impl == null) { + String message = String.format("PipeConsensus: unexpected consensusGroupId %s", groupId); + if (LOGGER.isErrorEnabled()) { + LOGGER.error(message); + } + return new TPipeConsensusTransferResp( + RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode(), message)); + } + if (impl.isReadOnly()) { + String message = + String.format( + "PipeConsensus-ConsensusGroupId-%s: fail to receive because system is read-only.", + groupId); + if (LOGGER.isErrorEnabled()) { + LOGGER.error(message); + } + return new TPipeConsensusTransferResp( + RpcUtils.getStatus(TSStatusCode.SYSTEM_READ_ONLY.getStatusCode(), message)); + } + if (!impl.isActive()) { + String message = + String.format( + "PipeConsensus-ConsensusGroupId-%s: fail to receive because peer is inactive and not ready.", + groupId); + if (LOGGER.isWarnEnabled()) { + LOGGER.warn(message); + } + return new TPipeConsensusTransferResp( + RpcUtils.getStatus(TSStatusCode.WRITE_PROCESS_REJECT.getStatusCode(), message)); + } + + return null; + } + + private TPipeConsensusTransferResp loadEvent(final TPipeConsensusTransferReq req) { + // synchronized load event, ensured by upper caller's lock. + try { + final short rawRequestType = req.getType(); + if (PipeConsensusRequestType.isValidatedRequestType(rawRequestType)) { + switch (PipeConsensusRequestType.valueOf(rawRequestType)) { + case TRANSFER_TABLET_INSERT_NODE: + return handleTransferTabletInsertNode( + PipeConsensusTabletInsertNodeReq.fromTPipeConsensusTransferReq(req)); + case TRANSFER_TABLET_BINARY: + return handleTransferTabletBinary( + PipeConsensusTabletBinaryReq.fromTPipeConsensusTransferReq(req)); + case TRANSFER_TS_FILE_PIECE: + return handleTransferFilePiece( + PipeConsensusTsFilePieceReq.fromTPipeConsensusTransferReq(req), true); + case TRANSFER_TS_FILE_SEAL: + return handleTransferFileSeal( + PipeConsensusTsFileSealReq.fromTPipeConsensusTransferReq(req)); + case TRANSFER_TS_FILE_PIECE_WITH_MOD: + return handleTransferFilePiece( + PipeConsensusTsFilePieceReq.fromTPipeConsensusTransferReq(req), false); + case TRANSFER_TS_FILE_SEAL_WITH_MOD: + return handleTransferFileSealWithMods( + PipeConsensusTsFileSealWithModReq.fromTPipeConsensusTransferReq(req)); + case TRANSFER_TABLET_BATCH: + LOGGER.info("PipeConsensus transfer batch hasn't been implemented yet."); + default: + break; + } + } + // Unknown request type, which means the request can not be handled by this receiver, + // maybe the version of the receiver is not compatible with the sender + final TSStatus status = + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_TYPE_ERROR, + String.format("Unknown PipeConsensusRequestType %s.", rawRequestType)); + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Unknown PipeRequestType, response status = {}.", + consensusGroupId, + status); + return new TPipeConsensusTransferResp(status); + } catch (Exception e) { + final String error = String.format("Serialization error during pipe receiving, %s", e); + LOGGER.warn("PipeConsensus-ConsensusGroupId-{}: {}", consensusGroupId, error, e); + return new TPipeConsensusTransferResp(RpcUtils.getStatus(TSStatusCode.PIPE_ERROR, error)); + } + } + + private TPipeConsensusTransferResp handleTransferTabletInsertNode( + final PipeConsensusTabletInsertNodeReq req) throws ConsensusGroupNotExistException { + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: starting to receive tablet insertNode", + consensusGroupId); + PipeConsensusServerImpl impl = + Optional.ofNullable(pipeConsensus.getImpl(consensusGroupId)) + .orElseThrow(() -> new ConsensusGroupNotExistException(consensusGroupId)); + final InsertNode insertNode = req.getInsertNode(); + insertNode.setProgressIndex( + ProgressIndexType.deserializeFrom(ByteBuffer.wrap(req.getProgressIndex()))); + return new TPipeConsensusTransferResp(impl.writeOnFollowerReplica(insertNode)); + } + + private TPipeConsensusTransferResp handleTransferTabletBinary( + final PipeConsensusTabletBinaryReq req) throws ConsensusGroupNotExistException { + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: starting to receive tablet binary", consensusGroupId); + PipeConsensusServerImpl impl = + Optional.ofNullable(pipeConsensus.getImpl(consensusGroupId)) + .orElseThrow(() -> new ConsensusGroupNotExistException(consensusGroupId)); + final InsertNode insertNode = req.convertToInsertNode(); + insertNode.setProgressIndex( + ProgressIndexType.deserializeFrom(ByteBuffer.wrap(req.getProgressIndex()))); + return new TPipeConsensusTransferResp(impl.writeOnFollowerReplica(insertNode)); + } + + private TPipeConsensusTransferResp handleTransferFilePiece( + final PipeConsensusTransferFilePieceReq req, final boolean isSingleFile) { + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: starting to receive tsFile pieces", consensusGroupId); + PipeConsensusTsFileWriter diskBuffer = + pipeConsensusTsFileWriterPool.borrowCorrespondingWriter(req.getCommitId()); + + try { + updateWritingFileIfNeeded(diskBuffer, req.getFileName(), isSingleFile); + final File writingFile = diskBuffer.getWritingFile(); + final RandomAccessFile writingFileWriter = diskBuffer.getWritingFileWriter(); + + if (isWritingFileOffsetNonCorrect(diskBuffer, req.getStartWritingOffset())) { + if (!writingFile.getName().endsWith(TsFileConstant.TSFILE_SUFFIX)) { + // If the file is a tsFile, then the content will not be changed for a specific + // filename. However, for other files (mod, snapshot, etc.) the content varies for the + // same name in different times, then we must rewrite the file to apply the newest + // version. + writingFileWriter.setLength(0); + } + + final TSStatus status = + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_OFFSET_RESET, + String.format( + "Request sender to reset file reader's offset from %s to %s.", + req.getStartWritingOffset(), writingFileWriter.length())); + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: File offset reset requested by receiver, response status = {}.", + consensusGroupId, + status); + return PipeConsensusTransferFilePieceResp.toTPipeConsensusTransferResp( + status, writingFileWriter.length()); + } + + writingFileWriter.write(req.getFilePiece()); + return PipeConsensusTransferFilePieceResp.toTPipeConsensusTransferResp( + RpcUtils.SUCCESS_STATUS, writingFileWriter.length()); + } catch (Exception e) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to write file piece from req {}.", + consensusGroupId, + req, + e); + final TSStatus status = + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR, + String.format("Failed to write file piece, because %s", e.getMessage())); + try { + return PipeConsensusTransferFilePieceResp.toTPipeConsensusTransferResp( + status, PipeTransferFilePieceResp.ERROR_END_OFFSET); + } catch (IOException ex) { + return PipeConsensusTransferFilePieceResp.toTPipeConsensusTransferResp(status); + } + } + } + + private TPipeConsensusTransferResp handleTransferFileSeal(final PipeConsensusTsFileSealReq req) { + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: starting to receive tsFile seal", consensusGroupId); + PipeConsensusTsFileWriter tsFileWriter = + pipeConsensusTsFileWriterPool.borrowCorrespondingWriter(req.getCommitId()); + File writingFile = tsFileWriter.getWritingFile(); + RandomAccessFile writingFileWriter = tsFileWriter.getWritingFileWriter(); + + try { + if (isWritingFileNonAvailable(tsFileWriter)) { + final TSStatus status = + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR, + String.format( + "Failed to seal file, because writing file %s is not available.", writingFile)); + LOGGER.warn(status.getMessage()); + return new TPipeConsensusTransferResp(status); + } + + final TPipeConsensusTransferResp resp = + checkFinalFileSeal(tsFileWriter, req.getFileName(), req.getFileLength()); + if (Objects.nonNull(resp)) { + return resp; + } + + final String fileAbsolutePath = writingFile.getAbsolutePath(); + + // 1. The writing file writer must be closed, otherwise it may cause concurrent errors during + // the process of loading tsfile when parsing tsfile. + // + // 2. The writing file must be set to null, otherwise if the next passed tsfile has the same + // name as the current tsfile, it will bypass the judgment logic of + // updateWritingFileIfNeeded#isFileExistedAndNameCorrect, and continue to write to the already + // loaded file. Since the writing file writer has already been closed, it will throw a Stream + // Close exception. + writingFileWriter.close(); + tsFileWriter.setWritingFileWriter(null); + + // writingFile will be deleted after load if no exception occurs + tsFileWriter.setWritingFile(null); + + final TSStatus status = + loadFileToDataRegion( + fileAbsolutePath, + ProgressIndexType.deserializeFrom(ByteBuffer.wrap(req.getProgressIndex()))); + if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + // if transfer success, disk buffer will be released. + tsFileWriter.returnSelf(); + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: Seal file {} successfully.", + consensusGroupId, + fileAbsolutePath); + } else { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {}, because {}.", + consensusGroupId, + fileAbsolutePath, + status.getMessage()); + } + return new TPipeConsensusTransferResp(status); + } catch (IOException e) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {} from req {}.", + consensusGroupId, + writingFile, + req, + e); + return new TPipeConsensusTransferResp( + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR, + String.format("Failed to seal file %s because %s", writingFile, e.getMessage()))); + } catch (LoadFileException e) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to load file {} from req {}.", + consensusGroupId, + writingFile, + req, + e); + return new TPipeConsensusTransferResp( + RpcUtils.getStatus( + TSStatusCode.LOAD_FILE_ERROR, + String.format("Failed to seal file %s because %s", writingFile, e.getMessage()))); + } finally { + // If the writing file is not sealed successfully, the writing file will be deleted. + // All pieces of the writing file and its mod (if exists) should be retransmitted by the + // sender. + closeCurrentWritingFileWriter(tsFileWriter); + deleteCurrentWritingFile(tsFileWriter); + } + } + + private TPipeConsensusTransferResp handleTransferFileSealWithMods( + final PipeConsensusTsFileSealWithModReq req) { + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: starting to receive tsFile seal with mods", + consensusGroupId); + PipeConsensusTsFileWriter tsFileWriter = + pipeConsensusTsFileWriterPool.borrowCorrespondingWriter(req.getCommitId()); + File writingFile = tsFileWriter.getWritingFile(); + RandomAccessFile writingFileWriter = tsFileWriter.getWritingFileWriter(); + + final List<File> files = + req.getFileNames().stream() + .map(fileName -> new File(receiverFileDirWithIdSuffix.get(), fileName)) + .collect(Collectors.toList()); + try { + if (isWritingFileNonAvailable(tsFileWriter)) { + final TSStatus status = + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR, + String.format( + "Failed to seal file %s, because writing file %s is not available.", + req.getFileNames(), writingFile)); + LOGGER.warn(status.getMessage()); + return new TPipeConsensusTransferResp(status); + } + + // Any of the transferred files cannot be empty, or else the receiver + // will not sense this file because no pieces are sent + for (int i = 0; i < req.getFileNames().size(); ++i) { + final TPipeConsensusTransferResp resp = + i == req.getFileNames().size() - 1 + ? checkFinalFileSeal( + tsFileWriter, req.getFileNames().get(i), req.getFileLengths().get(i)) + : checkNonFinalFileSeal( + tsFileWriter, + files.get(i), + req.getFileNames().get(i), + req.getFileLengths().get(i)); + if (Objects.nonNull(resp)) { + return resp; + } + } + + // 1. The writing file writer must be closed, otherwise it may cause concurrent errors during + // the process of loading tsfile when parsing tsfile. + // + // 2. The writing file must be set to null, otherwise if the next passed tsfile has the same + // name as the current tsfile, it will bypass the judgment logic of + // updateWritingFileIfNeeded#isFileExistedAndNameCorrect, and continue to write to the already + // loaded file. Since the writing file writer has already been closed, it will throw a Stream + // Close exception. + writingFileWriter.close(); + tsFileWriter.setWritingFileWriter(null); + + // WritingFile will be deleted after load if no exception occurs + tsFileWriter.setWritingFile(null); + + final List<String> fileAbsolutePaths = + files.stream().map(File::getAbsolutePath).collect(Collectors.toList()); + + final TSStatus status = + loadFileToDataRegion( + fileAbsolutePaths.get(1), + ProgressIndexType.deserializeFrom(ByteBuffer.wrap(req.getProgressIndex()))); + if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + // if transfer success, disk buffer will be released. + tsFileWriter.returnSelf(); + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: Seal file with mods {} successfully.", + consensusGroupId, + fileAbsolutePaths); + } else { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {}, status is {}.", + consensusGroupId, + fileAbsolutePaths, + status); + } + return new TPipeConsensusTransferResp(status); + } catch (Exception e) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {} from req {}.", + consensusGroupId, + files, + req, + e); + return new TPipeConsensusTransferResp( + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR, + String.format("Failed to seal file %s because %s", writingFile, e.getMessage()))); + } finally { + // If the writing file is not sealed successfully, the writing file will be deleted. + // All pieces of the writing file and its mod(if exists) should be retransmitted by the + // sender. + closeCurrentWritingFileWriter(tsFileWriter); + // Clear the directory instead of only deleting the referenced files in seal request + // to avoid previously undeleted file being redundant when transferring multi files + IoTDBReceiverAgent.cleanPipeReceiverDir(receiverFileDirWithIdSuffix.get()); + } + } + + private TPipeConsensusTransferResp checkNonFinalFileSeal( + final PipeConsensusTsFileWriter diskBuffer, + final File file, + final String fileName, + final long fileLength) + throws IOException { + final RandomAccessFile writingFileWriter = diskBuffer.getWritingFileWriter(); + + if (!file.exists()) { + final TSStatus status = + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR, + String.format("Failed to seal file %s, the file does not exist.", fileName)); + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {}, because the file does not exist.", + consensusGroupId, + fileName); + return new TPipeConsensusTransferResp(status); + } + + if (fileLength != file.length()) { + final TSStatus status = + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR, + String.format( + "Failed to seal file %s, because the length of file is not correct. " + + "The original file has length %s, but receiver file has length %s.", + fileName, fileLength, writingFileWriter.length())); + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {} when check non final seal, because the length of file is not correct. " + + "The original file has length {}, but receiver file has length {}.", + consensusGroupId, + fileName, + fileLength, + writingFileWriter.length()); + return new TPipeConsensusTransferResp(status); + } + + return null; + } + + private TSStatus loadFileToDataRegion(String filePath, ProgressIndex progressIndex) + throws IOException, LoadFileException { + StorageEngine.getInstance() + .getDataRegion(((DataRegionId) consensusGroupId)) + .loadNewTsFile(generateTsFileResource(filePath, progressIndex), true, false); + return RpcUtils.SUCCESS_STATUS; + } + + private TsFileResource generateTsFileResource(String filePath, ProgressIndex progressIndex) + throws IOException { + final File tsFile = new File(filePath); + + final TsFileResource tsFileResource = new TsFileResource(tsFile); + try (final TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + TsFileResourceUtils.updateTsFileResource(reader, tsFileResource); + } + + tsFileResource.setStatus(TsFileResourceStatus.NORMAL); + tsFileResource.setProgressIndex(progressIndex); + tsFileResource.serialize(); + return tsFileResource; + } + + private boolean isWritingFileNonAvailable(PipeConsensusTsFileWriter diskBuffer) { + File writingFile = diskBuffer.getWritingFile(); + RandomAccessFile writingFileWriter = diskBuffer.getWritingFileWriter(); + + final boolean isWritingFileAvailable = + writingFile != null && writingFile.exists() && writingFileWriter != null; + if (!isWritingFileAvailable) { + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: Writing file {} is not available. " + + "Writing file is null: {}, writing file exists: {}, writing file writer is null: {}.", + consensusGroupId, + writingFile, + writingFile == null, + writingFile != null && writingFile.exists(), + writingFileWriter == null); + } + return !isWritingFileAvailable; + } + + private TPipeConsensusTransferResp checkFinalFileSeal( + final PipeConsensusTsFileWriter diskBuffer, final String fileName, final long fileLength) + throws IOException { + final File writingFile = diskBuffer.getWritingFile(); + final RandomAccessFile writingFileWriter = diskBuffer.getWritingFileWriter(); + + if (!isFileExistedAndNameCorrect(diskBuffer, fileName)) { + final TSStatus status = + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR, + String.format( + "Failed to seal file %s, because writing file is %s.", fileName, writingFile)); + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {}, because writing file is {}.", + consensusGroupId, + fileName, + writingFile); + return new TPipeConsensusTransferResp(status); + } + + if (isWritingFileOffsetNonCorrect(diskBuffer, fileLength)) { + final TSStatus status = + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_TRANSFER_FILE_ERROR, + String.format( + "Failed to seal file %s, because the length of file is not correct. " + + "The original file has length %s, but receiver file has length %s.", + fileName, fileLength, writingFileWriter.length())); + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to seal file {} when check final seal file, because the length of file is not correct. " + + "The original file has length {}, but receiver file has length {}.", + consensusGroupId, + fileName, + fileLength, + writingFileWriter.length()); + return new TPipeConsensusTransferResp(status); + } + + return null; + } + + private boolean isFileExistedAndNameCorrect( + PipeConsensusTsFileWriter diskBuffer, String fileName) { + final File writingFile = diskBuffer.getWritingFile(); + return writingFile != null && writingFile.getName().equals(fileName); + } + + private boolean isWritingFileOffsetNonCorrect( + PipeConsensusTsFileWriter diskBuffer, final long offset) throws IOException { + final File writingFile = diskBuffer.getWritingFile(); + final RandomAccessFile writingFileWriter = diskBuffer.getWritingFileWriter(); + + final boolean offsetCorrect = writingFileWriter.length() == offset; + if (!offsetCorrect) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Writing file {}'s offset is {}, but request sender's offset is {}.", + consensusGroupId, + writingFile.getPath(), + writingFileWriter.length(), + offset); + } + return !offsetCorrect; + } + + private void closeCurrentWritingFileWriter(PipeConsensusTsFileWriter diskBuffer) { + if (diskBuffer.getWritingFileWriter() != null) { + try { + diskBuffer.getWritingFileWriter().close(); + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: Current writing file writer {} was closed.", + consensusGroupId, + diskBuffer.getWritingFile() == null ? "null" : diskBuffer.getWritingFile().getPath()); + } catch (IOException e) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to close current writing file writer {}, because {}.", + consensusGroupId, + diskBuffer.getWritingFile() == null ? "null" : diskBuffer.getWritingFile().getPath(), + e.getMessage(), + e); + } + diskBuffer.setWritingFileWriter(null); + } else { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-ConsensusGroupId-{}: Current writing file writer is null. No need to close.", + consensusGroupId.getId()); + } + } + } + + private void deleteFile(File file) { + if (file.exists()) { + try { + FileUtils.delete(file); + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: Original writing file {} was deleted.", + consensusGroupId, + file.getPath()); + } catch (IOException e) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to delete original writing file {}, because {}.", + consensusGroupId, + file.getPath(), + e.getMessage(), + e); + } + } else { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-ConsensusGroupId-{}: Original file {} is not existed. No need to delete.", + consensusGroupId, + file.getPath()); + } + } + } + + private void deleteCurrentWritingFile(PipeConsensusTsFileWriter diskBuffer) { + if (diskBuffer.getWritingFile() != null) { + deleteFile(diskBuffer.getWritingFile()); + } else { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-ConsensusGroupId-{}: Current writing file is null. No need to delete.", + consensusGroupId.getId()); + } + } + } + + private void updateWritingFileIfNeeded( + final PipeConsensusTsFileWriter diskBuffer, final String fileName, final boolean isSingleFile) + throws IOException { + if (isFileExistedAndNameCorrect(diskBuffer, fileName)) { + return; + } + + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: Writing file {} is not existed or name is not correct, try to create it. " + + "Current writing file is {}.", + consensusGroupId, + fileName, + diskBuffer.getWritingFile() == null ? "null" : diskBuffer.getWritingFile().getPath()); + + closeCurrentWritingFileWriter(diskBuffer); + // If there are multiple files we can not delete the current file + // instead they will be deleted after seal request + if (diskBuffer.getWritingFile() != null && isSingleFile) { + deleteCurrentWritingFile(diskBuffer); + } + + // Make sure receiver file dir exists + // This may be useless, because receiver file dir is created when receiver is initiated. just in + // case. + if (!receiverFileDirWithIdSuffix.get().exists()) { + if (receiverFileDirWithIdSuffix.get().mkdirs()) { + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: Receiver file dir {} was created.", + consensusGroupId, + receiverFileDirWithIdSuffix.get().getPath()); + } else { + LOGGER.error( + "PipeConsensus-ConsensusGroupId-{}: Failed to create receiver file dir {}.", + consensusGroupId, + receiverFileDirWithIdSuffix.get().getPath()); + } + } + + diskBuffer.setWritingFile(new File(receiverFileDirWithIdSuffix.get(), fileName)); + diskBuffer.setWritingFileWriter(new RandomAccessFile(diskBuffer.getWritingFile(), "rw")); + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: Writing file {} was created. Ready to write file pieces.", + consensusGroupId, + diskBuffer.getWritingFile().getPath()); + } + + private String getReceiverFileBaseDir() throws DiskSpaceInsufficientException { + // Get next receiver file base dir by folder manager + return Objects.isNull(folderManager) ? null : folderManager.getNextFolder(); + } + + private void initiateTsFileBufferFolder() throws DiskSpaceInsufficientException, IOException { + // Clear the original receiver file dir if exists + if (receiverFileDirWithIdSuffix.get() != null) { + if (receiverFileDirWithIdSuffix.get().exists()) { + try { + FileUtils.deleteDirectory(receiverFileDirWithIdSuffix.get()); + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: Original receiver file dir {} was deleted successfully.", + consensusGroupId, + receiverFileDirWithIdSuffix.get().getPath()); + } catch (IOException e) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to delete original receiver file dir {}, because {}.", + consensusGroupId, + receiverFileDirWithIdSuffix.get().getPath(), + e.getMessage(), + e); + } + } else { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-ConsensusGroupId-{}: Original receiver file dir {} is not existed. No need to delete.", + consensusGroupId, + receiverFileDirWithIdSuffix.get().getPath()); + } + } + receiverFileDirWithIdSuffix.set(null); + } else { + LOGGER.debug( + "PipeConsensus-ConsensusGroupId-{}: Current receiver file dir is null. No need to delete.", + consensusGroupId.getId()); + } + + // initiate receiverFileDirWithIdSuffix + try { + final String receiverFileBaseDir = getReceiverFileBaseDir(); + if (Objects.isNull(receiverFileBaseDir)) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to init pipeConsensus receiver file folder manager because all disks of folders are full.", + consensusGroupId.getId()); + throw new DiskSpaceInsufficientException(receiverBaseDirsName); + } + // Create a new receiver file dir + final File newReceiverDir = new File(receiverFileBaseDir, consensusGroupId.toString()); + if (!newReceiverDir.exists() && !newReceiverDir.mkdirs()) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Failed to create receiver file dir {}.", + newReceiverDir.getPath(), + consensusGroupId.getId()); + throw new IOException( + String.format( + "PipeConsensus-ConsensusGroupId-%s: Failed to create receiver file dir %s.", + newReceiverDir.getPath(), consensusGroupId.getId())); + } + receiverFileDirWithIdSuffix.set(newReceiverDir); + + } catch (Exception e) { + LOGGER.warn( + "Failed to init pipeConsensus receiver file folder manager because all disks of folders are full.", + e); + throw e; + } + } + + public PipeConsensusRequestVersion getVersion() { + return PipeConsensusRequestVersion.VERSION_1; + } + + public synchronized void handleExit() { + // Clear the diskBuffers + pipeConsensusTsFileWriterPool.handleExit(consensusGroupId); + + // Clear the original receiver file dir if exists + if (receiverFileDirWithIdSuffix.get() != null) { + if (receiverFileDirWithIdSuffix.get().exists()) { + try { + FileUtils.deleteDirectory(receiverFileDirWithIdSuffix.get()); + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: Receiver exit: Original receiver file dir {} was deleted.", + consensusGroupId, + receiverFileDirWithIdSuffix.get().getPath()); + } catch (IOException e) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: Receiver exit: Delete original receiver file dir {} error.", + consensusGroupId, + receiverFileDirWithIdSuffix.get().getPath(), + e); + } + } else { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-ConsensusGroupId-{}: Receiver exit: Original receiver file dir {} does not exist. No need to delete.", + consensusGroupId, + receiverFileDirWithIdSuffix.get().getPath()); + } + } + receiverFileDirWithIdSuffix.set(null); + } else { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-ConsensusGroupId-{}: Receiver exit: Original receiver file dir is null. No need to delete.", + consensusGroupId.getId()); + } + } + + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: Receiver exit: Receiver exited.", + consensusGroupId.getId()); + } + + private static class PipeConsensusTsFileWriterPool { + private final Lock lock = new ReentrantLock(); + private final List<PipeConsensusTsFileWriter> pipeConsensusTsFileWriterPool = new ArrayList<>(); + + public PipeConsensusTsFileWriterPool() { + for (int i = 0; i < IOTDB_CONFIG.getPipeConsensusPipelineSize(); i++) { + pipeConsensusTsFileWriterPool.add(new PipeConsensusTsFileWriter(i)); + } + } + + public PipeConsensusTsFileWriter borrowCorrespondingWriter(TCommitId commitId) { + Optional<PipeConsensusTsFileWriter> diskBuffer = + pipeConsensusTsFileWriterPool.stream() + .filter( + item -> Objects.equals(commitId, item.getCommitIdOfCorrespondingHolderEvent())) + .findFirst(); + + // If the TsFileInsertionEvent is first using diskBuffer, we will find the first available + // buffer for it. + if (!diskBuffer.isPresent()) { + // We should synchronously find the idle writer to avoid concurrency issues. + try { + lock.lock(); + diskBuffer = + pipeConsensusTsFileWriterPool.stream().filter(item -> !item.isUsed()).findFirst(); + // We don't need to check diskBuffer.isPresent() here. Since diskBuffers' length is equals + // to ReqExecutor's buffer, so the diskBuffer is always present. + diskBuffer.get().setUsed(true); + diskBuffer.get().setCommitIdOfCorrespondingHolderEvent(commitId); + } finally { + lock.unlock(); + } + } + + return diskBuffer.get(); + } + + public void handleExit(ConsensusGroupId consensusGroupId) { + pipeConsensusTsFileWriterPool.forEach( + diskBuffer -> { + // Wait until diskBuffer is not used by TsFileInsertionEvent or timeout. + long currentTime = System.currentTimeMillis(); + while (System.currentTimeMillis() - currentTime + < CLOSE_TSFILE_WRITER_MAX_WAIT_TIME_IN_MS + && diskBuffer.isUsed()) { + try { + Thread.sleep(RETRY_WAIT_TIME); + } catch (InterruptedException e) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: receiver thread get interrupted when exiting.", + consensusGroupId.getId()); + // avoid infinite loop + break; + } + } + diskBuffer.closeSelf(consensusGroupId); + }); + } + } + + private static class PipeConsensusTsFileWriter { + private final int index; + // whether this buffer is used. this will be updated when first transfer tsFile piece or + // when transfer seal. + private boolean isUsed = false; + // If isUsed is true, this variable will be set to the TCommitId of holderEvent + private TCommitId commitIdOfCorrespondingHolderEvent; + private File writingFile; + private RandomAccessFile writingFileWriter; + + public PipeConsensusTsFileWriter(int index) { + this.index = index; + } + + public File getWritingFile() { + return writingFile; + } + + public void setWritingFile(File writingFile) { + this.writingFile = writingFile; + } + + public RandomAccessFile getWritingFileWriter() { + return writingFileWriter; + } + + public void setWritingFileWriter(RandomAccessFile writingFileWriter) { + this.writingFileWriter = writingFileWriter; + } + + public TCommitId getCommitIdOfCorrespondingHolderEvent() { + return commitIdOfCorrespondingHolderEvent; + } + + public void setCommitIdOfCorrespondingHolderEvent( + TCommitId commitIdOfCorrespondingHolderEvent) { + this.commitIdOfCorrespondingHolderEvent = commitIdOfCorrespondingHolderEvent; + } + + public boolean isUsed() { + return isUsed; + } + + public void setUsed(boolean used) { + isUsed = used; + } + + public void returnSelf() { + this.isUsed = false; + this.commitIdOfCorrespondingHolderEvent = null; + } + + public void closeSelf(ConsensusGroupId consensusGroupId) { + // close file writer + if (writingFileWriter != null) { + try { + writingFileWriter.close(); + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: TsFileWriter-{} exit: Writing file writer was closed.", + consensusGroupId.getId(), + index); + } catch (Exception e) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: TsFileWriter-{} exit: Close Writing file writer error.", + consensusGroupId, + index, + e); + } + setWritingFileWriter(null); + } else { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-ConsensusGroupId-{}: TsFileWriter-{} exit: Writing file writer is null. No need to close.", + consensusGroupId.getId(), + index); + } + } + + // close file + if (writingFile != null) { + try { + FileUtils.delete(writingFile); + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: TsFileWriter exit: Writing file {} was deleted.", + consensusGroupId, + writingFile.getPath()); + } catch (Exception e) { + LOGGER.warn( + "PipeConsensus-ConsensusGroupId-{}: TsFileWriter exit: Delete writing file {} error.", + consensusGroupId, + writingFile.getPath(), + e); + } + setWritingFile(null); + } else { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-ConsensusGroupId-{}: TsFileWriter exit: Writing file is null. No need to delete.", + consensusGroupId.getId()); + } + } + } + } + + /** + * An executor component to ensure all events sent from connector can be loaded in sequence, + * although events can arrive receiver in a random sequence. + */ + private class RequestExecutor { + // An ordered set that buffers transfer requests' TCommitId, whose length is not larger than + // PIPE_CONSENSUS_PIPELINE_SIZE. + // Here we use set is to avoid duplicate events being received in some special cases + private final TreeSet<TCommitId> reqExecutionOrderBuffer; + private final Lock lock; + private final Condition condition; + private long onSyncedCommitIndex = -1; + private int connectorRebootTimes = 0; + + public RequestExecutor() { + reqExecutionOrderBuffer = + new TreeSet<>( + Comparator.comparingInt(TCommitId::getRebootTimes) + .thenComparingLong(TCommitId::getCommitIndex)); + lock = new ReentrantLock(); + condition = lock.newCondition(); + } + + private void onSuccess(long nextSyncedCommitIndex) { + LOGGER.info( + "PipeConsensus-ConsensusGroupId-{}: process no.{} event successfully!", + consensusGroupId, + nextSyncedCommitIndex); + reqExecutionOrderBuffer.pollFirst(); + onSyncedCommitIndex = nextSyncedCommitIndex; + } + + private TPipeConsensusTransferResp onRequest( + final TPipeConsensusTransferReq req, final boolean isTransferTsFilePiece) { + lock.lock(); + try { + TCommitId tCommitId = req.getCommitId(); + LOGGER.info( + "PipeConsensus-ConsensusGroup-{}: start to receive no.{} event", + consensusGroupId, + tCommitId.getCommitIndex()); + // if a req is deprecated, we will discard it + // This case may happen in this scenario: leader has transferred {1,2} and is intending to + // transfer {3, 4, 5, 6}. And in one moment, follower has received {4, 5, 6}, {3} is still + // transferring due to some network latency. + // At this time, leader restarts, and it will resend {3, 4, 5, 6} with incremental + // rebootTimes. If the {3} sent before the leader restart arrives after the follower + // receives + // the request with incremental rebootTimes, the {3} sent before the leader restart needs to + // be discarded. + if (tCommitId.getRebootTimes() < connectorRebootTimes) { + final TSStatus status = + new TSStatus( + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_DEPRECATED_REQUEST, + "PipeConsensus receiver received a deprecated request, which may be sent before the connector restart. Consider to discard it")); + LOGGER.info( + "PipeConsensus-ConsensusGroup-{}: received a deprecated request, which may be sent before the connector restart. Consider to discard it", + consensusGroupId); + return new TPipeConsensusTransferResp(status); + } + // Judge whether connector has rebooted or not, if the rebootTimes increases compared to + // connectorRebootTimes, need to reset receiver because connector has been restarted. + if (tCommitId.getRebootTimes() > connectorRebootTimes) { + resetWithNewestRebootTime(tCommitId.getRebootTimes()); + } + reqExecutionOrderBuffer.add(tCommitId); + // TsFilePieceTransferEvent will not enter further procedure, it just holds a place in + // buffer. Only after the corresponding sealing event is processed, this event can be + // dequeued. + if (isTransferTsFilePiece) { + return null; + } + + if (reqExecutionOrderBuffer.size() >= IOTDB_CONFIG.getPipeConsensusPipelineSize() + && !reqExecutionOrderBuffer.first().equals(tCommitId)) { + // If reqBuffer is full and current thread do not hold the reqBuffer's peek, this req + // is not supposed to be processed. So current thread should notify the corresponding + // threads to process the peek. + condition.signalAll(); + } + + // Polling to process + while (true) { + if (reqExecutionOrderBuffer.first().equals(tCommitId) + && tCommitId.getCommitIndex() == onSyncedCommitIndex + 1) { + // If current req is supposed to be process, load this event through + // DataRegionStateMachine. + TPipeConsensusTransferResp resp = loadEvent(req); + + // Only when event apply is successful and what is transmitted is not TsFilePiece, req + // will be removed from the buffer and onSyncedCommitIndex will be updated. Because pipe + // will transfer multi reqs with same commitId in a single TsFileInsertionEvent, only + // when the last seal req is applied, we can discard this event. + if (resp != null + && resp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + onSuccess(onSyncedCommitIndex + 1); + } + return resp; + } + + if (reqExecutionOrderBuffer.size() >= IOTDB_CONFIG.getPipeConsensusPipelineSize() + && reqExecutionOrderBuffer.first().equals(tCommitId)) { + // If the reqBuffer is full and its peek is hold by current thread, load this event. + TPipeConsensusTransferResp resp = loadEvent(req); + + if (resp != null + && resp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + onSuccess(tCommitId.getCommitIndex()); + // signal all other reqs that may wait for this event + condition.signalAll(); + } + return resp; + } else { + // if the req is not supposed to be processed and reqBuffer is not full, current thread + // should wait until reqBuffer is full, which indicates the receiver has received all + // the requests from the connector without duplication or leakage. + try { + boolean timeout = + !condition.await( + PIPE_CONSENSUS_RECEIVER_MAX_WAITING_TIME_IN_MS, TimeUnit.MILLISECONDS); + + // If the buffer is not full after waiting timeout, we suppose that the sender will + // not send any more events at this time, that is, the sender has sent all events. At + // this point we apply the event at reqBuffer's peek + if (timeout + && reqExecutionOrderBuffer.size() < IOTDB_CONFIG.getPipeConsensusPipelineSize() + && reqExecutionOrderBuffer.first() != null + && reqExecutionOrderBuffer.first().equals(tCommitId)) { + TPipeConsensusTransferResp resp = loadEvent(req); + + if (resp != null + && resp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + onSuccess(tCommitId.getCommitIndex()); + // signal all other reqs that may wait for this event + condition.signalAll(); + } + return resp; + } + } catch (InterruptedException e) { + LOGGER.warn( + "PipeConsensus-ConsensusGroup-{}: current waiting is interrupted. onSyncedCommitIndex: {}. Exception: ", + consensusGroupId, + tCommitId.getCommitIndex(), + e); + Thread.currentThread().interrupt(); + // Avoid infinite loop when RPC thread is killed by OS + return new TPipeConsensusTransferResp( + RpcUtils.getStatus( + TSStatusCode.SHUT_DOWN_ERROR, + "RPC processor is interrupted by shutdown hook when wait on condition!")); + } + } + } + } finally { + lock.unlock(); + } + } + + /** + * Reset all data to initial status and set connectorRebootTimes properly. This method is called + * when receiver identifies connector has rebooted. + */ + private void resetWithNewestRebootTime(int connectorRebootTimes) { + LOGGER.info( + "PipeConsensus-ConsensusGroup-{}: receiver detected an newer rebootTimes, which indicates the leader has rebooted. receiver will reset all its data.", + consensusGroupId); + this.reqExecutionOrderBuffer.clear(); + this.onSyncedCommitIndex = -1; + // sync the follower's connectorRebootTimes with connector's actual rebootTimes + this.connectorRebootTimes = connectorRebootTimes; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiverAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiverAgent.java new file mode 100644 index 000000000000..d4c7f37e697d --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiverAgent.java @@ -0,0 +1,169 @@ +/* + * 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.pipe.receiver.protocol.pipeconsensus; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.consensus.ConsensusGroupId; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestVersion; +import org.apache.iotdb.consensus.IConsensus; +import org.apache.iotdb.consensus.pipe.PipeConsensus; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeName; +import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeReceiver; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.commons.lang3.function.TriFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; + +public class PipeConsensusReceiverAgent implements ConsensusPipeReceiver { + + private static final Logger LOGGER = LoggerFactory.getLogger(PipeConsensusReceiverAgent.class); + + private static final Map< + Byte, + TriFunction<PipeConsensus, ConsensusGroupId, ConsensusPipeName, PipeConsensusReceiver>> + RECEIVER_CONSTRUCTORS = new HashMap<>(); + + private final int thisNodeId = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + + /** + * For each consensus Pipe task, there is an independent receiver. So for every replica, it has + * (n-1) receivers, n is the num of replicas. 1 DataNode --has--> 1 PipeConsensusReceiverAgent & n + * replicas 1 PipeConsensusReceiverAgent --manages--> n replicas' receivers 1 replica --has--> + * (n-1) receivers + */ + private final Map< + ConsensusGroupId, Map<ConsensusPipeName, AtomicReference<PipeConsensusReceiver>>> + replicaReceiverMap = new ConcurrentHashMap<>(); + + private PipeConsensus pipeConsensus; + + public PipeConsensusReceiverAgent() { + RECEIVER_CONSTRUCTORS.put( + PipeConsensusRequestVersion.VERSION_1.getVersion(), PipeConsensusReceiver::new); + } + + public void initConsensusInRuntime() { + IConsensus consensus = DataRegionConsensusImpl.getInstance(); + // If DataRegion uses PipeConsensus + if (consensus instanceof PipeConsensus) { + this.pipeConsensus = (PipeConsensus) consensus; + } + // If DataRegion uses other consensus such as IoTConsensus + else { + this.pipeConsensus = null; + } + } + + @Override + public TPipeConsensusTransferResp receive(TPipeConsensusTransferReq req) { + final byte reqVersion = req.getVersion(); + if (RECEIVER_CONSTRUCTORS.containsKey(reqVersion)) { + final ConsensusGroupId consensusGroupId = + ConsensusGroupId.Factory.createFromTConsensusGroupId(req.getConsensusGroupId()); + return getReceiver(consensusGroupId, req.getDataNodeId(), reqVersion).receive(req); + } else { + final TSStatus status = + RpcUtils.getStatus( + TSStatusCode.PIPE_CONSENSUS_VERSION_ERROR, + String.format("Unknown PipeConsensusRequestVersion %s.", reqVersion)); + LOGGER.warn( + "PipeConsensus: Unknown PipeConsensusRequestVersion, response status = {}.", status); + return new TPipeConsensusTransferResp(status); + } + } + + private PipeConsensusReceiver getReceiver( + ConsensusGroupId consensusGroupId, int leaderDataNodeId, byte reqVersion) { + // 1. Route to given consensusGroup's receiver map + Map<ConsensusPipeName, AtomicReference<PipeConsensusReceiver>> consensusPipe2ReceiverMap = + replicaReceiverMap.computeIfAbsent(consensusGroupId, key -> new ConcurrentHashMap<>()); + // 2. Route to given consensusPipeTask's receiver + ConsensusPipeName consensusPipeName = + new ConsensusPipeName(consensusGroupId, leaderDataNodeId, thisNodeId); + AtomicReference<PipeConsensusReceiver> receiverReference = + consensusPipe2ReceiverMap.computeIfAbsent( + consensusPipeName, key -> new AtomicReference<>(null)); + + if (receiverReference.get() == null) { + return internalSetAndGetReceiver(consensusGroupId, consensusPipeName, reqVersion); + } + + final byte receiverThreadLocalVersion = receiverReference.get().getVersion().getVersion(); + if (receiverThreadLocalVersion != reqVersion) { + LOGGER.warn( + "The pipeConsensus request version {} is different from the sender request version {}," + + " the receiver will be reset to the sender request version.", + receiverThreadLocalVersion, + reqVersion); + receiverReference.set(null); + return internalSetAndGetReceiver(consensusGroupId, consensusPipeName, reqVersion); + } + return receiverReference.get(); + } + + private PipeConsensusReceiver internalSetAndGetReceiver( + ConsensusGroupId consensusGroupId, ConsensusPipeName consensusPipeName, byte reqVersion) { + // 1. Route to given consensusGroup's receiver map + Map<ConsensusPipeName, AtomicReference<PipeConsensusReceiver>> consensusPipe2ReciverMap = + replicaReceiverMap.get(consensusGroupId); + // 2. Route to given consensusPipeTask's receiver + AtomicReference<PipeConsensusReceiver> receiverReference = + consensusPipe2ReciverMap.get(consensusPipeName); + + if (RECEIVER_CONSTRUCTORS.containsKey(reqVersion)) { + receiverReference.set( + RECEIVER_CONSTRUCTORS + .get(reqVersion) + .apply(pipeConsensus, consensusGroupId, consensusPipeName)); + } else { + throw new UnsupportedOperationException( + String.format("Unsupported pipeConsensus request version %d", reqVersion)); + } + return receiverReference.get(); + } + + /** Release receiver of given pipeConsensusTask */ + @Override + public final void handleDropPipeConsensusTask(ConsensusPipeName pipeName) { + // 1. Route to given consensusGroup's receiver map + Map<ConsensusPipeName, AtomicReference<PipeConsensusReceiver>> consensusPipe2ReciverMap = + replicaReceiverMap.getOrDefault(pipeName.getConsensusGroupId(), new ConcurrentHashMap<>()); + // 2. Route to given consensusPipeTask's receiver + AtomicReference<PipeConsensusReceiver> receiverReference = + consensusPipe2ReciverMap.getOrDefault(pipeName, null); + // 3. Release receiver + if (receiverReference != null) { + receiverReference.get().handleExit(); + receiverReference.set(null); + } + consensusPipe2ReciverMap.remove(pipeName); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/builder/PipeDataNodeTaskBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/builder/PipeDataNodeTaskBuilder.java index 7871038a1238..e84f93e34ce3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/builder/PipeDataNodeTaskBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/builder/PipeDataNodeTaskBuilder.java @@ -53,6 +53,8 @@ public class PipeDataNodeTaskBuilder { PipeType.USER, PipeSubtaskExecutorManager.getInstance().getConnectorExecutor()); CONNECTOR_EXECUTOR_MAP.put( PipeType.SUBSCRIPTION, PipeSubtaskExecutorManager.getInstance().getSubscriptionExecutor()); + CONNECTOR_EXECUTOR_MAP.put( + PipeType.CONSENSUS, PipeSubtaskExecutorManager.getInstance().getConnectorExecutor()); } protected final Map<String, String> systemParameters = new HashMap<>(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java index a3f2227e126b..1df20db17f63 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; import org.apache.iotdb.commons.pipe.progress.PipeEventCommitManager; import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; +import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; @@ -149,6 +150,9 @@ private void collectEvent(final Event event) { // Assign a commit id for this event in order to report progress in order. PipeEventCommitManager.getInstance() .enrichWithCommitterKeyAndCommitId((EnrichedEvent) event, creationTime, regionId); + + // Assign a rebootTime for pipeConsensus + ((EnrichedEvent) event).setRebootTimes(PipeAgent.runtime().getRebootTimes()); } if (event instanceof PipeHeartbeatEvent) { diff --git a/iotdb-core/node-commons/pom.xml b/iotdb-core/node-commons/pom.xml index c19924c2870d..37371509d0cf 100644 --- a/iotdb-core/node-commons/pom.xml +++ b/iotdb-core/node-commons/pom.xml @@ -75,6 +75,11 @@ <artifactId>iotdb-thrift-commons</artifactId> <version>1.3.2-SNAPSHOT</version> </dependency> + <dependency> + <groupId>org.apache.iotdb</groupId> + <artifactId>iotdb-thrift-consensus</artifactId> + <version>1.3.2-SNAPSHOT</version> + </dependency> <dependency> <groupId>org.apache.tsfile</groupId> <artifactId>tsfile</artifactId> diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties index 62d384990c9e..b92142ebe631 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties @@ -425,6 +425,19 @@ dn_seed_config_node=127.0.0.1:10710 # effectiveMode: restart # pipe_receiver_file_dirs=data/datanode/system/pipe/receiver +# pipe_consensus_receiver_file_dirs +# If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/${dn_system_dir}/pipe/consensus/receiver). +# If it is absolute, system will save the data in the exact location it points to. +# If it is relative, system will save the data in the relative path directory it indicates under the IoTDB folder. +# If there are more than one directory, please separate them by commas ",". +# Note: If pipe_consensus_receiver_file_dirs is assigned an empty string(i.e.,zero-size), it will be handled as a relative path. +# For windows platform +# If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative. +# pipe_consensus_receiver_file_dirs=data\\datanode\\system\\pipe\\consensus\\receiver +# For Linux platform +# If its prefix is "/", then the path is absolute. Otherwise, it is relative. +# pipe_consensus_receiver_file_dirs=data/datanode/system/pipe/consensus/receiver + #################### ### Metric Configuration #################### @@ -565,6 +578,7 @@ data_replication_factor=1 # 1. org.apache.iotdb.consensus.simple.SimpleConsensus (The data_replication_factor can only be set to 1) # 2. org.apache.iotdb.consensus.iot.IoTConsensus # 3. org.apache.iotdb.consensus.ratis.RatisConsensus +# 4. org.apache.iotdb.consensus.iot.FastIoTConsensus # effectiveMode: first_start # Datatype: string # data_region_consensus_protocol_class=org.apache.iotdb.consensus.iot.IoTConsensus @@ -1752,6 +1766,13 @@ data_replication_factor=1 # schema_region_ratis_periodic_snapshot_interval=86400 # data_region_ratis_periodic_snapshot_interval=86400 +#################### +### Fast IoTConsensus Configuration +#################### +# Default event buffer size for connector and receiver in pipe consensus +# DataType: int +# fast_iot_consensus_pipeline_size=5 + #################### ### Procedure Configuration #################### diff --git a/iotdb-core/node-commons/src/assembly/resources/sbin/destroy-datanode.bat b/iotdb-core/node-commons/src/assembly/resources/sbin/destroy-datanode.bat index 26aab580e851..46b6da01d97a 100644 --- a/iotdb-core/node-commons/src/assembly/resources/sbin/destroy-datanode.bat +++ b/iotdb-core/node-commons/src/assembly/resources/sbin/destroy-datanode.bat @@ -45,7 +45,7 @@ for /f "eol=# tokens=2 delims==" %%i in ('findstr /i "^dn_system_dir" set dn_system_dir=%%i ) if "%dn_system_dir%"=="" ( - set "dn_system_dir=data\confignode\system" + set "dn_system_dir=data\\datanode\\system" ) setlocal enabledelayedexpansion set "dn_system_dir=!dn_system_dir:%delimiter%= !" @@ -162,16 +162,36 @@ for %%i in (%dn_sync_dir%) do ( ) ) -for /f "eol=# tokens=2 delims==" %%i in ('findstr /i "^pipe_receiver_file_dir" +for /f "eol=# tokens=2 delims==" %%i in ('findstr /i "^pipe_receiver_file_dirs" %IOTDB_DATANODE_CONFIG%') do ( - set pipe_receiver_file_dir=%%i + set pipe_receiver_file_dirs=%%i ) -if "%pipe_receiver_file_dir%"=="" ( - set "pipe_receiver_file_dir=data\\datanode\\system\\pipe\\receiver" +if "%pipe_receiver_file_dirs%"=="" ( + set "pipe_receiver_file_dirs=data\\datanode\\system\\pipe\\receiver" ) -set "pipe_receiver_file_dir=!pipe_receiver_file_dir:%delimiter%= !" -for %%i in (%pipe_receiver_file_dir%) do ( +set "pipe_receiver_file_dirs=!pipe_receiver_file_dirs:%delimiter%= !" +for %%i in (%pipe_receiver_file_dirs%) do ( + set "var=%%i" + if "!var:~0,2!"=="\\" ( + rmdir /s /q "%%i" 2>nul + ) else if "!var:~1,3!"==":\\" ( + rmdir /s /q "%%i" 2>nul + ) else ( + rmdir /s /q "%IOTDB_HOME%\%%i" 2>nul + ) +) + +for /f "eol=# tokens=2 delims==" %%i in ('findstr /i "^pipe_consensus_receiver_file_dirs" + %IOTDB_DATANODE_CONFIG%') do ( + set pipe_consensus_receiver_file_dirs=%%i +) +if "%pipe_consensus_receiver_file_dirs%"=="" ( + set "pipe_consensus_receiver_file_dirs=data\\datanode\\system\\pipe\\consensus\\receiver" +) + +set "pipe_consensus_receiver_file_dirs=!pipe_consensus_receiver_file_dirs:%delimiter%= !" +for %%i in (%pipe_consensus_receiver_file_dirs%) do ( set "var=%%i" if "!var:~0,2!"=="\\" ( rmdir /s /q "%%i" 2>nul diff --git a/iotdb-core/node-commons/src/assembly/resources/sbin/destroy-datanode.sh b/iotdb-core/node-commons/src/assembly/resources/sbin/destroy-datanode.sh index 9a1580a2a9e6..39c0a1e612b5 100644 --- a/iotdb-core/node-commons/src/assembly/resources/sbin/destroy-datanode.sh +++ b/iotdb-core/node-commons/src/assembly/resources/sbin/destroy-datanode.sh @@ -46,6 +46,7 @@ dn_wal_dirs=$(echo $(grep '^dn_wal_dirs=' ${IOTDB_DATANODE_CONFIG} || echo "data dn_tracing_dir=$(echo $(grep '^dn_tracing_dir=' ${IOTDB_DATANODE_CONFIG} || echo "datanode/tracing") | sed 's/.*=//') dn_sync_dir=$(echo $(grep '^dn_sync_dir=' ${IOTDB_DATANODE_CONFIG} || echo "data/datanode/sync") | sed 's/.*=//') pipe_receiver_file_dirs=$(echo $(grep '^pipe_receiver_file_dirs=' ${IOTDB_DATANODE_CONFIG} || echo "data/datanode/system/pipe/receiver") | sed 's/.*=//') +pipe_consensus_receiver_file_dirs=$(echo $(grep '^pipe_consensus_receiver_file_dirs=' ${IOTDB_DATANODE_CONFIG} || echo "data/datanode/system/pipe/consensus/receiver") | sed 's/.*=//') sort_tmp_dir=$(echo $(grep '^sort_tmp_dir=' ${IOTDB_DATANODE_CONFIG} || echo "data/datanode/tmp") | sed 's/.*=//') function clearPath { @@ -70,6 +71,7 @@ clearPath $dn_wal_dirs clearPath $dn_tracing_dir clearPath $dn_sync_dir clearPath $pipe_receiver_file_dirs +clearPath $pipe_consensus_receiver_file_dirs clearPath $sort_tmp_dir echo "DataNode clean done ..." \ No newline at end of file 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 0277c84cb9ab..a157a9eaf63a 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 @@ -23,12 +23,16 @@ import org.apache.iotdb.commons.client.async.AsyncConfigNodeIServiceClient; import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient; import org.apache.iotdb.commons.client.async.AsyncDataNodeMPPDataExchangeServiceClient; +import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; import org.apache.iotdb.commons.client.property.ClientPoolProperty; +import org.apache.iotdb.commons.client.property.PipeConsensusClientProperty; import org.apache.iotdb.commons.client.property.ThriftClientProperty; +import org.apache.iotdb.commons.client.property.ThriftClientProperty.DefaultProperty; import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient; import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient; import org.apache.iotdb.commons.client.sync.SyncDataNodeMPPDataExchangeServiceClient; +import org.apache.iotdb.commons.client.sync.SyncPipeConsensusServiceClient; import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.conf.CommonConfig; import org.apache.iotdb.commons.conf.CommonDescriptor; @@ -259,4 +263,72 @@ public KeyedObjectPool<TEndPoint, AsyncPipeDataTransferServiceClient> createClie return clientPool; } } + + public static class SyncPipeConsensusServiceClientPoolFactory + implements IClientPoolFactory<TEndPoint, SyncPipeConsensusServiceClient> { + + private final PipeConsensusClientProperty config; + + public SyncPipeConsensusServiceClientPoolFactory(PipeConsensusClientProperty config) { + this.config = config; + } + + @Override + public KeyedObjectPool<TEndPoint, SyncPipeConsensusServiceClient> createClientPool( + ClientManager<TEndPoint, SyncPipeConsensusServiceClient> manager) { + GenericKeyedObjectPool<TEndPoint, SyncPipeConsensusServiceClient> clientPool = + new GenericKeyedObjectPool<>( + new SyncPipeConsensusServiceClient.Factory( + manager, + new ThriftClientProperty.Builder() + // TODO: consider timeout and evict strategy. + .setConnectionTimeoutMs(DefaultProperty.CONNECTION_NEVER_TIMEOUT_MS) + .setRpcThriftCompressionEnabled(config.isRpcThriftCompressionEnabled()) + .setPrintLogWhenEncounterException( + config.isPrintLogWhenThriftClientEncounterException()) + .build()), + new ClientPoolProperty.Builder<SyncPipeConsensusServiceClient>() + .setMaxClientNumForEachNode(config.getMaxClientNumForEachNode()) + .build() + .getConfig()); + ClientManagerMetrics.getInstance() + .registerClientManager(this.getClass().getSimpleName(), clientPool); + return clientPool; + } + } + + public static class AsyncPipeConsensusServiceClientPoolFactory + implements IClientPoolFactory<TEndPoint, AsyncPipeConsensusServiceClient> { + + private final PipeConsensusClientProperty config; + + public AsyncPipeConsensusServiceClientPoolFactory(PipeConsensusClientProperty config) { + this.config = config; + } + + @Override + public KeyedObjectPool<TEndPoint, AsyncPipeConsensusServiceClient> createClientPool( + ClientManager<TEndPoint, AsyncPipeConsensusServiceClient> manager) { + GenericKeyedObjectPool<TEndPoint, AsyncPipeConsensusServiceClient> clientPool = + new GenericKeyedObjectPool<>( + new AsyncPipeConsensusServiceClient.Factory( + manager, + new ThriftClientProperty.Builder() + // TODO: consider timeout and evict strategy. + .setConnectionTimeoutMs(DefaultProperty.CONNECTION_NEVER_TIMEOUT_MS) + .setRpcThriftCompressionEnabled(config.isRpcThriftCompressionEnabled()) + .setSelectorNumOfAsyncClientManager(config.getSelectorNumOfClientManager()) + .setPrintLogWhenEncounterException( + config.isPrintLogWhenThriftClientEncounterException()) + .build(), + ThreadName.ASYNC_DATANODE_PIPE_CONSENSUS_CLIENT_POOL.getName()), + new ClientPoolProperty.Builder<AsyncPipeConsensusServiceClient>() + .setMaxClientNumForEachNode(config.getMaxClientNumForEachNode()) + .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/AsyncPipeConsensusServiceClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeConsensusServiceClient.java new file mode 100644 index 000000000000..6f05350c0891 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeConsensusServiceClient.java @@ -0,0 +1,171 @@ +/* + * 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.consensus.pipe.thrift.PipeConsensusIService; +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; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +public class AsyncPipeConsensusServiceClient extends PipeConsensusIService.AsyncClient + implements ThriftClient { + + private static final Logger LOGGER = + LoggerFactory.getLogger(AsyncPipeConsensusServiceClient.class); + + private static final AtomicInteger idGenerator = new AtomicInteger(0); + private final int id = idGenerator.incrementAndGet(); + private final boolean printLogWhenEncounterException; + private final TEndPoint endpoint; + private final ClientManager<TEndPoint, AsyncPipeConsensusServiceClient> clientManager; + private final AtomicBoolean shouldReturnSelf = new AtomicBoolean(true); + + public AsyncPipeConsensusServiceClient( + ThriftClientProperty property, + TEndPoint endpoint, + TAsyncClientManager tAsyncClientManager, + ClientManager<TEndPoint, AsyncPipeConsensusServiceClient> clientManager) + throws IOException { + super( + property.getProtocolFactory(), + tAsyncClientManager, + TNonblockingSocketWrapper.wrap( + endpoint.getIp(), endpoint.getPort(), property.getConnectionTimeoutMs())); + setTimeout(property.getConnectionTimeoutMs()); + this.printLogWhenEncounterException = property.isPrintLogWhenEncounterException(); + this.endpoint = endpoint; + this.clientManager = 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(String.format("This client %d has been invalidated", id))); + } + } + + @Override + public void invalidateAll() { + clientManager.clear(endpoint); + } + + @Override + public boolean printLogWhenEncounterException() { + return printLogWhenEncounterException; + } + + public void returnSelf() { + if (shouldReturnSelf.get()) { + clientManager.returnClient(endpoint, this); + } + } + + public void setShouldReturnSelf(boolean shouldReturnSelf) { + this.shouldReturnSelf.set(shouldReturnSelf); + } + + private void close() { + ___transport.close(); + ___currentMethod = null; + } + + public boolean isReady() { + try { + checkReady(); + return true; + } catch (Exception e) { + LOGGER.info( + "Unexpected exception occurs in {}, error msg is {}", + this, + ExceptionUtils.getRootCause(e).toString()); + return false; + } + } + + @Override + public String toString() { + return String.format("AsyncPipeDataTransferServiceClient{%s}, id = {%d}", endpoint, id); + } + + public TEndPoint getTEndpoint() { + return endpoint; + } + + public static class Factory + extends AsyncThriftClientFactory<TEndPoint, AsyncPipeConsensusServiceClient> { + + public Factory( + ClientManager<TEndPoint, AsyncPipeConsensusServiceClient> clientManager, + ThriftClientProperty thriftClientProperty, + String threadName) { + super(clientManager, thriftClientProperty, threadName); + } + + @Override + public void destroyObject( + TEndPoint endPoint, PooledObject<AsyncPipeConsensusServiceClient> pooledObject) { + pooledObject.getObject().close(); + } + + @Override + public PooledObject<AsyncPipeConsensusServiceClient> makeObject(TEndPoint endPoint) + throws Exception { + return new DefaultPooledObject<>( + new AsyncPipeConsensusServiceClient( + thriftClientProperty, + endPoint, + tManagers[clientCnt.incrementAndGet() % tManagers.length], + clientManager)); + } + + @Override + public boolean validateObject( + TEndPoint endPoint, PooledObject<AsyncPipeConsensusServiceClient> pooledObject) { + return pooledObject.getObject().isReady(); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/container/PipeConsensusClientMgrContainer.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/container/PipeConsensusClientMgrContainer.java new file mode 100644 index 000000000000..5a52eb344093 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/container/PipeConsensusClientMgrContainer.java @@ -0,0 +1,91 @@ +/* + * 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.container; + +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.client.ClientPoolFactory.AsyncPipeConsensusServiceClientPoolFactory; +import org.apache.iotdb.commons.client.ClientPoolFactory.SyncPipeConsensusServiceClientPoolFactory; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; +import org.apache.iotdb.commons.client.property.PipeConsensusClientProperty; +import org.apache.iotdb.commons.client.sync.SyncPipeConsensusServiceClient; +import org.apache.iotdb.commons.conf.CommonConfig; +import org.apache.iotdb.commons.conf.CommonDescriptor; + +/** + * This class is used to hold the syncClientManager and asyncClientManager used by pipeConsensus. + * The purpose of designing this class is that both the consensus layer and the datanode layer of + * pipeConsensus use clientManager. + * + * <p>Note: we hope to create the corresponding clientManager only when the consensus is + * pipeConsensus to avoid unnecessary overhead. + */ +public class PipeConsensusClientMgrContainer { + private static final CommonConfig CONF = CommonDescriptor.getInstance().getConfig(); + private final IClientManager<TEndPoint, AsyncPipeConsensusServiceClient> asyncClientManager; + private final IClientManager<TEndPoint, SyncPipeConsensusServiceClient> syncClientManager; + + public PipeConsensusClientMgrContainer() { + // load rpc client config + PipeConsensusClientProperty config = + PipeConsensusClientProperty.newBuilder() + .setIsRpcThriftCompressionEnabled(CONF.isRpcThriftCompressionEnabled()) + .setMaxClientNumForEachNode(CONF.getMaxClientNumForEachNode()) + .setSelectorNumOfClientManager(CONF.getSelectorNumOfClientManager()) + .build(); + + this.asyncClientManager = + new IClientManager.Factory<TEndPoint, AsyncPipeConsensusServiceClient>() + .createClientManager(new AsyncPipeConsensusServiceClientPoolFactory(config)); + this.syncClientManager = + new IClientManager.Factory<TEndPoint, SyncPipeConsensusServiceClient>() + .createClientManager(new SyncPipeConsensusServiceClientPoolFactory(config)); + } + + public IClientManager<TEndPoint, AsyncPipeConsensusServiceClient> getAsyncClientManager() { + return asyncClientManager; + } + + public IClientManager<TEndPoint, SyncPipeConsensusServiceClient> getSyncClientManager() { + return syncClientManager; + } + + private static class PipeConsensusClientMgrContainerHolder { + private static PipeConsensusClientMgrContainer INSTANCE; + + private PipeConsensusClientMgrContainerHolder() {} + + public static void build() { + if (INSTANCE == null) { + INSTANCE = new PipeConsensusClientMgrContainer(); + } + } + } + + public static PipeConsensusClientMgrContainer getInstance() { + return PipeConsensusClientMgrContainerHolder.INSTANCE; + } + + // Only when consensus protocol is PipeConsensus, this method will be called once when construct + // consensus class. + public static void build() { + PipeConsensusClientMgrContainerHolder.build(); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/property/PipeConsensusClientProperty.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/property/PipeConsensusClientProperty.java new file mode 100644 index 000000000000..82f6165b710c --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/property/PipeConsensusClientProperty.java @@ -0,0 +1,99 @@ +/* + * 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.property; + +/** This class defines the configurations used by the PipeConsensus Client. */ +public class PipeConsensusClientProperty { + private final boolean isRpcThriftCompressionEnabled; + private final int selectorNumOfClientManager; + private final boolean printLogWhenThriftClientEncounterException; + private final int maxClientNumForEachNode; + + public PipeConsensusClientProperty( + boolean isRpcThriftCompressionEnabled, + int selectorNumOfClientManager, + boolean printLogWhenThriftClientEncounterException, + int maxClientNumForEachNode) { + this.isRpcThriftCompressionEnabled = isRpcThriftCompressionEnabled; + this.selectorNumOfClientManager = selectorNumOfClientManager; + this.printLogWhenThriftClientEncounterException = printLogWhenThriftClientEncounterException; + this.maxClientNumForEachNode = maxClientNumForEachNode; + } + + public boolean isRpcThriftCompressionEnabled() { + return isRpcThriftCompressionEnabled; + } + + public int getSelectorNumOfClientManager() { + return selectorNumOfClientManager; + } + + public boolean isPrintLogWhenThriftClientEncounterException() { + return printLogWhenThriftClientEncounterException; + } + + public int getMaxClientNumForEachNode() { + return maxClientNumForEachNode; + } + + public static PipeConsensusClientProperty.Builder newBuilder() { + return new PipeConsensusClientProperty.Builder(); + } + + public static class Builder { + private boolean isRpcThriftCompressionEnabled = false; + private int selectorNumOfClientManager = 1; + private boolean printLogWhenThriftClientEncounterException = true; + private int maxClientNumForEachNode = + ClientPoolProperty.DefaultProperty.MAX_CLIENT_NUM_FOR_EACH_NODE; + + public PipeConsensusClientProperty.Builder setIsRpcThriftCompressionEnabled( + boolean isRpcThriftCompressionEnabled) { + this.isRpcThriftCompressionEnabled = isRpcThriftCompressionEnabled; + return this; + } + + public PipeConsensusClientProperty.Builder setSelectorNumOfClientManager( + int selectorNumOfClientManager) { + this.selectorNumOfClientManager = selectorNumOfClientManager; + return this; + } + + public PipeConsensusClientProperty.Builder setPrintLogWhenThriftClientEncounterException( + boolean printLogWhenThriftClientEncounterException) { + this.printLogWhenThriftClientEncounterException = printLogWhenThriftClientEncounterException; + return this; + } + + public PipeConsensusClientProperty.Builder setMaxClientNumForEachNode( + int maxClientNumForEachNode) { + this.maxClientNumForEachNode = maxClientNumForEachNode; + return this; + } + + public PipeConsensusClientProperty build() { + return new PipeConsensusClientProperty( + isRpcThriftCompressionEnabled, + selectorNumOfClientManager, + printLogWhenThriftClientEncounterException, + maxClientNumForEachNode); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/sync/SyncPipeConsensusServiceClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/sync/SyncPipeConsensusServiceClient.java new file mode 100644 index 000000000000..6cc9897f23a8 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/sync/SyncPipeConsensusServiceClient.java @@ -0,0 +1,139 @@ +/* + * 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.sync; + +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.ThriftClientFactory; +import org.apache.iotdb.commons.client.property.ThriftClientProperty; +import org.apache.iotdb.consensus.pipe.thrift.PipeConsensusIService; +import org.apache.iotdb.rpc.DeepCopyRpcTransportFactory; +import org.apache.iotdb.rpc.TConfigurationConst; +import org.apache.iotdb.rpc.TimeoutChangeableTransport; + +import org.apache.commons.pool2.PooledObject; +import org.apache.commons.pool2.impl.DefaultPooledObject; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransportException; + +import java.net.SocketException; + +public class SyncPipeConsensusServiceClient extends PipeConsensusIService.Client + implements ThriftClient, AutoCloseable { + + private final boolean printLogWhenEncounterException; + private final TEndPoint endpoint; + private final ClientManager<TEndPoint, SyncPipeConsensusServiceClient> clientManager; + + public SyncPipeConsensusServiceClient( + ThriftClientProperty property, + TEndPoint endpoint, + ClientManager<TEndPoint, SyncPipeConsensusServiceClient> clientManager) + throws TTransportException { + super( + property + .getProtocolFactory() + .getProtocol( + DeepCopyRpcTransportFactory.INSTANCE.getTransport( + new TSocket( + TConfigurationConst.defaultTConfiguration, + endpoint.getIp(), + endpoint.getPort(), + property.getConnectionTimeoutMs())))); + this.printLogWhenEncounterException = property.isPrintLogWhenEncounterException(); + this.endpoint = endpoint; + this.clientManager = clientManager; + getInputProtocol().getTransport().open(); + } + + public int getTimeout() throws SocketException { + return ((TimeoutChangeableTransport) getInputProtocol().getTransport()).getTimeOut(); + } + + public void setTimeout(int timeout) { + // the same transport is used in both input and output + ((TimeoutChangeableTransport) (getInputProtocol().getTransport())).setTimeout(timeout); + } + + @Override + public void close() { + clientManager.returnClient(endpoint, this); + } + + @Override + public void invalidate() { + getInputProtocol().getTransport().close(); + } + + @Override + public void invalidateAll() { + clientManager.clear(endpoint); + } + + @Override + public boolean printLogWhenEncounterException() { + return printLogWhenEncounterException; + } + + @Override + public String toString() { + return String.format("SyncPipeConsensusServiceClient{%s}", endpoint); + } + + public TEndPoint getTEndpoint() { + return endpoint; + } + + public static class Factory + extends ThriftClientFactory<TEndPoint, SyncPipeConsensusServiceClient> { + + public Factory( + ClientManager<TEndPoint, SyncPipeConsensusServiceClient> clientManager, + ThriftClientProperty thriftClientProperty) { + super(clientManager, thriftClientProperty); + } + + @Override + public void destroyObject( + TEndPoint endpoint, PooledObject<SyncPipeConsensusServiceClient> pooledObject) { + pooledObject.getObject().invalidate(); + } + + @Override + public PooledObject<SyncPipeConsensusServiceClient> makeObject(TEndPoint endpoint) + throws Exception { + return new DefaultPooledObject<>( + SyncThriftClientWithErrorHandler.newErrorHandler( + SyncPipeConsensusServiceClient.class, + SyncPipeConsensusServiceClient.class.getConstructor( + thriftClientProperty.getClass(), endpoint.getClass(), clientManager.getClass()), + thriftClientProperty, + endpoint, + clientManager)); + } + + @Override + public boolean validateObject( + TEndPoint endpoint, PooledObject<SyncPipeConsensusServiceClient> pooledObject) { + return pooledObject.getObject().getInputProtocol().getTransport().isOpen(); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java index 5c664684250a..00701d163e69 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java @@ -98,6 +98,10 @@ public enum ThreadName { CONFIG_NODE_TIMEOUT_EXECUTOR("ProcedureTimeoutExecutor"), CONFIG_NODE_WORKER_THREAD_MONITOR("ProcedureWorkerThreadMonitor"), CONFIG_NODE_RETRY_FAILED_TASK("Cluster-RetryFailedTasks-Service"), + // -------------------------- PipeConsensus -------------------------- + PIPE_CONSENSUS_RPC_SERVICE("PipeConsensusRPC-Service"), + PIPE_CONSENSUS_RPC_PROCESSOR("PipeConsensusRPC-Processor"), + ASYNC_DATANODE_PIPE_CONSENSUS_CLIENT_POOL("AsyncDataNodePipeConsensusServiceClientPool"), // -------------------------- IoTConsensus -------------------------- IOT_CONSENSUS_RPC_SERVICE("IoTConsensusRPC-Service"), @@ -127,6 +131,7 @@ public enum ThreadName { PIPE_EXTRACTOR_DISRUPTOR("Pipe-Extractor-Disruptor"), PIPE_PROCESSOR_EXECUTOR_POOL("Pipe-Processor-Executor-Pool"), PIPE_CONNECTOR_EXECUTOR_POOL("Pipe-Connector-Executor-Pool"), + PIPE_CONSENSUS_EXECUTOR_POOL("Pipe-Consensus-Executor-Pool"), PIPE_CONFIGNODE_EXECUTOR_POOL("Pipe-ConfigNode-Executor-Pool"), PIPE_SUBTASK_CALLBACK_EXECUTOR_POOL("Pipe-SubTask-Callback-Executor-Pool"), PIPE_RUNTIME_META_SYNCER("Pipe-Runtime-Meta-Syncer"), @@ -240,6 +245,13 @@ public enum ThreadName { LOG_DISPATCHER, IOT_CONSENSUS_BACKGROUND_TASK_EXECUTOR)); + private static final Set<ThreadName> pipeConsensusThreadNames = + new HashSet<>( + Arrays.asList( + PIPE_CONSENSUS_RPC_SERVICE, + PIPE_CONSENSUS_RPC_PROCESSOR, + ASYNC_DATANODE_PIPE_CONSENSUS_CLIENT_POOL)); + private static final Set<ThreadName> ratisThreadNames = new HashSet<>( Arrays.asList( @@ -264,6 +276,7 @@ public enum ThreadName { PIPE_EXTRACTOR_DISRUPTOR, PIPE_PROCESSOR_EXECUTOR_POOL, PIPE_CONNECTOR_EXECUTOR_POOL, + PIPE_CONSENSUS_EXECUTOR_POOL, PIPE_CONFIGNODE_EXECUTOR_POOL, PIPE_SUBTASK_CALLBACK_EXECUTOR_POOL, PIPE_RUNTIME_META_SYNCER, @@ -361,6 +374,7 @@ public enum ThreadName { schemaEngineThreadNames, clientServiceThreadNames, iotConsensusThreadNames, + pipeConsensusThreadNames, ratisThreadNames, computeThreadNames, jvmThreadNames, diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConsensusGroupId.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConsensusGroupId.java index bc4c8c5484f6..6a771007b3ac 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConsensusGroupId.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConsensusGroupId.java @@ -92,6 +92,35 @@ public static ConsensusGroupId create(int type, int id) { return groupId; } + public static ConsensusGroupId createFromString(String groupIdString) { + ConsensusGroupId groupId; + if (groupIdString.startsWith(TConsensusGroupType.DataRegion.name())) { + groupId = + new DataRegionId( + Integer.parseInt( + groupIdString.substring( + TConsensusGroupType.DataRegion.name().length() + 1, + groupIdString.length() - 1))); + } else if (groupIdString.startsWith(TConsensusGroupType.SchemaRegion.name())) { + groupId = + new SchemaRegionId( + Integer.parseInt( + groupIdString.substring( + TConsensusGroupType.SchemaRegion.name().length() + 1, + groupIdString.length() - 1))); + } else if (groupIdString.startsWith(TConsensusGroupType.ConfigRegion.name())) { + groupId = + new ConfigRegionId( + Integer.parseInt( + groupIdString.substring( + TConsensusGroupType.ConfigRegion.name().length() + 1, + groupIdString.length() - 1))); + } else { + throw new IllegalArgumentException("Unrecognized ConsensusGroupId: " + groupIdString); + } + return groupId; + } + public static ConsensusGroupId createFromTConsensusGroupId( TConsensusGroupId tConsensusGroupId) { return create(tConsensusGroupId.getType().getValue(), tConsensusGroupId.getId()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/HybridProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/HybridProgressIndex.java index 8e4541e40380..70727c9dcf69 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/HybridProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/HybridProgressIndex.java @@ -55,6 +55,10 @@ public HybridProgressIndex(ProgressIndex progressIndex) { } } + public Map<Short, ProgressIndex> getType2Index() { + return type2Index; + } + @Override public void serialize(ByteBuffer byteBuffer) { lock.readLock().lock(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/RecoverProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/RecoverProgressIndex.java index d7dc1f022031..a67a7a9ec321 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/RecoverProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/RecoverProgressIndex.java @@ -51,6 +51,10 @@ public RecoverProgressIndex(int dataNodeId, SimpleProgressIndex simpleProgressIn dataNodeId2LocalIndex.put(dataNodeId, simpleProgressIndex); } + public Map<Integer, SimpleProgressIndex> getDataNodeId2LocalIndex() { + return dataNodeId2LocalIndex; + } + @Override public void serialize(ByteBuffer byteBuffer) { lock.readLock().lock(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java index a7da7c3ef68a..3d0bd4cab746 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java @@ -184,6 +184,8 @@ public class PipeConnectorConstant { public static final String SINK_TOPIC_KEY = "sink.topic"; public static final String SINK_CONSUMER_GROUP_KEY = "sink.consumer-group"; + public static final String CONNECTOR_CONSENSUS_GROUP_ID_KEY = "connector.consensus.group-id"; + private PipeConnectorConstant() { throw new IllegalStateException("Utility class"); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeExtractorConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeExtractorConstant.java index a01c24d3aecc..a67b594d361d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeExtractorConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeExtractorConstant.java @@ -91,6 +91,13 @@ public class PipeExtractorConstant { public static final String SOURCE_WATERMARK_INTERVAL_KEY = "source.watermark-interval-ms"; public static final long EXTRACTOR_WATERMARK_INTERVAL_DEFAULT_VALUE = -1; // -1 means no watermark + ///////////////////// pipe consensus ///////////////////// + public static final String EXTRACTOR_CONSENSUS_GROUP_ID_KEY = "extractor.consensus.group-id"; + public static final String EXTRACTOR_CONSENSUS_SENDER_DATANODE_ID_KEY = + "extractor.consensus.sender-dn-id"; + public static final String EXTRACTOR_CONSENSUS_RECEIVER_DATANODE_ID_KEY = + "extractor.consensus.receiver-dn-id"; + private PipeExtractorConstant() { throw new IllegalStateException("Utility class"); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusRequestType.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusRequestType.java new file mode 100644 index 000000000000..3ac3e8f2685f --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusRequestType.java @@ -0,0 +1,67 @@ +/* + * 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.pipe.connector.payload.pipeconsensus.request; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +public enum PipeConsensusRequestType { + + // Data region + TRANSFER_TABLET_INSERT_NODE((short) 100), + // PipeConsensus doesn't expect to handle rawTabletEvent. + TRANSFER_TS_FILE_PIECE((short) 101), + TRANSFER_TS_FILE_SEAL((short) 102), + TRANSFER_TABLET_BATCH((short) 103), + TRANSFER_TABLET_BINARY((short) 104), + TRANSFER_TS_FILE_PIECE_WITH_MOD((short) 105), + TRANSFER_TS_FILE_SEAL_WITH_MOD((short) 106), + +// Note: temporarily PipeConsensus only support data region. But we put this class in `node-common` +// to reserve the scalability +; + + private final short type; + + PipeConsensusRequestType(short type) { + this.type = type; + } + + public short getType() { + return type; + } + + private static final Map<Short, PipeConsensusRequestType> TYPE_MAP = + Arrays.stream(PipeConsensusRequestType.values()) + .collect( + HashMap::new, + (typeMap, PipeConsensusRequestType) -> + typeMap.put(PipeConsensusRequestType.getType(), PipeConsensusRequestType), + HashMap::putAll); + + public static boolean isValidatedRequestType(short type) { + return TYPE_MAP.containsKey(type); + } + + public static PipeConsensusRequestType valueOf(short type) { + return TYPE_MAP.get(type); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusRequestVersion.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusRequestVersion.java new file mode 100644 index 000000000000..9dc34c16361a --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusRequestVersion.java @@ -0,0 +1,35 @@ +/* + * 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.pipe.connector.payload.pipeconsensus.request; + +public enum PipeConsensusRequestVersion { + VERSION_1((byte) 1), + ; + + private final byte version; + + PipeConsensusRequestVersion(byte type) { + this.version = type; + } + + public byte getVersion() { + return version; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusTransferFilePieceReq.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusTransferFilePieceReq.java new file mode 100644 index 000000000000..4b15f31c668c --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusTransferFilePieceReq.java @@ -0,0 +1,139 @@ +/* + * 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.pipe.connector.payload.pipeconsensus.request; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; + +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.PublicBAOS; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Objects; + +public abstract class PipeConsensusTransferFilePieceReq extends TPipeConsensusTransferReq { + + private transient String fileName; + private transient long startWritingOffset; + private transient byte[] filePiece; + + public final String getFileName() { + return fileName; + } + + public final long getStartWritingOffset() { + return startWritingOffset; + } + + public final byte[] getFilePiece() { + return filePiece; + } + + protected abstract PipeConsensusRequestType getPlanType(); + + /////////////////////////////// Thrift /////////////////////////////// + + protected final PipeConsensusTransferFilePieceReq convertToTPipeConsensusTransferReq( + String snapshotName, + long startWritingOffset, + byte[] snapshotPiece, + TCommitId commitId, + TConsensusGroupId consensusGroupId, + int thisDataNodeId) + throws IOException { + + this.fileName = snapshotName; + this.startWritingOffset = startWritingOffset; + this.filePiece = snapshotPiece; + + this.commitId = commitId; + this.consensusGroupId = consensusGroupId; + this.dataNodeId = thisDataNodeId; + this.version = PipeConsensusRequestVersion.VERSION_1.getVersion(); + this.type = getPlanType().getType(); + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + ReadWriteIOUtils.write(snapshotName, outputStream); + ReadWriteIOUtils.write(startWritingOffset, outputStream); + ReadWriteIOUtils.write(new Binary(snapshotPiece), outputStream); + body = ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + + return this; + } + + protected final PipeConsensusTransferFilePieceReq translateFromTPipeConsensusTransferReq( + TPipeConsensusTransferReq transferReq) { + + fileName = ReadWriteIOUtils.readString(transferReq.body); + startWritingOffset = ReadWriteIOUtils.readLong(transferReq.body); + filePiece = ReadWriteIOUtils.readBinary(transferReq.body).getValues(); + + version = transferReq.version; + type = transferReq.type; + body = transferReq.body; + commitId = transferReq.commitId; + dataNodeId = transferReq.dataNodeId; + consensusGroupId = transferReq.consensusGroupId; + + return this; + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + PipeConsensusTransferFilePieceReq that = (PipeConsensusTransferFilePieceReq) obj; + return fileName.equals(that.fileName) + && startWritingOffset == that.startWritingOffset + && Arrays.equals(filePiece, that.filePiece) + && version == that.version + && type == that.type + && body.equals(that.body) + && Objects.equals(commitId, that.commitId) + && Objects.equals(consensusGroupId, that.consensusGroupId) + && Objects.equals(dataNodeId, that.dataNodeId); + } + + @Override + public int hashCode() { + return Objects.hash( + fileName, + startWritingOffset, + Arrays.hashCode(filePiece), + version, + type, + body, + commitId, + consensusGroupId, + dataNodeId); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusTransferFileSealReq.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusTransferFileSealReq.java new file mode 100644 index 000000000000..ab7f65db2a7d --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusTransferFileSealReq.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.pipe.connector.payload.pipeconsensus.request; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; + +import org.apache.tsfile.utils.PublicBAOS; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +public abstract class PipeConsensusTransferFileSealReq + extends org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq { + + private transient String fileName; + private transient long fileLength; + + public final String getFileName() { + return fileName; + } + + public final long getFileLength() { + return fileLength; + } + + protected abstract PipeConsensusRequestType getPlanType(); + + /////////////////////////////// Thrift /////////////////////////////// + + protected PipeConsensusTransferFileSealReq convertToTPipeConsensusTransferReq( + String fileName, + long fileLength, + TCommitId commitId, + TConsensusGroupId consensusGroupId, + ProgressIndex progressIndex, + int thisDataNodeId) + throws IOException { + + this.fileName = fileName; + this.fileLength = fileLength; + + this.commitId = commitId; + this.consensusGroupId = consensusGroupId; + this.dataNodeId = thisDataNodeId; + this.version = PipeConsensusRequestVersion.VERSION_1.getVersion(); + this.type = getPlanType().getType(); + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + ReadWriteIOUtils.write(fileName, outputStream); + ReadWriteIOUtils.write(fileLength, outputStream); + this.body = ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + progressIndex.serialize(outputStream); + this.progressIndex = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + + return this; + } + + public PipeConsensusTransferFileSealReq translateFromTPipeConsensusTransferReq( + TPipeConsensusTransferReq req) { + + fileName = ReadWriteIOUtils.readString(req.body); + fileLength = ReadWriteIOUtils.readLong(req.body); + + version = req.version; + type = req.type; + body = req.body; + commitId = req.commitId; + dataNodeId = req.dataNodeId; + consensusGroupId = req.consensusGroupId; + progressIndex = req.progressIndex; + + return this; + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + PipeConsensusTransferFileSealReq that = (PipeConsensusTransferFileSealReq) obj; + return fileName.equals(that.fileName) + && fileLength == that.fileLength + && version == that.version + && type == that.type + && body.equals(that.body) + && Objects.equals(commitId, that.commitId) + && Objects.equals(consensusGroupId, that.consensusGroupId) + && Objects.equals(dataNodeId, that.dataNodeId) + && Objects.equals(progressIndex, that.progressIndex); + } + + @Override + public int hashCode() { + return Objects.hash( + fileName, + fileLength, + version, + type, + body, + commitId, + consensusGroupId, + dataNodeId, + progressIndex); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusTransferFileSealWithModReq.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusTransferFileSealWithModReq.java new file mode 100644 index 000000000000..0d7138a46f8f --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusTransferFileSealWithModReq.java @@ -0,0 +1,177 @@ +/* + * 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.pipe.connector.payload.pipeconsensus.request; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; + +import org.apache.tsfile.utils.PublicBAOS; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public abstract class PipeConsensusTransferFileSealWithModReq extends TPipeConsensusTransferReq { + + private transient List<String> fileNames; + private transient List<Long> fileLengths; + private transient Map<String, String> parameters; + + public final List<String> getFileNames() { + return fileNames; + } + + public final List<Long> getFileLengths() { + return fileLengths; + } + + public final Map<String, String> getParameters() { + return parameters; + } + + protected abstract PipeConsensusRequestType getPlanType(); + + /////////////////////////////// Thrift /////////////////////////////// + + protected PipeConsensusTransferFileSealWithModReq convertToTPipeConsensusTransferReq( + List<String> fileNames, + List<Long> fileLengths, + Map<String, String> parameters, + TCommitId commitId, + TConsensusGroupId consensusGroupId, + ProgressIndex progressIndex, + int thisDataNodeId) + throws IOException { + + this.fileNames = fileNames; + this.fileLengths = fileLengths; + this.parameters = parameters; + + this.commitId = commitId; + this.consensusGroupId = consensusGroupId; + this.dataNodeId = thisDataNodeId; + this.version = PipeConsensusRequestVersion.VERSION_1.getVersion(); + this.type = getPlanType().getType(); + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + ReadWriteIOUtils.write(fileNames.size(), outputStream); + for (String fileName : fileNames) { + ReadWriteIOUtils.write(fileName, outputStream); + } + ReadWriteIOUtils.write(fileLengths.size(), outputStream); + for (Long fileLength : fileLengths) { + ReadWriteIOUtils.write(fileLength, outputStream); + } + ReadWriteIOUtils.write(parameters.size(), outputStream); + for (final Map.Entry<String, String> entry : parameters.entrySet()) { + ReadWriteIOUtils.write(entry.getKey(), outputStream); + ReadWriteIOUtils.write(entry.getValue(), outputStream); + } + this.body = ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + progressIndex.serialize(outputStream); + this.progressIndex = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + + return this; + } + + public PipeConsensusTransferFileSealWithModReq translateFromTPipeConsensusTransferReq( + TPipeConsensusTransferReq req) { + fileNames = new ArrayList<>(); + int size = ReadWriteIOUtils.readInt(req.body); + for (int i = 0; i < size; ++i) { + fileNames.add(ReadWriteIOUtils.readString(req.body)); + } + + fileLengths = new ArrayList<>(); + size = ReadWriteIOUtils.readInt(req.body); + for (int i = 0; i < size; ++i) { + fileLengths.add(ReadWriteIOUtils.readLong(req.body)); + } + + parameters = new HashMap<>(); + size = ReadWriteIOUtils.readInt(req.body); + for (int i = 0; i < size; ++i) { + final String key = ReadWriteIOUtils.readString(req.body); + final String value = ReadWriteIOUtils.readString(req.body); + parameters.put(key, value); + } + + version = req.version; + type = req.type; + body = req.body; + commitId = req.commitId; + dataNodeId = req.dataNodeId; + consensusGroupId = req.consensusGroupId; + progressIndex = req.progressIndex; + + return this; + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + PipeConsensusTransferFileSealWithModReq that = (PipeConsensusTransferFileSealWithModReq) obj; + return Objects.equals(fileNames, that.fileNames) + && Objects.equals(fileLengths, that.fileLengths) + && Objects.equals(parameters, that.parameters) + && Objects.equals(version, that.version) + && Objects.equals(type, that.type) + && Objects.equals(body, that.body) + && Objects.equals(commitId, that.commitId) + && Objects.equals(consensusGroupId, that.consensusGroupId) + && Objects.equals(progressIndex, that.progressIndex) + && Objects.equals(dataNodeId, that.dataNodeId); + } + + @Override + public int hashCode() { + return Objects.hash( + fileNames, + fileLengths, + parameters, + version, + type, + body, + commitId, + consensusGroupId, + dataNodeId, + progressIndex); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/response/PipeConsensusTransferFilePieceResp.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/response/PipeConsensusTransferFilePieceResp.java new file mode 100644 index 000000000000..cb9385584a90 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/response/PipeConsensusTransferFilePieceResp.java @@ -0,0 +1,110 @@ +/* + * 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.pipe.connector.payload.pipeconsensus.response; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; + +import org.apache.tsfile.utils.PublicBAOS; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +public class PipeConsensusTransferFilePieceResp extends TPipeConsensusTransferResp { + + public static final long ERROR_END_OFFSET = -1; + + private long endWritingOffset; + + private PipeConsensusTransferFilePieceResp() { + // Empty constructor + } + + public long getEndWritingOffset() { + return endWritingOffset; + } + + /////////////////////////////// Thrift /////////////////////////////// + + public static PipeConsensusTransferFilePieceResp toTPipeConsensusTransferResp( + TSStatus status, long endWritingOffset) throws IOException { + final PipeConsensusTransferFilePieceResp filePieceResp = + new PipeConsensusTransferFilePieceResp(); + + filePieceResp.status = status; + + filePieceResp.endWritingOffset = endWritingOffset; + try (PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + ReadWriteIOUtils.write(endWritingOffset, outputStream); + filePieceResp.body = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + + return filePieceResp; + } + + public static PipeConsensusTransferFilePieceResp toTPipeConsensusTransferResp(TSStatus status) { + final PipeConsensusTransferFilePieceResp filePieceResp = + new PipeConsensusTransferFilePieceResp(); + + filePieceResp.status = status; + + return filePieceResp; + } + + public static PipeConsensusTransferFilePieceResp fromTPipeConsensusTransferResp( + TPipeConsensusTransferResp transferResp) { + final PipeConsensusTransferFilePieceResp filePieceResp = + new PipeConsensusTransferFilePieceResp(); + + filePieceResp.status = transferResp.status; + + if (transferResp.isSetBody()) { + filePieceResp.endWritingOffset = ReadWriteIOUtils.readLong(transferResp.body); + filePieceResp.body = transferResp.body; + } + + return filePieceResp; + } + + /////////////////////////////// Object /////////////////////////////// + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + PipeConsensusTransferFilePieceResp that = (PipeConsensusTransferFilePieceResp) obj; + return endWritingOffset == that.endWritingOffset + && status.equals(that.status) + && body.equals(that.body); + } + + @Override + public int hashCode() { + return Objects.hash(endWritingOffset, status, body); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java index 9d1f0525ee1d..3742c3c60288 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java @@ -29,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -52,6 +53,7 @@ public abstract class EnrichedEvent implements Event { protected String committerKey; public static final long NO_COMMIT_ID = -1; protected long commitId = NO_COMMIT_ID; + protected int rebootTimes = 0; protected final PipePattern pipePattern; @@ -313,6 +315,14 @@ public void setCommitterKeyAndCommitId(final String committerKey, final long com this.commitId = commitId; } + public void setRebootTimes(int rebootTimes) { + this.rebootTimes = rebootTimes; + } + + public int getRebootTimes() { + return rebootTimes; + } + public String getCommitterKey() { return committerKey; } @@ -331,6 +341,23 @@ public boolean isReleased() { return isReleased.get(); } + /** + * Used for pipeConsensus. In PipeConsensus, we only need commiterKey, commitId and rebootTimes to + * uniquely identify an event + */ + public boolean equalsInPipeConsensus(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + EnrichedEvent otherEvent = (EnrichedEvent) o; + return Objects.equals(committerKey, otherEvent.committerKey) + && commitId == otherEvent.commitId + && rebootTimes == otherEvent.rebootTimes; + } + @Override public String toString() { return "EnrichedEvent{" diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/BuiltinPipePlugin.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/BuiltinPipePlugin.java index a6ae2e9fd382..2831dbc8b26a 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/BuiltinPipePlugin.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/BuiltinPipePlugin.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.pipe.plugin.builtin.connector.donothing.DoNothingConnector; import org.apache.iotdb.commons.pipe.plugin.builtin.connector.iotdb.airgap.IoTDBAirGapConnector; +import org.apache.iotdb.commons.pipe.plugin.builtin.connector.iotdb.consensus.PipeConsensusAsyncConnector; import org.apache.iotdb.commons.pipe.plugin.builtin.connector.iotdb.thrift.IoTDBLegacyPipeConnector; import org.apache.iotdb.commons.pipe.plugin.builtin.connector.iotdb.thrift.IoTDBThriftAsyncConnector; import org.apache.iotdb.commons.pipe.plugin.builtin.connector.iotdb.thrift.IoTDBThriftConnector; @@ -38,6 +39,7 @@ import org.apache.iotdb.commons.pipe.plugin.builtin.processor.downsampling.ChangingValueSamplingProcessor; import org.apache.iotdb.commons.pipe.plugin.builtin.processor.downsampling.SwingingDoorTrendingSamplingProcessor; import org.apache.iotdb.commons.pipe.plugin.builtin.processor.downsampling.TumblingTimeSamplingProcessor; +import org.apache.iotdb.commons.pipe.plugin.builtin.processor.pipeconsensus.PipeConsensusProcessor; import org.apache.iotdb.commons.pipe.plugin.builtin.processor.throwing.ThrowingExceptionProcessor; import org.apache.iotdb.commons.pipe.plugin.builtin.processor.twostage.TwoStageCountProcessor; @@ -69,6 +71,7 @@ public enum BuiltinPipePlugin { // Hidden-processors, which are plugins of the processors STANDARD_STATISTICS_PROCESSOR("standard-statistics-processor", StandardStatisticsProcessor.class), TUMBLING_WINDOWING_PROCESSOR("tumbling-windowing-processor", TumblingWindowingProcessor.class), + PIPE_CONSENSUS_PROCESSOR("pipe-consensus-processor", PipeConsensusProcessor.class), // connectors DO_NOTHING_CONNECTOR("do-nothing-connector", DoNothingConnector.class), @@ -78,6 +81,9 @@ public enum BuiltinPipePlugin { IOTDB_THRIFT_ASYNC_CONNECTOR("iotdb-thrift-async-connector", IoTDBThriftAsyncConnector.class), IOTDB_LEGACY_PIPE_CONNECTOR("iotdb-legacy-pipe-connector", IoTDBLegacyPipeConnector.class), IOTDB_AIR_GAP_CONNECTOR("iotdb-air-gap-connector", IoTDBAirGapConnector.class), + PIPE_CONSENSUS_ASYNC_CONNECTOR( + "pipe-consensus-async-connector", PipeConsensusAsyncConnector.class), + WEBSOCKET_CONNECTOR("websocket-connector", WebSocketConnector.class), OPC_UA_CONNECTOR("opc-ua-connector", OpcUaConnector.class), WRITE_BACK_CONNECTOR("write-back-connector", WriteBackConnector.class), @@ -93,6 +99,7 @@ public enum BuiltinPipePlugin { OPC_UA_SINK("opc-ua-sink", OpcUaConnector.class), WRITE_BACK_SINK("write-back-sink", WriteBackConnector.class), SUBSCRIPTION_SINK("subscription-sink", DoNothingConnector.class), + PIPE_CONSENSUS_ASYNC_SINK("pipe-consensus-async-sink", PipeConsensusAsyncConnector.class), ; private final String pipePluginName; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/connector/iotdb/consensus/PipeConsensusAsyncConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/connector/iotdb/consensus/PipeConsensusAsyncConnector.java new file mode 100644 index 000000000000..b6d1e85809ac --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/connector/iotdb/consensus/PipeConsensusAsyncConnector.java @@ -0,0 +1,30 @@ +/* + * 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.pipe.plugin.builtin.connector.iotdb.consensus; + +import org.apache.iotdb.commons.pipe.plugin.builtin.connector.PlaceholderConnector; + +/** + * This class is a placeholder and should not be initialized. It represents the + * PipeConsensusAsyncConnector. There is a real implementation in the server module but cannot be + * imported here. The pipe agent in the server module will replace this class with the real + * implementation when initializing the PipeConsensusAsyncConnector. + */ +public class PipeConsensusAsyncConnector extends PlaceholderConnector {} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/pipeconsensus/PipeConsensusProcessor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/pipeconsensus/PipeConsensusProcessor.java new file mode 100644 index 000000000000..18c4656428d9 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/pipeconsensus/PipeConsensusProcessor.java @@ -0,0 +1,30 @@ +/* + * 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.pipe.plugin.builtin.processor.pipeconsensus; + +import org.apache.iotdb.commons.pipe.plugin.builtin.processor.PlaceHolderProcessor; + +/** + * This class is a placeholder and should not be initialized. It represents the + * PipeConsensusProcessor. There is a real implementation in the server module but cannot be + * imported here. The pipe agent in the server module will replace this class with the real + * implementation when initializing the PipeConsensusProcessor. + */ +public class PipeConsensusProcessor extends PlaceHolderProcessor {} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeStaticMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeStaticMeta.java index 718ff85527d6..d35086c693f8 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeStaticMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeStaticMeta.java @@ -217,6 +217,7 @@ public String toString() { public static final String SYSTEM_PIPE_PREFIX = "__"; public static final String SUBSCRIPTION_PIPE_PREFIX = SYSTEM_PIPE_PREFIX + "subscription."; + public static final String CONSENSUS_PIPE_PREFIX = SYSTEM_PIPE_PREFIX + "consensus."; public static String generateSubscriptionPipeName( final String topicName, final String consumerGroupId) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeType.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeType.java index 5051cfcb5bdf..95fb893654ae 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeType.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeType.java @@ -22,6 +22,7 @@ public enum PipeType { USER((byte) 0), SUBSCRIPTION((byte) 1), + CONSENSUS((byte) 2), ; private final byte type; @@ -33,6 +34,8 @@ public enum PipeType { public static PipeType getPipeType(String pipeName) { if (pipeName.startsWith(PipeStaticMeta.SUBSCRIPTION_PIPE_PREFIX)) { return SUBSCRIPTION; + } else if (pipeName.startsWith(PipeStaticMeta.CONSENSUS_PIPE_PREFIX)) { + return CONSENSUS; } return USER; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/ServiceType.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/ServiceType.java index 53a24de56c70..24a1a915325e 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/ServiceType.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/ServiceType.java @@ -76,6 +76,7 @@ public enum ServiceType { MPP_DATA_EXCHANGE_SERVICE("MPP Data exchange manager", "MPPDataExchangeManager"), INTERNAL_SERVICE("Internal Service", "InternalService"), IOT_CONSENSUS_SERVICE("IoTConsensus Service", "IoTConsensusRPCService"), + PIPE_CONSENSUS_SERVICE("PipeConsensus Service", "PipeConsensusRPCService"), PIPE_PLUGIN_CLASSLOADER_MANAGER_SERVICE( "Pipe Plugin Classloader Manager Service", "PipePluginClassLoader"), PIPE_RUNTIME_DATA_NODE_AGENT("Pipe Runtime Data Node Agent", "PipeRuntimeDataNodeAgent"), diff --git a/iotdb-protocol/thrift-consensus/src/main/thrift/pipeconsensus.thrift b/iotdb-protocol/thrift-consensus/src/main/thrift/pipeconsensus.thrift new file mode 100644 index 000000000000..0651191a8af1 --- /dev/null +++ b/iotdb-protocol/thrift-consensus/src/main/thrift/pipeconsensus.thrift @@ -0,0 +1,109 @@ +/* + * 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. + */ + +include "common.thrift" +namespace java org.apache.iotdb.consensus.pipe.thrift + +struct TCommitId { + 1:required i64 commitIndex + 2:required i32 rebootTimes +} + +struct TPipeConsensusTransferReq { + 1:required i8 version + 2:required i16 type + 3:required TCommitId commitId + 4:required common.TConsensusGroupId consensusGroupId + 5:required i32 dataNodeId + 6:required binary body + 7:optional binary progressIndex +} + +struct TPipeConsensusTransferResp { + 1:required common.TSStatus status + 2:optional binary body +} + +struct TPipeConsensusBatchTransferReq { + 1:required list<TPipeConsensusTransferReq> batchReqs +} + +struct TPipeConsensusBatchTransferResp { + 1:required list<TPipeConsensusTransferResp> batchResps +} + +struct TSetActiveReq { + 1: required common.TConsensusGroupId consensusGroupId + 2: required bool isActive +} + +struct TSetActiveResp { + 1: required common.TSStatus status +} + +struct TNotifyPeerToCreateConsensusPipeReq { + 1: required common.TConsensusGroupId targetPeerConsensusGroupId + 2: required common.TEndPoint targetPeerEndPoint + 3: required i32 targetPeerNodeId +} + +struct TNotifyPeerToCreateConsensusPipeResp { + 1: required common.TSStatus status +} + +struct TNotifyPeerToDropConsensusPipeReq { + 1: required common.TConsensusGroupId targetPeerConsensusGroupId + 2: required common.TEndPoint targetPeerEndPoint + 3: required i32 targetPeerNodeId +} + +struct TNotifyPeerToDropConsensusPipeResp { + 1: required common.TSStatus status +} + +struct TCheckConsensusPipeCompletedReq { + 1: required common.TConsensusGroupId consensusGroupId + 2: required list<string> consensusPipeNames; + 3: required bool refreshCachedProgressIndex +} + +struct TCheckConsensusPipeCompletedResp { + 1: required common.TSStatus status + 2: required bool isCompleted +} + +service PipeConsensusIService { + /** + * Transfer stream data in a given ConsensusGroup, used by PipeConsensus + **/ + TPipeConsensusTransferResp pipeConsensusTransfer(TPipeConsensusTransferReq req) + + /** + * Transfer batch data in a given ConsensusGroup, used by PipeConsensus + **/ + TPipeConsensusBatchTransferResp pipeConsensusBatchTransfer(TPipeConsensusBatchTransferReq req) + + TSetActiveResp setActive(TSetActiveReq req) + + TNotifyPeerToCreateConsensusPipeResp notifyPeerToCreateConsensusPipe(TNotifyPeerToCreateConsensusPipeReq req) + + TNotifyPeerToDropConsensusPipeResp notifyPeerToDropConsensusPipe(TNotifyPeerToDropConsensusPipeReq req) + + TCheckConsensusPipeCompletedResp checkConsensusPipeCompleted(TCheckConsensusPipeCompletedReq req) +} \ No newline at end of file