Re: [PR] PipeConsensus: initialize thrift service & connector and receiver logic [iotdb]

2024-05-17 Thread via GitHub


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


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/PipeConsensusServerImpl.java:
##
@@ -0,0 +1,377 @@
+/*
+ * 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.TSStatus;
+import org.apache.iotdb.commons.consensus.index.ComparableConsensusRequest;
+import org.apache.iotdb.commons.consensus.index.ProgressIndex;
+import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex;
+import org.apache.iotdb.commons.pipe.task.meta.PipeStatus;
+import org.apache.iotdb.consensus.IStateMachine;
+import org.apache.iotdb.consensus.common.DataSet;
+import org.apache.iotdb.consensus.common.Peer;
+import org.apache.iotdb.consensus.common.request.IConsensusRequest;
+import org.apache.iotdb.consensus.config.PipeConsensusConfig;
+import org.apache.iotdb.consensus.exception.ConsensusGroupModifyPeerException;
+import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeManager;
+import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeName;
+import org.apache.iotdb.consensus.pipe.consensuspipe.ProgressIndexManager;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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;
+
+import static 
org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeManager.getConsensusPipeName;
+
+/** PipeConsensusServerImpl is a consensus server implementation for pipe 
consensus. */
+public class PipeConsensusServerImpl {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusServerImpl.class);
+
+  private final Peer thisNode;
+  private final IStateMachine stateMachine;
+  private final Lock stateMachineLock = new ReentrantLock();
+  private final PipeConsensusPeerManager peerManager;
+  private final PipeConsensusConfig config;
+  private final AtomicBoolean active;
+  private final AtomicBoolean isStarted;
+  private final String consensusGroupId;
+  private final ConsensusPipeManager consensusPipeManager;
+  private final ProgressIndexManager progressIndexManager;
+
+  private ProgressIndex cachedProgressIndex = MinimumProgressIndex.INSTANCE;
+
+  public PipeConsensusServerImpl(
+  Peer thisNode,
+  IStateMachine stateMachine,
+  String storageDir,
+  List configuration,
+  PipeConsensusConfig config,
+  ConsensusPipeManager consensusPipeManager)
+  throws IOException {
+this.thisNode = thisNode;
+this.stateMachine = stateMachine;
+this.peerManager = new PipeConsensusPeerManager(storageDir, configuration);
+this.config = config;
+this.active = new AtomicBoolean(true);
+this.isStarted = new AtomicBoolean(false);
+this.consensusGroupId = thisNode.getGroupId().toString();
+this.consensusPipeManager = consensusPipeManager;
+this.progressIndexManager = config.getPipe().getProgressIndexManager();
+
+if (configuration.isEmpty()) {
+  peerManager.recover();
+} else {
+  // create consensus pipes
+  configuration.remove(thisNode);
+  final List successfulPips = createConsensusPipes(configuration);

Review Comment:
   successfulPipes



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

Re: [PR] PipeConsensus: initialize thrift service & connector and receiver logic [iotdb]

2024-05-17 Thread via GitHub


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


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/PipeConsensusConfig.java:
##
@@ -0,0 +1,345 @@
+package org.apache.iotdb.consensus.config;

Review Comment:
   fixed, push it later to avoid code conflict



-- 
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: initialize thrift service & connector and receiver logic [iotdb]

2024-05-17 Thread via GitHub


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


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/PipeConsensusConfig.java:
##
@@ -0,0 +1,345 @@
+package org.apache.iotdb.consensus.config;

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: initialize thrift service & connector and receiver logic [iotdb]

2024-05-17 Thread via GitHub


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


##
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/PipeConsensusConfig.java:
##
@@ -0,0 +1,345 @@
+package org.apache.iotdb.consensus.config;

Review Comment:
   license?



-- 
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: initialize thrift service & connector and receiver logic [iotdb]

2024-04-17 Thread via GitHub


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


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,117 @@
+/*
+ * 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.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import 
org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector;
+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.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+
+public class PipeConsensusAsyncConnector extends IoTDBDataRegionAsyncConnector 
{
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusAsyncConnector.class);
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into transferBuffer 
failed, because transferBuffer is full";
+
+  private static final CommonConfig COMMON_CONFIG = 
CommonDescriptor.getInstance().getConfig();
+
+  private final BlockingQueue transferBuffer =
+  new 
LinkedBlockingDeque<>(COMMON_CONFIG.getPipeConsensusEventBufferSize());
+
+  /** Add an event to transferBuffer, whose events will be asynchronizedly 
transfer to receiver. */
+  private boolean addEvent2Buffer(Event event) {

Review Comment:
   In current design, we will increase the reference count when transfer method 
and decrease it in callback method of AsyncThrift RPC. but I think it's better 
to manage reference count in the way you said, since it's semantically clearer. 
 I will try to fix it as you suggested~



-- 
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: initialize thrift service & connector and receiver logic [iotdb]

2024-04-17 Thread via GitHub


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


##
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java:
##
@@ -284,6 +289,21 @@ public void onCommitted() {
 }
   }
 
+  /** Used for pipeConsensus */
+  @Override
+  public boolean equals(Object o) {
+if (this == o) {
+  return true;
+}
+if (o == null || getClass() != o.getClass()) {
+  return false;
+}
+EnrichedEvent otherEvent = (EnrichedEvent) o;
+return committerKey.equals(otherEvent.committerKey)

Review Comment:
   Nice 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: initialize thrift service & connector and receiver logic [iotdb]

2024-04-17 Thread via GitHub


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


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,117 @@
+/*
+ * 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.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import 
org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector;
+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.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+
+public class PipeConsensusAsyncConnector extends IoTDBDataRegionAsyncConnector 
{
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusAsyncConnector.class);
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into transferBuffer 
failed, because transferBuffer is full";
+
+  private static final CommonConfig COMMON_CONFIG = 
CommonDescriptor.getInstance().getConfig();
+
+  private final BlockingQueue transferBuffer =
+  new 
LinkedBlockingDeque<>(COMMON_CONFIG.getPipeConsensusEventBufferSize());
+
+  /** Add an event to transferBuffer, whose events will be asynchronizedly 
transfer to receiver. */
+  private boolean addEvent2Buffer(Event event) {

Review Comment:
   In current design, we will increase the reference count when transfer method 
and decrease it in callback method of AsyncThrift RPC. but I think it's better 
to manage reference count in the way you said, since it's semantically clearer. 
 I will try to fix it as you suggested~



-- 
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: initialize thrift service & connector and receiver logic [iotdb]

2024-04-17 Thread via GitHub


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


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,117 @@
+/*
+ * 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.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import 
org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector;
+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.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+
+public class PipeConsensusAsyncConnector extends IoTDBDataRegionAsyncConnector 
{
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusAsyncConnector.class);
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into transferBuffer 
failed, because transferBuffer is full";
+
+  private static final CommonConfig COMMON_CONFIG = 
CommonDescriptor.getInstance().getConfig();
+
+  private final BlockingQueue transferBuffer =
+  new 
LinkedBlockingDeque<>(COMMON_CONFIG.getPipeConsensusEventBufferSize());
+
+  /** Add an event to transferBuffer, whose events will be asynchronizedly 
transfer to receiver. */
+  private boolean addEvent2Buffer(Event event) {

Review Comment:
   > Do we need to increase the reference count of the event when adding it to 
buffer? And also decrease reference count when removing?
   
   @DanielWang2035 In current design, we will increase the reference count when 
transfer method and decrease it in callback method of AsyncThrift RPC. but I 
think it's better to manage reference count in the way you said, since it's 
semantically clearer. I will try to fix it as you suggested~



-- 
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: initialize thrift service & connector and receiver logic [iotdb]

2024-04-17 Thread via GitHub


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


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,113 @@
+/*
+ * 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.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import 
org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector;
+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.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+
+public class PipeConsensusAsyncConnector extends IoTDBDataRegionAsyncConnector 
{
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusAsyncConnector.class);
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into transferBuffer 
failed, because transferBuffer is full";
+
+  private static final CommonConfig COMMON_CONFIG = 
CommonDescriptor.getInstance().getConfig();
+
+  private final BlockingQueue transferBuffer =
+  new 
LinkedBlockingDeque<>(COMMON_CONFIG.getPipeConsensusEventBufferSize());
+
+  /** Add an event to transferBuffer, whose events will be asynchronizedly 
transfer to receiver. */
+  private boolean addEvent2Buffer(Event event) {
+try {
+  LOGGER.debug(

Review Comment:
   > It's better to print the debug message after `LOGGER.isDebugEnabled()`. 
Since the `transferBuffer.size()` and 
`COMMON_CONFIG.getPipeConsensusEventBufferSize()` will be computed and 
concatenated to the log string otherwise. See 
https://stackoverflow.com/questions/963492/in-log4j-does-checking-isdebugenabled-before-logging-improve-performance.
   
   nice 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: initialize thrift service & connector and receiver logic [iotdb]

2024-04-17 Thread via GitHub


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


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,117 @@
+/*
+ * 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.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import 
org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector;
+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.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+
+public class PipeConsensusAsyncConnector extends IoTDBDataRegionAsyncConnector 
{
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusAsyncConnector.class);
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into transferBuffer 
failed, because transferBuffer is full";
+
+  private static final CommonConfig COMMON_CONFIG = 
CommonDescriptor.getInstance().getConfig();
+
+  private final BlockingQueue transferBuffer =
+  new 
LinkedBlockingDeque<>(COMMON_CONFIG.getPipeConsensusEventBufferSize());
+
+  /** Add an event to transferBuffer, whose events will be asynchronizedly 
transfer to receiver. */
+  private boolean addEvent2Buffer(Event event) {

Review Comment:
   > Do we need to increase the reference count of the event when adding it to 
buffer? And also decrease reference count when removing?
   
   @DanielWang2035 In current design, we will increase the reference count when 
transfer method and decrease it in callback method of AsyncThrift RPC. but I 
think it's better to manage reference count in the way you said, since it's 
semantically clearer. I will try to fix it as you suggested.



-- 
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: initialize thrift service & connector and receiver logic [iotdb]

2024-04-17 Thread via GitHub


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


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,117 @@
+/*
+ * 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.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import 
org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector;
+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.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+
+public class PipeConsensusAsyncConnector extends IoTDBDataRegionAsyncConnector 
{
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusAsyncConnector.class);
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into transferBuffer 
failed, because transferBuffer is full";
+
+  private static final CommonConfig COMMON_CONFIG = 
CommonDescriptor.getInstance().getConfig();
+
+  private final BlockingQueue transferBuffer =
+  new 
LinkedBlockingDeque<>(COMMON_CONFIG.getPipeConsensusEventBufferSize());
+
+  /** Add an event to transferBuffer, whose events will be asynchronizedly 
transfer to receiver. */
+  private boolean addEvent2Buffer(Event event) {

Review Comment:
   Do we need to increase the reference count of the event when adding it to 
buffer? And also decrease reference count when removing?



-- 
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: initialize thrift service & connector and receiver logic [iotdb]

2024-04-17 Thread via GitHub


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


##
iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java:
##
@@ -284,6 +289,21 @@ public void onCommitted() {
 }
   }
 
+  /** Used for pipeConsensus */
+  @Override
+  public boolean equals(Object o) {
+if (this == o) {
+  return true;
+}
+if (o == null || getClass() != o.getClass()) {
+  return false;
+}
+EnrichedEvent otherEvent = (EnrichedEvent) o;
+return committerKey.equals(otherEvent.committerKey)

Review Comment:
   Maybe use Objects.equals(committerKey, otherEvent.committerKey)?



-- 
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: initialize thrift service & connector and receiver logic [iotdb]

2024-04-16 Thread via GitHub


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


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,113 @@
+/*
+ * 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.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import 
org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector;
+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.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+
+public class PipeConsensusAsyncConnector extends IoTDBDataRegionAsyncConnector 
{
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusAsyncConnector.class);
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into transferBuffer 
failed, because transferBuffer is full";
+
+  private static final CommonConfig COMMON_CONFIG = 
CommonDescriptor.getInstance().getConfig();
+
+  private final BlockingQueue transferBuffer =
+  new 
LinkedBlockingDeque<>(COMMON_CONFIG.getPipeConsensusEventBufferSize());
+
+  /** Add an event to transferBuffer, whose events will be asynchronizedly 
transfer to receiver. */
+  private boolean addEvent2Buffer(Event event) {
+try {
+  LOGGER.debug(

Review Comment:
   nice 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: initialize thrift service & connector and receiver logic [iotdb]

2024-04-16 Thread via GitHub


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


##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,113 @@
+/*
+ * 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.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import 
org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector;
+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.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+
+public class PipeConsensusAsyncConnector extends IoTDBDataRegionAsyncConnector 
{
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusAsyncConnector.class);
+
+  private static final String ENQUEUE_EXCEPTION_MSG =
+  "Timeout: PipeConsensusConnector offers an event into transferBuffer 
failed, because transferBuffer is full";
+
+  private static final CommonConfig COMMON_CONFIG = 
CommonDescriptor.getInstance().getConfig();
+
+  private final BlockingQueue transferBuffer =
+  new 
LinkedBlockingDeque<>(COMMON_CONFIG.getPipeConsensusEventBufferSize());
+
+  /** Add an event to transferBuffer, whose events will be asynchronizedly 
transfer to receiver. */
+  private boolean addEvent2Buffer(Event event) {
+try {
+  LOGGER.debug(

Review Comment:
   It's better to print the debug message after `LOGGER.isDebugEnabled()`. 
Since the `transferBuffer.size()` and 
`COMMON_CONFIG.getPipeConsensusEventBufferSize()` will be computed and 
concatenated to the log string otherwise. See 
https://stackoverflow.com/questions/963492/in-log4j-does-checking-isdebugenabled-before-logging-improve-performance.
 



##
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java:
##
@@ -0,0 +1,113 @@
+/*
+ * 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.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import 
org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector;
+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.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+
+public class PipeConsensusAsyncConnector extends IoTDBDataRegionAsyncConnector 
{
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PipeConsensusAsyncConnector.class);
+
+  private static final