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<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());
+    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() throws IOException {
+    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) {
+    return storageDir + File.separator + groupId.getType().getValue() + "_" + 
groupId.getId();
+  }
+
+  private ConsensusGroupId parsePeerDir(String dirName) {
+    String[] items = dirName.split("_");
+    return ConsensusGroupId.Factory.create(Integer.parseInt(items[0]), 
Integer.parseInt(items[1]));
+  }
+
+  @Override
+  public void createLocalPeer(ConsensusGroupId groupId, List<Peer> peers)
+      throws ConsensusException {
+    final int consensusGroupSize = peers.size();
+    if (consensusGroupSize == 0) {
+      throw new IllegalPeerNumException(consensusGroupSize);
+    }
+    if (!peers.contains(new Peer(groupId, thisNodeId, thisNode))) {
+      throw new IllegalPeerEndpointException(thisNode, peers);
+    }
+    if (stateMachineMap.containsKey(groupId)) {
+      throw new ConsensusGroupAlreadyExistException(groupId);
+    }
+
+    try {
+      stateMachineMapLock.lock();
+
+      final String path = buildPeerDir(groupId);
+      if (!new File(path).mkdirs()) {
+        LOGGER.warn("Unable to create consensus dir for group {} at {}", 
groupId, path);
+        throw new ConsensusException(
+            String.format("Unable to create consensus dir for group %s", 
groupId));
+      }
+
+      PipeConsensusServerImpl consensus =
+          new PipeConsensusServerImpl(
+              new Peer(groupId, thisNodeId, thisNode),
+              registry.apply(groupId),
+              path,
+              peers,
+              config,
+              consensusPipeManager,
+              syncClientManager);
+      stateMachineMap.put(groupId, consensus);
+      consensus.start(false); // pipe will start after creating
+    } catch (IOException e) {
+      LOGGER.warn("Cannot create local peer for group {} with peers {}", 
groupId, peers, e);
+      throw new ConsensusException(e);
+    } finally {
+      stateMachineMapLock.unlock();
+    }
+  }
+
+  @Override
+  public void deleteLocalPeer(ConsensusGroupId groupId) throws 
ConsensusException {
+    if (!stateMachineMap.containsKey(groupId)) {
+      throw new ConsensusGroupNotExistException(groupId);
+    }
+
+    try {
+      stateMachineMapLock.lock();
+
+      final PipeConsensusServerImpl consensus = stateMachineMap.get(groupId);
+      consensus.clear();
+
+      FileUtils.deleteFileOrDirectory(new File(buildPeerDir(groupId)));
+    } catch (IOException e) {
+      LOGGER.warn("Cannot delete local peer for group {}", groupId, e);
+      throw new ConsensusException(e);
+    } finally {
+      stateMachineMapLock.unlock();
+    }
+  }
+
+  @Override
+  public void addRemotePeer(ConsensusGroupId groupId, Peer peer) throws 
ConsensusException {
+    PipeConsensusServerImpl impl =
+        Optional.ofNullable(stateMachineMap.get(groupId))
+            .orElseThrow(() -> new ConsensusGroupNotExistException(groupId));
+    if (impl.containsPeer(peer)) {
+      throw new PeerAlreadyInConsensusGroupException(groupId, peer);
+    }
+    try {
+      // step 1: inactive new Peer to prepare for following steps
+      LOGGER.info("[{}] inactivate new peer: {}", CLASS_NAME, peer);
+      impl.setRemotePeerActive(peer, false);
+
+      // step 2: notify all the other Peers to create consensus pipes to 
newPeer
+      LOGGER.info("[{}] notify current peers to create consensus pipes...", 
CLASS_NAME);
+      impl.notifyPeersToCreateConsensusPipes(peer);
+
+      // step 3: wait until all the other Peers finish transferring
+      LOGGER.info("[{}] wait until all the other peers finish 
transferring...", CLASS_NAME);
+      impl.waitPeersToTargetPeerTransmissionCompleted(peer);
+
+      // step 4: active new Peer
+      LOGGER.info("[{}] activate new peer...", CLASS_NAME);
+      impl.setRemotePeerActive(peer, true);
+    } catch (ConsensusGroupModifyPeerException e) {
+      try {
+        LOGGER.info("[{}] add remote peer failed, automatic cleanup side 
effects...", CLASS_NAME);
+
+        // roll back
+        impl.notifyPeersToDropConsensusPipe(peer);
+
+      } catch (ConsensusGroupModifyPeerException mpe) {
+        LOGGER.error(
+            "[{}] failed to cleanup side effects after failed to add remote 
peer", CLASS_NAME, mpe);
+      }
+      throw new ConsensusException(e);
+    }
+  }
+
+  @Override
+  public void removeRemotePeer(ConsensusGroupId groupId, Peer peer) throws 
ConsensusException {
+    PipeConsensusServerImpl impl =
+        Optional.ofNullable(stateMachineMap.get(groupId))
+            .orElseThrow(() -> new ConsensusGroupNotExistException(groupId));
+    if (!impl.containsPeer(peer)) {
+      throw new PeerNotInConsensusGroupException(groupId, peer.toString());
+    }
+
+    try {
+      // let other peers remove the consensus pipe to target peer
+      impl.notifyPeersToDropConsensusPipe(peer);
+      // let target peer reject new write
+      impl.setRemotePeerActive(peer, false);
+      // wait its consensus pipes to complete
+      impl.waitTargetPeerToPeersTransmissionCompleted(peer);
+    } catch (ConsensusGroupModifyPeerException e) {
+      throw new ConsensusException(e.getMessage());
+    }
+  }
+
+  @Override
+  public void resetPeerList(ConsensusGroupId groupId, List<Peer> correctPeers)
+      throws ConsensusException {
+    PipeConsensusServerImpl impl =
+        Optional.ofNullable(stateMachineMap.get(groupId))
+            .orElseThrow(() -> new ConsensusGroupNotExistException(groupId));
+    if (!correctPeers.contains(new Peer(groupId, thisNodeId, thisNode))) {
+      LOGGER.warn(
+          "[RESET PEER LIST] Local peer is not in the correct configuration, 
delete local peer {}",
+          groupId);
+      deleteLocalPeer(groupId);
+      return;
+    }
+    String previousPeerListStr = impl.getPeers().toString();
+    for (Peer peer : impl.getPeers()) {
+      if (!correctPeers.contains(peer)) {
+        try {
+          impl.dropConsensusPipeToTargetPeer(peer);
+        } catch (ConsensusGroupModifyPeerException e) {
+          LOGGER.error(
+              "[RESET PEER LIST] Failed to remove peer {}'s consensus pipe 
from group {}",
+              peer,
+              groupId,
+              e);
+        }
+      }
+    }
+    LOGGER.info(
+        "[RESET PEER LIST] Local peer list has been reset: {} -> {}",
+        previousPeerListStr,
+        impl.getPeers());
+    for (Peer peer : correctPeers) {
+      if (!impl.containsPeer(peer)) {
+        LOGGER.warn("[RESET PEER LIST] \"Correct peer\" {} is not in local 
peer list", peer);
+      }
+    }
+  }
+
+  @Override
+  public void transferLeader(ConsensusGroupId groupId, Peer newLeader) throws 
ConsensusException {
+    throw new ConsensusException(String.format("%s does not support leader 
transfer", CLASS_NAME));
+  }
+
+  @Override
+  public void triggerSnapshot(ConsensusGroupId groupId, boolean force) throws 
ConsensusException {
+    Optional.ofNullable(stateMachineMap.get(groupId))
+        .orElseThrow(() -> new ConsensusGroupNotExistException(groupId));
+    // do nothing

Review Comment:
   Is it OK to do nothing here?



##########
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.<String, String>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



##########
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<TPipeConsensusTransferResp> resultHandler) {
+    try {
+      TPipeConsensusTransferResp resp = 
config.getConsensusPipeReceiver().receive(req);
+      // we need to call onComplete by hand
+      resultHandler.onComplete(resp);
+    } catch (Exception e) {
+      resultHandler.onError(e);
+    }
+  }
+
+  // TODO: consider batch transfer
+  @Override
+  public void pipeConsensusBatchTransfer(
+      TPipeConsensusBatchTransferReq req,
+      AsyncMethodCallback<TPipeConsensusBatchTransferResp> resultHandler)
+      throws TException {}
+
+  @Override
+  public void setActive(TSetActiveReq req, AsyncMethodCallback<TSetActiveResp> 
resultHandler)
+      throws TException {
+    ConsensusGroupId groupId =
+        
ConsensusGroupId.Factory.createFromTConsensusGroupId(req.consensusGroupId);
+    PipeConsensusServerImpl impl = pipeConsensus.getImpl(groupId);
+    if (impl == null) {
+      String message =
+          String.format("unexpected consensusGroupId %s for set active request 
%s", groupId, req);
+      LOGGER.error(message);
+      TSStatus status = new 
TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode());
+      status.setMessage(message);
+      resultHandler.onComplete(new TSetActiveResp(status));
+      return;
+    }
+    impl.setActive(req.isActive);
+    resultHandler.onComplete(new TSetActiveResp(RpcUtils.SUCCESS_STATUS));
+  }
+
+  @Override
+  public void notifyPeerToCreateConsensusPipe(
+      TNotifyPeerToCreateConsensusPipeReq req,
+      AsyncMethodCallback<TNotifyPeerToCreateConsensusPipeResp> resultHandler)
+      throws TException {
+    ConsensusGroupId groupId =
+        
ConsensusGroupId.Factory.createFromTConsensusGroupId(req.consensusGroupId);

Review Comment:
   Are there any conditions where req.consensusGroupId != 
req.targetPeerConsensusGroupId?



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