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<ConsensusGroupId, PipeConsensusServerImpl> stateMachineMap 
=
+      new ConcurrentHashMap<>();
+  private final PipeConsensusRPCService rpcService;
+  private final RegisterManager registerManager = new RegisterManager();
+  private final ReentrantLock stateMachineMapLock = new ReentrantLock();
+  private final PipeConsensusConfig config;
+  private final ConsensusPipeManager consensusPipeManager;
+  private final ConsensusPipeGuardian consensusPipeGuardian;
+  private final IClientManager<TEndPoint, AsyncPipeConsensusServiceClient> 
asyncClientManager;
+  private final IClientManager<TEndPoint, SyncPipeConsensusServiceClient> 
syncClientManager;
+
+  public PipeConsensus(ConsensusConfig config, IStateMachine.Registry 
registry) {
+    this.thisNode = config.getThisNodeEndPoint();
+    this.thisNodeId = config.getThisNodeId();
+    this.storageDir = new File(config.getStorageDir());
+    this.config = config.getPipeConsensusConfig();
+    this.registry = registry;
+    this.rpcService = new PipeConsensusRPCService(thisNode, 
config.getPipeConsensusConfig());
+    this.consensusPipeManager =
+        new ConsensusPipeManager(
+            config.getPipeConsensusConfig().getPipe(),
+            config.getPipeConsensusConfig().getReplicateMode());
+    this.consensusPipeGuardian =
+        config.getPipeConsensusConfig().getPipe().getConsensusPipeGuardian();
+    this.asyncClientManager =
+        new IClientManager.Factory<TEndPoint, 
AsyncPipeConsensusServiceClient>()
+            .createClientManager(
+                new 
PipeConsensusClientPool.AsyncPipeConsensusServiceClientPoolFactory(
+                    config.getPipeConsensusConfig()));
+    this.syncClientManager =
+        new IClientManager.Factory<TEndPoint, SyncPipeConsensusServiceClient>()
+            .createClientManager(
+                new 
PipeConsensusClientPool.SyncPipeConsensusServiceClientPoolFactory(
+                    config.getPipeConsensusConfig()));
+  }
+
+  @Override
+  public synchronized void start() throws IOException {
+    initAndRecover();
+
+    rpcService.initAsyncedServiceImpl(new 
PipeConsensusRPCServiceProcessor(this, config.getPipe()));
+    try {
+      registerManager.register(rpcService);
+    } catch (StartupException e) {
+      throw new IOException(e);
+    }
+
+    consensusPipeGuardian.start(
+        CONSENSUS_PIPE_GUARDIAN_TASK_ID,
+        this::checkAllConsensusPipe,
+        config.getPipe().getConsensusPipeGuardJobIntervalInSeconds());
+  }
+
+  private void initAndRecover() throws IOException {
+    if (!storageDir.exists()) {
+      if (!storageDir.mkdirs()) {
+        LOGGER.warn("Unable to create consensus dir at {}", storageDir);
+        throw new IOException(String.format("Unable to create consensus dir at 
%s", storageDir));
+      }
+    } else {
+      try (DirectoryStream<Path> stream = 
Files.newDirectoryStream(storageDir.toPath())) {
+        for (Path path : stream) {
+          ConsensusGroupId consensusGroupId = 
parsePeerDir(path.getFileName().toString());
+          PipeConsensusServerImpl consensus =
+              new PipeConsensusServerImpl(
+                  new Peer(consensusGroupId, thisNodeId, thisNode),
+                  registry.apply(consensusGroupId),
+                  path.toString(),
+                  new ArrayList<>(),
+                  config,
+                  consensusPipeManager,
+                  syncClientManager);
+          stateMachineMap.put(consensusGroupId, consensus);
+          consensus.start(true);
+        }
+      }
+    }
+  }
+
+  @Override
+  public synchronized void stop() {
+    asyncClientManager.close();
+    syncClientManager.close();
+    registerManager.deregisterAll();
+    consensusPipeGuardian.stop();
+    
stateMachineMap.values().parallelStream().forEach(PipeConsensusServerImpl::stop);
+  }
+
+  public void checkAllConsensusPipe() {
+    final Map<ConsensusGroupId, Map<ConsensusPipeName, PipeStatus>> 
existedPipes =
+        consensusPipeManager.getAllConsensusPipe().entrySet().stream()
+            .filter(entry -> entry.getKey().getSenderDataNodeId() == 
thisNodeId)
+            .collect(
+                Collectors.groupingBy(
+                    entry -> entry.getKey().getConsensusGroupId(),
+                    Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+    try {
+      stateMachineMapLock.lock();
+      stateMachineMap.entrySet().parallelStream()
+          .forEach(
+              entry ->
+                  entry
+                      .getValue()
+                      .checkConsensusPipe(
+                          existedPipes.getOrDefault(entry.getKey(), 
ImmutableMap.of())));
+      existedPipes.entrySet().stream()
+          .filter(entry -> !stateMachineMap.containsKey(entry.getKey()))
+          .flatMap(entry -> entry.getValue().keySet().stream())
+          .forEach(
+              consensusPipeName -> {
+                try {
+                  LOGGER.warn(
+                      "{} drop consensus pipe [{}]",
+                      consensusPipeName.getConsensusGroupId(),
+                      consensusPipeName);
+                  consensusPipeManager.updateConsensusPipe(consensusPipeName, 
PipeStatus.DROPPED);
+                } catch (Exception e) {
+                  LOGGER.warn(
+                      "{} cannot drop consensus pipe [{}]",
+                      consensusPipeName.getConsensusGroupId(),
+                      consensusPipeName,
+                      e);
+                }
+              });
+    } finally {
+      stateMachineMapLock.unlock();
+    }
+  }
+
+  @Override
+  public TSStatus write(ConsensusGroupId groupId, IConsensusRequest request)
+      throws ConsensusException {
+    final PipeConsensusServerImpl impl =
+        Optional.ofNullable(stateMachineMap.get(groupId))
+            .orElseThrow(() -> new ConsensusGroupNotExistException(groupId));
+    if (impl.isReadOnly()) {
+      return StatusUtils.getStatus(TSStatusCode.SYSTEM_READ_ONLY);
+    } else if (!impl.isActive()) {
+      return RpcUtils.getStatus(
+          TSStatusCode.WRITE_PROCESS_REJECT,
+          "peer is inactive and not ready to receive sync log request.");
+    } else {
+      return impl.write(request);
+    }
+  }
+
+  @Override
+  public DataSet read(ConsensusGroupId groupId, IConsensusRequest request)
+      throws ConsensusException {
+    return Optional.ofNullable(stateMachineMap.get(groupId))
+        .orElseThrow(() -> new ConsensusGroupNotExistException(groupId))
+        .read(request);
+  }
+
+  private String buildPeerDir(ConsensusGroupId groupId) {

Review Comment:
   Better rename it to "getPeerDir" since this does not make directory and just 
return a path.



##########
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<PipeConsensus, ConsensusGroupId, ConsensusPipeName, 
PipeConsensusReceiver>>
+      RECEIVER_CONSTRUCTORS = new HashMap<>();
+
+  private final int thisNodeId = 
IoTDBDescriptor.getInstance().getConfig().getDataNodeId();
+
+  // For each consensus Pipe task, there is an independent receiver. So for 
every replica, it has
+  // (n-1) receivers, n is the num of replicas.
+  // 1 DataNode --has--> 1 PipeConsensusReceiverAgent & n replicas
+  // 1 PipeConsensusReceiverAgent --manage--> n replicas' receivers
+  // 1 replica --has--> (n-1) receivers
+  private final Map<
+          ConsensusGroupId, Map<ConsensusPipeName, 
AtomicReference<PipeConsensusReceiver>>>
+      replicaReceiverMap = new ConcurrentHashMap<>();
+
+  private PipeConsensus pipeConsensus;
+
+  public PipeConsensusReceiverAgent() {
+    RECEIVER_CONSTRUCTORS.put(
+        PipeConsensusRequestVersion.VERSION_1.getVersion(), 
PipeConsensusReceiver::new);
+  }
+
+  public void initConsensusInRuntime() {
+    IConsensus consensus = DataRegionConsensusImpl.getInstance();
+    // If DataRegion uses PipeConsensus
+    if (consensus instanceof PipeConsensus) {
+      this.pipeConsensus = (PipeConsensus) consensus;
+    }
+    // If DataRegion uses other consensus such as IoTConsensus
+    else {
+      this.pipeConsensus = null;
+    }
+  }
+
+  @Override
+  public TPipeConsensusTransferResp receive(TPipeConsensusTransferReq req) {
+    final byte reqVersion = req.getVersion();
+    if (RECEIVER_CONSTRUCTORS.containsKey(reqVersion)) {
+      final ConsensusGroupId consensusGroupId =
+          
ConsensusGroupId.Factory.createFromTConsensusGroupId(req.getConsensusGroupId());
+      return getReceiver(consensusGroupId, req.getDataNodeId(), 
reqVersion).receive(req);
+    } else {
+      final TSStatus status =
+          RpcUtils.getStatus(
+              TSStatusCode.PIPE_CONSENSUS_VERSION_ERROR,
+              String.format("Unknown PipeConsensusRequestVersion %s.", 
reqVersion));
+      LOGGER.warn(
+          "PipeConsensus: Unknown PipeConsensusRequestVersion, response status 
= {}.", status);
+      return new TPipeConsensusTransferResp(status);
+    }
+  }
+
+  private PipeConsensusReceiver getReceiver(
+      ConsensusGroupId consensusGroupId, int leaderDataNodeId, byte 
reqVersion) {
+    // 1. Route to given consensusGroup's receiver map
+    Map<ConsensusPipeName, AtomicReference<PipeConsensusReceiver>> 
consensusPipe2ReciverMap =

Review Comment:
   receiver....



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to