Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-31 Thread via GitHub


OneSizeFitsQuorum merged PR #12355:
URL: https://github.com/apache/iotdb/pull/12355


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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-30 Thread via GitHub


OneSizeFitsQuorum commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1621740191


##
iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties:
##
@@ -1752,6 +1766,13 @@ data_replication_factor=1
 # schema_region_ratis_periodic_snapshot_interval=86400
 # data_region_ratis_periodic_snapshot_interval=86400
 
+
+### PipeConsensus Configuration
+
+# Default event buffer size for connector and receiver in pipe consensus
+# DataType: int
+# pipe_consensus_pipeline_size=5

Review Comment:
   fast_iot_consensus_pipeline_size?
   
   we do not need to expose pipeconsensus to users, we can just put this 
paramemters behind IoTConsensus
   
   
   ### Fast IoTConsensus Configuration
   



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-30 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1620763683


##
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RouteBalancer.java:
##
@@ -80,13 +80,17 @@ 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.isEnableAutoLeaderBalanceForFastIoTConsensus()
+  && 
ConsensusFactory.FAST_IOT_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 
=
   (CONF.isEnableAutoLeaderBalanceForRatisConsensus()
   && 
ConsensusFactory.RATIS_CONSENSUS.equals(SCHEMA_REGION_CONSENSUS_PROTOCOL_CLASS))
   || (CONF.isEnableAutoLeaderBalanceForIoTConsensus()
   && 
ConsensusFactory.IOT_CONSENSUS.equals(SCHEMA_REGION_CONSENSUS_PROTOCOL_CLASS))
+  || (CONF.isEnableAutoLeaderBalanceForFastIoTConsensus()

Review Comment:
   fixed



##
iotdb-core/node-commons/src/assembly/resources/conf/iotdb-common.properties:
##
@@ -138,6 +139,12 @@ data_replication_factor=1
 # Datatype: Boolean
 # enable_auto_leader_balance_for_iot_consensus=true
 
+# Whether to enable auto leader balance for FastIoTConsensus protocol.
+# The ConfigNode-leader will balance the leader of 
FastIoTConsensus-RegionGroups by leader_distribution_policy if set true.
+# Notice: Default is true because the FastIoTConsensus depends on this 
function to distribute leader.
+# Datatype: Boolean
+# enable_auto_leader_balance_for_fast_iot_consensus=true

Review Comment:
   fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-30 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1620763036


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeDispatcher.java:
##
@@ -33,5 +33,7 @@ void createPipe(
 
   void stopPipe(String pipeName) throws Exception;
 
-  void dropPipe(String pipeName) throws Exception;
+  // Use ConsensusPipeName instead of String to provide information for 
receiverAgent to release

Review Comment:
   fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-30 Thread via GitHub


OneSizeFitsQuorum commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1620726318


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeDispatcher.java:
##
@@ -33,5 +33,7 @@ void createPipe(
 
   void stopPipe(String pipeName) throws Exception;
 
-  void dropPipe(String pipeName) throws Exception;
+  // Use ConsensusPipeName instead of String to provide information for 
receiverAgent to release

Review Comment:
   use /** */



##
iotdb-core/node-commons/src/assembly/resources/conf/iotdb-common.properties:
##
@@ -138,6 +139,12 @@ data_replication_factor=1
 # Datatype: Boolean
 # enable_auto_leader_balance_for_iot_consensus=true
 
+# Whether to enable auto leader balance for FastIoTConsensus protocol.
+# The ConfigNode-leader will balance the leader of 
FastIoTConsensus-RegionGroups by leader_distribution_policy if set true.
+# Notice: Default is true because the FastIoTConsensus depends on this 
function to distribute leader.
+# Datatype: Boolean
+# enable_auto_leader_balance_for_fast_iot_consensus=true

Review Comment:
   It seems that the enable_auto_leader_balance_for_iot_consensus value can be 
used for both new consensus protocols, so we will not add this configuration



##
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/RouteBalancer.java:
##
@@ -80,13 +80,17 @@ 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.isEnableAutoLeaderBalanceForFastIoTConsensus()
+  && 
ConsensusFactory.FAST_IOT_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 
=
   (CONF.isEnableAutoLeaderBalanceForRatisConsensus()
   && 
ConsensusFactory.RATIS_CONSENSUS.equals(SCHEMA_REGION_CONSENSUS_PROTOCOL_CLASS))
   || (CONF.isEnableAutoLeaderBalanceForIoTConsensus()
   && 
ConsensusFactory.IOT_CONSENSUS.equals(SCHEMA_REGION_CONSENSUS_PROTOCOL_CLASS))
+  || (CONF.isEnableAutoLeaderBalanceForFastIoTConsensus()

Review Comment:
   do not need this?



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1619809943


##
iotdb-core/consensus/pom.xml:
##
@@ -117,12 +122,6 @@
 com.google.code.findbugs
 jsr305
 
-
-org.apache.tsfile
-tsfile
-${tsfile.version}

Review Comment:
   fixed
   



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1619809506


##
iotdb-core/node-commons/pom.xml:
##
@@ -75,6 +75,11 @@
 iotdb-thrift-commons
 1.3.2-SNAPSHOT
 
+

Review Comment:
   fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1619735791


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java:
##
@@ -0,0 +1,118 @@
+/*
+ * 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.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.builder()
+.put(PROCESSOR_KEY, config.getProcessorPluginName())
+.build(),
+ImmutableMap.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 startConsensusPipe(Peer senderPeer, Peer receiverPeer) throws 
Exception {

Review Comment:
   fixed



##

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1619735390


##
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 =

Review Comment:
   it's for debug use



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1619734518


##
iotdb-protocol/thrift-consensus/src/main/thrift/pipeconsensus.thrift:
##
@@ -0,0 +1,111 @@
+/*
+ * 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 batchReqs
+}
+
+struct TPipeConsensusBatchTransferResp {
+  1:required list 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 consensusGroupId
+  2: required common.TConsensusGroupId targetPeerConsensusGroupId
+  3: required common.TEndPoint targetPeerEndPoint
+  4: required i32 targetPeerNodeId
+}
+
+struct TNotifyPeerToCreateConsensusPipeResp {
+  1: required common.TSStatus status
+}
+
+struct TNotifyPeerToDropConsensusPipeReq {
+  1: required common.TConsensusGroupId consensusGroupId
+  2: required common.TConsensusGroupId targetPeerConsensusGroupId

Review Comment:
   fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1619734881


##
iotdb-protocol/thrift-consensus/src/main/thrift/pipeconsensus.thrift:
##
@@ -0,0 +1,111 @@
+/*
+ * 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 batchReqs
+}
+
+struct TPipeConsensusBatchTransferResp {
+  1:required list 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 consensusGroupId
+  2: required common.TConsensusGroupId targetPeerConsensusGroupId

Review Comment:
   fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618611230


##
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 {

Review Comment:
   todo with @Pengzna 



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618610512


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusTabletBatchReq.java:
##
@@ -0,0 +1,87 @@
+/*
+ * 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 {

Review Comment:
   todo with @Pengzna 



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618608920


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,585 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.AsyncPipeConsensusServiceClient;
+import org.apache.iotdb.consensus.pipe.thrift.TCommitId;
+import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawEventHandler;
+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.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletRawReq;
+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;
+import java.util.concurrent.atomic.AtomicInteger;
+
+// 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 by. Please check your construct parameters!";
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618512079


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensus.java:
##
@@ -0,0 +1,485 @@
+/*
+ * 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.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.client.AsyncPipeConsensusServiceClient;
+import org.apache.iotdb.consensus.pipe.client.PipeConsensusClientPool;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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 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 
asyncClientManager;
+  private final IClientManager 
syncClientManager;

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618467337


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusServerImpl.java:
##
@@ -0,0 +1,545 @@
+/*
+ * 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.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.client.SyncPipeConsensusServiceClient;
+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.TCheckConsensusPipeCompleteddReq;
+import 
org.apache.iotdb.consensus.pipe.thrift.TCheckConsensusPipeCompleteddResp;
+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 
syncClientManager;
+
+  private ProgressIndex cachedProgressIndex = MinimumProgressIndex.INSTANCE;
+
+  public PipeConsensusServerImpl(
+  Peer thisNode,
+  IStateMachine stateMachine,
+  String storageDir,
+  List configuration,
+  PipeConsensusConfig config,
+  ConsensusPipeManager consensusPipeManager,
+  IClientManager 
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 = 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618242621


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusServerImpl.java:
##
@@ -0,0 +1,545 @@
+/*
+ * 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.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.client.SyncPipeConsensusServiceClient;
+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.TCheckConsensusPipeCompleteddReq;
+import 
org.apache.iotdb.consensus.pipe.thrift.TCheckConsensusPipeCompleteddResp;
+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 
syncClientManager;
+
+  private ProgressIndex cachedProgressIndex = MinimumProgressIndex.INSTANCE;
+
+  public PipeConsensusServerImpl(
+  Peer thisNode,
+  IStateMachine stateMachine,
+  String storageDir,
+  List configuration,
+  PipeConsensusConfig config,
+  ConsensusPipeManager consensusPipeManager,
+  IClientManager 
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 = 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618434874


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java:
##
@@ -132,6 +139,39 @@ 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())
+  
.setSelectorNumOfClientManager(CONF.getSelectorNumOfClientManager())
+  .setThriftServerAwaitTimeForStopService(
+  CONF.getThriftServerAwaitTimeForStopService())
+  .setThriftMaxFrameSize(CONF.getThriftMaxFrameSize())
+  
.setMaxClientNumForEachNode(CONF.getMaxClientNumForEachNode())
+  .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

Review Comment:
   it makes sense. fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618432842


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusServerImpl.java:
##
@@ -0,0 +1,545 @@
+/*
+ * 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.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.client.SyncPipeConsensusServiceClient;
+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.TCheckConsensusPipeCompleteddReq;
+import 
org.apache.iotdb.consensus.pipe.thrift.TCheckConsensusPipeCompleteddResp;
+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 
syncClientManager;
+
+  private ProgressIndex cachedProgressIndex = MinimumProgressIndex.INSTANCE;
+
+  public PipeConsensusServerImpl(
+  Peer thisNode,
+  IStateMachine stateMachine,
+  String storageDir,
+  List configuration,
+  PipeConsensusConfig config,
+  ConsensusPipeManager consensusPipeManager,
+  IClientManager 
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 = 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618244595


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensus.java:
##
@@ -0,0 +1,485 @@
+/*
+ * 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.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.client.AsyncPipeConsensusServiceClient;
+import org.apache.iotdb.consensus.pipe.client.PipeConsensusClientPool;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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

Review Comment:
   I will discuss this with @Caideyipi 



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618242621


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusServerImpl.java:
##
@@ -0,0 +1,545 @@
+/*
+ * 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.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.client.SyncPipeConsensusServiceClient;
+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.TCheckConsensusPipeCompleteddReq;
+import 
org.apache.iotdb.consensus.pipe.thrift.TCheckConsensusPipeCompleteddResp;
+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 
syncClientManager;
+
+  private ProgressIndex cachedProgressIndex = MinimumProgressIndex.INSTANCE;
+
+  public PipeConsensusServerImpl(
+  Peer thisNode,
+  IStateMachine stateMachine,
+  String storageDir,
+  List configuration,
+  PipeConsensusConfig config,
+  ConsensusPipeManager consensusPipeManager,
+  IClientManager 
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 = 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618238655


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java:
##
@@ -0,0 +1,118 @@
+/*
+ * 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.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.builder()
+.put(PROCESSOR_KEY, config.getProcessorPluginName())
+.build(),
+ImmutableMap.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 startConsensusPipe(Peer senderPeer, Peer receiverPeer) throws 
Exception {

Review Comment:
   @yschengzi PTAL



##

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618237176


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeRuntimeAgentGuardian.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.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;
+  private String id;
+
+  @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;
+  this.id = id;
+  PipeAgent.runtime().registerPeriodicalJob(id, guardJob, 
intervalInSeconds);
+}
+  }
+
+  @Override
+  public synchronized void stop() {
+LOGGER.info("Try to stop periodical job {}, but it will not stop.", id);
+// TODO: add deregister in PipePeriodicalJobExecutor

Review Comment:
   when @Caideyipi  is available, I will discuss this todo with him



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618235983


##
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.TCheckConsensusPipeCompleteddReq;
+import 
org.apache.iotdb.consensus.pipe.thrift.TCheckConsensusPipeCompleteddResp;
+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 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 resultHandler)
+  throws TException {}
+
+  @Override
+  public void setActive(TSetActiveReq req, AsyncMethodCallback 
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 resultHandler)
+  throws TException {
+ConsensusGroupId groupId =
+

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618235692


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensus.java:
##
@@ -0,0 +1,477 @@
+/*
+ * 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.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.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.client.AsyncPipeConsensusServiceClient;
+import org.apache.iotdb.consensus.pipe.client.PipeConsensusClientPool;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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 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 
asyncClientManager;
+  private final IClientManager 
syncClientManager;
+
+  public PipeConsensus(ConsensusConfig config, 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-29 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618234304


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java:
##
@@ -153,24 +161,28 @@ 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 //
 
   public void assignProgressIndexForTsFileRecovery(TsFileResource 
tsFileResource) {
 tsFileResource.updateProgressIndex(
 new RecoverProgressIndex(
-DATA_NODE_ID,
-
simpleConsensusProgressIndexAssigner.getSimpleProgressIndexForTsFileRecovery()));
+DATA_NODE_ID, 
simpleProgressIndexAssigner.getSimpleProgressIndex()));

Review Comment:
   @yschengzi PTAL



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618226867


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618225716


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiverAgent.java:
##
@@ -0,0 +1,197 @@
+/*
+ * 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.commons.utils.TestOnly;
+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.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+public class PipeConsensusReceiverAgent implements ConsensusPipeReceiver {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusReceiverAgent.class);
+
+  private static final Map<
+  Byte,
+  TriFunction>
+  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 --manage--> n replicas' receivers
+  // 1 replica --has--> (n-1) receivers
+  private final Map<
+  ConsensusGroupId, Map>>
+  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> 
consensusPipe2ReciverMap =
+replicaReceiverMap.computeIfAbsent(consensusGroupId, key -> new 
ConcurrentHashMap<>());
+// 2. Route to 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618154632


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java:
##
@@ -0,0 +1,136 @@
+/*
+ * 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 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 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);
+  groupId2MaxProgressIndex.put(

Review Comment:
   fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618144182


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java:
##
@@ -0,0 +1,529 @@
+/*
+ * 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.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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawReq;
+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.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.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 TABLET_RAW_SCENARIO = "transfer raw tablet";
+
+  private static final String TSFILE_SCENARIO = "transfer tsfile";
+
+  private static final String TABLET_BATCH_SCENARIO 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618142680


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,585 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.AsyncPipeConsensusServiceClient;
+import org.apache.iotdb.consensus.pipe.thrift.TCommitId;
+import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawEventHandler;
+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.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletRawReq;
+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;
+import java.util.concurrent.atomic.AtomicInteger;
+
+// 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 by. Please check your construct parameters!";
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618142350


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,585 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.AsyncPipeConsensusServiceClient;
+import org.apache.iotdb.consensus.pipe.thrift.TCommitId;
+import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawEventHandler;
+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.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletRawReq;
+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;
+import java.util.concurrent.atomic.AtomicInteger;
+
+// 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 by. Please check your construct parameters!";
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1618135699


##
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java:
##
@@ -202,6 +202,7 @@ public void loadCommonProps(Properties properties) {
 properties.getProperty("datanode_token_timeout", String.valueOf(3 
* 60 * 1000;
 
 loadPipeProps(properties);
+loadPipeConsensusProps(properties);

Review Comment:
   fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617102941


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusServerImpl.java:
##
@@ -0,0 +1,545 @@
+/*
+ * 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.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.client.SyncPipeConsensusServiceClient;
+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.TCheckConsensusPipeCompleteddReq;
+import 
org.apache.iotdb.consensus.pipe.thrift.TCheckConsensusPipeCompleteddResp;
+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 
syncClientManager;
+
+  private ProgressIndex cachedProgressIndex = MinimumProgressIndex.INSTANCE;
+
+  public PipeConsensusServerImpl(
+  Peer thisNode,
+  IStateMachine stateMachine,
+  String storageDir,
+  List configuration,
+  PipeConsensusConfig config,
+  ConsensusPipeManager consensusPipeManager,
+  IClientManager 
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 = 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617102086


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/simple/SimpleConsensus.java:
##
@@ -61,7 +61,7 @@
  *
  * Notice: The stateMachine needs to implement WAL itself to ensure 
recovery after a restart
  */
-class SimpleConsensus implements IConsensus {
+public class SimpleConsensus implements IConsensus {

Review Comment:
   fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617101732


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensus.java:
##
@@ -0,0 +1,485 @@
+/*
+ * 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.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.client.AsyncPipeConsensusServiceClient;
+import org.apache.iotdb.consensus.pipe.client.PipeConsensusClientPool;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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 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 
asyncClientManager;
+  private final IClientManager 
syncClientManager;

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617101058


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/PipeConsensusConfig.java:
##
@@ -0,0 +1,404 @@
+/*
+ * 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.client.property.ClientPoolProperty;
+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 selectorNumOfClientManager;
+private final int connectionTimeoutInMs;
+private final boolean printLogWhenThriftClientEncounterException;
+private final int thriftMaxFrameSize;
+private final int maxClientNumForEachNode;
+
+public RPC(
+int rpcSelectorThreadNum,
+int rpcMinConcurrentClientNum,
+int rpcMaxConcurrentClientNum,
+int thriftServerAwaitTimeForStopService,
+boolean isRpcThriftCompressionEnabled,
+int selectorNumOfClientManager,
+int connectionTimeoutInMs,
+boolean printLogWhenThriftClientEncounterException,
+int thriftMaxFrameSize,
+int maxClientNumForEachNode) {
+  this.rpcSelectorThreadNum = rpcSelectorThreadNum;
+  this.rpcMinConcurrentClientNum = rpcMinConcurrentClientNum;
+  this.rpcMaxConcurrentClientNum = rpcMaxConcurrentClientNum;
+  this.thriftServerAwaitTimeForStopService = 
thriftServerAwaitTimeForStopService;
+  this.isRpcThriftCompressionEnabled = isRpcThriftCompressionEnabled;
+  this.selectorNumOfClientManager = selectorNumOfClientManager;
+  this.connectionTimeoutInMs = connectionTimeoutInMs;
+  this.printLogWhenThriftClientEncounterException = 
printLogWhenThriftClientEncounterException;
+  this.thriftMaxFrameSize = thriftMaxFrameSize;
+  this.maxClientNumForEachNode = maxClientNumForEachNode;
+}
+
+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 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617100553


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java:
##
@@ -35,6 +38,15 @@ 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";
+  public static final String STREAM_CONSENSUS = 
"org.apache.iotdb.consensus.pipe.StreamConsensus";
+  public static final String BATCH_CONSENSUS = 
"org.apache.iotdb.consensus.pipe.BatchConsensus";

Review Comment:
   got it



##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java:
##
@@ -35,6 +38,15 @@ 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";
+  public static final String STREAM_CONSENSUS = 
"org.apache.iotdb.consensus.pipe.StreamConsensus";

Review Comment:
   ditto



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617100285


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java:
##
@@ -0,0 +1,529 @@
+/*
+ * 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.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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawReq;
+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.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.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 TABLET_RAW_SCENARIO = "transfer raw tablet";
+
+  private static final String TSFILE_SCENARIO = "transfer tsfile";
+
+  private static final String TABLET_BATCH_SCENARIO 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617099794


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java:
##
@@ -0,0 +1,529 @@
+/*
+ * 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.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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawReq;
+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.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.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 TABLET_RAW_SCENARIO = "transfer raw tablet";
+
+  private static final String TSFILE_SCENARIO = "transfer tsfile";
+
+  private static final String TABLET_BATCH_SCENARIO 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617096503


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java:
##
@@ -0,0 +1,529 @@
+/*
+ * 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.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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawReq;
+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.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.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 TABLET_RAW_SCENARIO = "transfer raw tablet";
+
+  private static final String TSFILE_SCENARIO = "transfer tsfile";
+
+  private static final String TABLET_BATCH_SCENARIO 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617096201


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java:
##
@@ -0,0 +1,529 @@
+/*
+ * 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.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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawReq;
+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.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.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 TABLET_RAW_SCENARIO = "transfer raw tablet";
+
+  private static final String TSFILE_SCENARIO = "transfer tsfile";
+
+  private static final String TABLET_BATCH_SCENARIO 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617095725


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,585 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.AsyncPipeConsensusServiceClient;
+import org.apache.iotdb.consensus.pipe.thrift.TCommitId;
+import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawEventHandler;
+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.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletRawReq;
+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;
+import java.util.concurrent.atomic.AtomicInteger;
+
+// 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 by. Please check your construct parameters!";
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617088820


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java:
##
@@ -0,0 +1,529 @@
+/*
+ * 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.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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawReq;
+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.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.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 TABLET_RAW_SCENARIO = "transfer raw tablet";
+
+  private static final String TSFILE_SCENARIO = "transfer tsfile";
+
+  private static final String TABLET_BATCH_SCENARIO 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617088046


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,585 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.AsyncPipeConsensusServiceClient;
+import org.apache.iotdb.consensus.pipe.thrift.TCommitId;
+import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawEventHandler;
+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.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletRawReq;
+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;
+import java.util.concurrent.atomic.AtomicInteger;
+
+// 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 by. Please check your construct parameters!";
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617086716


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,585 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.AsyncPipeConsensusServiceClient;
+import org.apache.iotdb.consensus.pipe.thrift.TCommitId;
+import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawEventHandler;
+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.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletRawReq;
+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;
+import java.util.concurrent.atomic.AtomicInteger;
+
+// 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 by. Please check your construct parameters!";
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617080377


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617080377


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617080377


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617078557


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617079884


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617079194


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617077217


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617076843


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617075649


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617075350


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617073931


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617070230


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617064253


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1617059320


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiverAgent.java:
##
@@ -0,0 +1,197 @@
+/*
+ * 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.commons.utils.TestOnly;
+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.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+public class PipeConsensusReceiverAgent implements ConsensusPipeReceiver {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusReceiverAgent.class);
+
+  private static final Map<
+  Byte,
+  TriFunction>
+  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 --manage--> n replicas' receivers
+  // 1 replica --has--> (n-1) receivers
+  private final Map<
+  ConsensusGroupId, Map>>
+  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> 
consensusPipe2ReciverMap =
+replicaReceiverMap.computeIfAbsent(consensusGroupId, key -> new 
ConcurrentHashMap<>());
+// 2. Route to 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616760160


##
iotdb-core/node-commons/src/assembly/resources/sbin/destroy-datanode.bat:
##
@@ -158,26 +158,6 @@ for %%i in (%dn_sync_dir%) do (
 )
 )
 
-for /f  "eol=# tokens=2 delims==" %%i in ('findstr /i "^pipe_receiver_file_dir"

Review Comment:
   Good catch! fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616759480


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiverAgent.java:
##
@@ -0,0 +1,197 @@
+/*
+ * 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.commons.utils.TestOnly;
+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.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+public class PipeConsensusReceiverAgent implements ConsensusPipeReceiver {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusReceiverAgent.class);
+
+  private static final Map<
+  Byte,
+  TriFunction>
+  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 --manage--> n replicas' receivers
+  // 1 replica --has--> (n-1) receivers
+  private final Map<
+  ConsensusGroupId, Map>>
+  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> 
consensusPipe2ReciverMap =

Review Comment:
   fixed



##

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616757318


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java:
##
@@ -1669,7 +1671,9 @@ public SettableFuture 
createPipe(CreatePipeStatement createPip
 SettableFuture future = SettableFuture.create();
 
 // Validate pipe name
-if 
(createPipeStatement.getPipeName().startsWith(PipeStaticMeta.SYSTEM_PIPE_PREFIX))
 {
+if (!(DataRegionConsensusImpl.getInstance() instanceof PipeConsensus

Review Comment:
   I know what you mean, this was mostly written ad hoc for quick testing,  I 
will fix it



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616753456


##
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeRequestType.java:
##
@@ -30,6 +30,7 @@ public enum PipeRequestType {
   HANDSHAKE_DATANODE_V1((short) 1),
   HANDSHAKE_CONFIGNODE_V2((short) 50),
   HANDSHAKE_DATANODE_V2((short) 51),
+  PIPE_CONSENSUS_HANDSHAKE((short) 52),

Review Comment:
   no use. Have deleted.



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616752946


##
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 =

Review Comment:
   it seems no use. Can @yschengzi please give some point?



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616750147


##
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 recieverPeer) {

Review Comment:
   fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616749723


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/client/AsyncPipeConsensusServiceClient.java:
##
@@ -0,0 +1,168 @@
+/*
+ * 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.client;
+
+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;
+
+public class AsyncPipeConsensusServiceClient extends 
PipeConsensusIService.AsyncClient
+implements ThriftClient {
+
+  private static final Logger LOGGER =
+  LoggerFactory.getLogger(AsyncPipeConsensusServiceClient.class);

Review Comment:
   it makes sense, I will fixed it to keep consistency with 
`AsyncPipeDataTransferServiceClient`



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616750917


##
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 recieverPeer) {
+this.consensusGroupId = senderPeer.getGroupId();
+this.senderDataNodeId = senderPeer.getNodeId();
+this.receiverDataNodeId = recieverPeer.getNodeId();
+  }
+
+  public ConsensusPipeName(
+  ConsensusGroupId consensusGroupId, int senderDataNodeId, int 
receiverNodeId) {

Review Comment:
   fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616744837


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusServerImpl.java:
##
@@ -0,0 +1,545 @@
+/*
+ * 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.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.client.SyncPipeConsensusServiceClient;
+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.TCheckConsensusPipeCompleteddReq;
+import 
org.apache.iotdb.consensus.pipe.thrift.TCheckConsensusPipeCompleteddResp;
+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 
syncClientManager;
+
+  private ProgressIndex cachedProgressIndex = MinimumProgressIndex.INSTANCE;
+
+  public PipeConsensusServerImpl(
+  Peer thisNode,
+  IStateMachine stateMachine,
+  String storageDir,
+  List configuration,
+  PipeConsensusConfig config,
+  ConsensusPipeManager consensusPipeManager,
+  IClientManager 
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 = 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616744519


##
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java:
##
@@ -137,31 +145,52 @@ public void 
executeFromCalculateInfoForTask(ConfigNodeProcedureEnv env) {
 final ConcurrentMap 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 leaderDataNodeId =

Review Comment:
   > Better be renamed to senderDataNodeId because this is not the "leader".
   
   fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616727438


##
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),
+  TRANSFER_TABLET_RAW((short) 101),
+  TRANSFER_TS_FILE_PIECE((short) 102),
+  TRANSFER_TS_FILE_SEAL((short) 103),
+  TRANSFER_TABLET_BATCH((short) 104),
+  TRANSFER_TABLET_BINARY((short) 105),
+  TRANSFER_TS_FILE_PIECE_WITH_MOD((short) 106),
+  TRANSFER_TS_FILE_SEAL_WITH_MOD((short) 107),
+
+// 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 TYPE_MAP =

Review Comment:
   we intend to use `Short` as key here, since we can get short value from 
thrift reqs directly. So maybe it's more convenient for us to use HashMap here?



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616707988


##
iotdb-core/node-commons/src/assembly/resources/sbin/destroy-datanode.bat:
##
@@ -158,16 +158,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%=
 !"

Review Comment:
   ditto



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616706529


##
iotdb-core/node-commons/src/assembly/resources/sbin/destroy-datanode.sh:
##
@@ -40,6 +40,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/.*=//')

Review Comment:
   pipe_consensus_receiver_file_dirs can be personalized and configured, so it 
may not be under the system folder



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616702691


##
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java:
##
@@ -226,6 +226,9 @@ public class CommonConfig {
   private long pipeMemoryExpanderIntervalSeconds = (long) 3 * 60; // 3Min
   private float pipeLeaderCacheMemoryUsagePercentage = 0.1F;
   private long pipeListeningQueueTransferSnapshotThreshold = 1000;
+  private int pipeConsensusEventBufferSize = 5;
+  private long pipeConsensusEventEnqueueTimeoutInMs = 5000;
+  private long pipeConsensusReceiverMaxWaitingTimeForEventsInMs = 1;

Review Comment:
   fixed



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616702220


##
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java:
##
@@ -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-Connector-Executor-Pool"),

Review Comment:
   fixed.



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616701827


##
iotdb-core/node-commons/src/assembly/resources/conf/iotdb-common.properties:
##
@@ -1046,6 +1046,21 @@ data_replication_factor=1
 # schema_region_ratis_periodic_snapshot_interval=86400
 # data_region_ratis_periodic_snapshot_interval=86400
 
+
+### PipeConsensus Configuration
+
+# Default event buffer size for connector and receiver in pipe consensus
+# DataType: int
+# pipe_consensus_event_buffer_size=5
+
+# Default max wait time for event enqueue in pipe consensus
+# DataType: long
+# pipe_consensus_event_enqueue_timeout_in_ms=5000

Review Comment:
   It makes sense, I will modify it according your suggestions



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616699652


##
iotdb-core/node-commons/src/assembly/resources/conf/iotdb-common.properties:
##
@@ -1046,6 +1046,21 @@ data_replication_factor=1
 # schema_region_ratis_periodic_snapshot_interval=86400
 # data_region_ratis_periodic_snapshot_interval=86400
 
+
+### PipeConsensus Configuration
+
+# Default event buffer size for connector and receiver in pipe consensus

Review Comment:
   sure, maybe `pipe_consensus_pipeline_size`?



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616697727


##
iotdb-protocol/thrift-consensus/src/main/thrift/pipeconsensus.thrift:
##
@@ -0,0 +1,111 @@
+/*
+ * 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 batchReqs
+}
+
+struct TPipeConsensusBatchTransferResp {
+  1:required list 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 consensusGroupId
+  2: required common.TConsensusGroupId targetPeerConsensusGroupId

Review Comment:
   ditto



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616697033


##
iotdb-protocol/thrift-consensus/src/main/thrift/pipeconsensus.thrift:
##
@@ -0,0 +1,111 @@
+/*
+ * 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 batchReqs
+}
+
+struct TPipeConsensusBatchTransferResp {
+  1:required list 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 consensusGroupId
+  2: required common.TConsensusGroupId targetPeerConsensusGroupId
+  3: required common.TEndPoint targetPeerEndPoint
+  4: required i32 targetPeerNodeId
+}
+
+struct TNotifyPeerToCreateConsensusPipeResp {
+  1: required common.TSStatus status
+}
+
+struct TNotifyPeerToDropConsensusPipeReq {
+  1: required common.TConsensusGroupId consensusGroupId
+  2: required common.TConsensusGroupId targetPeerConsensusGroupId

Review Comment:
   as you said, these two vars seems to be same. can @yschengzi please give 
some point?



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616687820


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/service/PipeConsensusRPCService.java:
##
@@ -0,0 +1,120 @@
+/*
+ * 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.ShutdownException;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.InvocationTargetException;
+
+public class PipeConsensusRPCService extends ThriftService implements 
PipeConsensusRPCServiceMBean {
+
+  private static final Logger logger = 
LoggerFactory.getLogger(PipeConsensusRPCService.class);
+
+  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 void waitAndStop(long milliseconds) {
+super.waitAndStop(milliseconds);
+  }
+
+  @Override
+  public void shutdown(long milliseconds) throws ShutdownException {

Review Comment:
   currently, yes.



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616515725


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java:
##
@@ -0,0 +1,529 @@
+/*
+ * 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.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.PipeConsensus;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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.consensus.DataRegionConsensusImpl;
+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.PipeConsensusTabletRawReq;
+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.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.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 TABLET_RAW_SCENARIO = "transfer raw tablet";
+
+  private static final String TSFILE_SCENARIO = "transfer tsfile";
+
+  private static final String 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1616006182


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-28 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615854418


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java:
##
@@ -0,0 +1,1251 @@
+/*
+ * 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.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+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.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 CommonConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-27 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615566875


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeRuntimeAgentGuardian.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.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;
+  private String id;
+
+  @Override
+  public synchronized void start(String id, Runnable guardJob, long 
intervalInSeconds) {
+if (!registered) {
+  LOGGER.info(
+  "Registering periodical job {} with interval in seconds {}.", id, 
intervalInSeconds);

Review Comment:
   This may contain the same information with "PipePeriodicalJobExecutor". This 
may better be deleted or enriched with more precise information.



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-27 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1609920591


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java:
##
@@ -0,0 +1,114 @@
+/*
+ * 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 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_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_HYBRID_VALUE;
+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 ConsensusPipeDispatcher dispatcher;
+  private final ConsensusPipeSelector selector;
+
+  public ConsensusPipeManager(PipeConsensusConfig.Pipe config) {
+this.config = config;
+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.builder()
+.put(EXTRACTOR_KEY, config.getExtractorPluginName())
+.put(
+EXTRACTOR_CONSENSUS_GROUP_ID_KEY,
+consensusPipeName.getConsensusGroupId().toString())
+.put(
+EXTRACTOR_CONSENSUS_SENDER_DATANODE_ID_KEY,

Review Comment:
   Todo: Add non-forwarding parameter here



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-27 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615573623


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java:
##
@@ -132,6 +139,39 @@ 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())
+  
.setSelectorNumOfClientManager(CONF.getSelectorNumOfClientManager())
+  .setThriftServerAwaitTimeForStopService(
+  CONF.getThriftServerAwaitTimeForStopService())
+  .setThriftMaxFrameSize(CONF.getThriftMaxFrameSize())
+  
.setMaxClientNumForEachNode(CONF.getMaxClientNumForEachNode())

Review Comment:
   Be sure to check if these configurations are suitable for your connector 
algorithm, for example, by default there will be 300 concurrent client per 
receiver node



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-27 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615788425


##
iotdb-core/node-commons/src/assembly/resources/sbin/destroy-datanode.bat:
##
@@ -158,26 +158,6 @@ for %%i in (%dn_sync_dir%) do (
 )
 )
 
-for /f  "eol=# tokens=2 delims==" %%i in ('findstr /i "^pipe_receiver_file_dir"

Review Comment:
   This cannot be removed since users can configure this to other directories 
than "dn_system_dir".



##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensus.java:
##
@@ -0,0 +1,485 @@
+/*
+ * 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.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.client.AsyncPipeConsensusServiceClient;
+import org.apache.iotdb.consensus.pipe.client.PipeConsensusClientPool;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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 stateMachineMap 
=
+  new ConcurrentHashMap<>();
+  private final PipeConsensusRPCService rpcService;
+  private final RegisterManager registerManager = new 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-27 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1609917302


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensus.java:
##
@@ -0,0 +1,477 @@
+/*
+ * 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.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.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.client.AsyncPipeConsensusServiceClient;
+import org.apache.iotdb.consensus.pipe.client.PipeConsensusClientPool;
+import org.apache.iotdb.consensus.pipe.client.SyncPipeConsensusServiceClient;
+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 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 
asyncClientManager;
+  private final IClientManager 
syncClientManager;
+
+  public PipeConsensus(ConsensusConfig 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-27 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615573623


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java:
##
@@ -132,6 +139,39 @@ 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())
+  
.setSelectorNumOfClientManager(CONF.getSelectorNumOfClientManager())
+  .setThriftServerAwaitTimeForStopService(
+  CONF.getThriftServerAwaitTimeForStopService())
+  .setThriftMaxFrameSize(CONF.getThriftMaxFrameSize())
+  
.setMaxClientNumForEachNode(CONF.getMaxClientNumForEachNode())

Review Comment:
   Be sure to check if these configurations are suitable for your connector 
algorithm, for example, by default there will be 300 concurrent client per 
receiver node



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-27 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615526255


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java:
##
@@ -582,4 +586,47 @@ public Set 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 getAllConsensusPipe() {
+if (!tryReadLockWithTimeOut(10)) {
+  throw new PipeException("Failed to get all consensus pipe.");

Review Comment:
   What will happen if you throw an exception in "buildConsensusConfig"?



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-27 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615526255


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java:
##
@@ -582,4 +586,47 @@ public Set 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 getAllConsensusPipe() {
+if (!tryReadLockWithTimeOut(10)) {
+  throw new PipeException("Failed to get all consensus pipe.");

Review Comment:
   What will happen if you throw an exception in "buildConsensusConfig"?



##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeRuntimeAgentGuardian.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.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;
+  private String id;
+
+  @Override
+  public synchronized void start(String id, Runnable guardJob, long 
intervalInSeconds) {
+if (!registered) {
+  LOGGER.info(
+  "Registering periodical job {} with interval in seconds {}.", id, 
intervalInSeconds);

Review Comment:
   This may contain the same information with "PipePeriodicalJobExecutor". This 
may better be deleted or enriched with more precise information.



##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java:
##
@@ -132,6 +139,39 @@ 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())
+  
.setSelectorNumOfClientManager(CONF.getSelectorNumOfClientManager())
+  .setThriftServerAwaitTimeForStopService(
+  CONF.getThriftServerAwaitTimeForStopService())
+  .setThriftMaxFrameSize(CONF.getThriftMaxFrameSize())
+  
.setMaxClientNumForEachNode(CONF.getMaxClientNumForEachNode())

Review Comment:
   Be sure to check if these configurations is suitable for your connector 
algorithm, for example, by default there will be 300 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-27 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615451552


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeRuntimeAgentGuardian.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.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;
+  private String id;
+
+  @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;
+  this.id = id;
+  PipeAgent.runtime().registerPeriodicalJob(id, guardJob, 
intervalInSeconds);
+}
+  }
+
+  @Override
+  public synchronized void stop() {
+LOGGER.info("Try to stop periodical job {}, but it will not stop.", id);
+// TODO: add deregister in PipePeriodicalJobExecutor

Review Comment:
   Currently there are no “deregister” in pipePeriodicalJobExecutor



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-27 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615511598


##
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java:
##
@@ -137,31 +145,52 @@ public void 
executeFromCalculateInfoForTask(ConfigNodeProcedureEnv env) {
 final ConcurrentMap 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 leaderDataNodeId =

Review Comment:
   Remember to enrich the "RouteBalancer" and check if there are other places 
mentioning consensus name constant 
   ;-)



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-26 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615509875


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java:
##
@@ -1669,7 +1671,9 @@ public SettableFuture 
createPipe(CreatePipeStatement createPip
 SettableFuture future = SettableFuture.create();
 
 // Validate pipe name
-if 
(createPipeStatement.getPipeName().startsWith(PipeStaticMeta.SYSTEM_PIPE_PREFIX))
 {
+if (!(DataRegionConsensusImpl.getInstance() instanceof PipeConsensus

Review Comment:
   This is the normal path of pipe creation, thus it cannot be restricted to 
"PipeConsensus". If you want to let the consensus pipe pass, you can call the 
"configNodeClient" directly or just prohibit the subscription pipe. The former 
one is more recommended because it can avoid users from manully creating 
consensus pipes.



##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java:
##
@@ -1669,7 +1671,9 @@ public SettableFuture 
createPipe(CreatePipeStatement createPip
 SettableFuture future = SettableFuture.create();
 
 // Validate pipe name
-if 
(createPipeStatement.getPipeName().startsWith(PipeStaticMeta.SYSTEM_PIPE_PREFIX))
 {
+if (!(DataRegionConsensusImpl.getInstance() instanceof PipeConsensus

Review Comment:
   ……



##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ConsensusPipeDataNodeRuntimeAgentGuardian.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.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;
+  private String id;
+
+  @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;
+  this.id = id;
+  PipeAgent.runtime().registerPeriodicalJob(id, guardJob, 
intervalInSeconds);
+}
+  }
+
+  @Override
+  public synchronized void stop() {
+LOGGER.info("Try to stop periodical job {}, but it will not stop.", id);
+// TODO: add deregister in PipePeriodicalJobExecutor

Review Comment:
   Complete it before merge



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-26 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615408916


##
iotdb-core/consensus/pom.xml:
##
@@ -117,12 +122,6 @@
 com.google.code.findbugs
 jsr305
 
-
-org.apache.tsfile
-tsfile
-${tsfile.version}

Review Comment:
   Why delete it?



##
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java:
##
@@ -137,31 +145,52 @@ public void 
executeFromCalculateInfoForTask(ConfigNodeProcedureEnv env) {
 final ConcurrentMap 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 leaderDataNodeId =

Review Comment:
   Better be renamed to senderDataNodeId because this is not the "leader".



##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/client/AsyncPipeConsensusServiceClient.java:
##
@@ -0,0 +1,168 @@
+/*
+ * 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.client;
+
+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;
+
+public class AsyncPipeConsensusServiceClient extends 
PipeConsensusIService.AsyncClient
+implements ThriftClient {
+
+  private static final Logger LOGGER =
+  LoggerFactory.getLogger(AsyncPipeConsensusServiceClient.class);

Review Comment:
   Since there are multiple clients runtime, ignorance of the "id" in 
AsyncPipeDataTransferServiceClient may make the logger harder to comprehend and 
reduce consistency.



##
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 

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-26 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615400577


##
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java:
##
@@ -137,31 +145,52 @@ public void 
executeFromCalculateInfoForTask(ConfigNodeProcedureEnv env) {
 final ConcurrentMap 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 leaderDataNodeId =
+  Integer.parseInt(
+  createPipeRequest
+  .getExtractorAttributes()
+  .get(EXTRACTOR_CONSENSUS_SENDER_DATANODE_ID_KEY));
+  consensusGroupIdToTaskMetaMap.put(
+  groupId.getId(),
+  new PipeTaskMeta(
+  new RecoverProgressIndex(leaderDataNodeId, new 
SimpleProgressIndex(0, 0)),
+  leaderDataNodeId));
+} 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(

Review Comment:
   That is pipe related codes and has nothing to do with consensus... Only the 
codes managed by the "if" is related.



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-26 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615399756


##
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 {}

Review Comment:
   In pipe, all of the plugins are list both in "common" and "db". The former 
one is to enrich the enum name because the latter one cannot be seen here. Here 
we just continue to use that to reserve consistency.



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-26 Thread via GitHub


Caideyipi commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615398949


##
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 {

Review Comment:
   This non-thrift structure is already used by pipe... First because there are 
to many "req"s, and the second advantage is to easily reused by the air gap 
socket. The most important reason is that we need the receiver to open the 
compression function on 6667, which may not take positive effects. Besides, now 
we can already have backwards and forwards compatibility by "version" and 
"type".



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

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

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



Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-26 Thread via GitHub


OneSizeFitsQuorum commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615148198


##
iotdb-protocol/thrift-consensus/src/main/thrift/pipeconsensus.thrift:
##
@@ -0,0 +1,111 @@
+/*
+ * 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 batchReqs
+}
+
+struct TPipeConsensusBatchTransferResp {
+  1:required list 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 consensusGroupId
+  2: required common.TConsensusGroupId targetPeerConsensusGroupId

Review Comment:
   targetPeerConsensusGroupId and consensusGroupId always seems to be same? 
What's the difference with TBuildSyncLogChannelReq?



##
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java:
##
@@ -226,6 +226,9 @@ public class CommonConfig {
   private long pipeMemoryExpanderIntervalSeconds = (long) 3 * 60; // 3Min
   private float pipeLeaderCacheMemoryUsagePercentage = 0.1F;
   private long pipeListeningQueueTransferSnapshotThreshold = 1000;
+  private int pipeConsensusEventBufferSize = 5;
+  private long pipeConsensusEventEnqueueTimeoutInMs = 5000;
+  private long pipeConsensusReceiverMaxWaitingTimeForEventsInMs = 1;

Review Comment:
   not equal in conf file



##
iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java:
##
@@ -137,31 +145,52 @@ public void 
executeFromCalculateInfoForTask(ConfigNodeProcedureEnv env) {
 final ConcurrentMap 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 leaderDataNodeId =
+  Integer.parseInt(
+  createPipeRequest
+  .getExtractorAttributes()
+  .get(EXTRACTOR_CONSENSUS_SENDER_DATANODE_ID_KEY));
+  

Re: [PR] PipeConsensus: complete consensus prodedure and pipe components with new thrift service [iotdb]

2024-05-26 Thread via GitHub


Pengzna commented on code in PR #12355:
URL: https://github.com/apache/iotdb/pull/12355#discussion_r1615148745


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionConnectorConstructor.java:
##
@@ -93,5 +94,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_CONNECTOR.getPipePluginName(),

Review Comment:
   fixed



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

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

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



  1   2   >