You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ji...@apache.org on 2023/02/16 03:02:55 UTC

[iotdb] branch native_raft created (now 2494ffaf24)

This is an automated email from the ASF dual-hosted git repository.

jiangtian pushed a change to branch native_raft
in repository https://gitbox.apache.org/repos/asf/iotdb.git


      at 2494ffaf24 add native raft imp

This branch includes the following new commits:

     new 2494ffaf24 add native raft imp

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[iotdb] 01/01: add native raft imp

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jiangtian pushed a commit to branch native_raft
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 2494ffaf24f7918aa265a25408abfd6acecd9615
Author: Tian Jiang <jt...@163.com>
AuthorDate: Thu Feb 16 11:04:07 2023 +0800

    add native raft imp
---
 consensus/pom.xml                                  |   11 +
 .../apache/iotdb/consensus/ConsensusFactory.java   |    2 +
 .../common/request/IConsensusRequest.java          |    4 +
 .../iotdb/consensus/config/ConsensusConfig.java    |    7 +
 .../apache/iotdb/consensus/config/RPCConfig.java   |  137 ++
 .../iotdb/consensus/natraft/RaftConsensus.java     |  286 ++++
 .../iotdb/consensus/natraft/Utils/IOUtils.java     |   98 ++
 .../iotdb/consensus/natraft/Utils/StatusUtils.java |  228 +++
 .../natraft/client/AsyncRaftServiceClient.java     |  144 ++
 .../natraft/client/ForwardRequestHandler.java      |   54 +
 .../consensus/natraft/client/GenericHandler.java   |   87 ++
 .../natraft/client/RaftConsensusClientPool.java    |   64 +
 .../natraft/client/SyncClientAdaptor.java          |   71 +
 .../exception/CheckConsistencyException.java       |   39 +
 .../exception/LeaderUnknownException.java}         |   25 +-
 .../exception/LogExecutionException.java}          |   21 +-
 .../exception/UnknownLogTypeException.java}        |   22 +-
 .../consensus/natraft/protocol/HardState.java      |  113 ++
 .../iotdb/consensus/natraft/protocol/PeerInfo.java |   69 +
 .../consensus/natraft/protocol/RaftConfig.java     |  426 ++++++
 .../consensus/natraft/protocol/RaftMember.java     | 1078 ++++++++++++++
 .../protocol/RaftRole.java}                        |   26 +-
 .../consensus/natraft/protocol/RaftStatus.java     |   80 +
 .../iotdb/consensus/natraft/protocol/Response.java |   68 +
 .../protocol/consistency/CheckConsistency.java}    |   25 +-
 .../protocol/consistency/ConsistencyLevel.java     |   67 +
 .../protocol/consistency/MidCheckConsistency.java  |   48 +
 .../consistency/StrongCheckConsistency.java        |   27 +
 .../protocol/heartbeat/ElectionReqHandler.java     |  174 +++
 .../protocol/heartbeat/ElectionRespHandler.java    |  148 ++
 .../protocol/heartbeat/HeartbeatReqHandler.java    |   94 ++
 .../protocol/heartbeat/HeartbeatRespHandler.java   |  148 ++
 .../protocol/heartbeat/HeartbeatThread.java        |  418 ++++++
 .../protocol/log/CommitLogCallback.java}           |   46 +-
 .../natraft/protocol/log/CommitLogTask.java        |   65 +
 .../consensus/natraft/protocol/log/Entry.java      |  144 ++
 .../consensus/natraft/protocol/log/LogParser.java  |   76 +
 .../consensus/natraft/protocol/log/VotingLog.java  |  119 ++
 .../protocol/log/appender/BlockingLogAppender.java |  231 +++
 .../protocol/log/appender/LogAppender.java}        |   31 +-
 .../protocol/log/appender/LogAppenderFactory.java} |   21 +-
 .../protocol/log/applier/AsyncLogApplier.java      |  224 +++
 .../natraft/protocol/log/applier/BaseApplier.java  |   67 +
 .../protocol/log/applier/LogApplier.java}          |   25 +-
 .../protocol/log/catchup/CatchUpManager.java       |  130 ++
 .../natraft/protocol/log/catchup/CatchUpTask.java  |  384 +++++
 .../log/catchup/LogCatchUpInBatchHandler.java      |  117 ++
 .../protocol/log/catchup/LogCatchUpTask.java       |  204 +++
 .../log/catchup/SnapshotCatchUpHandler.java        |   61 +
 .../protocol/log/catchup/SnapshotCatchUpTask.java  |  129 ++
 .../log/dispatch/AppendNodeEntryHandler.java       |  158 ++
 .../protocol/log/dispatch/LogDispatcher.java       |  328 +++++
 .../protocol/log/dispatch/VotingLogList.java       |  126 ++
 .../protocol/log/flowcontrol/FlowBalancer.java     |  140 ++
 .../protocol/log/flowcontrol/FlowMonitor.java      |  144 ++
 .../log/flowcontrol/FlowMonitorManager.java        |   79 +
 .../natraft/protocol/log/logtype/EmptyEntry.java   |   67 +
 .../natraft/protocol/log/logtype/RequestEntry.java |  121 ++
 .../protocol/log/manager/RaftLogManager.java       |  913 ++++++++++++
 .../log/manager/SnapshotRaftLogManager.java}       |   39 +-
 .../protocol/log/sequencing/LogSequencer.java      |   44 +
 .../log/sequencing/LogSequencerFactory.java}       |   23 +-
 .../log/sequencing/SynchronousSequencer.java       |  179 +++
 .../protocol/log/serialization/LogManagerMeta.java |  155 ++
 .../log/serialization/StableEntryManager.java      |   62 +
 .../log/serialization/SyncLogDequeSerializer.java  | 1556 ++++++++++++++++++++
 .../natraft/protocol/log/snapshot/Snapshot.java    |   73 +
 .../consensus/natraft/service/RaftRPCService.java  |   99 ++
 .../natraft/service/RaftRPCServiceHandler.java     |   51 +
 .../service/RaftRPCServiceMBean.java}              |   20 +-
 .../natraft/service/RaftRPCServiceProcessor.java   |  118 ++
 .../apache/iotdb/commons/service/ServiceType.java  |    3 +-
 pom.xml                                            |    2 +
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |    1 +
 thrift-raft/pom.xml                                |   67 +
 thrift-raft/src/main/thrift/raft.thrift            |  162 ++
 76 files changed, 10920 insertions(+), 193 deletions(-)

diff --git a/consensus/pom.xml b/consensus/pom.xml
index 23f5a5cb1b..efc1842ac1 100644
--- a/consensus/pom.xml
+++ b/consensus/pom.xml
@@ -65,6 +65,17 @@
             <artifactId>thrift-multi-leader-consensus</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>thrift-raft-consensus</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>thrift-raft-consensus</artifactId>
+            <version>0.14.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java b/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java
index 8146fcf3a4..e6d535aa47 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java
@@ -37,6 +37,8 @@ public class ConsensusFactory {
   public static final String RatisConsensus = "org.apache.iotdb.consensus.ratis.RatisConsensus";
   public static final String MultiLeaderConsensus =
       "org.apache.iotdb.consensus.multileader.MultiLeaderConsensus";
+  public static final String RaftConsensus =
+      "org.apache.iotdb.consensus.natraft.RaftConsensus";
 
   private static final Logger logger = LoggerFactory.getLogger(ConsensusFactory.class);
 
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
index daa2a7d7d0..f2729471d5 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
@@ -35,4 +35,8 @@ public interface IConsensusRequest {
    * changed or an error may occur
    */
   ByteBuffer serializeToByteBuffer();
+
+  default long estimateSize() {
+    return 0;
+  }
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java b/consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java
index be3b4bf342..e58b4779b9 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java
@@ -27,6 +27,7 @@ public class ConsensusConfig {
   private final String storageDir;
   private final RatisConfig ratisConfig;
   private final MultiLeaderConfig multiLeaderConfig;
+  private final RPCConfig rpcConfig;
 
   private ConsensusConfig(
       TEndPoint thisNode,
@@ -37,6 +38,8 @@ public class ConsensusConfig {
     this.storageDir = storageDir;
     this.ratisConfig = ratisConfig;
     this.multiLeaderConfig = multiLeaderConfig;
+    // TODO-Raft: unify rpc config for all protocols
+    this.rpcConfig = RPCConfig.newBuilder().build();
   }
 
   public TEndPoint getThisNode() {
@@ -59,6 +62,10 @@ public class ConsensusConfig {
     return new ConsensusConfig.Builder();
   }
 
+  public RPCConfig getRPCConfig() {
+    return rpcConfig;
+  }
+
   public static class Builder {
 
     private TEndPoint thisNode;
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/config/RPCConfig.java b/consensus/src/main/java/org/apache/iotdb/consensus/config/RPCConfig.java
new file mode 100644
index 0000000000..495519cdfe
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/config/RPCConfig.java
@@ -0,0 +1,137 @@
+/*
+ * 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 requ [...]
+ */
+
+package org.apache.iotdb.consensus.config;
+
+import java.util.concurrent.TimeUnit;
+
+public class RPCConfig {
+  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 int thriftMaxFrameSize;
+
+  private RPCConfig(
+      int rpcSelectorThreadNum,
+      int rpcMinConcurrentClientNum,
+      int rpcMaxConcurrentClientNum,
+      int thriftServerAwaitTimeForStopService,
+      boolean isRpcThriftCompressionEnabled,
+      int selectorNumOfClientManager,
+      int connectionTimeoutInMs,
+      int thriftMaxFrameSize) {
+    this.rpcSelectorThreadNum = rpcSelectorThreadNum;
+    this.rpcMinConcurrentClientNum = rpcMinConcurrentClientNum;
+    this.rpcMaxConcurrentClientNum = rpcMaxConcurrentClientNum;
+    this.thriftServerAwaitTimeForStopService = thriftServerAwaitTimeForStopService;
+    this.isRpcThriftCompressionEnabled = isRpcThriftCompressionEnabled;
+    this.selectorNumOfClientManager = selectorNumOfClientManager;
+    this.connectionTimeoutInMs = connectionTimeoutInMs;
+    this.thriftMaxFrameSize = thriftMaxFrameSize;
+  }
+
+  public int getRpcSelectorThreadNum() {
+    return rpcSelectorThreadNum;
+  }
+
+  public int getRpcMinConcurrentClientNum() {
+    return rpcMinConcurrentClientNum;
+  }
+
+  public int getRpcMaxConcurrentClientNum() {
+    return rpcMaxConcurrentClientNum;
+  }
+
+  public int getThriftServerAwaitTimeForStopService() {
+    return thriftServerAwaitTimeForStopService;
+  }
+
+  public boolean isRpcThriftCompressionEnabled() {
+    return isRpcThriftCompressionEnabled;
+  }
+
+  public int getSelectorNumOfClientManager() {
+    return selectorNumOfClientManager;
+  }
+
+  public int getConnectionTimeoutInMs() {
+    return connectionTimeoutInMs;
+  }
+
+  public int getThriftMaxFrameSize() {
+    return thriftMaxFrameSize;
+  }
+
+  public static RPCConfig.Builder newBuilder() {
+    return new RPCConfig.Builder();
+  }
+
+  public static class Builder {
+    private int rpcSelectorThreadNum = 1;
+    private int rpcMinConcurrentClientNum = Runtime.getRuntime().availableProcessors();
+    private int rpcMaxConcurrentClientNum = 65535;
+    private int thriftServerAwaitTimeForStopService = 60;
+    private boolean isRpcThriftCompressionEnabled = false;
+    private int selectorNumOfClientManager = 1;
+    private int connectionTimeoutInMs = (int) TimeUnit.SECONDS.toMillis(20);
+    private int thriftMaxFrameSize = 536870912;
+
+    public RPCConfig.Builder setRpcSelectorThreadNum(int rpcSelectorThreadNum) {
+      this.rpcSelectorThreadNum = rpcSelectorThreadNum;
+      return this;
+    }
+
+    public RPCConfig.Builder setRpcMinConcurrentClientNum(int rpcMinConcurrentClientNum) {
+      this.rpcMinConcurrentClientNum = rpcMinConcurrentClientNum;
+      return this;
+    }
+
+    public RPCConfig.Builder setRpcMaxConcurrentClientNum(int rpcMaxConcurrentClientNum) {
+      this.rpcMaxConcurrentClientNum = rpcMaxConcurrentClientNum;
+      return this;
+    }
+
+    public RPCConfig.Builder setThriftServerAwaitTimeForStopService(
+        int thriftServerAwaitTimeForStopService) {
+      this.thriftServerAwaitTimeForStopService = thriftServerAwaitTimeForStopService;
+      return this;
+    }
+
+    public RPCConfig.Builder setRpcThriftCompressionEnabled(boolean rpcThriftCompressionEnabled) {
+      isRpcThriftCompressionEnabled = rpcThriftCompressionEnabled;
+      return this;
+    }
+
+    public RPCConfig.Builder setSelectorNumOfClientManager(int selectorNumOfClientManager) {
+      this.selectorNumOfClientManager = selectorNumOfClientManager;
+      return this;
+    }
+
+    public RPCConfig.Builder setConnectionTimeoutInMs(int connectionTimeoutInMs) {
+      this.connectionTimeoutInMs = connectionTimeoutInMs;
+      return this;
+    }
+
+    public RPCConfig.Builder setThriftMaxFrameSize(int thriftMaxFrameSize) {
+      this.thriftMaxFrameSize = thriftMaxFrameSize;
+      return this;
+    }
+
+    public RPCConfig build() {
+      return new RPCConfig(
+          rpcSelectorThreadNum,
+          rpcMinConcurrentClientNum,
+          rpcMaxConcurrentClientNum,
+          thriftServerAwaitTimeForStopService,
+          isRpcThriftCompressionEnabled,
+          selectorNumOfClientManager,
+          connectionTimeoutInMs,
+          thriftMaxFrameSize);
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/RaftConsensus.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/RaftConsensus.java
new file mode 100644
index 0000000000..2185c39667
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/RaftConsensus.java
@@ -0,0 +1,286 @@
+/*
+ * 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.natraft;
+
+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.service.RegisterManager;
+import org.apache.iotdb.commons.utils.FileUtils;
+import org.apache.iotdb.consensus.IConsensus;
+import org.apache.iotdb.consensus.IStateMachine;
+import org.apache.iotdb.consensus.IStateMachine.Registry;
+import org.apache.iotdb.consensus.common.Peer;
+import org.apache.iotdb.consensus.common.request.IConsensusRequest;
+import org.apache.iotdb.consensus.common.response.ConsensusGenericResponse;
+import org.apache.iotdb.consensus.common.response.ConsensusReadResponse;
+import org.apache.iotdb.consensus.common.response.ConsensusWriteResponse;
+import org.apache.iotdb.consensus.config.ConsensusConfig;
+import org.apache.iotdb.consensus.exception.ConsensusGroupAlreadyExistException;
+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.natraft.client.AsyncRaftServiceClient;
+import org.apache.iotdb.consensus.natraft.client.RaftConsensusClientPool.AsyncRaftServiceClientPoolFactory;
+import org.apache.iotdb.consensus.natraft.exception.CheckConsistencyException;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.service.RaftRPCService;
+import org.apache.iotdb.rpc.TSStatusCode;
+
+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.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+public class RaftConsensus implements IConsensus {
+
+  private static final Logger logger = LoggerFactory.getLogger(RaftConsensus.class);
+  private final TEndPoint thisNode;
+  private final File storageDir;
+  private final IStateMachine.Registry registry;
+  private final Map<ConsensusGroupId, RaftMember> stateMachineMap = new ConcurrentHashMap<>();
+  private final RaftRPCService service;
+  private final RegisterManager registerManager = new RegisterManager();
+  private final RaftConfig config;
+  private final IClientManager<TEndPoint, AsyncRaftServiceClient> clientManager;
+
+  public RaftConsensus(ConsensusConfig config, Registry registry) {
+    this.thisNode = config.getThisNode();
+    this.storageDir = new File(config.getStorageDir());
+    this.config = new RaftConfig(config);
+    this.registry = registry;
+    this.service = new RaftRPCService(thisNode, this.config);
+    this.clientManager =
+        new IClientManager.Factory<TEndPoint, AsyncRaftServiceClient>()
+            .createClientManager(new AsyncRaftServiceClientPoolFactory(this.config));
+  }
+
+  @Override
+  public void start() throws IOException {
+    initAndRecover();
+    service.initAsyncedServiceImpl(new RaftRPCService(thisNode, config));
+    try {
+      registerManager.register(service);
+    } catch (StartupException e) {
+      throw new IOException(e);
+    }
+  }
+
+  private void initAndRecover() throws IOException {
+    if (!storageDir.exists()) {
+      if (!storageDir.mkdirs()) {
+        logger.warn("Unable to create consensus dir at {}", storageDir);
+      }
+    } else {
+      try (DirectoryStream<Path> stream = Files.newDirectoryStream(storageDir.toPath())) {
+        for (Path path : stream) {
+          Path fileName = path.getFileName();
+          String[] items = fileName.toString().split("_");
+          ConsensusGroupId consensusGroupId =
+              ConsensusGroupId.Factory.create(
+                  Integer.parseInt(items[0]), Integer.parseInt(items[1]));
+          RaftMember raftMember =
+              new RaftMember(
+                  config,
+                  thisNode,
+                  new ArrayList<>(),
+                  consensusGroupId,
+                  registry.apply(consensusGroupId),
+                  clientManager);
+          stateMachineMap.put(consensusGroupId, raftMember);
+          raftMember.start();
+        }
+      }
+    }
+  }
+
+  @Override
+  public void stop() throws IOException {
+    clientManager.close();
+    stateMachineMap.values().parallelStream().forEach(RaftMember::stop);
+    registerManager.deregisterAll();
+  }
+
+  @Override
+  public ConsensusWriteResponse write(ConsensusGroupId groupId, IConsensusRequest request) {
+    RaftMember impl = stateMachineMap.get(groupId);
+    if (impl == null) {
+      return ConsensusWriteResponse.newBuilder()
+          .setException(new ConsensusGroupNotExistException(groupId))
+          .build();
+    }
+
+    TSStatus status;
+    if (impl.isReadOnly()) {
+      status = new TSStatus(TSStatusCode.READ_ONLY_SYSTEM_ERROR.getStatusCode());
+      status.setMessage("Fail to do non-query operations because system is read-only.");
+    } else {
+      status = impl.processRequest(request);
+    }
+    return ConsensusWriteResponse.newBuilder().setStatus(status).build();
+  }
+
+  @Override
+  public ConsensusReadResponse read(ConsensusGroupId groupId, IConsensusRequest request) {
+    RaftMember impl = stateMachineMap.get(groupId);
+    if (impl == null) {
+      return ConsensusReadResponse.newBuilder()
+          .setException(new ConsensusGroupNotExistException(groupId))
+          .build();
+    }
+    try {
+      return ConsensusReadResponse.newBuilder().setDataSet(impl.read(request)).build();
+    } catch (CheckConsistencyException e) {
+      return ConsensusReadResponse.newBuilder().setException(e).build();
+    }
+  }
+
+  @Override
+  public ConsensusGenericResponse createPeer(ConsensusGroupId groupId, List<Peer> peers) {
+    int consensusGroupSize = peers.size();
+    if (consensusGroupSize == 0) {
+      return ConsensusGenericResponse.newBuilder()
+          .setException(new IllegalPeerNumException(consensusGroupSize))
+          .build();
+    }
+    if (!peers.contains(new Peer(groupId, thisNode))) {
+      return ConsensusGenericResponse.newBuilder()
+          .setException(new IllegalPeerEndpointException(thisNode, peers))
+          .build();
+    }
+    AtomicBoolean exist = new AtomicBoolean(true);
+    stateMachineMap.computeIfAbsent(
+        groupId,
+        k -> {
+          exist.set(false);
+          String path = buildPeerDir(groupId);
+          File file = new File(path);
+          if (!file.mkdirs()) {
+            logger.warn("Unable to create consensus dir for group {} at {}", groupId, path);
+          }
+          RaftMember impl =
+              new RaftMember(
+                  config,
+                  thisNode,
+                  peers.stream().map(Peer::getEndpoint).collect(Collectors.toList()),
+                  groupId,
+                  registry.apply(groupId),
+                  clientManager);
+          impl.start();
+          return impl;
+        });
+    if (exist.get()) {
+      return ConsensusGenericResponse.newBuilder()
+          .setException(new ConsensusGroupAlreadyExistException(groupId))
+          .build();
+    }
+    return ConsensusGenericResponse.newBuilder().setSuccess(true).build();
+  }
+
+  private String buildPeerDir(ConsensusGroupId groupId) {
+    return storageDir + File.separator + groupId.getType().getValue() + "_" + groupId.getId();
+  }
+
+  @Override
+  public ConsensusGenericResponse deletePeer(ConsensusGroupId groupId) {
+    AtomicBoolean exist = new AtomicBoolean(false);
+    stateMachineMap.computeIfPresent(
+        groupId,
+        (k, v) -> {
+          exist.set(true);
+          v.stop();
+          FileUtils.deleteDirectory(new File(buildPeerDir(groupId)));
+          return null;
+        });
+
+    if (!exist.get()) {
+      return ConsensusGenericResponse.newBuilder()
+          .setException(new ConsensusGroupNotExistException(groupId))
+          .build();
+    }
+    return ConsensusGenericResponse.newBuilder().setSuccess(true).build();
+  }
+
+  @Override
+  public ConsensusGenericResponse addPeer(ConsensusGroupId groupId, Peer peer) {
+    return ConsensusGenericResponse.newBuilder().setSuccess(false).build();
+  }
+
+  @Override
+  public ConsensusGenericResponse removePeer(ConsensusGroupId groupId, Peer peer) {
+    return ConsensusGenericResponse.newBuilder().setSuccess(false).build();
+  }
+
+  @Override
+  public ConsensusGenericResponse changePeer(ConsensusGroupId groupId, List<Peer> newPeers) {
+    return ConsensusGenericResponse.newBuilder().setSuccess(false).build();
+  }
+
+  @Override
+  public ConsensusGenericResponse transferLeader(ConsensusGroupId groupId, Peer newLeader) {
+    return ConsensusGenericResponse.newBuilder().setSuccess(false).build();
+  }
+
+  @Override
+  public ConsensusGenericResponse triggerSnapshot(ConsensusGroupId groupId) {
+    return ConsensusGenericResponse.newBuilder().setSuccess(false).build();
+  }
+
+  @Override
+  public boolean isLeader(ConsensusGroupId groupId) {
+    RaftMember impl = stateMachineMap.get(groupId);
+    if (impl == null) {
+      return false;
+    }
+    return Objects.equals(impl.getStatus().getLeader().get(), impl.getThisNode());
+  }
+
+  @Override
+  public Peer getLeader(ConsensusGroupId groupId) {
+    RaftMember impl = stateMachineMap.get(groupId);
+    if (impl == null) {
+      return null;
+    }
+    return new Peer(groupId, impl.getStatus().getLeader().get());
+  }
+
+  @Override
+  public List<ConsensusGroupId> getAllConsensusGroupIds() {
+    return new ArrayList<>(stateMachineMap.keySet());
+  }
+
+  public RaftMember getMember(ConsensusGroupId groupId) {
+    return stateMachineMap.get(groupId);
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/Utils/IOUtils.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/Utils/IOUtils.java
new file mode 100644
index 0000000000..cc57b4a0e2
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/Utils/IOUtils.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.natraft.Utils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+
+@SuppressWarnings("java:S1135")
+public class IOUtils {
+
+  private static final Logger logger = LoggerFactory.getLogger(IOUtils.class);
+
+  private IOUtils() {
+    // util class
+  }
+
+  /**
+   * An interface that is used for a node to pull chunks of files like TsFiles. The file should be a
+   * temporary hard link, and once the file is totally read, it will be removed.
+   */
+  public static ByteBuffer readFile(String filePath, long offset, int length) throws IOException {
+    // TODO-Cluster: hold if the file is an unclosed TsFile
+    File file = new File(filePath);
+    if (!file.exists()) {
+      logger.warn("Reading a non-existing snapshot file {}", filePath);
+      return ByteBuffer.allocate(0);
+    }
+
+    ByteBuffer result;
+    boolean fileExhausted;
+    int len;
+    try (BufferedInputStream bufferedInputStream =
+        new BufferedInputStream(new FileInputStream(file))) {
+      skipExactly(bufferedInputStream, offset);
+      byte[] bytes = new byte[length];
+      result = ByteBuffer.wrap(bytes);
+      len = bufferedInputStream.read(bytes);
+      result.limit(Math.max(len, 0));
+      fileExhausted = bufferedInputStream.available() <= 0;
+    }
+
+    if (fileExhausted) {
+      try {
+        Files.delete(file.toPath());
+        if (logger.isInfoEnabled()) {
+          logger.info(
+              "Snapshot file {} is exhausted, offset: {}, read length: {}, file length: {}",
+              filePath,
+              offset,
+              len,
+              file.length());
+        }
+      } catch (IOException e) {
+        logger.warn("Cannot delete an exhausted file {}", filePath, e);
+      }
+    }
+    return result;
+  }
+
+  private static void skipExactly(InputStream stream, long byteToSkip) throws IOException {
+    while (byteToSkip > 0) {
+      byteToSkip -= stream.skip(byteToSkip);
+    }
+  }
+
+  public static Throwable getRootCause(Throwable e) {
+    Throwable curr = e;
+    while (curr.getCause() != null) {
+      curr = curr.getCause();
+    }
+    return curr;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/Utils/StatusUtils.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/Utils/StatusUtils.java
new file mode 100644
index 0000000000..9b4abd7eed
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/Utils/StatusUtils.java
@@ -0,0 +1,228 @@
+/*
+ * 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.natraft.Utils;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.rpc.TSStatusCode;
+
+public class StatusUtils {
+
+  private StatusUtils() {
+    // util class
+  }
+
+  public static final TSStatus PARTITION_TABLE_NOT_READY =
+      getStatus(TSStatusCode.PARTITION_NOT_READY);
+  public static final TSStatus OK = getStatus(TSStatusCode.SUCCESS_STATUS);
+  public static final TSStatus TIME_OUT = getStatus(TSStatusCode.TIME_OUT);
+  public static final TSStatus NO_LEADER = getStatus(TSStatusCode.NO_LEADER);
+  public static final TSStatus INTERNAL_ERROR = getStatus(TSStatusCode.INTERNAL_SERVER_ERROR);
+  public static final TSStatus UNSUPPORTED_OPERATION =
+      getStatus(TSStatusCode.UNSUPPORTED_OPERATION);
+  public static final TSStatus EXECUTE_STATEMENT_ERROR =
+      getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR);
+  public static final TSStatus NO_STORAGE_GROUP = getStatus(TSStatusCode.STORAGE_GROUP_ERROR);
+  public static final TSStatus NODE_READ_ONLY = getStatus(TSStatusCode.NODE_READ_ONLY);
+  public static final TSStatus CONSISTENCY_FAILURE = getStatus(TSStatusCode.CONSISTENCY_FAILURE);
+  public static final TSStatus TIMESERIES_NOT_EXIST_ERROR =
+      getStatus(TSStatusCode.TIMESERIES_NOT_EXIST);
+  public static final TSStatus NO_CONNECTION = getStatus(TSStatusCode.NO_CONNECTION);
+  public static final TSStatus PARSE_LOG_ERROR = getStatus(TSStatusCode.PARSE_LOG_ERROR);
+  public static final TSStatus DUPLICATED_TEMPLATE = getStatus(TSStatusCode.DUPLICATED_TEMPLATE);
+
+  public static TSStatus getStatus(TSStatusCode statusCode) {
+    TSStatus status = new TSStatus();
+    status.setCode(statusCode.getStatusCode());
+    switch (statusCode) {
+      case SUCCESS_STATUS:
+        status.setMessage("Executed successfully. ");
+        break;
+      case TIME_OUT:
+        status.setMessage("Request timed out. ");
+        break;
+      case NO_LEADER:
+        status.setMessage("Leader cannot be found. ");
+        break;
+      case PARTITION_NOT_READY:
+        status.setMessage("Partition table is not ready. ");
+        break;
+      case NODE_READ_ONLY:
+        status.setMessage(
+            "Current node is read-only, please retry to find another available node. ");
+        break;
+      case INCOMPATIBLE_VERSION:
+        status.setMessage("Incompatible version. ");
+        break;
+      case NODE_DELETE_FAILED_ERROR:
+        status.setMessage("Failed while deleting node. ");
+        break;
+      case ALIAS_ALREADY_EXIST_ERROR:
+        status.setMessage("Alias already exists. ");
+        break;
+      case PATH_ALREADY_EXIST_ERROR:
+        status.setMessage("Path already exist. ");
+        break;
+      case PATH_NOT_EXIST_ERROR:
+      case TIMESERIES_NOT_EXIST:
+        status.setMessage("Path does not exist. ");
+        break;
+      case UNSUPPORTED_FETCH_METADATA_OPERATION_ERROR:
+        status.setMessage("Unsupported fetch metadata operation. ");
+        break;
+      case METADATA_ERROR:
+        status.setMessage("Meet error when dealing with metadata. ");
+        break;
+      case OUT_OF_TTL_ERROR:
+        status.setMessage("Insertion time is less than TTL time bound. ");
+        break;
+      case CONFIG_ADJUSTER:
+        status.setMessage("IoTDB system load is too large. ");
+        break;
+      case MERGE_ERROR:
+        status.setMessage("Meet error while merging. ");
+        break;
+      case SYSTEM_CHECK_ERROR:
+        status.setMessage("Meet error while system checking. ");
+        break;
+      case SYNC_CONNECTION_EXCEPTION:
+        status.setMessage("Meet error while sync connecting. ");
+        break;
+      case DATA_REGION_ERROR:
+        status.setMessage("Storage group processor related error. ");
+        break;
+      case STORAGE_GROUP_ERROR:
+        status.setMessage("No associated storage group. ");
+        break;
+      case STORAGE_ENGINE_ERROR:
+        status.setMessage("Storage engine related error. ");
+        break;
+      case TSFILE_PROCESSOR_ERROR:
+        status.setMessage("TsFile processor related error. ");
+        break;
+      case PATH_ILLEGAL:
+        status.setMessage("Illegal path. ");
+        break;
+      case LOAD_FILE_ERROR:
+        status.setMessage("Meet error while loading file. ");
+        break;
+      case EXECUTE_STATEMENT_ERROR:
+        status.setMessage("Execute statement error. ");
+        break;
+      case SQL_PARSE_ERROR:
+        status.setMessage("Meet error while parsing SQL. ");
+        break;
+      case GENERATE_TIME_ZONE_ERROR:
+        status.setMessage("Meet error while generating time zone. ");
+        break;
+      case SET_TIME_ZONE_ERROR:
+        status.setMessage("Meet error while setting time zone. ");
+        break;
+      case NOT_STORAGE_GROUP_ERROR:
+        status.setMessage("Operating object is not a storage group. ");
+        break;
+      case QUERY_NOT_ALLOWED:
+        status.setMessage("Query statements are not allowed error. ");
+        break;
+      case AST_FORMAT_ERROR:
+        status.setMessage("AST format related error. ");
+        break;
+      case LOGICAL_OPERATOR_ERROR:
+        status.setMessage("Logical operator related error. ");
+        break;
+      case LOGICAL_OPTIMIZE_ERROR:
+        status.setMessage("Logical optimize related error. ");
+        break;
+      case UNSUPPORTED_FILL_TYPE_ERROR:
+        status.setMessage("Unsupported fill type related error. ");
+        break;
+      case PATH_ERROR:
+        status.setMessage("Path related error. ");
+        break;
+      case QUERY_PROCESS_ERROR:
+        status.setMessage("Query process related error. ");
+        break;
+      case WRITE_PROCESS_ERROR:
+        status.setMessage("Writing data related error. ");
+        break;
+      case INTERNAL_SERVER_ERROR:
+        status.setMessage("Internal server error. ");
+        break;
+      case CLOSE_OPERATION_ERROR:
+        status.setMessage("Meet error in close operation. ");
+        break;
+      case READ_ONLY_SYSTEM_ERROR:
+        status.setMessage("Database is read-only. ");
+        break;
+      case DISK_SPACE_INSUFFICIENT_ERROR:
+        status.setMessage("Disk space is insufficient. ");
+        break;
+      case START_UP_ERROR:
+        status.setMessage("Meet error while starting up. ");
+        break;
+      case WRONG_LOGIN_PASSWORD_ERROR:
+        status.setMessage("Username or password is wrong. ");
+        break;
+      case NOT_LOGIN_ERROR:
+        status.setMessage("Has not logged in. ");
+        break;
+      case NO_PERMISSION_ERROR:
+        status.setMessage("No permissions for this operation. ");
+        break;
+      case UNINITIALIZED_AUTH_ERROR:
+        status.setMessage("Uninitialized authorizer. ");
+        break;
+      case UNSUPPORTED_OPERATION:
+        status.setMessage("Unsupported operation. ");
+        break;
+      case NO_CONNECTION:
+        status.setMessage("Node cannot be reached.");
+        break;
+      case PARSE_LOG_ERROR:
+        status.setMessage("Parse log error.");
+        break;
+      case PIPESINK_ERROR:
+        status.setMessage("PipeSink error.");
+        break;
+      default:
+        status.setMessage("");
+        break;
+    }
+    return status;
+  }
+
+  public static TSStatus getStatus(TSStatusCode statusCode, TEndPoint redirectedNode) {
+    TSStatus status = getStatus(statusCode);
+    status.setRedirectNode(redirectedNode);
+    return status;
+  }
+
+  public static TSStatus getStatus(TSStatus status, String message) {
+    TSStatus newStatus = status.deepCopy();
+    newStatus.setMessage(message);
+    return newStatus;
+  }
+
+  public static TSStatus getStatus(TSStatus status, TEndPoint redirectedNode) {
+    TSStatus newStatus = status.deepCopy();
+    newStatus.setRedirectNode(redirectedNode);
+    return newStatus;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/AsyncRaftServiceClient.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/AsyncRaftServiceClient.java
new file mode 100644
index 0000000000..95e92deebf
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/AsyncRaftServiceClient.java
@@ -0,0 +1,144 @@
+/*
+ * 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.natraft.client;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.commons.client.AsyncBaseClientFactory;
+import org.apache.iotdb.commons.client.ClientFactoryProperty;
+import org.apache.iotdb.commons.client.ClientManager;
+import org.apache.iotdb.consensus.raft.thrift.RaftService;
+import org.apache.iotdb.rpc.TNonblockingSocketWrapper;
+
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+import org.apache.thrift.async.TAsyncClientManager;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class AsyncRaftServiceClient extends RaftService.AsyncClient {
+
+  private static final Logger logger = LoggerFactory.getLogger(AsyncRaftServiceClient.class);
+
+  private final TEndPoint endpoint;
+  private final ClientManager<TEndPoint, AsyncRaftServiceClient> clientManager;
+
+  public AsyncRaftServiceClient(
+      TProtocolFactory protocolFactory,
+      int connectionTimeout,
+      TEndPoint endpoint,
+      TAsyncClientManager tClientManager,
+      ClientManager<TEndPoint, AsyncRaftServiceClient> clientManager)
+      throws IOException {
+    super(
+        protocolFactory,
+        tClientManager,
+        TNonblockingSocketWrapper.wrap(endpoint.getIp(), endpoint.getPort(), connectionTimeout));
+    this.endpoint = endpoint;
+    this.clientManager = clientManager;
+  }
+
+  public void close() {
+    ___transport.close();
+    ___currentMethod = null;
+  }
+
+  /**
+   * return self if clientManager is not null, the method doesn't need to call by user, it will
+   * trigger once client transport complete.
+   */
+  private void returnSelf() {
+    if (clientManager != null) {
+      clientManager.returnClient(endpoint, this);
+    }
+  }
+
+  /**
+   * This method will be automatically called by the thrift selector thread, and we'll just simulate
+   * the behavior in our test
+   */
+  @Override
+  public void onComplete() {
+    super.onComplete();
+    returnSelf();
+  }
+
+  /**
+   * This method will be automatically called by the thrift selector thread, and we'll just simulate
+   * the behavior in our test
+   */
+  @Override
+  public void onError(Exception e) {
+    super.onError(e);
+    returnSelf();
+  }
+
+  public boolean isReady() {
+    try {
+      checkReady();
+      return true;
+    } catch (Exception e) {
+      logger.info("Unexpected exception occurs in {} :", this, e);
+      return false;
+    }
+  }
+
+  @Override
+  public String toString() {
+    return String.format("AsyncConfigNodeIServiceClient{%s}", endpoint);
+  }
+
+  public static class Factory extends AsyncBaseClientFactory<TEndPoint, AsyncRaftServiceClient> {
+
+    public Factory(
+        ClientManager<TEndPoint, AsyncRaftServiceClient> clientManager,
+        ClientFactoryProperty clientFactoryProperty,
+        String threadName) {
+      super(clientManager, clientFactoryProperty, threadName);
+    }
+
+    @Override
+    public void destroyObject(
+        TEndPoint endPoint, PooledObject<AsyncRaftServiceClient> pooledObject) {
+      pooledObject.getObject().close();
+    }
+
+    @Override
+    public PooledObject<AsyncRaftServiceClient> makeObject(TEndPoint endPoint) throws Exception {
+      TAsyncClientManager tManager = tManagers[clientCnt.incrementAndGet() % tManagers.length];
+      tManager = tManager == null ? new TAsyncClientManager() : tManager;
+      return new DefaultPooledObject<>(
+          new AsyncRaftServiceClient(
+              clientFactoryProperty.getProtocolFactory(),
+              clientFactoryProperty.getConnectionTimeoutMs(),
+              endPoint,
+              tManager,
+              clientManager));
+    }
+
+    @Override
+    public boolean validateObject(
+        TEndPoint endPoint, PooledObject<AsyncRaftServiceClient> pooledObject) {
+      return pooledObject.getObject() != null && pooledObject.getObject().isReady();
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/ForwardRequestHandler.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/ForwardRequestHandler.java
new file mode 100644
index 0000000000..cab6e2e97c
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/ForwardRequestHandler.java
@@ -0,0 +1,54 @@
+/*
+ * 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 requ [...]
+ */
+
+package org.apache.iotdb.consensus.natraft.client;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.consensus.common.request.IConsensusRequest;
+import org.apache.iotdb.consensus.natraft.Utils.StatusUtils;
+
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class ForwardRequestHandler implements AsyncMethodCallback<TSStatus> {
+
+  private static final Logger logger = LoggerFactory.getLogger(ForwardRequestHandler.class);
+  private IConsensusRequest request;
+  private AtomicReference<TSStatus> result;
+  private TEndPoint node;
+
+  public ForwardRequestHandler(
+      AtomicReference<TSStatus> result, IConsensusRequest request, TEndPoint node) {
+    this.result = result;
+    this.request = request;
+    this.node = node;
+  }
+
+  @Override
+  public void onComplete(TSStatus response) {
+    synchronized (result) {
+      result.set(response);
+      result.notifyAll();
+    }
+  }
+
+  @Override
+  public void onError(Exception exception) {
+    if (exception instanceof IOException) {
+      logger.warn("Cannot send plan {} to node {}: {}", request, node, exception.getMessage());
+    } else {
+      logger.error("Cannot send plan {} to node {}", request, node, exception);
+    }
+    synchronized (result) {
+      TSStatus status = StatusUtils.getStatus(StatusUtils.INTERNAL_ERROR, exception.getMessage());
+      result.set(status);
+      result.notifyAll();
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/GenericHandler.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/GenericHandler.java
new file mode 100644
index 0000000000..f5b3370ac7
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/GenericHandler.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 requ [...]
+ */
+
+package org.apache.iotdb.consensus.natraft.client;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.ConnectException;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * GenericHandler simply put the response into an AtomicReference and wake up the caller. Notice:
+ * the caller should wait on "result" if it wants to get the result. Please refer to the current
+ * usages before using this class.
+ *
+ * @param <T>
+ */
+public class GenericHandler<T> implements AsyncMethodCallback<T> {
+
+  private static final Logger logger = LoggerFactory.getLogger(GenericHandler.class);
+
+  private TEndPoint source;
+  private AtomicReference<T> result;
+  private Exception e;
+
+  public GenericHandler(TEndPoint source) {
+    this.source = source;
+    this.result = new AtomicReference<>();
+  }
+
+  @Override
+  public void onComplete(T response) {
+    if (result != null) {
+      synchronized (result) {
+        result.set(response);
+        result.notifyAll();
+      }
+    }
+  }
+
+  @Override
+  public void onError(Exception exception) {
+    if (!(exception instanceof ConnectException)) {
+      logger.error("Cannot receive result from {}", source, exception);
+    } else {
+      logger.warn("Cannot receive result from {} : {}", source, exception.getMessage());
+    }
+
+    if (result != null) {
+      synchronized (result) {
+        result.notifyAll();
+        e = exception;
+      }
+    }
+  }
+
+  public Exception getException() {
+    return e;
+  }
+
+  @SuppressWarnings("java:S2274") // enable timeout
+  public T getResult(long timeout) throws InterruptedException, TException {
+    long waitStart = System.currentTimeMillis();
+    long elapsedTime = 0;
+    while (elapsedTime < timeout) {
+      if (result.get() == null && getException() == null) {
+        synchronized (result) {
+          result.wait(1000);
+        }
+      } else {
+        break;
+      }
+      elapsedTime = System.currentTimeMillis() - waitStart;
+    }
+
+    if (getException() != null) {
+      throw new TException(getException());
+    }
+    return result.get();
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/RaftConsensusClientPool.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/RaftConsensusClientPool.java
new file mode 100644
index 0000000000..4b7be10441
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/RaftConsensusClientPool.java
@@ -0,0 +1,64 @@
+/*
+ * 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.natraft.client;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.commons.client.ClientFactoryProperty;
+import org.apache.iotdb.commons.client.ClientManager;
+import org.apache.iotdb.commons.client.ClientPoolProperty;
+import org.apache.iotdb.commons.client.IClientPoolFactory;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+
+import org.apache.commons.pool2.KeyedObjectPool;
+import org.apache.commons.pool2.impl.GenericKeyedObjectPool;
+
+public class RaftConsensusClientPool {
+
+  private RaftConsensusClientPool() {}
+
+  public static class AsyncRaftServiceClientPoolFactory
+      implements IClientPoolFactory<TEndPoint, AsyncRaftServiceClient> {
+
+    private final RaftConfig config;
+    private static final String RAFT_CONSENSUS_CLIENT_POOL_THREAD_NAME =
+        "MultiLeaderConsensusClientPool";
+
+    public AsyncRaftServiceClientPoolFactory(RaftConfig config) {
+      this.config = config;
+    }
+
+    @Override
+    public KeyedObjectPool<TEndPoint, AsyncRaftServiceClient> createClientPool(
+        ClientManager<TEndPoint, AsyncRaftServiceClient> manager) {
+      return new GenericKeyedObjectPool<>(
+          new AsyncRaftServiceClient.Factory(
+              manager,
+              new ClientFactoryProperty.Builder()
+                  .setConnectionTimeoutMs(config.getRpcConfig().getConnectionTimeoutInMs())
+                  .setRpcThriftCompressionEnabled(
+                      config.getRpcConfig().isRpcThriftCompressionEnabled())
+                  .setSelectorNumOfAsyncClientManager(
+                      config.getRpcConfig().getSelectorNumOfClientManager())
+                  .build(),
+              RAFT_CONSENSUS_CLIENT_POOL_THREAD_NAME),
+          new ClientPoolProperty.Builder<AsyncRaftServiceClient>().build().getConfig());
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/SyncClientAdaptor.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/SyncClientAdaptor.java
new file mode 100644
index 0000000000..3d780bbbfe
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/client/SyncClientAdaptor.java
@@ -0,0 +1,71 @@
+/*
+ * 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 requ [...]
+ */
+
+package org.apache.iotdb.consensus.natraft.client;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.consensus.ConsensusGroupId;
+import org.apache.iotdb.consensus.common.request.IConsensusRequest;
+import org.apache.iotdb.consensus.raft.thrift.ExecuteReq;
+
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * SyncClientAdaptor convert the async of AsyncClient method call to a sync one by synchronizing on
+ * an AtomicReference of the return value of an RPC, and wait for at most connectionTimeoutInMS
+ * until the reference is set by the handler or the request timeouts.
+ */
+@SuppressWarnings("java:S2274") // enable timeout
+public class SyncClientAdaptor {
+
+  private static final Logger logger = LoggerFactory.getLogger(SyncClientAdaptor.class);
+
+  private SyncClientAdaptor() {
+    // static class
+  }
+
+  public static Boolean matchTerm(
+      AsyncRaftServiceClient client,
+      TEndPoint target,
+      long prevLogIndex,
+      long prevLogTerm,
+      ConsensusGroupId groupId)
+      throws TException, InterruptedException {
+    try {
+      GenericHandler<Boolean> matchTermHandler = new GenericHandler<>(target);
+      client.matchTerm(
+          prevLogIndex, prevLogTerm, groupId.convertToTConsensusGroupId(), matchTermHandler);
+      return matchTermHandler.getResult(client.getTimeout());
+    } catch (NullPointerException e) {
+      logger.error("match term null exception", e);
+      return false;
+    }
+  }
+
+  public static TSStatus executeRequest(
+      AsyncRaftServiceClient client,
+      IConsensusRequest request,
+      ConsensusGroupId groupId,
+      TEndPoint receiver)
+      throws IOException, TException, InterruptedException {
+    AtomicReference<TSStatus> status = new AtomicReference<>();
+    ExecuteReq req = new ExecuteReq();
+    req.requestBytes = request.serializeToByteBuffer();
+    req.setGroupId(groupId.convertToTConsensusGroupId());
+
+    client.executeRequest(req, new ForwardRequestHandler(status, request, receiver));
+    synchronized (status) {
+      if (status.get() == null) {
+        status.wait(client.getTimeout());
+      }
+    }
+    return status.get();
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/exception/CheckConsistencyException.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/exception/CheckConsistencyException.java
new file mode 100644
index 0000000000..b2c5e5edd4
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/exception/CheckConsistencyException.java
@@ -0,0 +1,39 @@
+/*
+ * 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.natraft.exception;
+
+import org.apache.iotdb.consensus.exception.ConsensusException;
+
+/**
+ * Raised when check consistency failed, now only happens if there is a strong-consistency and
+ * syncLeader failed
+ */
+public class CheckConsistencyException extends ConsensusException {
+
+  public CheckConsistencyException(String errMsg) {
+    super(String.format("check consistency failed, error message=%s ", errMsg));
+  }
+
+  public static final CheckConsistencyException CHECK_STRONG_CONSISTENCY_EXCEPTION =
+      new CheckConsistencyException("strong consistency, sync with leader failed");
+
+  public static final CheckConsistencyException CHECK_MID_CONSISTENCY_EXCEPTION =
+      new CheckConsistencyException(
+          "mid consistency, localAppliedId is smaller than the leaderCommitId");
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/exception/LeaderUnknownException.java
similarity index 56%
copy from consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
copy to consensus/src/main/java/org/apache/iotdb/consensus/natraft/exception/LeaderUnknownException.java
index daa2a7d7d0..24b1aefe90 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/exception/LeaderUnknownException.java
@@ -16,23 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.consensus.natraft.exception;
 
-package org.apache.iotdb.consensus.common.request;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 
-import java.nio.ByteBuffer;
+import java.util.Collection;
 
-public interface IConsensusRequest {
-  /**
-   * Serialize all the data to a ByteBuffer.
-   *
-   * <p>In a specific implementation, ByteBuf or PublicBAOS can be used to reduce the number of
-   * memory copies.
-   *
-   * <p>To improve efficiency, a specific implementation could return a DirectByteBuffer to reduce
-   * the memory copy required to send an RPC
-   *
-   * <p>Note: The implementation needs to ensure that the data in the returned Bytebuffer cannot be
-   * changed or an error may occur
-   */
-  ByteBuffer serializeToByteBuffer();
+/** Raised when a request should be forwarded to the leader but the leader cannot be found. */
+public class LeaderUnknownException extends Exception {
+
+  public LeaderUnknownException(Collection<TEndPoint> nodes) {
+    super(String.format("The leader is unknown in this group %s", nodes));
+  }
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/exception/LogExecutionException.java
similarity index 56%
copy from consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
copy to consensus/src/main/java/org/apache/iotdb/consensus/natraft/exception/LogExecutionException.java
index daa2a7d7d0..0a6002c4a7 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/exception/LogExecutionException.java
@@ -17,22 +17,11 @@
  * under the License.
  */
 
-package org.apache.iotdb.consensus.common.request;
+package org.apache.iotdb.consensus.natraft.exception;
 
-import java.nio.ByteBuffer;
+public class LogExecutionException extends Exception {
 
-public interface IConsensusRequest {
-  /**
-   * Serialize all the data to a ByteBuffer.
-   *
-   * <p>In a specific implementation, ByteBuf or PublicBAOS can be used to reduce the number of
-   * memory copies.
-   *
-   * <p>To improve efficiency, a specific implementation could return a DirectByteBuffer to reduce
-   * the memory copy required to send an RPC
-   *
-   * <p>Note: The implementation needs to ensure that the data in the returned Bytebuffer cannot be
-   * changed or an error may occur
-   */
-  ByteBuffer serializeToByteBuffer();
+  public LogExecutionException(Throwable cause) {
+    super(cause);
+  }
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/exception/UnknownLogTypeException.java
similarity index 56%
copy from consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
copy to consensus/src/main/java/org/apache/iotdb/consensus/natraft/exception/UnknownLogTypeException.java
index daa2a7d7d0..86e12b85b9 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/exception/UnknownLogTypeException.java
@@ -17,22 +17,12 @@
  * under the License.
  */
 
-package org.apache.iotdb.consensus.common.request;
+package org.apache.iotdb.consensus.natraft.exception;
 
-import java.nio.ByteBuffer;
+/** Raised when the type of a raft log is unknown. */
+public class UnknownLogTypeException extends Exception {
 
-public interface IConsensusRequest {
-  /**
-   * Serialize all the data to a ByteBuffer.
-   *
-   * <p>In a specific implementation, ByteBuf or PublicBAOS can be used to reduce the number of
-   * memory copies.
-   *
-   * <p>To improve efficiency, a specific implementation could return a DirectByteBuffer to reduce
-   * the memory copy required to send an RPC
-   *
-   * <p>Note: The implementation needs to ensure that the data in the returned Bytebuffer cannot be
-   * changed or an error may occur
-   */
-  ByteBuffer serializeToByteBuffer();
+  public UnknownLogTypeException(int logType) {
+    super("Unknown log type: " + logType);
+  }
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/HardState.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/HardState.java
new file mode 100644
index 0000000000..9228c96404
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/HardState.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.consensus.natraft.protocol;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class HardState {
+
+  private long currentTerm;
+  private TEndPoint voteFor;
+
+  public HardState() {
+    this.voteFor = null;
+  }
+
+  public static HardState deserialize(ByteBuffer buffer) {
+    HardState res = new HardState();
+    res.setCurrentTerm(ReadWriteIOUtils.readLong(buffer));
+    int isNull = buffer.get();
+    if (isNull == 1) {
+      TEndPoint endPoint = ThriftCommonsSerDeUtils.deserializeTEndPoint(buffer);
+      res.setVoteFor(endPoint);
+    } else {
+      res.setVoteFor(null);
+    }
+    return res;
+  }
+
+  public ByteBuffer serialize() {
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    DataOutputStream dataOutputStream = new DataOutputStream(outputStream);
+    try {
+      dataOutputStream.writeLong(currentTerm);
+      if (voteFor == null) {
+        dataOutputStream.writeByte(0);
+      } else {
+        dataOutputStream.writeByte(1);
+        ThriftCommonsSerDeUtils.serializeTEndPoint(voteFor, dataOutputStream);
+      }
+    } catch (IOException e) {
+      // unreachable
+    }
+    return ByteBuffer.wrap(outputStream.toByteArray());
+  }
+
+  public long getCurrentTerm() {
+    return currentTerm;
+  }
+
+  public void setCurrentTerm(long currentTerm) {
+    this.currentTerm = currentTerm;
+  }
+
+  public TEndPoint getVoteFor() {
+    return voteFor;
+  }
+
+  public void setVoteFor(TEndPoint voteFor) {
+    this.voteFor = voteFor;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof HardState)) {
+      return false;
+    }
+    HardState that = (HardState) o;
+    return new EqualsBuilder()
+        .append(currentTerm, that.currentTerm)
+        .append(voteFor, that.voteFor)
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37).append(currentTerm).append(voteFor).toHashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "HardState{" + "currentTerm=" + currentTerm + ", voteFor=" + voteFor + '}';
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/PeerInfo.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/PeerInfo.java
new file mode 100644
index 0000000000..c82a0722f6
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/PeerInfo.java
@@ -0,0 +1,69 @@
+/*
+ * 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.natraft.protocol;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class PeerInfo {
+  private long nextIndex;
+  private long matchIndex;
+  private AtomicInteger inconsistentHeartbeatNum = new AtomicInteger();
+  // lastLogIndex from the last heartbeat
+  private long lastHeartBeatIndex;
+
+  public PeerInfo(long nextIndex) {
+    this.nextIndex = nextIndex;
+    this.matchIndex = -1;
+  }
+
+  public synchronized long getNextIndex() {
+    return nextIndex;
+  }
+
+  public synchronized void setNextIndex(long nextIndex) {
+    this.nextIndex = nextIndex;
+  }
+
+  public synchronized long getMatchIndex() {
+    return matchIndex;
+  }
+
+  public synchronized void setMatchIndex(long matchIndex) {
+    this.matchIndex = matchIndex;
+    this.setNextIndex(Math.max(nextIndex, matchIndex + 1));
+    this.notifyAll();
+  }
+
+  public int incInconsistentHeartbeatNum() {
+    return inconsistentHeartbeatNum.incrementAndGet();
+  }
+
+  public void resetInconsistentHeartbeatNum() {
+    inconsistentHeartbeatNum.set(0);
+  }
+
+  public long getLastHeartBeatIndex() {
+    return lastHeartBeatIndex;
+  }
+
+  public void setLastHeartBeatIndex(long lastHeartBeatIndex) {
+    this.lastHeartBeatIndex = lastHeartBeatIndex;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/RaftConfig.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/RaftConfig.java
new file mode 100644
index 0000000000..684c559e84
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/RaftConfig.java
@@ -0,0 +1,426 @@
+/*
+
+* 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.natraft.protocol;
+
+import org.apache.iotdb.consensus.config.ConsensusConfig;
+import org.apache.iotdb.consensus.config.RPCConfig;
+import org.apache.iotdb.consensus.natraft.protocol.consistency.ConsistencyLevel;
+
+import java.io.File;
+import java.util.concurrent.TimeUnit;
+
+public class RaftConfig {
+
+  private boolean enableWeakAcceptance = true;
+  private int maxNumOfLogsInMem = 10000;
+  private int minNumOfLogsInMem = 1000;
+  private long maxMemorySizeForRaftLog = 512 * 1024 * 1024L;
+  private int logDeleteCheckIntervalSecond = 1;
+  private boolean enableRaftLogPersistence;
+  private int catchUpTimeoutMS = 60_000;
+  private boolean useFollowerSlidingWindow = false;
+  private int uncommittedRaftLogNumForRejectThreshold = 10000;
+  private int heartbeatIntervalMs = 1000;
+  private int electionTimeoutMs = 20_000;
+  /** max number of clients in a ClientPool of a member for one node. */
+  private int maxClientPerNodePerMember = 1000;
+
+  /** max number of idle clients in a ClientPool of a member for one node. */
+  private int maxIdleClientPerNodePerMember = 500;
+
+  /**
+   * If the number of connections created for a node exceeds `max_client_pernode_permember_number`,
+   * we need to wait so much time for other connections to be released until timeout, or a new
+   * connection will be created.
+   */
+  private long waitClientTimeoutMS = 5 * 1000L;
+
+  /**
+   * ClientPool will have so many selector threads (TAsyncClientManager) to distribute to its
+   * clients.
+   */
+  private int selectorNumOfClientPool =
+      Runtime.getRuntime().availableProcessors() / 3 > 0
+          ? Runtime.getRuntime().availableProcessors() / 3
+          : 1;
+
+  private int connectionTimeoutInMS = (int) TimeUnit.SECONDS.toMillis(20);
+  private boolean enableUsePersistLogOnDiskToCatchUp;
+  private long writeOperationTimeoutMS = 20_000L;
+  // TODO-raft: apply to thrift
+  private int thriftMaxFrameSize = 512 * 1024 * 1024;
+  private int logNumInBatch = 100;
+  private int dispatcherBindingThreadNum = 64;
+  private int followerLoadBalanceWindowsToUse = 1;
+  private double followerLoadBalanceOverestimateFactor = 1.1;
+  private int flowMonitorMaxWindowSize = 1000;
+  private long flowMonitorWindowInterval = 1000;
+  private String storageDir = "data";
+  private long electionMaxWaitMs = 5000;
+  private long unAppliedRaftLogNumForRejectThreshold = 10000;
+  private long checkPeriodWhenInsertBlocked = 100;
+  private long maxWaitingTimeWhenInsertBlocked = 10000;
+  private boolean useFollowerLoadBalance;
+  private int raftLogBufferSize = 512 * 1024 * 1024;
+  private int maxNumberOfLogsPerFetchOnDisk = 1000;
+  private int maxRaftLogIndexSizeInMemory = 64 * 1024 * 1024;
+  private int maxNumberOfPersistRaftLogFiles = 128;
+  private int maxPersistRaftLogNumberOnDisk = 10_000_000;
+  private int flushRaftLogThreshold = 100_000;
+  private long maxSyncLogLag = 100_000;
+  private long syncLeaderMaxWaitMs = 30_000;
+  private ConsistencyLevel consistencyLevel = ConsistencyLevel.STRONG_CONSISTENCY;
+  private RPCConfig rpcConfig;
+
+  public RaftConfig(ConsensusConfig config) {
+    this.storageDir = config.getStorageDir() + File.separator + "system";
+    new File(this.storageDir).mkdirs();
+    this.rpcConfig = config.getRPCConfig();
+  }
+
+  public boolean isEnableWeakAcceptance() {
+    return enableWeakAcceptance;
+  }
+
+  public void setEnableWeakAcceptance(boolean enableWeakAcceptance) {
+    this.enableWeakAcceptance = enableWeakAcceptance;
+  }
+
+  public int getMaxNumOfLogsInMem() {
+    return maxNumOfLogsInMem;
+  }
+
+  public void setMaxNumOfLogsInMem(int maxNumOfLogsInMem) {
+    this.maxNumOfLogsInMem = maxNumOfLogsInMem;
+  }
+
+  public int getMinNumOfLogsInMem() {
+    return minNumOfLogsInMem;
+  }
+
+  public void setMinNumOfLogsInMem(int minNumOfLogsInMem) {
+    this.minNumOfLogsInMem = minNumOfLogsInMem;
+  }
+
+  public long getMaxMemorySizeForRaftLog() {
+    return maxMemorySizeForRaftLog;
+  }
+
+  public void setMaxMemorySizeForRaftLog(long maxMemorySizeForRaftLog) {
+    this.maxMemorySizeForRaftLog = maxMemorySizeForRaftLog;
+  }
+
+  public int getLogDeleteCheckIntervalSecond() {
+    return logDeleteCheckIntervalSecond;
+  }
+
+  public void setLogDeleteCheckIntervalSecond(int logDeleteCheckIntervalSecond) {
+    this.logDeleteCheckIntervalSecond = logDeleteCheckIntervalSecond;
+  }
+
+  public boolean isEnableRaftLogPersistence() {
+    return enableRaftLogPersistence;
+  }
+
+  public void setEnableRaftLogPersistence(boolean enableRaftLogPersistence) {
+    this.enableRaftLogPersistence = enableRaftLogPersistence;
+  }
+
+  public int getCatchUpTimeoutMS() {
+    return catchUpTimeoutMS;
+  }
+
+  public void setCatchUpTimeoutMS(int catchUpTimeoutMS) {
+    this.catchUpTimeoutMS = catchUpTimeoutMS;
+  }
+
+  public boolean isUseFollowerSlidingWindow() {
+    return useFollowerSlidingWindow;
+  }
+
+  public void setUseFollowerSlidingWindow(boolean useFollowerSlidingWindow) {
+    this.useFollowerSlidingWindow = useFollowerSlidingWindow;
+  }
+
+  public int getUncommittedRaftLogNumForRejectThreshold() {
+    return uncommittedRaftLogNumForRejectThreshold;
+  }
+
+  public void setUncommittedRaftLogNumForRejectThreshold(
+      int uncommittedRaftLogNumForRejectThreshold) {
+    this.uncommittedRaftLogNumForRejectThreshold = uncommittedRaftLogNumForRejectThreshold;
+  }
+
+  public int getHeartbeatIntervalMs() {
+    return heartbeatIntervalMs;
+  }
+
+  public void setHeartbeatIntervalMs(int heartbeatIntervalMs) {
+    this.heartbeatIntervalMs = heartbeatIntervalMs;
+  }
+
+  public int getElectionTimeoutMs() {
+    return electionTimeoutMs;
+  }
+
+  public void setElectionTimeoutMs(int electionTimeoutMs) {
+    this.electionTimeoutMs = electionTimeoutMs;
+  }
+
+  public int getMaxClientPerNodePerMember() {
+    return maxClientPerNodePerMember;
+  }
+
+  public void setMaxClientPerNodePerMember(int maxClientPerNodePerMember) {
+    this.maxClientPerNodePerMember = maxClientPerNodePerMember;
+  }
+
+  public int getMaxIdleClientPerNodePerMember() {
+    return maxIdleClientPerNodePerMember;
+  }
+
+  public void setMaxIdleClientPerNodePerMember(int maxIdleClientPerNodePerMember) {
+    this.maxIdleClientPerNodePerMember = maxIdleClientPerNodePerMember;
+  }
+
+  public long getWaitClientTimeoutMS() {
+    return waitClientTimeoutMS;
+  }
+
+  public void setWaitClientTimeoutMS(long waitClientTimeoutMS) {
+    this.waitClientTimeoutMS = waitClientTimeoutMS;
+  }
+
+  public int getSelectorNumOfClientPool() {
+    return selectorNumOfClientPool;
+  }
+
+  public void setSelectorNumOfClientPool(int selectorNumOfClientPool) {
+    this.selectorNumOfClientPool = selectorNumOfClientPool;
+  }
+
+  public int getConnectionTimeoutInMS() {
+    return connectionTimeoutInMS;
+  }
+
+  public void setConnectionTimeoutInMS(int connectionTimeoutInMS) {
+    this.connectionTimeoutInMS = connectionTimeoutInMS;
+  }
+
+  public boolean isEnableUsePersistLogOnDiskToCatchUp() {
+    return enableUsePersistLogOnDiskToCatchUp;
+  }
+
+  public void setEnableUsePersistLogOnDiskToCatchUp(boolean enableUsePersistLogOnDiskToCatchUp) {
+    this.enableUsePersistLogOnDiskToCatchUp = enableUsePersistLogOnDiskToCatchUp;
+  }
+
+  public long getWriteOperationTimeoutMS() {
+    return writeOperationTimeoutMS;
+  }
+
+  public void setWriteOperationTimeoutMS(long writeOperationTimeoutMS) {
+    this.writeOperationTimeoutMS = writeOperationTimeoutMS;
+  }
+
+  public int getThriftMaxFrameSize() {
+    return thriftMaxFrameSize;
+  }
+
+  public void setThriftMaxFrameSize(int thriftMaxFrameSize) {
+    this.thriftMaxFrameSize = thriftMaxFrameSize;
+  }
+
+  public int getLogNumInBatch() {
+    return logNumInBatch;
+  }
+
+  public void setLogNumInBatch(int logNumInBatch) {
+    this.logNumInBatch = logNumInBatch;
+  }
+
+  public int getDispatcherBindingThreadNum() {
+    return dispatcherBindingThreadNum;
+  }
+
+  public void setDispatcherBindingThreadNum(int dispatcherBindingThreadNum) {
+    this.dispatcherBindingThreadNum = dispatcherBindingThreadNum;
+  }
+
+  public int getFollowerLoadBalanceWindowsToUse() {
+    return followerLoadBalanceWindowsToUse;
+  }
+
+  public void setFollowerLoadBalanceWindowsToUse(int followerLoadBalanceWindowsToUse) {
+    this.followerLoadBalanceWindowsToUse = followerLoadBalanceWindowsToUse;
+  }
+
+  public double getFollowerLoadBalanceOverestimateFactor() {
+    return followerLoadBalanceOverestimateFactor;
+  }
+
+  public void setFollowerLoadBalanceOverestimateFactor(
+      double followerLoadBalanceOverestimateFactor) {
+    this.followerLoadBalanceOverestimateFactor = followerLoadBalanceOverestimateFactor;
+  }
+
+  public int getFlowMonitorMaxWindowSize() {
+    return flowMonitorMaxWindowSize;
+  }
+
+  public void setFlowMonitorMaxWindowSize(int flowMonitorMaxWindowSize) {
+    this.flowMonitorMaxWindowSize = flowMonitorMaxWindowSize;
+  }
+
+  public long getFlowMonitorWindowInterval() {
+    return flowMonitorWindowInterval;
+  }
+
+  public void setFlowMonitorWindowInterval(long flowMonitorWindowInterval) {
+    this.flowMonitorWindowInterval = flowMonitorWindowInterval;
+  }
+
+  public String getStorageDir() {
+    return storageDir;
+  }
+
+  public void setStorageDir(String storageDir) {
+    this.storageDir = storageDir;
+  }
+
+  public long getElectionMaxWaitMs() {
+    return electionMaxWaitMs;
+  }
+
+  public void setElectionMaxWaitMs(long electionMaxWaitMs) {
+    this.electionMaxWaitMs = electionMaxWaitMs;
+  }
+
+  public long getUnAppliedRaftLogNumForRejectThreshold() {
+    return unAppliedRaftLogNumForRejectThreshold;
+  }
+
+  public void setUnAppliedRaftLogNumForRejectThreshold(long unAppliedRaftLogNumForRejectThreshold) {
+    this.unAppliedRaftLogNumForRejectThreshold = unAppliedRaftLogNumForRejectThreshold;
+  }
+
+  public long getCheckPeriodWhenInsertBlocked() {
+    return checkPeriodWhenInsertBlocked;
+  }
+
+  public void setCheckPeriodWhenInsertBlocked(long checkPeriodWhenInsertBlocked) {
+    this.checkPeriodWhenInsertBlocked = checkPeriodWhenInsertBlocked;
+  }
+
+  public long getMaxWaitingTimeWhenInsertBlocked() {
+    return maxWaitingTimeWhenInsertBlocked;
+  }
+
+  public void setMaxWaitingTimeWhenInsertBlocked(long maxWaitingTimeWhenInsertBlocked) {
+    this.maxWaitingTimeWhenInsertBlocked = maxWaitingTimeWhenInsertBlocked;
+  }
+
+  public boolean isUseFollowerLoadBalance() {
+    return useFollowerLoadBalance;
+  }
+
+  public void setUseFollowerLoadBalance(boolean useFollowerLoadBalance) {
+    this.useFollowerLoadBalance = useFollowerLoadBalance;
+  }
+
+  public int getRaftLogBufferSize() {
+    return raftLogBufferSize;
+  }
+
+  public void setRaftLogBufferSize(int raftLogBufferSize) {
+    this.raftLogBufferSize = raftLogBufferSize;
+  }
+
+  public int getMaxNumberOfLogsPerFetchOnDisk() {
+    return maxNumberOfLogsPerFetchOnDisk;
+  }
+
+  public void setMaxNumberOfLogsPerFetchOnDisk(int maxNumberOfLogsPerFetchOnDisk) {
+    this.maxNumberOfLogsPerFetchOnDisk = maxNumberOfLogsPerFetchOnDisk;
+  }
+
+  public int getMaxRaftLogIndexSizeInMemory() {
+    return maxRaftLogIndexSizeInMemory;
+  }
+
+  public void setMaxRaftLogIndexSizeInMemory(int maxRaftLogIndexSizeInMemory) {
+    this.maxRaftLogIndexSizeInMemory = maxRaftLogIndexSizeInMemory;
+  }
+
+  public int getMaxNumberOfPersistRaftLogFiles() {
+    return maxNumberOfPersistRaftLogFiles;
+  }
+
+  public void setMaxNumberOfPersistRaftLogFiles(int maxNumberOfPersistRaftLogFiles) {
+    this.maxNumberOfPersistRaftLogFiles = maxNumberOfPersistRaftLogFiles;
+  }
+
+  public int getMaxPersistRaftLogNumberOnDisk() {
+    return maxPersistRaftLogNumberOnDisk;
+  }
+
+  public void setMaxPersistRaftLogNumberOnDisk(int maxPersistRaftLogNumberOnDisk) {
+    this.maxPersistRaftLogNumberOnDisk = maxPersistRaftLogNumberOnDisk;
+  }
+
+  public int getFlushRaftLogThreshold() {
+    return flushRaftLogThreshold;
+  }
+
+  public void setFlushRaftLogThreshold(int flushRaftLogThreshold) {
+    this.flushRaftLogThreshold = flushRaftLogThreshold;
+  }
+
+  public long getMaxSyncLogLag() {
+    return maxSyncLogLag;
+  }
+
+  public void setMaxSyncLogLag(long maxSyncLogLag) {
+    this.maxSyncLogLag = maxSyncLogLag;
+  }
+
+  public long getSyncLeaderMaxWaitMs() {
+    return syncLeaderMaxWaitMs;
+  }
+
+  public void setSyncLeaderMaxWaitMs(long syncLeaderMaxWaitMs) {
+    this.syncLeaderMaxWaitMs = syncLeaderMaxWaitMs;
+  }
+
+  public ConsistencyLevel getConsistencyLevel() {
+    return consistencyLevel;
+  }
+
+  public void setConsistencyLevel(ConsistencyLevel consistencyLevel) {
+    this.consistencyLevel = consistencyLevel;
+  }
+
+  public RPCConfig getRpcConfig() {
+    return rpcConfig;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/RaftMember.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/RaftMember.java
new file mode 100644
index 0000000000..549c41bcff
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/RaftMember.java
@@ -0,0 +1,1078 @@
+/*
+
+* 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.natraft.protocol;
+
+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.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.commons.consensus.ConsensusGroupId;
+import org.apache.iotdb.commons.exception.IoTDBException;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
+import org.apache.iotdb.consensus.IStateMachine;
+import org.apache.iotdb.consensus.common.DataSet;
+import org.apache.iotdb.consensus.common.request.IConsensusRequest;
+import org.apache.iotdb.consensus.common.response.ConsensusWriteResponse;
+import org.apache.iotdb.consensus.natraft.Utils.IOUtils;
+import org.apache.iotdb.consensus.natraft.Utils.StatusUtils;
+import org.apache.iotdb.consensus.natraft.client.AsyncRaftServiceClient;
+import org.apache.iotdb.consensus.natraft.client.GenericHandler;
+import org.apache.iotdb.consensus.natraft.client.SyncClientAdaptor;
+import org.apache.iotdb.consensus.natraft.exception.CheckConsistencyException;
+import org.apache.iotdb.consensus.natraft.exception.LogExecutionException;
+import org.apache.iotdb.consensus.natraft.exception.UnknownLogTypeException;
+import org.apache.iotdb.consensus.natraft.protocol.consistency.CheckConsistency;
+import org.apache.iotdb.consensus.natraft.protocol.consistency.MidCheckConsistency;
+import org.apache.iotdb.consensus.natraft.protocol.consistency.StrongCheckConsistency;
+import org.apache.iotdb.consensus.natraft.protocol.heartbeat.ElectionReqHandler;
+import org.apache.iotdb.consensus.natraft.protocol.heartbeat.HeartbeatReqHandler;
+import org.apache.iotdb.consensus.natraft.protocol.heartbeat.HeartbeatThread;
+import org.apache.iotdb.consensus.natraft.protocol.log.CommitLogCallback;
+import org.apache.iotdb.consensus.natraft.protocol.log.CommitLogTask;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+import org.apache.iotdb.consensus.natraft.protocol.log.LogParser;
+import org.apache.iotdb.consensus.natraft.protocol.log.VotingLog;
+import org.apache.iotdb.consensus.natraft.protocol.log.appender.BlockingLogAppender;
+import org.apache.iotdb.consensus.natraft.protocol.log.appender.LogAppender;
+import org.apache.iotdb.consensus.natraft.protocol.log.appender.LogAppenderFactory;
+import org.apache.iotdb.consensus.natraft.protocol.log.applier.AsyncLogApplier;
+import org.apache.iotdb.consensus.natraft.protocol.log.applier.BaseApplier;
+import org.apache.iotdb.consensus.natraft.protocol.log.catchup.CatchUpManager;
+import org.apache.iotdb.consensus.natraft.protocol.log.dispatch.LogDispatcher;
+import org.apache.iotdb.consensus.natraft.protocol.log.dispatch.VotingLogList;
+import org.apache.iotdb.consensus.natraft.protocol.log.flowcontrol.FlowBalancer;
+import org.apache.iotdb.consensus.natraft.protocol.log.flowcontrol.FlowMonitorManager;
+import org.apache.iotdb.consensus.natraft.protocol.log.logtype.RequestEntry;
+import org.apache.iotdb.consensus.natraft.protocol.log.manager.RaftLogManager;
+import org.apache.iotdb.consensus.natraft.protocol.log.manager.SnapshotRaftLogManager;
+import org.apache.iotdb.consensus.natraft.protocol.log.sequencing.LogSequencer;
+import org.apache.iotdb.consensus.natraft.protocol.log.sequencing.LogSequencerFactory;
+import org.apache.iotdb.consensus.natraft.protocol.log.sequencing.SynchronousSequencer;
+import org.apache.iotdb.consensus.natraft.protocol.log.serialization.SyncLogDequeSerializer;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntriesRequest;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntryResult;
+import org.apache.iotdb.consensus.raft.thrift.ElectionRequest;
+import org.apache.iotdb.consensus.raft.thrift.HeartBeatRequest;
+import org.apache.iotdb.consensus.raft.thrift.HeartBeatResponse;
+import org.apache.iotdb.consensus.raft.thrift.RequestCommitIndexResponse;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+public class RaftMember {
+
+  private static final String CONFIGURATION_FILE_NAME = "configuration.dat";
+  private static final Logger logger = LoggerFactory.getLogger(RaftMember.class);
+
+  private RaftConfig config;
+  private boolean enableWeakAcceptance;
+  protected static final LogAppenderFactory appenderFactory = new BlockingLogAppender.Factory();
+
+  protected static final LogSequencerFactory SEQUENCER_FACTORY = new SynchronousSequencer.Factory();
+
+  private static final String MSG_FORWARD_TIMEOUT = "{}: Forward {} to {} time out";
+  private static final String MSG_FORWARD_ERROR =
+      "{}: encountered an error when forwarding {} to" + " {}";
+  private static final String MSG_NO_LEADER_COMMIT_INDEX =
+      "{}: Cannot request commit index from {}";
+  private static final String MSG_NO_LEADER_IN_SYNC = "{}: No leader is found when synchronizing";
+  public static final String MSG_LOG_IS_ACCEPTED = "{}: log {} is accepted";
+
+  /**
+   * when the leader of this node changes, the condition will be notified so other threads that wait
+   * on this may be woken.
+   */
+  private final Object waitLeaderCondition = new Object();
+
+  /** the lock is to make sure that only one thread can apply snapshot at the same time */
+  private final Lock snapshotApplyLock = new ReentrantLock();
+  /**
+   * when the commit progress is updated by a heartbeat, this object is notified so that we may know
+   * if this node is up-to-date with the leader, and whether the given consistency is reached
+   */
+  private Object syncLock = new Object();
+
+  protected TEndPoint thisNode;
+  /** the nodes that belong to the same raft group as thisNode. */
+  protected List<TEndPoint> allNodes;
+
+  protected ConsensusGroupId groupId;
+  protected String name;
+  protected String storageDir;
+
+  protected RaftStatus status = new RaftStatus();
+
+  /** the raft logs are all stored and maintained in the log manager */
+  protected RaftLogManager logManager;
+
+  protected HeartbeatThread heartbeatThread;
+  protected HeartbeatReqHandler heartbeatReqHandler;
+  protected ElectionReqHandler electionReqHandler;
+  /**
+   * if set to true, the node will reject all log appends when the header of a group is removed from
+   * the cluster, the members of the group should no longer accept writes, but they still can be
+   * candidates for weak consistency reads and provide snapshots for the new data holders
+   */
+  volatile boolean readOnly = false;
+  /**
+   * lastLogIndex when generating the previous member report, to show the log ingestion rate of the
+   * member by comparing it with the current last log index.
+   */
+  long lastReportedLogIndex;
+
+  /**
+   * client manager that provides reusable Thrift clients to connect to other RaftMembers and
+   * execute RPC requests. It will be initialized according to the implementation of the subclasses
+   */
+  protected IClientManager<TEndPoint, AsyncRaftServiceClient> clientManager;
+
+  protected CatchUpManager catchUpManager;
+  /** a thread pool that is used to do commit log tasks asynchronous in heartbeat thread */
+  private ExecutorService commitLogPool;
+
+  /**
+   * logDispatcher buff the logs orderly according to their log indexes and send them sequentially,
+   * which avoids the followers receiving out-of-order logs, forcing them to wait for previous logs.
+   */
+  private volatile LogDispatcher logDispatcher;
+
+  private VotingLogList votingLogList;
+  private volatile boolean stopped;
+  protected IStateMachine stateMachine;
+  protected LogSequencer logSequencer;
+  private volatile LogAppender logAppender;
+  private FlowBalancer flowBalancer;
+
+  public RaftMember(
+      RaftConfig config,
+      TEndPoint thisNode,
+      List<TEndPoint> allNodes,
+      ConsensusGroupId groupId,
+      IStateMachine stateMachine,
+      IClientManager<TEndPoint, AsyncRaftServiceClient> clientManager) {
+    this.config = config;
+    initConfig();
+
+    this.thisNode = thisNode;
+    this.allNodes = allNodes;
+    if (allNodes.isEmpty()) {
+      recoverConfiguration();
+    } else {
+      persistConfiguration();
+    }
+
+    this.groupId = groupId;
+    this.name = groupId + "-" + thisNode;
+
+    this.clientManager = clientManager;
+    this.stateMachine = stateMachine;
+    this.logManager =
+        new SnapshotRaftLogManager(
+            new SyncLogDequeSerializer(groupId, config),
+            new AsyncLogApplier(new BaseApplier(stateMachine), name, config),
+            name,
+            stateMachine);
+    this.votingLogList = new VotingLogList(allNodes.size() / 2, this);
+    this.logAppender = appenderFactory.create(this, config);
+    this.logSequencer = SEQUENCER_FACTORY.create(this, logManager, config);
+    this.heartbeatReqHandler = new HeartbeatReqHandler(this);
+
+    initPeerMap();
+  }
+
+  public void recoverConfiguration() {
+    ByteBuffer buffer;
+    try {
+      buffer =
+          ByteBuffer.wrap(
+              Files.readAllBytes(
+                  Paths.get(new File(storageDir, CONFIGURATION_FILE_NAME).getAbsolutePath())));
+      int size = buffer.getInt();
+      for (int i = 0; i < size; i++) {
+        allNodes.add(ThriftCommonsSerDeUtils.deserializeTEndPoint(buffer));
+      }
+      logger.info("Recover Raft, configuration: {}", allNodes);
+    } catch (IOException e) {
+      logger.error("Unexpected error occurs when recovering configuration", e);
+    }
+  }
+
+  public void persistConfiguration() {
+    try (PublicBAOS publicBAOS = new PublicBAOS();
+        DataOutputStream outputStream = new DataOutputStream(publicBAOS)) {
+      outputStream.writeInt(allNodes.size());
+      for (TEndPoint node : allNodes) {
+        ThriftCommonsSerDeUtils.serializeTEndPoint(node, outputStream);
+      }
+      Files.write(
+          Paths.get(new File(storageDir, CONFIGURATION_FILE_NAME).getAbsolutePath()),
+          publicBAOS.getBuf());
+    } catch (IOException e) {
+      logger.error("Unexpected error occurs when persisting configuration", e);
+    }
+  }
+
+  /**
+   * Start the heartbeat thread and the catch-up thread pool. Calling the method twice does not
+   * induce side effects.
+   */
+  public void start() {
+    if (heartbeatThread != null) {
+      return;
+    }
+
+    stateMachine.start();
+    getLogDispatcher();
+
+    startBackGroundThreads();
+    setStopped(false);
+    logger.info("{} started", name);
+  }
+
+  void startBackGroundThreads() {
+    catchUpManager = new CatchUpManager(this, config);
+    catchUpManager.start();
+    commitLogPool = IoTDBThreadPoolFactory.newSingleThreadExecutor("RaftCommitLog");
+    if (config.isUseFollowerLoadBalance()) {
+      FlowMonitorManager.INSTANCE.register(thisNode);
+      flowBalancer = new FlowBalancer(logDispatcher, this, config);
+      flowBalancer.start();
+    }
+  }
+
+  private void initConfig() {
+    this.enableWeakAcceptance = config.isEnableWeakAcceptance();
+    this.storageDir = config.getStorageDir();
+  }
+
+  public void initPeerMap() {
+    status.peerMap = new ConcurrentHashMap<>();
+    for (TEndPoint entry : allNodes) {
+      status.peerMap.computeIfAbsent(entry, k -> new PeerInfo(logManager.getLastLogIndex()));
+    }
+  }
+
+  /**
+   * Stop the heartbeat thread and the catch-up thread pool. Calling the method twice does not
+   * induce side effects.
+   */
+  public void stop() {
+    setStopped(true);
+    closeLogManager();
+    if (clientManager != null) {
+      clientManager.close();
+    }
+    if (logSequencer != null) {
+      logSequencer.close();
+    }
+
+    if (heartbeatThread == null) {
+      return;
+    }
+
+    heartbeatThread.stop();
+    catchUpManager.stop();
+
+    if (commitLogPool != null) {
+      commitLogPool.shutdownNow();
+      try {
+        commitLogPool.awaitTermination(60, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        logger.error("Unexpected interruption when waiting for commitLogPool to end", e);
+      }
+    }
+
+    status.leader.set(null);
+    catchUpManager = null;
+    heartbeatThread = null;
+
+    if (flowBalancer != null) {
+      flowBalancer.stop();
+    }
+    logger.info("Member {} stopped", name);
+  }
+
+  public void closeLogManager() {
+    if (logManager != null) {
+      logManager.close();
+    }
+  }
+
+  /**
+   * If "newTerm" is larger than the local term, give up the leadership, become a follower and reset
+   * heartbeat timer.
+   */
+  public void stepDown(long newTerm, TEndPoint newLeader) {
+    try {
+      logManager.getLock().writeLock().lock();
+      long currTerm = status.term.get();
+      // confirm that the heartbeat of the new leader hasn't come
+      if (currTerm < newTerm) {
+        logger.info("{} has update it's term to {}", getName(), newTerm);
+        status.term.set(newTerm);
+        status.setVoteFor(null);
+        status.setRole(RaftRole.CANDIDATE);
+        status.getLeader().set(null);
+        updateHardState(newTerm, status.getVoteFor());
+      }
+
+      if (currTerm <= newTerm && newLeader != null) {
+        // only when the request is from a leader should we update lastHeartbeatReceivedTime,
+        // otherwise the node may be stuck in FOLLOWER state by a stale node.
+        status.getLeader().set(newLeader);
+        status.setRole(RaftRole.FOLLOWER);
+        heartbeatThread.setLastHeartbeatReceivedTime(System.currentTimeMillis());
+      }
+    } finally {
+      logManager.getLock().writeLock().unlock();
+    }
+  }
+
+  public void updateHardState(long currentTerm, TEndPoint voteFor) {
+    HardState state = logManager.getHardState();
+    state.setCurrentTerm(currentTerm);
+    state.setVoteFor(voteFor);
+    logManager.updateHardState(state);
+  }
+
+  public void tryUpdateCommitIndex(long leaderTerm, long commitIndex, long commitTerm) {
+    if (leaderTerm >= status.term.get() && logManager.getCommitLogIndex() < commitIndex) {
+      // there are more local logs that can be committed, commit them in a ThreadPool so the
+      // heartbeat response will not be blocked
+      CommitLogTask commitLogTask = new CommitLogTask(logManager, commitIndex, commitTerm);
+      commitLogTask.registerCallback(new CommitLogCallback(this));
+      // if the log is not consistent, the commitment will be blocked until the leader makes the
+      // node catch up
+      if (commitLogPool != null && !commitLogPool.isShutdown()) {
+        commitLogPool.submit(commitLogTask);
+      }
+
+      logger.debug(
+          "{}: Inconsistent log found, leaderCommit: {}-{}, localCommit: {}-{}, "
+              + "localLast: {}-{}",
+          name,
+          commitIndex,
+          commitTerm,
+          logManager.getCommitLogIndex(),
+          logManager.getCommitLogTerm(),
+          logManager.getLastLogIndex(),
+          logManager.getLastLogTerm());
+    }
+  }
+
+  public void catchUp(TEndPoint follower, long lastLogIdx) {
+    catchUpManager.catchUp(follower, lastLogIdx);
+  }
+
+  /**
+   * Process the HeartBeatRequest from the leader. If the term of the leader is smaller than the
+   * local term, reject the request by telling it the newest term. Else if the local logs are
+   * consistent with the leader's, commit them. Else help the leader find the last matched log. Also
+   * update the leadership, heartbeat timer and term of the local node.
+   */
+  public HeartBeatResponse processHeartbeatRequest(HeartBeatRequest request) {
+    return heartbeatReqHandler.processHeartbeatRequest(request);
+  }
+
+  /**
+   * Process an ElectionRequest. If the request comes from the last leader, accept it. Else decide
+   * whether to accept by examining the log status of the elector.
+   */
+  public long processElectionRequest(ElectionRequest electionRequest) {
+    return electionReqHandler.processElectionRequest(electionRequest);
+  }
+
+  public AppendEntryResult appendEntries(AppendEntriesRequest request)
+      throws UnknownLogTypeException {
+    return appendEntriesInternal(request);
+  }
+
+  /** Similar to appendEntry, while the incoming load is batch of logs instead of a single log. */
+  private AppendEntryResult appendEntriesInternal(AppendEntriesRequest request)
+      throws UnknownLogTypeException {
+    logger.debug("{} received an AppendEntriesRequest", name);
+
+    // the term checked here is that of the leader, not that of the log
+    long checkResult = checkRequestTerm(request.term, request.leader);
+    if (checkResult != Response.RESPONSE_AGREE) {
+      return new AppendEntryResult(checkResult)
+          .setGroupId(getRaftGroupId().convertToTConsensusGroupId());
+    }
+
+    AppendEntryResult response;
+    List<Entry> entries = new ArrayList<>();
+    for (ByteBuffer buffer : request.getEntries()) {
+      buffer.mark();
+      Entry e;
+      try {
+        e = LogParser.getINSTANCE().parse(buffer);
+        e.setByteSize(buffer.limit() - buffer.position());
+      } catch (BufferUnderflowException ex) {
+        buffer.reset();
+        throw ex;
+      }
+      entries.add(e);
+    }
+
+    response = logAppender.appendEntries(request, entries);
+
+    if (logger.isDebugEnabled()) {
+      logger.debug(
+          "{} AppendEntriesRequest of log size {} completed with result {}",
+          name,
+          request.getEntries().size(),
+          response);
+    }
+
+    return response;
+  }
+
+  /**
+   * Check the term of the AppendEntryRequest. The term checked is the term of the leader, not the
+   * term of the log. A new leader can still send logs of old leaders.
+   *
+   * @return -1 if the check is passed, >0 otherwise
+   */
+  private long checkRequestTerm(long leaderTerm, TEndPoint leader) {
+    long localTerm;
+
+    // if the request comes before the heartbeat arrives, the local term may be smaller than the
+    // leader term
+    localTerm = status.term.get();
+    if (leaderTerm < localTerm) {
+      logger.debug(
+          "{} rejected the AppendEntriesRequest for term: {}/{}", name, leaderTerm, localTerm);
+      return localTerm;
+    } else {
+      if (leaderTerm > localTerm) {
+        stepDown(leaderTerm, leader);
+      } else {
+        heartbeatThread.setLastHeartbeatReceivedTime(System.currentTimeMillis());
+      }
+      status.getLeader().set(leader);
+    }
+
+    logger.debug("{} accepted the AppendEntryRequest for term: {}", name, localTerm);
+    return Response.RESPONSE_AGREE;
+  }
+
+  private boolean checkLogSize(Entry entry) {
+    return !config.isEnableRaftLogPersistence()
+        || entry.serialize().capacity() + Integer.BYTES < config.getRaftLogBufferSize();
+  }
+
+  public boolean isReadOnly() {
+    return readOnly;
+  }
+
+  public DataSet read(IConsensusRequest request) throws CheckConsistencyException {
+    syncLeaderWithConsistencyCheck(false);
+    return stateMachine.read(request);
+  }
+
+  protected enum AppendLogResult {
+    OK,
+    TIME_OUT,
+    LEADERSHIP_STALE,
+    WEAK_ACCEPT
+  }
+
+  public TSStatus processRequest(IConsensusRequest request) {
+    if (readOnly) {
+      return StatusUtils.NODE_READ_ONLY;
+    }
+
+    logger.debug("{}: Processing request {}", name, request);
+    Entry entry = new RequestEntry(request);
+
+    // just like processPlanLocally,we need to check the size of log
+    if (!checkLogSize(entry)) {
+      logger.error(
+          "Log cannot fit into buffer, please increase raft_log_buffer_size;"
+              + "or reduce the size of requests you send.");
+      return StatusUtils.INTERNAL_ERROR.deepCopy().setMessage("Log cannot fit into buffer");
+    }
+
+    // assign term and index to the new log and append it
+    VotingLog sendLogRequest = logSequencer.sequence(entry);
+    FlowMonitorManager.INSTANCE.report(thisNode, entry.estimateSize());
+
+    if (sendLogRequest == null) {
+      return StatusUtils.getStatus(TSStatusCode.WRITE_PROCESS_REJECT);
+    }
+
+    try {
+      AppendLogResult appendLogResult =
+          waitAppendResult(sendLogRequest, sendLogRequest.getQuorumSize());
+      switch (appendLogResult) {
+        case WEAK_ACCEPT:
+          return includeLogNumbersInStatus(
+              StatusUtils.getStatus(TSStatusCode.WEAKLY_ACCEPTED), entry);
+        case OK:
+          waitApply(entry);
+          return includeLogNumbersInStatus(StatusUtils.OK.deepCopy(), entry);
+        case TIME_OUT:
+          logger.debug("{}: log {} timed out...", name, entry);
+          break;
+        case LEADERSHIP_STALE:
+          // abort the appending, the new leader will fix the local logs by catch-up
+        default:
+          break;
+      }
+    } catch (LogExecutionException e) {
+      return handleLogExecutionException(entry, IOUtils.getRootCause(e));
+    }
+    return StatusUtils.TIME_OUT;
+  }
+
+  protected void waitApply(Entry entry) throws LogExecutionException {
+    // when using async applier, the log here may not be applied. To return the execution
+    // result, we must wait until the log is applied.
+    synchronized (entry) {
+      while (!entry.isApplied()) {
+        // wait until the log is applied
+        try {
+          entry.wait(1);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new LogExecutionException(e);
+        }
+      }
+    }
+    if (entry.getException() != null) {
+      throw new LogExecutionException(entry.getException());
+    }
+  }
+
+  private TSStatus includeLogNumbersInStatus(TSStatus status, Entry entry) {
+    return status.setMessage(
+        getRaftGroupId() + "-" + entry.getCurrLogIndex() + "-" + entry.getCurrLogTerm());
+  }
+
+  protected AppendLogResult waitAppendResult(VotingLog log, int quorumSize) {
+    // wait for the followers to vote
+    int totalAccepted = votingLogList.totalAcceptedNodeNum(log);
+    int weaklyAccepted = log.getWeaklyAcceptedNodes().size();
+    int stronglyAccepted = totalAccepted - weaklyAccepted;
+
+    if (log.getEntry().getCurrLogIndex() == Long.MIN_VALUE
+        || ((stronglyAccepted < quorumSize
+            || (!enableWeakAcceptance || (totalAccepted < quorumSize)) && !log.isHasFailed()))) {
+      waitAppendResultLoop(log, quorumSize);
+    }
+    totalAccepted = votingLogList.totalAcceptedNodeNum(log);
+    weaklyAccepted = log.getWeaklyAcceptedNodes().size();
+    stronglyAccepted = totalAccepted - weaklyAccepted;
+
+    // a node has a larger status.term than the local node, so this node is no longer a valid leader
+    if (status.term.get() != log.getEntry().getCurrLogTerm()) {
+      return AppendLogResult.LEADERSHIP_STALE;
+    }
+    // the node knows it is no longer the leader from other requests
+    if (status.role != RaftRole.LEADER) {
+      return AppendLogResult.LEADERSHIP_STALE;
+    }
+
+    if (totalAccepted >= quorumSize && stronglyAccepted < quorumSize) {
+      return AppendLogResult.WEAK_ACCEPT;
+    }
+
+    // cannot get enough agreements within a certain amount of time
+    if (totalAccepted < quorumSize) {
+      return AppendLogResult.TIME_OUT;
+    }
+
+    // voteCounter has counted down to zero
+    return AppendLogResult.OK;
+  }
+
+  protected TSStatus handleLogExecutionException(Object log, Throwable cause) {
+    TSStatus tsStatus =
+        StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, cause.getMessage());
+    if (cause instanceof RuntimeException) {
+      logger.error("RuntimeException during executing {}", log, cause);
+    }
+    if (cause instanceof IoTDBException) {
+      tsStatus.setCode(((IoTDBException) cause).getErrorCode());
+    }
+    return tsStatus;
+  }
+
+  /**
+   * wait until "voteCounter" counts down to zero, which means the quorum has received the log, or
+   * one follower tells the node that it is no longer a valid leader, or a timeout is triggered.
+   */
+  @SuppressWarnings({"java:S2445"}) // safe synchronized
+  private void waitAppendResultLoop(VotingLog log, int quorumSize) {
+    int totalAccepted = votingLogList.totalAcceptedNodeNum(log);
+    int weaklyAccepted = log.getWeaklyAcceptedNodes().size();
+    int stronglyAccepted = totalAccepted - weaklyAccepted;
+    long nextTimeToPrint = 5000;
+
+    long waitStart = System.nanoTime();
+    long alreadyWait = 0;
+
+    String threadBaseName = Thread.currentThread().getName();
+    long waitTime = 1;
+    synchronized (log.getEntry()) {
+      while (log.getEntry().getCurrLogIndex() == Long.MIN_VALUE
+          || (stronglyAccepted < quorumSize
+              && (!(enableWeakAcceptance || (totalAccepted < quorumSize))
+                  && alreadyWait < config.getWriteOperationTimeoutMS()
+                  && !log.isHasFailed()))) {
+        try {
+          log.getEntry().wait(waitTime);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          logger.warn("Unexpected interruption when sending a log", e);
+        }
+        waitTime = waitTime * 2;
+
+        alreadyWait = (System.nanoTime() - waitStart) / 1000000;
+        if (alreadyWait > nextTimeToPrint) {
+          logger.info(
+              "Still not receive enough votes for {}, weakly " + "accepted {}, wait {}ms ",
+              log,
+              log.getWeaklyAcceptedNodes(),
+              alreadyWait);
+          nextTimeToPrint *= 2;
+        }
+        totalAccepted = votingLogList.totalAcceptedNodeNum(log);
+        weaklyAccepted = log.getWeaklyAcceptedNodes().size();
+        stronglyAccepted = totalAccepted - weaklyAccepted;
+      }
+    }
+    if (logger.isDebugEnabled()) {
+      Thread.currentThread().setName(threadBaseName);
+    }
+
+    if (alreadyWait > 15000) {
+      logger.info(
+          "Slow entry {}, weakly " + "accepted {}, waited time {}ms",
+          log,
+          log.getWeaklyAcceptedNodes(),
+          alreadyWait);
+    }
+  }
+
+  public ConsensusWriteResponse executeForwardedRequest(IConsensusRequest request) {
+    return new ConsensusWriteResponse(null, processRequest(request));
+  }
+
+  /**
+   * according to the consistency configuration, decide whether to execute syncLeader or not and
+   * throws exception when failed. Note that the write request will always try to sync leader
+   */
+  public void syncLeaderWithConsistencyCheck(boolean isWriteRequest)
+      throws CheckConsistencyException {
+    if (isWriteRequest) {
+      syncLeader(new StrongCheckConsistency());
+    } else {
+      switch (config.getConsistencyLevel()) {
+        case STRONG_CONSISTENCY:
+          syncLeader(new StrongCheckConsistency());
+          return;
+        case MID_CONSISTENCY:
+          // if leaderCommitId bigger than localAppliedId a value,
+          // will throw CHECK_MID_CONSISTENCY_EXCEPTION
+          syncLeader(new MidCheckConsistency(config.getMaxSyncLogLag()));
+          return;
+        case WEAK_CONSISTENCY:
+          // do nothing
+          return;
+        default:
+          // this should not happen in theory
+          throw new CheckConsistencyException(
+              "unknown consistency=" + config.getConsistencyLevel().name());
+      }
+    }
+  }
+
+  public boolean syncLeader(CheckConsistency checkConsistency) throws CheckConsistencyException {
+    if (status.role == RaftRole.LEADER) {
+      return true;
+    }
+    waitLeader();
+    if (status.leader.get() == null || status.leader.get() == null) {
+      // the leader has not been elected, we must assume the node falls behind
+      logger.warn(MSG_NO_LEADER_IN_SYNC, name);
+      return false;
+    }
+    if (status.role == RaftRole.LEADER) {
+      return true;
+    }
+    logger.debug("{}: try synchronizing with the leader {}", name, status.leader.get());
+    return waitUntilCatchUp(checkConsistency);
+  }
+
+  /**
+   * Request the leader's commit index and wait until the local commit index becomes not less than
+   * it.
+   *
+   * @return true if this node has caught up before timeout, false otherwise
+   * @throws CheckConsistencyException if leaderCommitId bigger than localAppliedId a threshold
+   *     value after timeout
+   */
+  protected boolean waitUntilCatchUp(CheckConsistency checkConsistency)
+      throws CheckConsistencyException {
+    long leaderCommitId = Long.MIN_VALUE;
+    RequestCommitIndexResponse response;
+    try {
+      response = requestCommitIdAsync();
+      leaderCommitId = response.getCommitLogIndex();
+
+      tryUpdateCommitIndex(
+          response.getTerm(), response.getCommitLogIndex(), response.getCommitLogTerm());
+
+      return syncLocalApply(leaderCommitId, true);
+    } catch (TException e) {
+      logger.error(MSG_NO_LEADER_COMMIT_INDEX, name, status.leader.get(), e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      logger.error(MSG_NO_LEADER_COMMIT_INDEX, name, status.leader.get(), e);
+    } finally {
+      if (checkConsistency != null) {
+        checkConsistency.postCheckConsistency(leaderCommitId, logManager.getAppliedIndex());
+      }
+    }
+    logger.debug("Start to sync with leader, leader commit id is {}", leaderCommitId);
+    return false;
+  }
+
+  /**
+   * sync local applyId to leader commitId
+   *
+   * @param leaderCommitId leader commit id
+   * @param fastFail if enabled, when log differ too much, return false directly.
+   * @return true if leaderCommitId <= localAppliedId
+   */
+  public boolean syncLocalApply(long leaderCommitId, boolean fastFail) {
+    long startTime = System.currentTimeMillis();
+    long waitedTime = 0;
+    long localAppliedId;
+
+    if (fastFail && leaderCommitId - logManager.getAppliedIndex() > config.getMaxSyncLogLag()) {
+      logger.info(
+          "{}: The raft log of this member is too backward to provide service directly.", name);
+      return false;
+    }
+
+    while (waitedTime < config.getSyncLeaderMaxWaitMs()) {
+      try {
+        localAppliedId = logManager.getAppliedIndex();
+        logger.debug("{}: synchronizing commitIndex {}/{}", name, localAppliedId, leaderCommitId);
+        if (leaderCommitId <= localAppliedId) {
+          // this node has caught up
+          if (logger.isDebugEnabled()) {
+            waitedTime = System.currentTimeMillis() - startTime;
+            logger.debug(
+                "{}: synchronized to target index {} after {}ms", name, leaderCommitId, waitedTime);
+          }
+          return true;
+        }
+        // wait for next heartbeat to catch up
+        // the local node will not perform a commit here according to the leaderCommitId because
+        // the node may have some inconsistent logs with the leader
+        waitedTime = System.currentTimeMillis() - startTime;
+        synchronized (syncLock) {
+          syncLock.wait(config.getHeartbeatIntervalMs());
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        logger.error(MSG_NO_LEADER_COMMIT_INDEX, name, status.leader.get(), e);
+      }
+    }
+    logger.warn(
+        "{}: Failed to synchronize to target index {} after {}ms",
+        name,
+        leaderCommitId,
+        waitedTime);
+    return false;
+  }
+
+  /** Wait until the leader of this node becomes known or time out. */
+  public void waitLeader() {
+    long startTime = System.currentTimeMillis();
+    while (status.leader.get() == null || status.leader.get() == null) {
+      synchronized (waitLeaderCondition) {
+        try {
+          waitLeaderCondition.wait(10);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          logger.error("Unexpected interruption when waiting for a leader", e);
+        }
+      }
+      long consumedTime = System.currentTimeMillis() - startTime;
+      if (consumedTime >= config.getConnectionTimeoutInMS()) {
+        logger.warn("{}: leader is still offline after {}ms", name, consumedTime);
+        break;
+      }
+    }
+    logger.debug("{}: current leader is {}", name, status.leader.get());
+  }
+
+  protected RequestCommitIndexResponse requestCommitIdAsync()
+      throws TException, InterruptedException {
+    // use Long.MAX_VALUE to indicate a timeout
+    RequestCommitIndexResponse response = new RequestCommitIndexResponse(-1, -1, -1);
+    AsyncRaftServiceClient client = getClient(status.leader.get());
+    if (client == null) {
+      // cannot connect to the leader
+      logger.warn(MSG_NO_LEADER_IN_SYNC, name);
+      return response;
+    }
+    GenericHandler<RequestCommitIndexResponse> handler = new GenericHandler<>(status.leader.get());
+    client.requestCommitIndex(getRaftGroupId().convertToTConsensusGroupId(), handler);
+    return handler.getResult(config.getConnectionTimeoutInMS());
+  }
+
+  /** @return true if there is a log whose index is "index" and term is "term", false otherwise */
+  public boolean matchLog(long index, long term) {
+    boolean matched = logManager.matchTerm(term, index);
+    logger.debug("Log {}-{} matched: {}", index, term, matched);
+    return matched;
+  }
+
+  /**
+   * Forward a non-query plan to a node using the default client.
+   *
+   * @param plan a non-query plan
+   * @param node cannot be the local node
+   * @param groupId must be set for data group communication, set to null for meta group
+   *     communication
+   * @return a TSStatus indicating if the forwarding is successful.
+   */
+  public TSStatus forwardRequest(IConsensusRequest plan, TEndPoint node, ConsensusGroupId groupId) {
+    if (node == null || node.equals(thisNode)) {
+      logger.debug("{}: plan {} has no where to be forwarded", name, plan);
+      return StatusUtils.NO_LEADER;
+    }
+    logger.debug("{}: Forward {} to node {}", name, plan, node);
+
+    TSStatus status;
+    status = forwardPlanAsync(plan, node, groupId);
+    if (status.getCode() == TSStatusCode.NO_CONNECTION.getStatusCode()
+        && (groupId == null || groupId.equals(getRaftGroupId()))
+        && (this.status.leader.get() != null)
+        && this.status.leader.get().equals(node)) {
+      // leader is down, trigger a new election by resetting heartbeat
+      heartbeatThread.setLastHeartbeatReceivedTime(-1);
+      this.status.leader.set(null);
+      waitLeader();
+    }
+    return status;
+  }
+
+  /**
+   * Forward a non-query plan to "receiver" using "client".
+   *
+   * @param plan a non-query plan
+   * @param groupId to determine which DataGroupMember of "receiver" will process the request.
+   * @return a TSStatus indicating if the forwarding is successful.
+   */
+  private TSStatus forwardPlanAsync(
+      IConsensusRequest plan, TEndPoint receiver, ConsensusGroupId groupId) {
+    AsyncRaftServiceClient client = getClient(receiver);
+    if (client == null) {
+      logger.debug("{}: can not get client for node={}", name, receiver);
+      return StatusUtils.NO_CONNECTION
+          .deepCopy()
+          .setMessage(String.format("%s cannot be reached", receiver));
+    }
+    return forwardPlanAsync(plan, receiver, groupId, client);
+  }
+
+  public TSStatus forwardPlanAsync(
+      IConsensusRequest request,
+      TEndPoint receiver,
+      ConsensusGroupId header,
+      AsyncRaftServiceClient client) {
+    try {
+      TSStatus tsStatus = SyncClientAdaptor.executeRequest(client, request, header, receiver);
+      if (tsStatus == null) {
+        tsStatus = StatusUtils.TIME_OUT;
+        logger.warn(MSG_FORWARD_TIMEOUT, name, request, receiver);
+      }
+      return tsStatus;
+    } catch (IOException | TException e) {
+      logger.error(MSG_FORWARD_ERROR, name, request, receiver, e);
+      return StatusUtils.getStatus(StatusUtils.INTERNAL_ERROR, e.getMessage());
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      logger.warn("{}: forward {} to {} interrupted", name, request, receiver);
+      return StatusUtils.TIME_OUT;
+    }
+  }
+
+  public void installSnapshot(ByteBuffer snapshotBytes) {
+    // TODO-raft: implement
+  }
+
+  public boolean containsNode(TEndPoint node) {
+    for (TEndPoint localNode : allNodes) {
+      if (localNode.equals(node)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public TEndPoint getLeader() {
+    return status.leader.get();
+  }
+
+  public void setLeader(TEndPoint leader) {
+    if (!Objects.equals(leader, this.status.leader.get())) {
+      if (leader == null) {
+        logger.info("{} has been set to null in term {}", getName(), status.term.get());
+      } else if (!Objects.equals(leader, this.thisNode)) {
+        logger.info(
+            "{} has become a follower of {} in term {}", getName(), leader, status.term.get());
+      }
+      synchronized (waitLeaderCondition) {
+        this.status.leader.set(leader);
+        if (this.status.leader.get() != null) {
+          waitLeaderCondition.notifyAll();
+        }
+      }
+    }
+  }
+
+  public RequestCommitIndexResponse requestCommitIndex() {
+    long commitIndex;
+    long commitTerm;
+    long curTerm;
+    try {
+      logManager.getLock().readLock().lock();
+      commitIndex = logManager.getCommitLogIndex();
+      commitTerm = logManager.getCommitLogTerm();
+      curTerm = status.getTerm().get();
+    } finally {
+      logManager.getLock().readLock().unlock();
+    }
+
+    return new RequestCommitIndexResponse(curTerm, commitIndex, commitTerm);
+  }
+
+  public IStateMachine getStateMachine() {
+    return stateMachine;
+  }
+
+  public ConsensusGroupId getRaftGroupId() {
+    return groupId;
+  }
+
+  public VotingLogList getVotingLogList() {
+    return votingLogList;
+  }
+
+  public boolean isStopped() {
+    return stopped;
+  }
+
+  public void setStopped(boolean stopped) {
+    this.stopped = stopped;
+  }
+
+  public LogDispatcher getLogDispatcher() {
+    return logDispatcher;
+  }
+
+  public VotingLog buildVotingLog(Entry e) {
+    return new VotingLog(e, allNodes.size(), null, allNodes.size() / 2, config);
+  }
+
+  public HeartbeatThread getHeartbeatThread() {
+    return heartbeatThread;
+  }
+
+  public Lock getSnapshotApplyLock() {
+    return snapshotApplyLock;
+  }
+
+  public Object getSyncLock() {
+    return syncLock;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public RaftRole getRole() {
+    return status.role;
+  }
+
+  public RaftStatus getStatus() {
+    return status;
+  }
+
+  public RaftLogManager getLogManager() {
+    return logManager;
+  }
+
+  public TEndPoint getThisNode() {
+    return thisNode;
+  }
+
+  public List<TEndPoint> getAllNodes() {
+    return allNodes;
+  }
+
+  public AsyncRaftServiceClient getHeartbeatClient(TEndPoint node) {
+    try {
+      return clientManager.borrowClient(node);
+    } catch (Exception e) {
+      logger.error("borrow async heartbeat client fail", e);
+      return null;
+    }
+  }
+
+  public AsyncRaftServiceClient getClient(TEndPoint node) {
+    try {
+      return clientManager.borrowClient(node);
+    } catch (Exception e) {
+      logger.error("borrow async heartbeat client fail", e);
+      return null;
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/RaftRole.java
similarity index 56%
copy from consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
copy to consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/RaftRole.java
index daa2a7d7d0..aee98d7559 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/RaftRole.java
@@ -16,23 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.consensus.natraft.protocol;
 
-package org.apache.iotdb.consensus.common.request;
-
-import java.nio.ByteBuffer;
-
-public interface IConsensusRequest {
-  /**
-   * Serialize all the data to a ByteBuffer.
-   *
-   * <p>In a specific implementation, ByteBuf or PublicBAOS can be used to reduce the number of
-   * memory copies.
-   *
-   * <p>To improve efficiency, a specific implementation could return a DirectByteBuffer to reduce
-   * the memory copy required to send an RPC
-   *
-   * <p>Note: The implementation needs to ensure that the data in the returned Bytebuffer cannot be
-   * changed or an error may occur
-   */
-  ByteBuffer serializeToByteBuffer();
+public enum RaftRole {
+  // the leader in this group is unknown
+  CANDIDATE,
+  // this node has known the leader in the group
+  FOLLOWER,
+  // this node is the leader in the group
+  LEADER
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/RaftStatus.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/RaftStatus.java
new file mode 100644
index 0000000000..b20a526fb7
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/RaftStatus.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.consensus.natraft.protocol;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class RaftStatus {
+  /** when the node is a leader, this map is used to track log progress of each follower. */
+  protected Map<TEndPoint, PeerInfo> peerMap;
+  /**
+   * the current term of the node, this object also works as lock of some transactions of the member
+   * like elections.
+   */
+  protected AtomicLong term = new AtomicLong(0);
+
+  volatile RaftRole role = RaftRole.CANDIDATE;
+  AtomicReference<TEndPoint> leader = new AtomicReference<>(null);
+  volatile TEndPoint voteFor;
+
+  public Map<TEndPoint, PeerInfo> getPeerMap() {
+    return peerMap;
+  }
+
+  public void setPeerMap(Map<TEndPoint, PeerInfo> peerMap) {
+    this.peerMap = peerMap;
+  }
+
+  public AtomicLong getTerm() {
+    return term;
+  }
+
+  public void setTerm(AtomicLong term) {
+    this.term = term;
+  }
+
+  public RaftRole getRole() {
+    return role;
+  }
+
+  public void setRole(RaftRole role) {
+    this.role = role;
+  }
+
+  public AtomicReference<TEndPoint> getLeader() {
+    return leader;
+  }
+
+  public void setLeader(AtomicReference<TEndPoint> leader) {
+    this.leader = leader;
+  }
+
+  public TEndPoint getVoteFor() {
+    return voteFor;
+  }
+
+  public void setVoteFor(TEndPoint voteFor) {
+    this.voteFor = voteFor;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/Response.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/Response.java
new file mode 100644
index 0000000000..b1ac24e407
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/Response.java
@@ -0,0 +1,68 @@
+/*
+ * 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.natraft.protocol;
+
+/**
+ * Response defines the numeric responses that have special meanings. Enum class is not used for
+ * thrift compatibility and to reduce communication cost.
+ */
+public class Response {
+
+  // the request is successfully accepted
+  public static final long RESPONSE_AGREE = -1;
+  // cannot find the previous log of the log sent to the follower, a catch-up is required
+  public static final long RESPONSE_LOG_MISMATCH = -2;
+  // the request is rejected but the detailed reason depends on the type of the request
+  public static final long RESPONSE_REJECT = -3;
+  // the partition table is not established yet, so related requests are not available
+  public static final long RESPONSE_PARTITION_TABLE_UNAVAILABLE = -4;
+  // the identifier of the node which wants to join conflicts with one of the joined node's
+  public static final long RESPONSE_IDENTIFIER_CONFLICT = -5;
+  // the requested node is unreachable in the network
+  public static final long RESPONSE_NO_CONNECTION = -6;
+  // the node does not give a vote because its leader does not time out. This is to avoid a
+  // node which cannot connect to the leader changing the leader in the group frequently.
+  public static final long RESPONSE_LEADER_STILL_ONLINE = -7;
+  // the operation is rejected because the cluster will not be able to have enough replicas after
+  // this operation
+  public static final long RESPONSE_CLUSTER_TOO_SMALL = -8;
+  // the new node, which tries to join the cluster, contains conflicted parameters with the
+  // cluster, so the operation is rejected.
+  public static final long RESPONSE_NEW_NODE_PARAMETER_CONFLICT = -9;
+  // the data migration of previous add/remove node operations is not finished.
+  public static final long RESPONSE_DATA_MIGRATION_NOT_FINISH = -10;
+  // the node has removed from the group, so the operation is rejected.
+  public static final long RESPONSE_NODE_IS_NOT_IN_GROUP = -11;
+  public static final long RESPONSE_STRONG_ACCEPT = -12;
+  public static final long RESPONSE_WEAK_ACCEPT = -13;
+  public static final long RESPONSE_OUT_OF_WINDOW = -14;
+  // the request is not executed locally anc should be forwarded
+  // the meta engine is not ready (except for the partitionTable is ready)
+  public static final long RESPONSE_META_NOT_READY = -15;
+  // the cluster is too busy to reject new committed logs
+  public static final long RESPONSE_TOO_BUSY = -16;
+  public static final long RESPONSE_HASH_INCONSISTENT = -17;
+  public static final long RESPONSE_SIGNATURE_INCONSISTENT = -18;
+  public static final long RESPONSE_NULL = Long.MIN_VALUE;
+
+  private Response() {
+    // enum-like class
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/consistency/CheckConsistency.java
similarity index 57%
copy from consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
copy to consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/consistency/CheckConsistency.java
index daa2a7d7d0..279a813a78 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/consistency/CheckConsistency.java
@@ -17,22 +17,21 @@
  * under the License.
  */
 
-package org.apache.iotdb.consensus.common.request;
+package org.apache.iotdb.consensus.natraft.protocol.consistency;
 
-import java.nio.ByteBuffer;
+import org.apache.iotdb.consensus.natraft.exception.CheckConsistencyException;
+
+/** call back after syncLeader */
+public interface CheckConsistency {
 
-public interface IConsensusRequest {
   /**
-   * Serialize all the data to a ByteBuffer.
-   *
-   * <p>In a specific implementation, ByteBuf or PublicBAOS can be used to reduce the number of
-   * memory copies.
-   *
-   * <p>To improve efficiency, a specific implementation could return a DirectByteBuffer to reduce
-   * the memory copy required to send an RPC
+   * deal leaderCommitId and localAppliedId after syncLeader
    *
-   * <p>Note: The implementation needs to ensure that the data in the returned Bytebuffer cannot be
-   * changed or an error may occur
+   * @param leaderCommitId leader commit id
+   * @param localAppliedId local applied id
+   * @throws CheckConsistencyException maybe throw CheckConsistencyException, which is defined in
+   *     implements.
    */
-  ByteBuffer serializeToByteBuffer();
+  void postCheckConsistency(long leaderCommitId, long localAppliedId)
+      throws CheckConsistencyException;
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/consistency/ConsistencyLevel.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/consistency/ConsistencyLevel.java
new file mode 100644
index 0000000000..ddc35ccde9
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/consistency/ConsistencyLevel.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.consensus.natraft.protocol.consistency;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public enum ConsistencyLevel {
+  /**
+   * Strong consistency means the server will first try to synchronize with the leader to get the
+   * newest metadata, if failed(timeout), directly report an error to the user;
+   */
+  STRONG_CONSISTENCY("strong"),
+
+  /**
+   * mid-consistency means the server will first try to synchronize with the leader, but if
+   * failed(timeout), it will give up and just use current data it has cached before;
+   */
+  MID_CONSISTENCY("mid"),
+
+  /** weak consistency do not synchronize with the leader and simply use the local data */
+  WEAK_CONSISTENCY("weak"),
+  ;
+
+  private String consistencyLevelName;
+  private static final Logger logger = LoggerFactory.getLogger(ConsistencyLevel.class);
+
+  ConsistencyLevel(String consistencyLevelName) {
+    this.consistencyLevelName = consistencyLevelName;
+  }
+
+  public static ConsistencyLevel getConsistencyLevel(String consistencyLevel) {
+    if (consistencyLevel == null) {
+      return ConsistencyLevel.MID_CONSISTENCY;
+    }
+    switch (consistencyLevel.toLowerCase()) {
+      case "strong":
+        return ConsistencyLevel.STRONG_CONSISTENCY;
+      case "mid":
+        return ConsistencyLevel.MID_CONSISTENCY;
+      case "weak":
+        return ConsistencyLevel.WEAK_CONSISTENCY;
+      default:
+        logger.warn(
+            "Unsupported consistency level={}, use default consistency level={}",
+            consistencyLevel,
+            ConsistencyLevel.MID_CONSISTENCY.consistencyLevelName);
+        return ConsistencyLevel.MID_CONSISTENCY;
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/consistency/MidCheckConsistency.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/consistency/MidCheckConsistency.java
new file mode 100644
index 0000000000..689e44d3bd
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/consistency/MidCheckConsistency.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.consensus.natraft.protocol.consistency;
+
+import org.apache.iotdb.consensus.natraft.exception.CheckConsistencyException;
+
+public class MidCheckConsistency implements CheckConsistency {
+
+  private long maxLogLag;
+
+  public MidCheckConsistency(long maxLogLag) {
+    this.maxLogLag = maxLogLag;
+  }
+
+  /**
+   * if leaderCommitId - localAppliedId > MaxReadLogLag, will throw CHECK_MID_CONSISTENCY_EXCEPTION
+   *
+   * @param leaderCommitId leader commit id
+   * @param localAppliedId local applied id
+   * @throws CheckConsistencyException
+   */
+  @Override
+  public void postCheckConsistency(long leaderCommitId, long localAppliedId)
+      throws CheckConsistencyException {
+    if (leaderCommitId == Long.MAX_VALUE
+        || leaderCommitId == Long.MIN_VALUE
+        || leaderCommitId - localAppliedId > maxLogLag) {
+      throw CheckConsistencyException.CHECK_MID_CONSISTENCY_EXCEPTION;
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/consistency/StrongCheckConsistency.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/consistency/StrongCheckConsistency.java
new file mode 100644
index 0000000000..7cdf02f54e
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/consistency/StrongCheckConsistency.java
@@ -0,0 +1,27 @@
+/*
+ * 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 requ [...]
+ */
+
+package org.apache.iotdb.consensus.natraft.protocol.consistency;
+
+import org.apache.iotdb.consensus.natraft.exception.CheckConsistencyException;
+
+public class StrongCheckConsistency implements CheckConsistency {
+
+  /**
+   * if leaderCommitId > localAppliedId, will throw CHECK_STRONG_CONSISTENCY_EXCEPTION
+   *
+   * @param leaderCommitId leader commit id
+   * @param localAppliedId local applied id
+   * @throws CheckConsistencyException
+   */
+  @Override
+  public void postCheckConsistency(long leaderCommitId, long localAppliedId)
+      throws CheckConsistencyException {
+    if (leaderCommitId > localAppliedId
+        || leaderCommitId == Long.MAX_VALUE
+        || leaderCommitId == Long.MIN_VALUE) {
+      throw CheckConsistencyException.CHECK_STRONG_CONSISTENCY_EXCEPTION;
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/ElectionReqHandler.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/ElectionReqHandler.java
new file mode 100644
index 0000000000..9b8dd4c3d1
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/ElectionReqHandler.java
@@ -0,0 +1,174 @@
+/*
+ * 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.natraft.protocol.heartbeat;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.protocol.RaftRole;
+import org.apache.iotdb.consensus.natraft.protocol.Response;
+import org.apache.iotdb.consensus.raft.thrift.ElectionRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Objects;
+
+public class ElectionReqHandler {
+
+  private static final Logger logger = LoggerFactory.getLogger(ElectionReqHandler.class);
+  private RaftMember member;
+
+  public ElectionReqHandler(RaftMember member) {
+    this.member = member;
+  }
+
+  public long processElectionRequest(ElectionRequest electionRequest) {
+    if (logger.isDebugEnabled()) {
+      logger.debug(
+          "{}: start to handle request from elector {}",
+          member.getName(),
+          electionRequest.getElector());
+    }
+    long currentTerm = member.getStatus().getTerm().get();
+    long response =
+        checkElectorTerm(currentTerm, electionRequest.getTerm(), electionRequest.getElector());
+    if (response != Response.RESPONSE_AGREE) {
+      return response;
+    }
+
+    // compare the log progress of the elector with this node
+    response = checkElectorLogProgress(electionRequest);
+    logger.info(
+        "{} sending response {} to the elector {}",
+        member.getName(),
+        response,
+        electionRequest.getElector());
+    return response;
+  }
+
+  private long checkElectorTerm(long currentTerm, long electorTerm, TEndPoint candidate) {
+    if (electorTerm < currentTerm) {
+      // the elector has a smaller term thus the request is invalid
+      logger.info(
+          "{} sending localTerm {} to the elector {} because it's term {} is smaller.",
+          member.getName(),
+          currentTerm,
+          candidate,
+          electorTerm);
+      return currentTerm;
+    }
+    if (currentTerm == electorTerm
+        && member.getStatus().getVoteFor() != null
+        && !Objects.equals(member.getStatus().getVoteFor(), candidate)) {
+      // this node has voted in this round, but not for the elector, as one node cannot vote
+      // twice, reject the request
+      logger.info(
+          "{} sending rejection to the elector {} because member already has voted {} in this term {}.",
+          member.getName(),
+          candidate,
+          member.getStatus().getVoteFor(),
+          currentTerm);
+      return Response.RESPONSE_REJECT;
+    }
+    if (electorTerm > currentTerm) {
+      // the elector has a larger term, this node should update its term first
+      logger.info(
+          "{} received an election from elector {} which has bigger term {} than localTerm {}, raftMember should step down first and then continue to decide whether to grant it's vote by log status.",
+          member.getName(),
+          candidate,
+          electorTerm,
+          currentTerm);
+      member.stepDown(electorTerm, null);
+    }
+    return Response.RESPONSE_AGREE;
+  }
+
+  /**
+   * Verify the validity of an ElectionRequest, and make itself a follower of the elector if the
+   * request is valid.
+   *
+   * @return Response.RESPONSE_AGREE if the elector is valid or the local term if the elector has a
+   *     smaller term or Response.RESPONSE_LOG_MISMATCH if the elector has older logs.
+   */
+  long checkElectorLogProgress(ElectionRequest electionRequest) {
+    TEndPoint candidate = electionRequest.getElector();
+    // check if the node is in the group
+    if (!member.containsNode(candidate)) {
+      logger.info(
+          "{}: the elector {} is not in the data group {}, so reject this election.",
+          member.getName(),
+          member.getAllNodes(),
+          candidate);
+      return Response.RESPONSE_NODE_IS_NOT_IN_GROUP;
+    }
+
+    long thatTerm = electionRequest.getTerm();
+    long thatLastLogIndex = electionRequest.getLastLogIndex();
+    long thatLastLogTerm = electionRequest.getLastLogTerm();
+
+    // check the log progress of the elector
+    long resp = checkLogProgress(thatLastLogIndex, thatLastLogTerm);
+    if (resp == Response.RESPONSE_AGREE) {
+      logger.info(
+          "{} accepted an election request, term:{}/{}, logIndex:{}/{}, logTerm:{}/{}",
+          member.getName(),
+          thatTerm,
+          member.getStatus().getTerm().get(),
+          thatLastLogIndex,
+          member.getLogManager().getLastLogIndex(),
+          thatLastLogTerm,
+          member.getLogManager().getLastLogTerm());
+      member.getStatus().setRole(RaftRole.FOLLOWER);
+      member.getHeartbeatThread().setLastHeartbeatReceivedTime(System.currentTimeMillis());
+      member.getStatus().setVoteFor(candidate);
+      member.updateHardState(thatTerm, member.getStatus().getVoteFor());
+    } else {
+      logger.info(
+          "{} rejected an election request, term:{}/{}, logIndex:{}/{}, logTerm:{}/{}",
+          member.getName(),
+          thatTerm,
+          member.getStatus().getTerm().get(),
+          thatLastLogIndex,
+          member.getLogManager().getLastLogIndex(),
+          thatLastLogTerm,
+          member.getLogManager().getLastLogTerm());
+    }
+    return resp;
+  }
+
+  /**
+   * Reject the election if the lastLogTerm of the candidate equals to the voter's but its
+   * lastLogIndex is smaller than the voter's Otherwise accept the election.
+   *
+   * @return Response.RESPONSE_AGREE if the elector is valid or the local term if the elector has a
+   *     smaller term or Response.RESPONSE_LOG_MISMATCH if the elector has older logs.
+   */
+  long checkLogProgress(long lastLogIndex, long lastLogTerm) {
+    long response;
+    synchronized (member.getLogManager()) {
+      if (member.getLogManager().isLogUpToDate(lastLogTerm, lastLogIndex)) {
+        response = Response.RESPONSE_AGREE;
+      } else {
+        response = Response.RESPONSE_LOG_MISMATCH;
+      }
+    }
+    return response;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/ElectionRespHandler.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/ElectionRespHandler.java
new file mode 100644
index 0000000000..d693e9bd52
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/ElectionRespHandler.java
@@ -0,0 +1,148 @@
+/*
+ * 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.natraft.protocol.heartbeat;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.ConnectException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_AGREE;
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_LEADER_STILL_ONLINE;
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_NODE_IS_NOT_IN_GROUP;
+
+/**
+ * ElectionHandler checks the result from a voter and decides whether the election goes on, succeeds
+ * or fails.
+ */
+public class ElectionRespHandler implements AsyncMethodCallback<Long> {
+
+  private static final Logger logger = LoggerFactory.getLogger(ElectionRespHandler.class);
+
+  private RaftMember raftMember;
+  private String memberName;
+  private TEndPoint voter;
+  private long currTerm;
+  private AtomicInteger requiredVoteNum;
+  private AtomicBoolean terminated;
+  // when set to true, the elector wins the election
+  private AtomicBoolean electionValid;
+  private AtomicInteger failingVoteCounter;
+
+  public ElectionRespHandler(
+      RaftMember raftMember,
+      TEndPoint voter,
+      long currTerm,
+      AtomicInteger requiredVoteNum,
+      AtomicBoolean terminated,
+      AtomicBoolean electionValid,
+      AtomicInteger failingVoteCounter) {
+    this.raftMember = raftMember;
+    this.voter = voter;
+    this.currTerm = currTerm;
+    this.requiredVoteNum = requiredVoteNum;
+    this.terminated = terminated;
+    this.electionValid = electionValid;
+    this.memberName = raftMember.getName();
+    this.failingVoteCounter = failingVoteCounter;
+  }
+
+  @Override
+  public void onComplete(Long resp) {
+    long voterResp = resp;
+
+    if (terminated.get()) {
+      // a voter has rejected this election, which means the term or the log id falls behind
+      // this node is not able to be the leader
+      logger.info(
+          "{}: Terminated election received a election response {} from {}",
+          memberName,
+          voterResp,
+          voter);
+      return;
+    }
+
+    if (voterResp == RESPONSE_AGREE) {
+      long remaining = requiredVoteNum.decrementAndGet();
+      logger.info(
+          "{}: Received a grant vote from {}, remaining votes to succeed: {}",
+          memberName,
+          voter,
+          remaining);
+      if (remaining == 0) {
+        // the election is valid
+        electionValid.set(true);
+        terminated.set(true);
+        synchronized (terminated) {
+          terminated.notifyAll();
+        }
+        logger.info("{}: Election {} is won", memberName, currTerm);
+      }
+      // still need more votes
+    } else if (voterResp != RESPONSE_LEADER_STILL_ONLINE) {
+      if (voterResp < currTerm) {
+        // the rejection from a node with a smaller term means the log of this node falls behind
+        logger.info("{}: Election {} rejected: code {}", memberName, currTerm, voterResp);
+        onFail();
+      } else if (voterResp == RESPONSE_NODE_IS_NOT_IN_GROUP) {
+        logger.info("{}: This node has removed from the group", memberName);
+        onFail();
+      } else {
+        // the election is rejected by a node with a bigger term, update current term to it
+        logger.info(
+            "{}: Election {} rejected from {}: The term of this node is no bigger than {}",
+            memberName,
+            currTerm,
+            voter,
+            voterResp);
+        raftMember.stepDown(voterResp, null);
+        // the election is rejected
+        terminated.set(true);
+        terminated.notifyAll();
+      }
+    }
+  }
+
+  @Override
+  public void onError(Exception exception) {
+    if (exception instanceof ConnectException) {
+      logger.warn("{}: Cannot connect to {}: {}", memberName, voter, exception.getMessage());
+    } else {
+      logger.warn("{}: A voter {} encountered an error:", memberName, voter, exception);
+    }
+    onFail();
+  }
+
+  private void onFail() {
+    int failingVoteRemaining = failingVoteCounter.decrementAndGet();
+    if (failingVoteRemaining <= 0) {
+      synchronized (terminated) {
+        // wake up heartbeat thread to start the next election
+        terminated.notifyAll();
+      }
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/HeartbeatReqHandler.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/HeartbeatReqHandler.java
new file mode 100644
index 0000000000..7d7ca4cbed
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/HeartbeatReqHandler.java
@@ -0,0 +1,94 @@
+package org.apache.iotdb.consensus.natraft.protocol.heartbeat;
+
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.protocol.RaftRole;
+import org.apache.iotdb.consensus.natraft.protocol.Response;
+import org.apache.iotdb.consensus.raft.thrift.HeartBeatRequest;
+import org.apache.iotdb.consensus.raft.thrift.HeartBeatResponse;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HeartbeatReqHandler {
+
+  private static final Logger logger = LoggerFactory.getLogger(HeartbeatReqHandler.class);
+  private RaftMember member;
+
+  public HeartbeatReqHandler(RaftMember member) {
+    this.member = member;
+  }
+
+  /**
+   * Process the HeartBeatRequest from the leader. If the term of the leader is smaller than the
+   * local term, reject the request by telling it the newest term. Else if the local logs are
+   * consistent with the leader's, commit them. Else help the leader find the last matched log. Also
+   * update the leadership, heartbeat timer and term of the local node.
+   */
+  public HeartBeatResponse processHeartbeatRequest(HeartBeatRequest request) {
+    logger.trace("{} received a heartbeat", member.getName());
+    long thisTerm = member.getStatus().getTerm().get();
+    long leaderTerm = request.getTerm();
+    HeartBeatResponse response = new HeartBeatResponse();
+
+    if (leaderTerm < thisTerm) {
+      // a leader with a term lower than this node is invalid, send it the local term to inform
+      // it to resign
+      response.setTerm(thisTerm);
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "{} received a heartbeat from a stale leader {}",
+            member.getName(),
+            request.getLeader());
+      }
+    } else if (!(leaderTerm == thisTerm && member.getStatus().getLeader().get() != null)) {
+      // try updating local term or leader
+      try {
+        member.getLogManager().getLock().writeLock().lock();
+        member.stepDown(leaderTerm, request.leader);
+        member.getStatus().getLeader().set(request.getLeader());
+        if (member.getStatus().getRole() != RaftRole.FOLLOWER) {
+          // interrupt current election
+          Object electionWaitObject = member.getHeartbeatThread().getElectionWaitObject();
+          if (electionWaitObject != null) {
+            synchronized (electionWaitObject) {
+              electionWaitObject.notifyAll();
+            }
+          }
+        }
+      } finally {
+        member.getLogManager().getLock().writeLock().unlock();
+      }
+
+      response.setTerm(Response.RESPONSE_AGREE);
+      // tell the leader who I am in case of catch-up
+      response.setFollower(member.getThisNode());
+      // tell the leader the local log progress, so it may decide whether to perform a catch-up
+      response.setLastLogIndex(member.getLogManager().getLastLogIndex());
+      response.setLastLogTerm(member.getLogManager().getLastLogTerm());
+      response.setCommitIndex(member.getLogManager().getCommitLogIndex());
+
+      // if the snapshot apply lock is held, it means that a snapshot is installing now.
+      boolean isFree = member.getSnapshotApplyLock().tryLock();
+      if (isFree) {
+        member.getSnapshotApplyLock().unlock();
+      }
+      response.setInstallingSnapshot(!isFree);
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "{}: log commit log index = {}, max have applied commit index = {}",
+            member.getName(),
+            member.getLogManager().getCommitLogIndex(),
+            member.getLogManager().getAppliedIndex());
+      }
+
+      member.tryUpdateCommitIndex(
+          leaderTerm, request.getCommitLogIndex(), request.getCommitLogTerm());
+
+      if (logger.isTraceEnabled()) {
+        logger.trace(
+            "{} received heartbeat from a valid leader {}", member.getName(), request.getLeader());
+      }
+    }
+    return response;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/HeartbeatRespHandler.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/HeartbeatRespHandler.java
new file mode 100644
index 0000000000..91c7ebddc0
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/HeartbeatRespHandler.java
@@ -0,0 +1,148 @@
+/*
+ * 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.natraft.protocol.heartbeat;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.consensus.natraft.protocol.PeerInfo;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.raft.thrift.HeartBeatResponse;
+
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.ConnectException;
+
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_AGREE;
+
+/**
+ * HeartbeatHandler checks the response of a heartbeat and decides whether to start a catch-up or
+ * give up the leadership due to the term is stale.
+ */
+public class HeartbeatRespHandler implements AsyncMethodCallback<HeartBeatResponse> {
+
+  private static final Logger logger = LoggerFactory.getLogger(HeartbeatRespHandler.class);
+
+  private RaftMember localMember;
+  private String memberName;
+  private TEndPoint receiver;
+
+  public HeartbeatRespHandler(RaftMember localMember, TEndPoint receiver) {
+    this.localMember = localMember;
+    this.receiver = receiver;
+    this.memberName = localMember.getName();
+  }
+
+  @Override
+  public void onComplete(HeartBeatResponse resp) {
+    long followerTerm = resp.getTerm();
+    if (logger.isTraceEnabled()) {
+      logger.trace(
+          "{}: Received a heartbeat response {} for last log index {}",
+          memberName,
+          followerTerm,
+          resp.getLastLogIndex());
+    }
+    if (followerTerm == RESPONSE_AGREE) {
+      // current leadership is still valid
+      handleNormalHeartbeatResponse(resp);
+    } else {
+      // current leadership is invalid because the follower has a larger term
+      long currTerm = localMember.getStatus().getTerm().get();
+      if (currTerm < followerTerm) {
+        logger.info(
+            "{}: Losing leadership because current term {} is smaller than {}",
+            memberName,
+            currTerm,
+            followerTerm);
+        localMember.stepDown(followerTerm, null);
+      }
+    }
+  }
+
+  private void handleNormalHeartbeatResponse(HeartBeatResponse resp) {
+
+    // check the necessity of performing a catch-up
+    TEndPoint follower = resp.getFollower();
+    long lastLogIdx = resp.getLastLogIndex();
+    long lastLogTerm = resp.getLastLogTerm();
+    long localLastLogIdx = localMember.getLogManager().getLastLogIndex();
+    long localLastLogTerm = localMember.getLogManager().getLastLogTerm();
+    if (logger.isTraceEnabled()) {
+      logger.trace(
+          "{}: Node {} is still alive, log index: {}/{}, log term: {}/{}",
+          memberName,
+          follower,
+          lastLogIdx,
+          localLastLogIdx,
+          lastLogTerm,
+          localLastLogTerm);
+    }
+
+    PeerInfo peerInfo = localMember.getStatus().getPeerMap().get(follower);
+    if (!localMember.getLogManager().isLogUpToDate(lastLogTerm, lastLogIdx)
+        || !localMember.getLogManager().matchTerm(lastLogTerm, lastLogIdx)) {
+      // the follower is not up-to-date
+      if (lastLogIdx == -1 || lastLogIdx < peerInfo.getMatchIndex()) {
+        // maybe the follower has restarted, so we need to find its match index again, because
+        // some logs may be lost due to restart
+        peerInfo.setMatchIndex(lastLogIdx);
+      }
+
+      // only start a catch-up when the follower's lastLogIndex remains stall and unchanged for 5
+      // heartbeats. If the follower is installing snapshot currently, we reset the counter.
+      if (lastLogIdx == peerInfo.getLastHeartBeatIndex() && !resp.isInstallingSnapshot()) {
+        // the follower's lastLogIndex is unchanged, increase inconsistent counter
+        int inconsistentNum = peerInfo.incInconsistentHeartbeatNum();
+        if (inconsistentNum >= 1000) {
+          logger.info(
+              "{}: catching up node {}, index-term: {}-{}/{}-{}, peer match index {}",
+              memberName,
+              follower,
+              lastLogIdx,
+              lastLogTerm,
+              localLastLogIdx,
+              localLastLogTerm,
+              peerInfo.getMatchIndex());
+          localMember.catchUp(follower, lastLogIdx);
+        }
+      } else {
+        // the follower's lastLogIndex is changed, which means the follower is not down yet, we
+        // reset the counter to see if it can eventually catch up by itself
+        peerInfo.resetInconsistentHeartbeatNum();
+      }
+    } else {
+      // the follower is up-to-date
+      peerInfo.setMatchIndex(Math.max(peerInfo.getMatchIndex(), lastLogIdx));
+      peerInfo.resetInconsistentHeartbeatNum();
+    }
+    peerInfo.setLastHeartBeatIndex(lastLogIdx);
+  }
+
+  @Override
+  public void onError(Exception exception) {
+    if (exception instanceof ConnectException) {
+      logger.warn("{}: Cannot connect to {}: {}", memberName, receiver, exception.getMessage());
+    } else {
+      logger.error(
+          "{}: Heart beat error, receiver {}, {}", memberName, receiver, exception.getMessage());
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/HeartbeatThread.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/HeartbeatThread.java
new file mode 100644
index 0000000000..f7e130bf78
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/heartbeat/HeartbeatThread.java
@@ -0,0 +1,418 @@
+/*
+ * 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.natraft.protocol.heartbeat;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.consensus.natraft.client.AsyncRaftServiceClient;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.protocol.RaftRole;
+import org.apache.iotdb.consensus.raft.thrift.ElectionRequest;
+import org.apache.iotdb.consensus.raft.thrift.HeartBeatRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * HeartbeatThread takes the responsibility to send heartbeats (when this node is a leader), check
+ * if the leader is still online (when this node is a follower) or start elections (when this node
+ * is a elector).
+ */
+public class HeartbeatThread implements Runnable {
+
+  private static final Logger logger = LoggerFactory.getLogger(HeartbeatThread.class);
+
+  private RaftMember localMember;
+  private String memberName;
+  HeartBeatRequest request = new HeartBeatRequest();
+  ElectionRequest electionRequest = new ElectionRequest();
+  /**
+   * when this node is a follower, this records the unix-epoch timestamp when the last heartbeat
+   * arrived, and is reported in the timed member report to show how long the leader has been
+   * offline.
+   */
+  volatile long lastHeartbeatReceivedTime;
+
+  private Random random = new Random();
+  boolean hasHadLeader = false;
+  private final Object heartBeatWaitObject = new Object();
+  private Object electionWaitObject;
+  /**
+   * the single thread pool that runs the heartbeat thread, which send heartbeats to the follower
+   * when this node is a leader, or start elections when this node is an elector.
+   */
+  protected ExecutorService heartBeatService;
+
+  protected RaftConfig config;
+
+  HeartbeatThread(RaftMember localMember, RaftConfig config) {
+    this.localMember = localMember;
+    memberName = localMember.getName();
+    heartBeatService =
+        IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor(
+            localMember.getName() + "-Heartbeat");
+    this.config = config;
+  }
+
+  public void start() {
+    heartBeatService.submit(this);
+  }
+
+  public void stop() {
+    heartBeatService.shutdownNow();
+    try {
+      heartBeatService.awaitTermination(60, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      logger.error("Unexpected interruption when waiting for heartbeat to end", e);
+    }
+  }
+
+  @Override
+  public void run() {
+    logger.info("{}: Heartbeat thread starts...", memberName);
+    // sleep random time to reduce first election conflicts
+    long electionWait = getElectionRandomWaitMs();
+    try {
+      logger.info("{}: Sleep {}ms before first election", memberName, electionWait);
+      Thread.sleep(electionWait);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+    while (!Thread.interrupted()) {
+      try {
+        switch (localMember.getRole()) {
+          case LEADER:
+            // send heartbeats to the followers
+            sendHeartbeats();
+            synchronized (heartBeatWaitObject) {
+              heartBeatWaitObject.wait(config.getHeartbeatIntervalMs());
+            }
+            hasHadLeader = true;
+            break;
+          case FOLLOWER:
+            // check if heartbeat times out
+            long heartbeatInterval = System.currentTimeMillis() - lastHeartbeatReceivedTime;
+
+            long randomElectionTimeout = config.getElectionTimeoutMs() + getElectionRandomWaitMs();
+            if (heartbeatInterval >= randomElectionTimeout) {
+              // the leader is considered dead, an election will be started in the next loop
+              logger.info(
+                  "{}: The leader {} timed out", memberName, localMember.getStatus().getLeader());
+              localMember.getStatus().setRole(RaftRole.CANDIDATE);
+              localMember.getStatus().setLeader(null);
+            } else {
+              logger.trace(
+                  "{}: Heartbeat from leader {} is still valid",
+                  memberName,
+                  localMember.getStatus().getLeader());
+              synchronized (heartBeatWaitObject) {
+                // we sleep to next possible heartbeat timeout point
+                long leastWaitTime =
+                    lastHeartbeatReceivedTime + randomElectionTimeout - System.currentTimeMillis();
+                heartBeatWaitObject.wait(leastWaitTime);
+              }
+            }
+            hasHadLeader = true;
+            break;
+          case CANDIDATE:
+          default:
+            onElectionsStart();
+            startElections();
+            onElectionsEnd();
+            break;
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        break;
+      } catch (Exception e) {
+        logger.error("{}: Unexpected heartbeat exception:", memberName, e);
+      }
+    }
+
+    logger.info("{}: Heartbeat thread exits", memberName);
+  }
+
+  protected void onElectionsStart() {
+    logger.info("{}: Start elections", memberName);
+  }
+
+  protected void onElectionsEnd() {
+    logger.info("{}: End elections", memberName);
+  }
+
+  /** Send each node (except the local node) in the group of the member a heartbeat. */
+  protected void sendHeartbeats() {
+    try {
+      localMember.getLogManager().getLock().readLock().lock();
+      request.setTerm(localMember.getStatus().getTerm().get());
+      request.setLeader(localMember.getThisNode());
+      request.setCommitLogIndex(localMember.getLogManager().getCommitLogIndex());
+      request.setCommitLogTerm(localMember.getLogManager().getCommitLogTerm());
+    } finally {
+      localMember.getLogManager().getLock().readLock().unlock();
+    }
+    sendHeartbeats(localMember.getAllNodes());
+  }
+
+  /** Send each node (except the local node) in list a heartbeat. */
+  @SuppressWarnings("java:S2445")
+  private void sendHeartbeats(Collection<TEndPoint> nodes) {
+    if (logger.isDebugEnabled()) {
+      logger.trace(
+          "{}: Send heartbeat to {} followers, commit log index = {}",
+          memberName,
+          nodes.size() - 1,
+          request.getCommitLogIndex());
+    }
+    synchronized (nodes) {
+      // avoid concurrent modification
+      for (TEndPoint node : nodes) {
+        if (node.equals(localMember.getThisNode())) {
+          continue;
+        }
+        if (Thread.currentThread().isInterrupted()) {
+          Thread.currentThread().interrupt();
+          return;
+        }
+
+        if (localMember.getRole() != RaftRole.LEADER) {
+          // if the character changes, abort the remaining heartbeats
+          logger.warn("The leadership of node {} is ended.", localMember.getThisNode());
+          return;
+        }
+
+        sendHeartbeatAsync(node);
+      }
+    }
+  }
+
+  /**
+   * Send a heartbeat to "node" through "client".
+   *
+   * @param node
+   */
+  void sendHeartbeatAsync(TEndPoint node) {
+    AsyncRaftServiceClient client = localMember.getHeartbeatClient(node);
+    if (client != null) {
+      // connecting to the local node results in a null
+      try {
+        logger.trace("{}: Sending heartbeat to {}", memberName, node);
+        client.sendHeartbeat(request, new HeartbeatRespHandler(localMember, node));
+      } catch (Exception e) {
+        logger.warn("{}: Cannot send heart beat to node {}", memberName, node, e);
+      }
+    }
+  }
+
+  /**
+   * Start elections until this node becomes a leader or a follower.
+   *
+   * @throws InterruptedException
+   */
+  private void startElections() throws InterruptedException {
+    if (localMember.getAllNodes().size() == 1) {
+      // single node group, this node is always the leader
+      localMember.getStatus().setRole(RaftRole.LEADER);
+      localMember.getStatus().getLeader().set(localMember.getThisNode());
+      logger.info("{}: Winning the election because the node is the only node.", memberName);
+    }
+
+    if (!localMember.getThisNode().equals(localMember.getAllNodes().get(0))) {
+      long electionWait = getElectionRandomWaitMs() + 5000;
+      logger.info(
+          "{}: Sleep {}ms before the first election as this node is not the preferred " + "leader",
+          memberName,
+          electionWait);
+      Thread.sleep(electionWait);
+    }
+
+    // the election goes on until this node becomes a follower or a leader
+    while (localMember.getRole() == RaftRole.CANDIDATE) {
+      startElection();
+      if (localMember.getRole() == RaftRole.CANDIDATE) {
+        // sleep random time to reduce election conflicts
+        long electionWait = getElectionRandomWaitMs();
+        logger.info("{}: Sleep {}ms until next election", memberName, electionWait);
+        Thread.sleep(electionWait);
+      }
+    }
+    // take the election request as the first heartbeat
+    setLastHeartbeatReceivedTime(System.currentTimeMillis());
+  }
+
+  /**
+   * Start one round of election. Increase the local term, ask for vote from each of the nodes in
+   * the group and become the leader if at least half of them agree.
+   */
+  @SuppressWarnings({"java:S2274"})
+  // enable timeout
+  void startElection() {
+    if (localMember.isStopped()) {
+      logger.info("{}: Skip election because this node has stopped.", memberName);
+      return;
+    }
+
+    int quorumNum = localMember.getAllNodes().size() / 2;
+    // set to true when the election has a result (rejected or succeeded)
+    AtomicBoolean electionTerminated = new AtomicBoolean(false);
+    // set to true when the election is won
+    AtomicBoolean electionValid = new AtomicBoolean(false);
+    // a decreasing vote counter
+    AtomicInteger quorum = new AtomicInteger(quorumNum);
+    long nextTerm;
+    try {
+      localMember.getLogManager().getLock().writeLock().lock();
+
+      nextTerm = localMember.getStatus().getTerm().incrementAndGet();
+      localMember.getStatus().setVoteFor(localMember.getThisNode());
+      localMember.updateHardState(nextTerm, this.localMember.getStatus().getVoteFor());
+      // erase the log index, so it can be updated in the next heartbeat
+      electionRequest.unsetLastLogIndex();
+    } finally {
+      localMember.getLogManager().getLock().writeLock().unlock();
+    }
+
+    // the number of votes needed to become a leader,
+    // quorumNum should be equal to localMember.getAllNodes().size() / 2 + 1,
+    // but since it doesn’t need to vote for itself here, it directly decreases 1
+    logger.info("{}: Election {} starts, quorum: {}", memberName, nextTerm, quorumNum);
+    // NOTICE, failingVoteCounter should be equal to quorumNum + 1
+    AtomicInteger failingVoteCounter = new AtomicInteger(quorumNum + 1);
+
+    electionRequest.setTerm(nextTerm);
+    electionRequest.setElector(localMember.getThisNode());
+    electionRequest.setLastLogTerm(localMember.getLogManager().getLastLogTerm());
+    electionRequest.setLastLogIndex(localMember.getLogManager().getLastLogIndex());
+
+    requestVote(
+        localMember.getAllNodes(),
+        electionRequest,
+        nextTerm,
+        quorum,
+        electionTerminated,
+        electionValid,
+        failingVoteCounter);
+
+    try {
+      logger.info(
+          "{}: Wait for {}ms until election time out", memberName, config.getElectionTimeoutMs());
+      synchronized (electionTerminated) {
+        electionWaitObject = electionTerminated;
+        electionTerminated.wait(config.getElectionTimeoutMs());
+      }
+      electionWaitObject = null;
+    } catch (InterruptedException e) {
+      logger.info("{}: Unexpected interruption when waiting the result of election", memberName);
+      Thread.currentThread().interrupt();
+    } finally {
+      localMember.getLogManager().getLock().writeLock().lock();
+    }
+
+    // if the election times out, the remaining votes do not matter
+    electionTerminated.set(true);
+    if (electionValid.get()) {
+      logger.info("{}: Election {} accepted", memberName, nextTerm);
+      localMember.getStatus().setRole(RaftRole.LEADER);
+      localMember.getStatus().getLeader().set(localMember.getThisNode());
+    }
+  }
+
+  /**
+   * Request a vote from each of the "nodes". Each for vote will decrease the counter "quorum" and
+   * when it reaches 0, the flag "electionValid" and "electionTerminated" will be set to true. Any
+   * against vote will set the flag "electionTerminated" to true and ends the election.
+   *
+   * @param nodes
+   * @param request
+   * @param nextTerm the term of the election
+   * @param quorum
+   * @param electionTerminated
+   * @param electionValid
+   */
+  @SuppressWarnings("java:S2445")
+  private void requestVote(
+      Collection<TEndPoint> nodes,
+      ElectionRequest request,
+      long nextTerm,
+      AtomicInteger quorum,
+      AtomicBoolean electionTerminated,
+      AtomicBoolean electionValid,
+      AtomicInteger failingVoteCounter) {
+    synchronized (nodes) {
+      // avoid concurrent modification
+      for (TEndPoint node : nodes) {
+        if (node.equals(localMember.getThisNode())) {
+          continue;
+        }
+
+        ElectionRespHandler handler =
+            new ElectionRespHandler(
+                localMember,
+                node,
+                nextTerm,
+                quorum,
+                electionTerminated,
+                electionValid,
+                failingVoteCounter);
+        requestVoteAsync(node, handler, request);
+      }
+    }
+  }
+
+  private void requestVoteAsync(
+      TEndPoint node, ElectionRespHandler handler, ElectionRequest request) {
+    AsyncRaftServiceClient client = localMember.getHeartbeatClient(node);
+    if (client != null) {
+      logger.info("{}: Requesting a vote from {}", memberName, node);
+      try {
+        client.startElection(request, handler);
+      } catch (Exception e) {
+        logger.error("{}: Cannot request a vote from {}", memberName, node, e);
+      }
+    }
+  }
+
+  private long getElectionRandomWaitMs() {
+    return Math.abs(random.nextLong() % config.getElectionMaxWaitMs());
+  }
+
+  public void notifyHeartbeat() {
+    synchronized (heartBeatWaitObject) {
+      heartBeatWaitObject.notifyAll();
+    }
+  }
+
+  public void setLastHeartbeatReceivedTime(long currentTimeMillis) {
+    lastHeartbeatReceivedTime = currentTimeMillis;
+  }
+
+  public Object getElectionWaitObject() {
+    return electionWaitObject;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/CommitLogCallback.java
similarity index 51%
copy from consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
copy to consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/CommitLogCallback.java
index daa2a7d7d0..78ed30db8b 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/CommitLogCallback.java
@@ -17,22 +17,32 @@
  * under the License.
  */
 
-package org.apache.iotdb.consensus.common.request;
-
-import java.nio.ByteBuffer;
-
-public interface IConsensusRequest {
-  /**
-   * Serialize all the data to a ByteBuffer.
-   *
-   * <p>In a specific implementation, ByteBuf or PublicBAOS can be used to reduce the number of
-   * memory copies.
-   *
-   * <p>To improve efficiency, a specific implementation could return a DirectByteBuffer to reduce
-   * the memory copy required to send an RPC
-   *
-   * <p>Note: The implementation needs to ensure that the data in the returned Bytebuffer cannot be
-   * changed or an error may occur
-   */
-  ByteBuffer serializeToByteBuffer();
+package org.apache.iotdb.consensus.natraft.protocol.log;
+
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommitLogCallback implements AsyncMethodCallback<Void> {
+
+  private static final Logger logger = LoggerFactory.getLogger(CommitLogCallback.class);
+  private final RaftMember raftMember;
+
+  public CommitLogCallback(RaftMember raftMember) {
+    this.raftMember = raftMember;
+  }
+
+  @Override
+  public void onComplete(Void v) {
+    synchronized (raftMember.getSyncLock()) {
+      raftMember.getSyncLock().notifyAll();
+    }
+  }
+
+  @Override
+  public void onError(Exception e) {
+    logger.error("async commit log failed", e);
+  }
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/CommitLogTask.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/CommitLogTask.java
new file mode 100644
index 0000000000..3b4d33d44d
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/CommitLogTask.java
@@ -0,0 +1,65 @@
+/*
+ * 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.natraft.protocol.log;
+
+import org.apache.iotdb.consensus.natraft.protocol.log.manager.RaftLogManager;
+
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommitLogTask implements Runnable {
+
+  private static final Logger logger = LoggerFactory.getLogger(CommitLogTask.class);
+  private RaftLogManager logManager;
+  private long leaderCommit;
+  private long term;
+
+  public CommitLogTask(RaftLogManager logManager, long leaderCommit, long term) {
+    this.logManager = logManager;
+    this.leaderCommit = leaderCommit;
+    this.term = term;
+  }
+
+  /** listener field */
+  private AsyncMethodCallback<Void> callback;
+
+  /** @param callback the event listener */
+  public void registerCallback(AsyncMethodCallback<Void> callback) {
+    this.callback = callback;
+  }
+
+  private void doCommitLog() {
+    if (callback == null) {
+      logger.error("callback is not registered");
+      return;
+    }
+
+    boolean success = logManager.maybeCommit(leaderCommit, term);
+    if (success) {
+      callback.onComplete(null);
+    }
+  }
+
+  @Override
+  public void run() {
+    doCommitLog();
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/Entry.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/Entry.java
new file mode 100644
index 0000000000..1a357441d5
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/Entry.java
@@ -0,0 +1,144 @@
+/*
+ * 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.natraft.protocol.log;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+import java.util.Objects;
+
+/**
+ * Log records operations that are made on this cluster. Each log records 2 longs: currLogIndex,
+ * currLogTerm, so that the logs in a cluster will form a log chain and abnormal operations can thus
+ * be distinguished and removed.
+ */
+public abstract class Entry implements Comparable<Entry> {
+
+  private static final Comparator<Entry> COMPARATOR =
+      Comparator.comparingLong(Entry::getCurrLogIndex).thenComparing(Entry::getCurrLogTerm);
+
+  // make this configurable or adaptive
+  protected static final int DEFAULT_SERIALIZATION_BUFFER_SIZE = 16 * 1024;
+  private volatile long currLogIndex = Long.MIN_VALUE;
+  private long currLogTerm = -1;
+  private long prevTerm = -1;
+
+  // for async application
+  private volatile boolean applied;
+
+  @SuppressWarnings("java:S3077")
+  private volatile Exception exception;
+
+  private long byteSize = 0;
+
+  public int getDefaultSerializationBufferSize() {
+    return DEFAULT_SERIALIZATION_BUFFER_SIZE;
+  }
+
+  public abstract ByteBuffer serialize();
+
+  public abstract void deserialize(ByteBuffer buffer);
+
+  public void serialize(ByteBuffer buffer) {
+    buffer.put(serialize());
+  }
+
+  public enum Types {
+    // DO CHECK LogParser when you add a new type of log
+    CLIENT_REQUEST,
+    EMPTY
+  }
+
+  public long getCurrLogIndex() {
+    return currLogIndex;
+  }
+
+  public void setCurrLogIndex(long currLogIndex) {
+    this.currLogIndex = currLogIndex;
+  }
+
+  public long getCurrLogTerm() {
+    return currLogTerm;
+  }
+
+  public void setCurrLogTerm(long currLogTerm) {
+    this.currLogTerm = currLogTerm;
+  }
+
+  @SuppressWarnings("java:S2886") // synchronized outside
+  public boolean isApplied() {
+    return applied;
+  }
+
+  public void setApplied(boolean applied) {
+    synchronized (this) {
+      this.applied = applied;
+      this.notifyAll();
+    }
+  }
+
+  public Exception getException() {
+    return exception;
+  }
+
+  public void setException(Exception exception) {
+    this.exception = exception;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Entry e = (Entry) o;
+    return currLogIndex == e.currLogIndex && currLogTerm == e.currLogTerm;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(currLogIndex, currLogTerm);
+  }
+
+  @Override
+  public int compareTo(Entry o) {
+    return COMPARATOR.compare(this, o);
+  }
+
+  public long estimateSize() {
+    return byteSize;
+  };
+
+  public long getByteSize() {
+    return byteSize;
+  }
+
+  public void setByteSize(long byteSize) {
+    this.byteSize = byteSize;
+  }
+
+  public long getPrevTerm() {
+    return prevTerm;
+  }
+
+  public void setPrevTerm(long prevTerm) {
+    this.prevTerm = prevTerm;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/LogParser.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/LogParser.java
new file mode 100644
index 0000000000..84ffc3b59e
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/LogParser.java
@@ -0,0 +1,76 @@
+/*
+ * 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.natraft.protocol.log;
+
+import org.apache.iotdb.consensus.natraft.exception.UnknownLogTypeException;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry.Types;
+import org.apache.iotdb.consensus.natraft.protocol.log.logtype.EmptyEntry;
+import org.apache.iotdb.consensus.natraft.protocol.log.logtype.RequestEntry;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+
+/** LogParser transform a ByteBuffer into a Entry. */
+public class LogParser {
+
+  private static final Logger logger = LoggerFactory.getLogger(LogParser.class);
+  private static final LogParser INSTANCE = new LogParser();
+
+  private LogParser() {
+    // singleton class
+  }
+
+  public static LogParser getINSTANCE() {
+    return INSTANCE;
+  }
+
+  public Entry parse(ByteBuffer buffer) throws UnknownLogTypeException {
+    if (logger.isDebugEnabled()) {
+      logger.debug("Received a log buffer, pos:{}, limit:{}", buffer.position(), buffer.limit());
+    }
+    int typeInt = buffer.get();
+    Types type;
+    try {
+      type = Types.values()[typeInt];
+    } catch (ArrayIndexOutOfBoundsException e) {
+      throw new UnknownLogTypeException(typeInt);
+    }
+    logger.debug("The log type is {}", type);
+    Entry log;
+    switch (type) {
+      case CLIENT_REQUEST:
+        RequestEntry requestLog = new RequestEntry();
+        requestLog.deserialize(buffer);
+        log = requestLog;
+        break;
+      case EMPTY:
+        EmptyEntry emptyLog = new EmptyEntry();
+        emptyLog.deserialize(buffer);
+        log = emptyLog;
+        break;
+      default:
+        throw new IllegalArgumentException(type.toString());
+    }
+    logger.debug("Parsed a log {}", log);
+    return log;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/VotingLog.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/VotingLog.java
new file mode 100644
index 0000000000..393616c34d
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/VotingLog.java
@@ -0,0 +1,119 @@
+/*
+ * 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.natraft.protocol.log;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntryRequest;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.Future;
+
+public class VotingLog {
+
+  protected Entry entry;
+  // for NB-Raft
+  protected Set<TEndPoint> weaklyAcceptedNodes;
+  protected Set<TEndPoint> failedNodes;
+  private boolean hasFailed;
+  private AppendEntryRequest appendEntryRequest;
+  private Future<ByteBuffer> serializedLogFuture;
+  private int quorumSize;
+
+  public VotingLog(
+      Entry entry,
+      int groupSize,
+      AppendEntryRequest appendEntryRequest,
+      int quorumSize,
+      RaftConfig config) {
+    this.entry = entry;
+    failedNodes = new HashSet<>(groupSize);
+    if (config.isUseFollowerSlidingWindow()) {
+      weaklyAcceptedNodes = new HashSet<>(groupSize);
+    }
+    this.setAppendEntryRequest(appendEntryRequest);
+    this.setQuorumSize(quorumSize);
+  }
+
+  public VotingLog(VotingLog another) {
+    this.entry = another.entry;
+    this.weaklyAcceptedNodes = another.weaklyAcceptedNodes;
+    this.failedNodes = another.failedNodes;
+    this.setAppendEntryRequest(another.appendEntryRequest);
+    this.setQuorumSize(another.quorumSize);
+    this.setSerializedLogFuture(another.getSerializedLogFuture());
+  }
+
+  public Entry getEntry() {
+    return entry;
+  }
+
+  public void setEntry(Entry entry) {
+    this.entry = entry;
+  }
+
+  public Set<TEndPoint> getWeaklyAcceptedNodes() {
+    return weaklyAcceptedNodes != null ? weaklyAcceptedNodes : Collections.emptySet();
+  }
+
+  @Override
+  public String toString() {
+    return entry.toString();
+  }
+
+  public Set<TEndPoint> getFailedNodes() {
+    return failedNodes;
+  }
+
+  public boolean isHasFailed() {
+    return hasFailed;
+  }
+
+  public void setHasFailed(boolean hasFailed) {
+    this.hasFailed = hasFailed;
+  }
+
+  public AppendEntryRequest getAppendEntryRequest() {
+    return appendEntryRequest;
+  }
+
+  public void setAppendEntryRequest(AppendEntryRequest appendEntryRequest) {
+    this.appendEntryRequest = appendEntryRequest;
+  }
+
+  public Future<ByteBuffer> getSerializedLogFuture() {
+    return serializedLogFuture;
+  }
+
+  public void setSerializedLogFuture(Future<ByteBuffer> serializedLogFuture) {
+    this.serializedLogFuture = serializedLogFuture;
+  }
+
+  public int getQuorumSize() {
+    return quorumSize;
+  }
+
+  public void setQuorumSize(int quorumSize) {
+    this.quorumSize = quorumSize;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/appender/BlockingLogAppender.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/appender/BlockingLogAppender.java
new file mode 100644
index 0000000000..b59bc090e6
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/appender/BlockingLogAppender.java
@@ -0,0 +1,231 @@
+/*
+ * 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.natraft.protocol.log.appender;
+
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.protocol.Response;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+import org.apache.iotdb.consensus.natraft.protocol.log.manager.RaftLogManager;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntriesRequest;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntryRequest;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntryResult;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * BlockingLogAppender wait for a certain amount of time when it receives out-of-order entries
+ * (entries with indices larger than local last entry's index + 1), if the local log is updated
+ * during the waiting and the received entries are now appendable, it appends them normally.
+ * Otherwise, a LOG_MISMATCH is reported to the leader.
+ */
+public class BlockingLogAppender implements LogAppender {
+
+  private static final Logger logger = LoggerFactory.getLogger(BlockingLogAppender.class);
+
+  private RaftMember member;
+  private RaftLogManager logManager;
+  private RaftConfig config;
+
+  public BlockingLogAppender(RaftMember member, RaftConfig config) {
+    this.member = member;
+    this.logManager = member.getLogManager();
+    this.config = config;
+  }
+
+  /**
+   * Find the local previous log of "log". If such log is found, discard all local logs behind it
+   * and append "log" to it. Otherwise report a log mismatch.
+   *
+   * @return Response.RESPONSE_AGREE when the log is successfully appended or Response
+   *     .RESPONSE_LOG_MISMATCH if the previous log of "log" is not found.
+   */
+  public AppendEntryResult appendEntry(AppendEntryRequest request, Entry log) {
+    long resp = checkPrevLogIndex(request.prevLogIndex);
+    if (resp != Response.RESPONSE_AGREE) {
+      return new AppendEntryResult(resp)
+          .setGroupId(member.getRaftGroupId().convertToTConsensusGroupId());
+    }
+
+    long startWaitingTime = System.currentTimeMillis();
+    long success;
+    AppendEntryResult result = new AppendEntryResult();
+    while (true) {
+      // TODO: Consider memory footprint to execute a precise rejection
+      if ((logManager.getCommitLogIndex() - logManager.getAppliedIndex())
+          <= config.getUnAppliedRaftLogNumForRejectThreshold()) {
+        success =
+            logManager.maybeAppend(
+                request.prevLogIndex,
+                request.prevLogTerm,
+                request.leaderCommit,
+                Collections.singletonList(log));
+        break;
+      }
+      try {
+        TimeUnit.MILLISECONDS.sleep(config.getCheckPeriodWhenInsertBlocked());
+        if (System.currentTimeMillis() - startWaitingTime
+            > config.getMaxWaitingTimeWhenInsertBlocked()) {
+          result.status = Response.RESPONSE_TOO_BUSY;
+          return result;
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      }
+    }
+    if (success != -1) {
+      logger.debug("{} append a new log {}", member.getName(), log);
+      result.status = Response.RESPONSE_STRONG_ACCEPT;
+    } else {
+      // the incoming log points to an illegal position, reject it
+      result.status = Response.RESPONSE_LOG_MISMATCH;
+    }
+    result.setGroupId(request.getGroupId());
+    return result;
+  }
+
+  /** Wait until all logs before "prevLogIndex" arrive or a timeout is reached. */
+  private boolean waitForPrevLog(long prevLogIndex) {
+    long waitStart = System.currentTimeMillis();
+    long alreadyWait = 0;
+    Object logUpdateCondition = logManager.getLogUpdateCondition(prevLogIndex);
+    long lastLogIndex = logManager.getLastLogIndex();
+    long waitTime = 1;
+    while (lastLogIndex < prevLogIndex && alreadyWait <= config.getWriteOperationTimeoutMS()) {
+      try {
+        // each time new logs are appended, this will be notified
+        synchronized (logUpdateCondition) {
+          logUpdateCondition.wait(waitTime);
+        }
+        lastLogIndex = logManager.getLastLogIndex();
+        if (lastLogIndex >= prevLogIndex) {
+          return true;
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        return false;
+      }
+      waitTime = waitTime * 2;
+      alreadyWait = System.currentTimeMillis() - waitStart;
+    }
+
+    return alreadyWait <= config.getWriteOperationTimeoutMS();
+  }
+
+  protected long checkPrevLogIndex(long prevLogIndex) {
+    long lastLogIndex = logManager.getLastLogIndex();
+    if (lastLogIndex < prevLogIndex && !waitForPrevLog(prevLogIndex)) {
+      // there are logs missing between the incoming log and the local last log, and such logs
+      // did not come within a timeout, report a mismatch to the sender and it shall fix this
+      // through catch-up
+      return Response.RESPONSE_LOG_MISMATCH;
+    }
+    return Response.RESPONSE_AGREE;
+  }
+
+  /**
+   * Find the local previous log of "log". If such log is found, discard all local logs behind it
+   * and append "log" to it. Otherwise report a log mismatch.
+   *
+   * @param logs append logs
+   * @return Response.RESPONSE_AGREE when the log is successfully appended or Response
+   *     .RESPONSE_LOG_MISMATCH if the previous log of "log" is not found.
+   */
+  public AppendEntryResult appendEntries(AppendEntriesRequest request, List<Entry> logs) {
+    logger.debug(
+        "{}, prevLogIndex={}, prevLogTerm={}, leaderCommit={}",
+        member.getName(),
+        request.prevLogIndex,
+        request.prevLogTerm,
+        request.leaderCommit);
+    if (logs.isEmpty()) {
+      return new AppendEntryResult(Response.RESPONSE_AGREE)
+          .setGroupId(member.getRaftGroupId().convertToTConsensusGroupId());
+    }
+
+    long resp = checkPrevLogIndex(request.prevLogIndex);
+    if (resp != Response.RESPONSE_AGREE) {
+      return new AppendEntryResult(resp)
+          .setGroupId(member.getRaftGroupId().convertToTConsensusGroupId());
+    }
+
+    AppendEntryResult result = new AppendEntryResult();
+    long startWaitingTime = System.currentTimeMillis();
+    while (true) {
+      synchronized (logManager) {
+        // TODO: Consider memory footprint to execute a precise rejection
+        if ((logManager.getCommitLogIndex() - logManager.getAppliedIndex())
+            <= config.getUnAppliedRaftLogNumForRejectThreshold()) {
+          resp =
+              logManager.maybeAppend(
+                  request.prevLogIndex, request.prevLogTerm, request.leaderCommit, logs);
+          break;
+        }
+      }
+
+      try {
+        TimeUnit.MILLISECONDS.sleep(config.getCheckPeriodWhenInsertBlocked());
+        if (System.currentTimeMillis() - startWaitingTime
+            > config.getMaxWaitingTimeWhenInsertBlocked()) {
+          result.status = Response.RESPONSE_TOO_BUSY;
+          return result;
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      }
+    }
+
+    if (resp != -1) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "{} append a new log list {}, commit to {}",
+            member.getName(),
+            logs,
+            request.leaderCommit);
+      }
+      result.status = Response.RESPONSE_STRONG_ACCEPT;
+      result.setLastLogIndex(logManager.getLastLogIndex());
+      result.setLastLogTerm(logManager.getLastLogTerm());
+
+    } else {
+      // the incoming log points to an illegal position, reject it
+      result.status = Response.RESPONSE_LOG_MISMATCH;
+    }
+    return result;
+  }
+
+  @Override
+  public void reset() {
+    // no states maintained by this implementation
+  }
+
+  public static class Factory implements LogAppenderFactory {
+
+    @Override
+    public LogAppender create(RaftMember member, RaftConfig config) {
+      return new BlockingLogAppender(member, config);
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/appender/LogAppender.java
similarity index 56%
copy from consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
copy to consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/appender/LogAppender.java
index daa2a7d7d0..2ec3eceb52 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/appender/LogAppender.java
@@ -17,22 +17,21 @@
  * under the License.
  */
 
-package org.apache.iotdb.consensus.common.request;
+package org.apache.iotdb.consensus.natraft.protocol.log.appender;
 
-import java.nio.ByteBuffer;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntriesRequest;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntryResult;
 
-public interface IConsensusRequest {
-  /**
-   * Serialize all the data to a ByteBuffer.
-   *
-   * <p>In a specific implementation, ByteBuf or PublicBAOS can be used to reduce the number of
-   * memory copies.
-   *
-   * <p>To improve efficiency, a specific implementation could return a DirectByteBuffer to reduce
-   * the memory copy required to send an RPC
-   *
-   * <p>Note: The implementation needs to ensure that the data in the returned Bytebuffer cannot be
-   * changed or an error may occur
-   */
-  ByteBuffer serializeToByteBuffer();
+import java.util.List;
+
+/**
+ * LogAppender appends newly incoming entries to the local log of a member, providing different
+ * policies for out-of-order entries and other cases.
+ */
+public interface LogAppender {
+
+  AppendEntryResult appendEntries(AppendEntriesRequest request, List<Entry> entries);
+
+  void reset();
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/appender/LogAppenderFactory.java
similarity index 56%
copy from consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
copy to consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/appender/LogAppenderFactory.java
index daa2a7d7d0..6f90eeec78 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/appender/LogAppenderFactory.java
@@ -17,22 +17,11 @@
  * under the License.
  */
 
-package org.apache.iotdb.consensus.common.request;
+package org.apache.iotdb.consensus.natraft.protocol.log.appender;
 
-import java.nio.ByteBuffer;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
 
-public interface IConsensusRequest {
-  /**
-   * Serialize all the data to a ByteBuffer.
-   *
-   * <p>In a specific implementation, ByteBuf or PublicBAOS can be used to reduce the number of
-   * memory copies.
-   *
-   * <p>To improve efficiency, a specific implementation could return a DirectByteBuffer to reduce
-   * the memory copy required to send an RPC
-   *
-   * <p>Note: The implementation needs to ensure that the data in the returned Bytebuffer cannot be
-   * changed or an error may occur
-   */
-  ByteBuffer serializeToByteBuffer();
+public interface LogAppenderFactory {
+  LogAppender create(RaftMember member, RaftConfig config);
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/applier/AsyncLogApplier.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/applier/AsyncLogApplier.java
new file mode 100644
index 0000000000..9d11c8feb6
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/applier/AsyncLogApplier.java
@@ -0,0 +1,224 @@
+/*
+ * 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.natraft.protocol.log.applier;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+public class AsyncLogApplier implements LogApplier {
+
+  private static final Logger logger = LoggerFactory.getLogger(AsyncLogApplier.class);
+  private static final int CONCURRENT_CONSUMER_NUM = Runtime.getRuntime().availableProcessors();
+  private RaftConfig config;
+  private LogApplier embeddedApplier;
+  private Map<PartialPath, DataLogConsumer> consumerMap;
+  private ExecutorService consumerPool;
+  private String name;
+
+  // a plan that affects multiple sgs should wait until all consumers become empty to assure all
+  // previous logs are applied, such a plan will wait on this condition if it finds any
+  // consumers nonempty, and each time a consumer becomes empty, this will be notified so the
+  // waiting log can start another round of check
+  private final Object consumerEmptyCondition = new Object();
+
+  public AsyncLogApplier(LogApplier embeddedApplier, String name, RaftConfig config) {
+    this.embeddedApplier = embeddedApplier;
+    consumerMap = new HashMap<>();
+    consumerPool =
+        new ThreadPoolExecutor(
+            CONCURRENT_CONSUMER_NUM,
+            Integer.MAX_VALUE,
+            0,
+            TimeUnit.SECONDS,
+            new SynchronousQueue<>());
+    this.name = name;
+    this.config = config;
+  }
+
+  @Override
+  public void close() {
+    consumerPool.shutdownNow();
+  }
+
+  @Override
+  // synchronized: when a log is draining consumers, avoid other threads adding more logs so that
+  // the consumers will never be drained
+  public synchronized void apply(Entry e) {
+
+    PartialPath logKey = getLogKey(e);
+
+    if (logKey != null) {
+      // this plan only affects one sg, so we can run it with other plans in parallel
+      provideLogToConsumers(logKey, e);
+      return;
+    }
+
+    logger.debug("{}: {} is waiting for consumers to drain", name, e);
+    drainConsumers();
+    applyInternal(e);
+  }
+
+  private PartialPath getLogKey(Entry e) {
+    // TODO-raft: implement
+    return null;
+  }
+
+  private void provideLogToConsumers(PartialPath planKey, Entry e) {
+    consumerMap.computeIfAbsent(planKey, d -> new DataLogConsumer(name + "-" + d)).accept(e);
+  }
+
+  private void drainConsumers() {
+    synchronized (consumerEmptyCondition) {
+      while (!allConsumersEmpty()) {
+        // wait until all consumers empty
+        try {
+          consumerEmptyCondition.wait(5);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          return;
+        }
+      }
+    }
+  }
+
+  private boolean allConsumersEmpty() {
+    for (DataLogConsumer consumer : consumerMap.values()) {
+      if (!consumer.isEmpty()) {
+        if (logger.isDebugEnabled()) {
+          logger.debug("Consumer not empty: {}", consumer);
+        }
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private void applyInternal(Entry e) {
+    embeddedApplier.apply(e);
+  }
+
+  private class DataLogConsumer implements Runnable, Consumer<Entry> {
+
+    private BlockingQueue<Entry> logQueue = new ArrayBlockingQueue<>(config.getMaxNumOfLogsInMem());
+    private volatile long lastLogIndex;
+    private volatile long lastAppliedLogIndex;
+    private String name;
+    private Future<?> future;
+
+    public DataLogConsumer(String name) {
+      this.name = name;
+    }
+
+    public boolean isEmpty() {
+      return lastLogIndex == lastAppliedLogIndex;
+    }
+
+    @Override
+    public void run() {
+      // appliers have a higher priority than normal threads (like client threads and low
+      // priority background threads), to assure fast ingestion, but a lower priority than
+      // heartbeat threads
+      Thread.currentThread().setPriority(8);
+      if (logger.isDebugEnabled()) {
+        Thread.currentThread().setName(name);
+      }
+      while (!Thread.currentThread().isInterrupted()) {
+        try {
+          Entry e = logQueue.take();
+          try {
+            applyInternal(e);
+          } finally {
+            lastAppliedLogIndex = e.getCurrLogIndex();
+            if (isEmpty()) {
+              synchronized (consumerEmptyCondition) {
+                consumerEmptyCondition.notifyAll();
+              }
+            }
+          }
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          break;
+        } catch (Exception e) {
+          logger.error("DataLogConsumer exits", e);
+          return;
+        }
+      }
+      logger.info("DataLogConsumer exits");
+    }
+
+    @Override
+    public void accept(Entry e) {
+      if (future == null || future.isCancelled() || future.isDone()) {
+        if (future != null) {
+          try {
+            future.get();
+          } catch (InterruptedException ex) {
+            logger.error("Last applier thread exits unexpectedly", ex);
+            Thread.currentThread().interrupt();
+          } catch (ExecutionException ex) {
+            logger.error("Last applier thread exits unexpectedly", ex);
+          }
+        }
+        future = consumerPool.submit(this);
+      }
+
+      try {
+        lastLogIndex = e.getCurrLogIndex();
+        logQueue.put(e);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        e.setException(ex);
+        e.setApplied(true);
+        lastAppliedLogIndex = e.getCurrLogIndex();
+      }
+    }
+
+    @Override
+    public String toString() {
+      return "DataLogConsumer{"
+          + "logQueue="
+          + logQueue.size()
+          + ", lastLogIndex="
+          + lastLogIndex
+          + ", lastAppliedLogIndex="
+          + lastAppliedLogIndex
+          + ", name='"
+          + name
+          + '\''
+          + '}';
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/applier/BaseApplier.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/applier/BaseApplier.java
new file mode 100644
index 0000000000..bb371c2c3b
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/applier/BaseApplier.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.consensus.natraft.protocol.log.applier;
+
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.consensus.IStateMachine;
+import org.apache.iotdb.consensus.common.request.IConsensusRequest;
+import org.apache.iotdb.consensus.exception.ConsensusException;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+import org.apache.iotdb.consensus.natraft.protocol.log.logtype.RequestEntry;
+import org.apache.iotdb.rpc.TSStatusCode;
+
+/** BaseApplier use PlanExecutor to execute PhysicalPlans. */
+public class BaseApplier implements LogApplier {
+
+  IStateMachine stateMachine;
+
+  public BaseApplier(IStateMachine stateMachine) {
+    this.stateMachine = stateMachine;
+  }
+
+  @TestOnly
+  public void setStateMachine(IStateMachine stateMachine) {
+    this.stateMachine = stateMachine;
+  }
+
+  @Override
+  public void apply(Entry e) {
+
+    try {
+      if (e instanceof RequestEntry) {
+        RequestEntry requestLog = (RequestEntry) e;
+        IConsensusRequest request = requestLog.getRequest();
+        TSStatus status = applyRequest(request);
+        if (status.code != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+          e.setException(new ConsensusException(status.message + ":" + status.code));
+        }
+      }
+    } catch (Exception ex) {
+      e.setException(ex);
+    } finally {
+      e.setApplied(true);
+    }
+  }
+
+  public TSStatus applyRequest(IConsensusRequest request) {
+    return stateMachine.write(request);
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/applier/LogApplier.java
similarity index 57%
copy from consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
copy to consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/applier/LogApplier.java
index daa2a7d7d0..e2ddfdacf0 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/applier/LogApplier.java
@@ -17,22 +17,21 @@
  * under the License.
  */
 
-package org.apache.iotdb.consensus.common.request;
+package org.apache.iotdb.consensus.natraft.protocol.log.applier;
 
-import java.nio.ByteBuffer;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+
+/** LogApplier applies the log to the local node to make it take effect. */
+public interface LogApplier {
 
-public interface IConsensusRequest {
   /**
-   * Serialize all the data to a ByteBuffer.
-   *
-   * <p>In a specific implementation, ByteBuf or PublicBAOS can be used to reduce the number of
-   * memory copies.
+   * Apply the given log, if any exception is thrown during the execution, the exception will be
+   * recorded in the log. Either an exception is thrown or the log is executed successfully, log
+   * .setApplied(true) must be called.
    *
-   * <p>To improve efficiency, a specific implementation could return a DirectByteBuffer to reduce
-   * the memory copy required to send an RPC
-   *
-   * <p>Note: The implementation needs to ensure that the data in the returned Bytebuffer cannot be
-   * changed or an error may occur
+   * @param e
    */
-  ByteBuffer serializeToByteBuffer();
+  void apply(Entry e);
+
+  default void close() {}
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/CatchUpManager.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/CatchUpManager.java
new file mode 100644
index 0000000000..8633b31b2e
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/CatchUpManager.java
@@ -0,0 +1,130 @@
+/*
+ * 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.natraft.protocol.log.catchup;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+public class CatchUpManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(CatchUpManager.class);
+  private ExecutorService catchUpService;
+  /**
+   * lastCatchUpResponseTime records when is the latest response of each node's catch-up. There
+   * should be only one catch-up task for each node to avoid duplication, but the task may time out
+   * or the task may corrupt unexpectedly, and in that case, the next catch up should be enabled. So
+   * if we find a catch-up task that does not respond for long, we will start a new one instead of
+   * waiting for the previous one to finish.
+   */
+  private Map<TEndPoint, Long> lastCatchUpResponseTime = new ConcurrentHashMap<>();
+
+  private RaftMember member;
+  private RaftConfig config;
+
+  public CatchUpManager(RaftMember member, RaftConfig config) {
+    this.member = member;
+    this.config = config;
+  }
+
+  public void start() {
+    catchUpService = IoTDBThreadPoolFactory.newCachedThreadPool(member.getName() + "-CatchUp");
+  }
+
+  public void stop() {
+    catchUpService.shutdownNow();
+    try {
+      catchUpService.awaitTermination(60, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      logger.error(
+          "Unexpected interruption when waiting for heartBeatService and catchUpService "
+              + "to end",
+          e);
+    }
+    catchUpService = null;
+  }
+
+  public void registerTask(TEndPoint node) {
+    lastCatchUpResponseTime.put(node, System.currentTimeMillis());
+  }
+
+  public void unregisterTask(TEndPoint node) {
+    lastCatchUpResponseTime.remove(node);
+  }
+
+  /**
+   * Update the followers' log by sending logs whose index >= followerLastMatchedLogIndex to the
+   * follower. If some required logs are removed, also send the snapshot. <br>
+   * notice that if a part of data is in the snapshot, then it is not in the logs.
+   */
+  public void catchUp(TEndPoint follower, long lastLogIdx) {
+    // for one follower, there is at most one ongoing catch-up, so the same data will not be sent
+    // twice to the node
+    synchronized (this) {
+      // check if the last catch-up is still ongoing and does not time out yet
+      Long lastCatchupResp = lastCatchUpResponseTime.get(follower);
+      if (lastCatchupResp != null
+          && System.currentTimeMillis() - lastCatchupResp < config.getCatchUpTimeoutMS()) {
+        logger.debug("{}: last catch up of {} is ongoing", member, follower);
+        return;
+      } else {
+        // record the start of the catch-up
+        lastCatchUpResponseTime.put(follower, System.currentTimeMillis());
+      }
+    }
+    logger.info("{}: Start to make {} catch up", member, follower);
+    if (!catchUpService.isShutdown()) {
+      Future<?> future =
+          catchUpService.submit(
+              new CatchUpTask(
+                  follower,
+                  member.getStatus().getPeerMap().get(follower),
+                  this,
+                  lastLogIdx,
+                  config));
+      catchUpService.submit(
+          () -> {
+            try {
+              future.get();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            } catch (ExecutionException e) {
+              logger.error("{}: Catch up task exits with unexpected exception", member, e);
+            }
+          });
+    }
+  }
+
+  public RaftMember getMember() {
+    return member;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/CatchUpTask.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/CatchUpTask.java
new file mode 100644
index 0000000000..6cfdc431c3
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/CatchUpTask.java
@@ -0,0 +1,384 @@
+/*
+ * 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.natraft.protocol.log.catchup;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.consensus.natraft.client.AsyncRaftServiceClient;
+import org.apache.iotdb.consensus.natraft.client.SyncClientAdaptor;
+import org.apache.iotdb.consensus.natraft.exception.LeaderUnknownException;
+import org.apache.iotdb.consensus.natraft.protocol.PeerInfo;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.protocol.RaftRole;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+import org.apache.iotdb.consensus.natraft.protocol.log.logtype.EmptyEntry;
+import org.apache.iotdb.consensus.natraft.protocol.log.snapshot.Snapshot;
+
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.ConcurrentModificationException;
+import java.util.List;
+
+public class CatchUpTask implements Runnable {
+
+  private static final Logger logger = LoggerFactory.getLogger(CatchUpTask.class);
+
+  private TEndPoint node;
+  private PeerInfo peerInfo;
+  private RaftMember raftMember;
+  private Snapshot snapshot;
+  private List<Entry> logs;
+  private long lastLogIndex;
+  private boolean abort;
+  private String name;
+
+  private long startTime;
+  private CatchUpManager catchUpManager;
+  private RaftConfig config;
+
+  public CatchUpTask(
+      TEndPoint node,
+      PeerInfo peerInfo,
+      CatchUpManager catchUpManager,
+      long lastLogIdx,
+      RaftConfig config) {
+    this.node = node;
+    this.peerInfo = peerInfo;
+    this.raftMember = catchUpManager.getMember();
+    this.catchUpManager = catchUpManager;
+    this.logs = Collections.emptyList();
+    this.snapshot = null;
+    this.lastLogIndex = lastLogIdx;
+    this.name = raftMember.getName() + "@" + System.currentTimeMillis();
+    this.config = config;
+  }
+
+  /**
+   * @return true if a matched index is found so that we can use logs only to catch up, or false if
+   *     the catch up must be done with a snapshot.
+   * @throws TException
+   * @throws InterruptedException
+   */
+  private boolean checkMatchIndex()
+      throws TException, InterruptedException, LeaderUnknownException {
+
+    logger.debug("Checking the match index of {}", node);
+    long lo = 0;
+    long hi = 0;
+    long localFirstIndex = 0;
+    try {
+      // to avoid snapshot catch up when index is volatile
+      localFirstIndex = raftMember.getLogManager().getFirstIndex();
+      lo = Math.max(localFirstIndex, peerInfo.getMatchIndex() + 1);
+      hi = raftMember.getLogManager().getLastLogIndex() + 1;
+      logs = raftMember.getLogManager().getEntries(lo, hi);
+
+      // this may result from peer's match index being changed concurrently, making the peer
+      // actually catch up now
+      if (logger.isInfoEnabled()) {
+        logger.info(
+            "{}: use {} logs of [{}, {}] to fix log inconsistency with node [{}], "
+                + "local first index: {}",
+            raftMember.getName(),
+            logs.size(),
+            lo,
+            hi,
+            node,
+            localFirstIndex);
+      }
+    } catch (ConcurrentModificationException e) {
+      // ignore
+    } catch (Exception e) {
+      logger.error("Unexpected error in logManager's getEntries during matchIndexCheck", e);
+    }
+
+    if (logs.isEmpty()) {
+      return true;
+    }
+
+    int index = findLastMatchIndex(logs);
+    if (index == -1) {
+      logger.info("{}, Cannot find matched of {} within [{}, {}] in memory", name, node, lo, hi);
+      if (!judgeUseLogsInDiskToCatchUp()) {
+        return false;
+      }
+      long startIndex = peerInfo.getMatchIndex() + 1;
+      long endIndex = raftMember.getLogManager().getCommitLogIndex();
+      List<Entry> logsInDisk = getLogsInStableEntryManager(startIndex, endIndex);
+      if (!logsInDisk.isEmpty()) {
+        logger.info(
+            "{}, found {} logs in disk to catch up {} , startIndex={}, endIndex={}, memoryFirstIndex={}, getFirstLogIndex={}",
+            name,
+            logsInDisk.size(),
+            node,
+            startIndex,
+            endIndex,
+            localFirstIndex,
+            logsInDisk.get(0).getCurrLogIndex());
+        logs = logsInDisk;
+        index = findLastMatchIndex(logs);
+        // the follower's matchIndex may have been updated
+        if (index == -1) {
+          return false;
+        }
+      } else {
+        logger.info(
+            "{}, Cannot find matched of {} within [{}, {}] in disk",
+            name,
+            node,
+            startIndex,
+            endIndex);
+        return false;
+      }
+    }
+    long newMatchedIndex = logs.get(index).getCurrLogIndex() - 1;
+    if (newMatchedIndex > lastLogIndex) {
+      logger.info(
+          "{}: matched index of {} has moved beyond last log index, node is "
+              + "self-catching-up, abort this catch up to avoid duplicates",
+          name,
+          node);
+      abort = true;
+      return true;
+    }
+    logger.info("{}: {} matches at {}", name, node, newMatchedIndex);
+
+    peerInfo.setMatchIndex(newMatchedIndex);
+    // if follower return RESPONSE.AGREE with this empty log, then start sending real logs from
+    // index.
+    logs.subList(0, index).clear();
+    if (logger.isInfoEnabled()) {
+      if (logs.isEmpty()) {
+        logger.info("{}: {} has caught up by previous catch up", name, node);
+      } else {
+        logger.info(
+            "{}: makes {} catch up with {} and other {} logs",
+            name,
+            node,
+            logs.get(0),
+            logs.size());
+      }
+    }
+    return true;
+  }
+
+  @SuppressWarnings("squid:S1135")
+  private boolean judgeUseLogsInDiskToCatchUp() {
+    // TODO use log in disk to snapshot first, if the log not found on disk, then use snapshot.
+    if (!config.isEnableRaftLogPersistence()) {
+      return false;
+    }
+    // TODO judge the cost of snapshot and logs in disk
+    return config.isEnableUsePersistLogOnDiskToCatchUp();
+  }
+
+  private List<Entry> getLogsInStableEntryManager(long startIndex, long endIndex) {
+    List<Entry> logsInDisk =
+        raftMember.getLogManager().getStableEntryManager().getEntries(startIndex, endIndex);
+    logger.debug(
+        "{}, found {} logs in disk to catchup {}, startIndex={}, endIndex={}",
+        raftMember.getName(),
+        logsInDisk.size(),
+        node,
+        startIndex,
+        endIndex);
+    return logsInDisk;
+  }
+
+  /**
+   * return the index of log whose previous log is matched, or -1 when can not found
+   *
+   * @param logs
+   * @return
+   * @throws LeaderUnknownException
+   * @throws TException
+   * @throws InterruptedException
+   */
+  public int findLastMatchIndex(List<Entry> logs)
+      throws LeaderUnknownException, TException, InterruptedException {
+    int start = 0;
+    int end = logs.size() - 1;
+    int matchedIndex = -1;
+    while (start <= end) {
+      int mid = start + (end - start) / 2;
+      if (checkMatchIndex(mid)) {
+        start = mid + 1;
+        matchedIndex = mid;
+      } else {
+        end = mid - 1;
+      }
+    }
+    return matchedIndex;
+  }
+
+  /**
+   * @param index the index of a log in logs
+   * @return true if the previous log at logs[index] matches a log in the remote node, false if the
+   *     corresponding log cannot be found
+   * @throws LeaderUnknownException
+   * @throws TException
+   * @throws InterruptedException
+   */
+  private boolean checkMatchIndex(int index)
+      throws LeaderUnknownException, TException, InterruptedException {
+    Entry log = logs.get(index);
+    if (raftMember.getRole() != RaftRole.LEADER) {
+      throw new LeaderUnknownException(raftMember.getAllNodes());
+    }
+
+    long prevLogIndex = log.getCurrLogIndex() - 1;
+    long prevLogTerm = getPrevLogTerm(index);
+
+    if (prevLogTerm == -1) {
+      // prev log cannot be found, we cannot know whether is matches if it is not the first log
+      return prevLogIndex == -1;
+    }
+
+    boolean matched = checkLogIsMatch(prevLogIndex, prevLogTerm);
+    catchUpManager.registerTask(node);
+    logger.info(
+        "{} check {}'s matchIndex {} with log [{}]",
+        raftMember.getName(),
+        node,
+        matched ? "succeed" : "failed",
+        log);
+    return matched;
+  }
+
+  /**
+   * @param logIndex the log index needs to check
+   * @param logTerm the log term need to check
+   * @return true if the log's index and term matches a log in the remote node, false if the
+   *     corresponding log cannot be found
+   * @throws TException
+   * @throws InterruptedException
+   */
+  private boolean checkLogIsMatch(long logIndex, long logTerm)
+      throws TException, InterruptedException {
+    boolean matched;
+    AsyncRaftServiceClient client = raftMember.getClient(node);
+    if (client == null) {
+      return false;
+    }
+    matched =
+        SyncClientAdaptor.matchTerm(client, node, logIndex, logTerm, raftMember.getRaftGroupId());
+    return matched;
+  }
+
+  private long getPrevLogTerm(int index) {
+    long prevLogTerm = -1;
+    if (index > 0) {
+      prevLogTerm = logs.get(index - 1).getCurrLogTerm();
+    } else {
+      prevLogTerm = raftMember.getLogManager().getTerm(logs.get(0).getCurrLogIndex() - 1);
+    }
+    return prevLogTerm;
+  }
+
+  private void doSnapshot() {
+    raftMember.getLogManager().takeSnapshot();
+    snapshot = raftMember.getLogManager().getSnapshot(peerInfo.getMatchIndex());
+    if (logger.isInfoEnabled()) {
+      logger.info("{}: Logs in {} are too old, catch up with snapshot", raftMember.getName(), node);
+    }
+  }
+
+  /** Remove logs that are contained in the snapshot. */
+  private void removeSnapshotLogs() {
+    Entry logToSearch = new EmptyEntry(snapshot.getLastLogIndex(), snapshot.getLastLogTerm());
+    int pos =
+        Collections.binarySearch(
+            logs, logToSearch, Comparator.comparingLong(Entry::getCurrLogIndex));
+    int prevSize = logs.size();
+    if (pos >= 0) {
+      logs.subList(0, pos + 1).clear();
+    } else {
+      int insertPos = -pos - 1;
+      if (insertPos > 0) {
+        logs.subList(0, insertPos).clear();
+      }
+    }
+    logger.info("Logs are reduced from {} to {}", prevSize, logs.size());
+  }
+
+  @Override
+  public void run() {
+    startTime = System.currentTimeMillis();
+    try {
+      boolean findMatchedIndex = checkMatchIndex();
+      if (abort) {
+        peerInfo.resetInconsistentHeartbeatNum();
+        catchUpManager.unregisterTask(node);
+        return;
+      }
+      boolean catchUpSucceeded;
+      if (!findMatchedIndex) {
+        logger.info("{}: performing a snapshot catch-up to {}", raftMember.getName(), node);
+        doSnapshot();
+        // snapshot may overlap with logs
+        removeSnapshotLogs();
+        SnapshotCatchUpTask task =
+            new SnapshotCatchUpTask(logs, snapshot, node, catchUpManager, config);
+        catchUpSucceeded = task.call();
+      } else {
+        logger.info("{}: performing a log catch-up to {}", raftMember.getName(), node);
+        LogCatchUpTask task = new LogCatchUpTask(logs, node, catchUpManager, config);
+        catchUpSucceeded = task.call();
+      }
+      if (catchUpSucceeded) {
+        // the catch up may be triggered by an old heartbeat, and the node may have already
+        // caught up, so logs can be empty
+        if (!logs.isEmpty() || snapshot != null) {
+          long lastIndex =
+              !logs.isEmpty()
+                  ? logs.get(logs.size() - 1).getCurrLogIndex()
+                  : snapshot.getLastLogIndex();
+          peerInfo.setMatchIndex(lastIndex);
+        }
+        if (logger.isInfoEnabled()) {
+          logger.info(
+              "{}: Catch up {} finished, update it's matchIndex to {}, time consumption: {}ms",
+              raftMember.getName(),
+              node,
+              peerInfo.getMatchIndex(),
+              System.currentTimeMillis() - startTime);
+        }
+        peerInfo.resetInconsistentHeartbeatNum();
+      }
+
+    } catch (LeaderUnknownException e) {
+      logger.warn("Catch up {} failed because leadership is lost", node);
+    } catch (Exception e) {
+      logger.error("Catch up {} errored", node, e);
+    }
+    // the next catch-up is enabled
+    catchUpManager.unregisterTask(node);
+  }
+
+  @TestOnly
+  public void setLogs(List<Entry> logs) {
+    this.logs = logs;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/LogCatchUpInBatchHandler.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/LogCatchUpInBatchHandler.java
new file mode 100644
index 0000000000..8ed1451394
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/LogCatchUpInBatchHandler.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.consensus.natraft.protocol.log.catchup;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntryResult;
+
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_AGREE;
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_LOG_MISMATCH;
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_STRONG_ACCEPT;
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_WEAK_ACCEPT;
+
+public class LogCatchUpInBatchHandler implements AsyncMethodCallback<AppendEntryResult> {
+
+  private static final Logger logger = LoggerFactory.getLogger(LogCatchUpInBatchHandler.class);
+
+  private TEndPoint follower;
+  private List<ByteBuffer> logs;
+  private AtomicBoolean appendSucceed;
+  private String memberName;
+  private RaftMember raftMember;
+
+  @Override
+  public void onComplete(AppendEntryResult response) {
+    logger.debug(
+        "{}: Received a catch-up result size of {} from {}", memberName, logs.size(), follower);
+
+    long resp = response.status;
+    if (resp == RESPONSE_AGREE || resp == RESPONSE_STRONG_ACCEPT) {
+      synchronized (appendSucceed) {
+        appendSucceed.set(true);
+        appendSucceed.notifyAll();
+      }
+      logger.debug("{}: Succeeded to send logs, size is {}", memberName, logs.size());
+
+    } else if (resp == RESPONSE_LOG_MISMATCH || resp == RESPONSE_WEAK_ACCEPT) {
+      // this is not probably possible
+      logger.error(
+          "{}: Log mismatch occurred when sending logs, whose size is {}, resp: {}",
+          memberName,
+          logs.size(),
+          resp);
+      synchronized (appendSucceed) {
+        appendSucceed.notifyAll();
+      }
+    } else {
+      // the follower's term has updated, which means a new leader is elected
+      logger.debug(
+          "{}: Received a rejection because term is updated to {} when sending {} logs",
+          memberName,
+          resp,
+          logs.size());
+      raftMember.stepDown(resp, null);
+
+      synchronized (appendSucceed) {
+        appendSucceed.notifyAll();
+      }
+      logger.warn(
+          "{}: Catch-up with {} logs aborted because leadership is lost", logs.size(), memberName);
+    }
+  }
+
+  @Override
+  public void onError(Exception exception) {
+    synchronized (appendSucceed) {
+      appendSucceed.notifyAll();
+    }
+    logger.warn(
+        "{}: Catch-up fails when sending log, whose size is {}",
+        memberName,
+        logs.size(),
+        exception);
+  }
+
+  public void setAppendSucceed(AtomicBoolean appendSucceed) {
+    this.appendSucceed = appendSucceed;
+  }
+
+  public void setRaftMember(RaftMember raftMember) {
+    this.raftMember = raftMember;
+    this.memberName = raftMember.getName();
+  }
+
+  public void setFollower(TEndPoint follower) {
+    this.follower = follower;
+  }
+
+  public void setLogs(List<ByteBuffer> logs) {
+    this.logs = logs;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/LogCatchUpTask.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/LogCatchUpTask.java
new file mode 100644
index 0000000000..a07937efa5
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/LogCatchUpTask.java
@@ -0,0 +1,204 @@
+/*
+ * 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.natraft.protocol.log.catchup;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.consensus.natraft.client.AsyncRaftServiceClient;
+import org.apache.iotdb.consensus.natraft.exception.LeaderUnknownException;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.protocol.RaftRole;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntriesRequest;
+
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** LogCatchUpTask sends a list of logs to a node to make the node keep up with the leader. */
+@SuppressWarnings("java:S2274") // enable timeout
+public class LogCatchUpTask implements Callable<Boolean> {
+
+  // sending logs may take longer than normal communications
+  private long sendLogsWaitMs;
+  private static final Logger logger = LoggerFactory.getLogger(LogCatchUpTask.class);
+  TEndPoint node;
+  RaftMember raftMember;
+  CatchUpManager catchUpManager;
+  private List<Entry> logs;
+  boolean abort = false;
+  protected RaftConfig config;
+
+  LogCatchUpTask(
+      List<Entry> logs, TEndPoint node, CatchUpManager catchUpManager, RaftConfig config) {
+    this.logs = logs;
+    this.node = node;
+    this.raftMember = catchUpManager.getMember();
+    this.catchUpManager = catchUpManager;
+    this.sendLogsWaitMs = config.getWriteOperationTimeoutMS();
+    this.config = config;
+  }
+
+  private AppendEntriesRequest prepareRequest(List<ByteBuffer> logList, int startPos) {
+    AppendEntriesRequest request = new AppendEntriesRequest();
+
+    request.setGroupId(raftMember.getRaftGroupId().convertToTConsensusGroupId());
+    request.setLeader(raftMember.getThisNode());
+    request.setLeaderCommit(raftMember.getLogManager().getCommitLogIndex());
+
+    synchronized (raftMember.getStatus().getTerm()) {
+      // make sure this node is still a leader
+      if (raftMember.getRole() != RaftRole.LEADER) {
+        logger.debug("Leadership is lost when doing a catch-up to {}, aborting", node);
+        abort = true;
+        return null;
+      }
+      request.setTerm(raftMember.getStatus().getTerm().get());
+    }
+
+    request.setEntries(logList);
+    // set index for raft
+    request.setPrevLogIndex(logs.get(startPos).getCurrLogIndex() - 1);
+    if (startPos != 0) {
+      request.setPrevLogTerm(logs.get(startPos - 1).getCurrLogTerm());
+    } else {
+      try {
+        request.setPrevLogTerm(
+            raftMember.getLogManager().getTerm(logs.get(0).getCurrLogIndex() - 1));
+      } catch (Exception e) {
+        logger.error("getTerm failed for newly append entries", e);
+      }
+    }
+    logger.debug("{}, node={} catchup request={}", raftMember.getName(), node, request);
+    return request;
+  }
+
+  protected void doLogCatchUpInBatch() throws TException, InterruptedException {
+    List<ByteBuffer> logList = new ArrayList<>();
+    long totalLogSize = 0;
+    int firstLogPos = 0;
+    boolean batchFull;
+
+    for (int i = 0; i < logs.size() && !abort; i++) {
+
+      ByteBuffer logData = logs.get(i).serialize();
+      int logSize = logData.array().length;
+      if (logSize > config.getThriftMaxFrameSize() - IoTDBConstant.LEFT_SIZE_IN_REQUEST) {
+        logger.warn("the frame size {} of thrift is too small", config.getThriftMaxFrameSize());
+        abort = true;
+        return;
+      }
+
+      totalLogSize += logSize;
+      // we should send logs who's size is smaller than the max frame size of thrift
+      // left 200 byte for other fields of AppendEntriesRequest
+      // send at most 100 logs a time to avoid long latency
+      if (totalLogSize > config.getThriftMaxFrameSize() - IoTDBConstant.LEFT_SIZE_IN_REQUEST) {
+        // batch oversize, send previous batch and add the log to a new batch
+        sendBatchLogs(logList, firstLogPos);
+        logList.add(logData);
+        firstLogPos = i;
+        totalLogSize = logSize;
+      } else {
+        // just add the log the batch
+        logList.add(logData);
+      }
+
+      batchFull = logList.size() >= config.getLogNumInBatch();
+      if (batchFull) {
+        sendBatchLogs(logList, firstLogPos);
+        firstLogPos = i + 1;
+        totalLogSize = 0;
+      }
+    }
+
+    if (!logList.isEmpty()) {
+      sendBatchLogs(logList, firstLogPos);
+    }
+  }
+
+  private void sendBatchLogs(List<ByteBuffer> logList, int firstLogPos)
+      throws TException, InterruptedException {
+    if (logger.isInfoEnabled()) {
+      logger.info(
+          "{} send logs from {} num {} for {}",
+          raftMember.getThisNode(),
+          logs.get(firstLogPos).getCurrLogIndex(),
+          logList.size(),
+          node);
+    }
+    AppendEntriesRequest request = prepareRequest(logList, firstLogPos);
+    if (request == null) {
+      return;
+    }
+    // do append entries
+    if (logger.isInfoEnabled()) {
+      logger.info("{}: sending {} logs to {}", raftMember.getName(), logList.size(), node);
+    }
+    abort = !appendEntriesAsync(logList, request);
+    if (!abort && logger.isInfoEnabled()) {
+      logger.info("{}: sent {} logs to {}", raftMember.getName(), logList.size(), node);
+    }
+    logList.clear();
+  }
+
+  private boolean appendEntriesAsync(List<ByteBuffer> logList, AppendEntriesRequest request)
+      throws TException, InterruptedException {
+    AtomicBoolean appendSucceed = new AtomicBoolean(false);
+
+    LogCatchUpInBatchHandler handler = new LogCatchUpInBatchHandler();
+    handler.setAppendSucceed(appendSucceed);
+    handler.setRaftMember(raftMember);
+    handler.setFollower(node);
+    handler.setLogs(logList);
+    synchronized (appendSucceed) {
+      appendSucceed.set(false);
+      AsyncRaftServiceClient client = raftMember.getClient(node);
+      if (client == null) {
+        return false;
+      }
+      client.appendEntries(request, handler);
+      catchUpManager.registerTask(node);
+      appendSucceed.wait(sendLogsWaitMs);
+    }
+    return appendSucceed.get();
+  }
+
+  @Override
+  public Boolean call() throws TException, InterruptedException, LeaderUnknownException {
+    if (logs.isEmpty()) {
+      return true;
+    }
+
+    doLogCatchUpInBatch();
+    logger.info("{}: Catch up {} finished with result {}", raftMember.getName(), node, !abort);
+
+    // the next catch up is enabled
+    catchUpManager.unregisterTask(node);
+    return !abort;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/SnapshotCatchUpHandler.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/SnapshotCatchUpHandler.java
new file mode 100644
index 0000000000..a799b9683d
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/SnapshotCatchUpHandler.java
@@ -0,0 +1,61 @@
+/*
+ * 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.natraft.protocol.log.catchup;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.consensus.natraft.protocol.log.snapshot.Snapshot;
+
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** SnapshotCatchUpHandler receives the result of sending a snapshot to a stale node. */
+public class SnapshotCatchUpHandler implements AsyncMethodCallback<Void> {
+
+  private static final Logger logger = LoggerFactory.getLogger(SnapshotCatchUpHandler.class);
+
+  private AtomicBoolean succeed;
+  private TEndPoint receiver;
+  private Snapshot snapshot;
+
+  public SnapshotCatchUpHandler(AtomicBoolean succeed, TEndPoint receiver, Snapshot snapshot) {
+    this.succeed = succeed;
+    this.receiver = receiver;
+    this.snapshot = snapshot;
+  }
+
+  @Override
+  public void onComplete(Void resp) {
+    synchronized (succeed) {
+      succeed.set(true);
+      succeed.notifyAll();
+    }
+  }
+
+  @Override
+  public void onError(Exception exception) {
+    logger.error("Cannot send snapshot {} to {}", snapshot, receiver, exception);
+    synchronized (succeed) {
+      succeed.notifyAll();
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/SnapshotCatchUpTask.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/SnapshotCatchUpTask.java
new file mode 100644
index 0000000000..0d4bd04bcd
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/catchup/SnapshotCatchUpTask.java
@@ -0,0 +1,129 @@
+/*
+ * 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.natraft.protocol.log.catchup;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.consensus.natraft.client.AsyncRaftServiceClient;
+import org.apache.iotdb.consensus.natraft.exception.LeaderUnknownException;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.RaftRole;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+import org.apache.iotdb.consensus.natraft.protocol.log.snapshot.Snapshot;
+import org.apache.iotdb.consensus.raft.thrift.SendSnapshotRequest;
+
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * SnapshotCatchUpTask first sends the snapshot to the stale node then sends the logs to the node.
+ */
+public class SnapshotCatchUpTask extends LogCatchUpTask implements Callable<Boolean> {
+
+  private static final Logger logger = LoggerFactory.getLogger(SnapshotCatchUpTask.class);
+
+  // sending a snapshot may take longer than normal communications
+  private long sendSnapshotWaitMs;
+  private Snapshot snapshot;
+
+  SnapshotCatchUpTask(
+      List<Entry> logs,
+      Snapshot snapshot,
+      TEndPoint node,
+      CatchUpManager catchUpManager,
+      RaftConfig config) {
+    super(logs, node, catchUpManager, config);
+    this.snapshot = snapshot;
+    sendSnapshotWaitMs = config.getCatchUpTimeoutMS();
+  }
+
+  private void doSnapshotCatchUp() throws TException, InterruptedException, LeaderUnknownException {
+    SendSnapshotRequest request = new SendSnapshotRequest();
+    request.setGroupId(raftMember.getRaftGroupId().convertToTConsensusGroupId());
+    logger.info("Start to send snapshot to {}", node);
+    ByteBuffer data = snapshot.serialize();
+    if (logger.isInfoEnabled()) {
+      logger.info("Do snapshot catch up with size {}", data.array().length);
+    }
+    request.setSnapshotBytes(data);
+
+    synchronized (raftMember.getStatus().getTerm()) {
+      // make sure this node is still a leader
+      if (raftMember.getRole() != RaftRole.LEADER) {
+        throw new LeaderUnknownException(raftMember.getAllNodes());
+      }
+    }
+
+    abort = !sendSnapshotAsync(request);
+  }
+
+  @SuppressWarnings("java:S2274") // enable timeout
+  private boolean sendSnapshotAsync(SendSnapshotRequest request)
+      throws TException, InterruptedException {
+    AtomicBoolean succeed = new AtomicBoolean(false);
+    SnapshotCatchUpHandler handler = new SnapshotCatchUpHandler(succeed, node, snapshot);
+    AsyncRaftServiceClient client = raftMember.getClient(node);
+    if (client == null) {
+      logger.info("{}: client null for node {}", raftMember.getThisNode(), node);
+      abort = true;
+      return false;
+    }
+
+    logger.info(
+        "{}: the snapshot request size={}",
+        raftMember.getName(),
+        request.getSnapshotBytes().length);
+    synchronized (succeed) {
+      client.sendSnapshot(request, handler);
+      catchUpManager.registerTask(node);
+      succeed.wait(sendSnapshotWaitMs);
+    }
+    if (logger.isInfoEnabled()) {
+      logger.info("send snapshot to node {} success {}", raftMember.getThisNode(), succeed.get());
+    }
+    return succeed.get();
+  }
+
+  @Override
+  public Boolean call() throws InterruptedException, TException, LeaderUnknownException {
+    doSnapshotCatchUp();
+    if (abort) {
+      logger.warn("{}: Snapshot catch up {} failed", raftMember.getName(), node);
+      catchUpManager.unregisterTask(node);
+      return false;
+    }
+    logger.info(
+        "{}: Snapshot catch up {} finished, begin to catch up log", raftMember.getName(), node);
+    doLogCatchUpInBatch();
+    if (!abort) {
+      logger.info("{}: Catch up {} finished", raftMember.getName(), node);
+    } else {
+      logger.warn("{}: Log catch up {} failed", raftMember.getName(), node);
+    }
+    // the next catch up is enabled
+    catchUpManager.unregisterTask(node);
+    return !abort;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/dispatch/AppendNodeEntryHandler.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/dispatch/AppendNodeEntryHandler.java
new file mode 100644
index 0000000000..17999fba23
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/dispatch/AppendNodeEntryHandler.java
@@ -0,0 +1,158 @@
+/*
+ * 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.natraft.protocol.log.dispatch;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.protocol.log.VotingLog;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntryResult;
+
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.ConnectException;
+
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_AGREE;
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_LOG_MISMATCH;
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_OUT_OF_WINDOW;
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_STRONG_ACCEPT;
+import static org.apache.iotdb.consensus.natraft.protocol.Response.RESPONSE_WEAK_ACCEPT;
+
+/**
+ * AppendNodeEntryHandler checks if the log is successfully appended by the quorum or some node has
+ * rejected it for some reason when one node has finished the AppendEntryRequest. The target of the
+ * log is the single nodes, it requires the agreement from the quorum of the nodes to reach
+ * consistency.
+ */
+public class AppendNodeEntryHandler implements AsyncMethodCallback<AppendEntryResult> {
+
+  private static final Logger logger = LoggerFactory.getLogger(AppendNodeEntryHandler.class);
+
+  protected RaftMember member;
+  protected VotingLog log;
+  protected TEndPoint directReceiver;
+  protected int quorumSize;
+
+  public AppendNodeEntryHandler() {}
+
+  @Override
+  public void onComplete(AppendEntryResult response) {
+    if (log.isHasFailed()) {
+      return;
+    }
+
+    TEndPoint trueReceiver = response.isSetReceiver() ? response.receiver : directReceiver;
+
+    logger.debug(
+        "{}: Append response {} from {} for log {}", member.getName(), response, trueReceiver, log);
+
+    long resp = response.status;
+
+    if (resp == RESPONSE_STRONG_ACCEPT || resp == RESPONSE_AGREE) {
+      member
+          .getVotingLogList()
+          .onStronglyAccept(
+              log.getEntry().getCurrLogIndex(), log.getEntry().getCurrLogTerm(), trueReceiver);
+
+      member.getStatus().getPeerMap().get(trueReceiver).setMatchIndex(response.lastLogIndex);
+    } else if (resp > 0) {
+      // a response > 0 is the follower's term
+      // the leadership is stale, wait for the new leader's heartbeat
+      logger.debug(
+          "{}: Received a rejection from {} because term is stale: {}, log: {}",
+          member.getName(),
+          trueReceiver,
+          resp,
+          log);
+      member.stepDown(resp, null);
+      synchronized (log) {
+        log.notifyAll();
+      }
+    } else if (resp == RESPONSE_WEAK_ACCEPT) {
+      synchronized (log) {
+        log.getWeaklyAcceptedNodes().add(trueReceiver);
+        log.notifyAll();
+      }
+    } else {
+      // e.g., Response.RESPONSE_LOG_MISMATCH
+      if (resp == RESPONSE_LOG_MISMATCH || resp == RESPONSE_OUT_OF_WINDOW) {
+        logger.debug(
+            "{}: The log {} is rejected by {} because: {}",
+            member.getName(),
+            log,
+            trueReceiver,
+            resp);
+      } else {
+        logger.warn(
+            "{}: The log {} is rejected by {} because: {}",
+            member.getName(),
+            log,
+            trueReceiver,
+            resp);
+        onFail(trueReceiver);
+      }
+    }
+    // rejected because the receiver's logs are stale or the receiver has no cluster info, just
+    // wait for the heartbeat to handle
+  }
+
+  @Override
+  public void onError(Exception exception) {
+    if (exception instanceof ConnectException) {
+      logger.warn(
+          "{}: Cannot append log {}: cannot connect to {}: {}",
+          member.getName(),
+          log,
+          directReceiver,
+          exception.getMessage());
+    } else {
+      logger.warn(
+          "{}: Cannot append log {} to {}", member.getName(), log, directReceiver, exception);
+    }
+    onFail(directReceiver);
+  }
+
+  private void onFail(TEndPoint trueReceiver) {
+    synchronized (log) {
+      log.getFailedNodes().add(trueReceiver);
+      if (log.getFailedNodes().size() > quorumSize) {
+        // quorum members have failed, there is no need to wait for others
+        log.setHasFailed(true);
+        log.notifyAll();
+      }
+    }
+  }
+
+  public void setLog(VotingLog log) {
+    this.log = log;
+  }
+
+  public void setMember(RaftMember member) {
+    this.member = member;
+  }
+
+  public void setDirectReceiver(TEndPoint follower) {
+    this.directReceiver = follower;
+  }
+
+  public void setQuorumSize(int quorumSize) {
+    this.quorumSize = quorumSize;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/dispatch/LogDispatcher.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/dispatch/LogDispatcher.java
new file mode 100644
index 0000000000..a05edacfb0
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/dispatch/LogDispatcher.java
@@ -0,0 +1,328 @@
+/*
+ * 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.natraft.protocol.log.dispatch;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.consensus.natraft.client.AsyncRaftServiceClient;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.protocol.log.VotingLog;
+import org.apache.iotdb.consensus.natraft.protocol.log.flowcontrol.FlowMonitorManager;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntriesRequest;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntryResult;
+
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A LogDispatcher serves a raft leader by queuing logs that the leader wants to send to its
+ * followers and send the logs in an ordered manner so that the followers will not wait for previous
+ * logs for too long. For example: if the leader send 3 logs, log1, log2, log3, concurrently to
+ * follower A, the actual reach order may be log3, log2, and log1. According to the protocol, log3
+ * and log2 must halt until log1 reaches, as a result, the total delay may increase significantly.
+ */
+public class LogDispatcher {
+
+  private static final Logger logger = LoggerFactory.getLogger(LogDispatcher.class);
+  protected RaftMember member;
+  private RaftConfig config;
+  protected Map<TEndPoint, BlockingQueue<VotingLog>> nodesLogQueuesMap = new HashMap<>();
+  protected Map<TEndPoint, Boolean> nodesEnabled;
+  protected Map<TEndPoint, RateLimiter> nodesRateLimiter = new HashMap<>();
+  protected Map<TEndPoint, Double> nodesRate = new HashMap<>();
+  protected Map<TEndPoint, ExecutorService> executorServices = new HashMap<>();
+  protected ExecutorService resultHandlerThread =
+      IoTDBThreadPoolFactory.newFixedThreadPool(2, "AppendResultHandler");
+  protected boolean queueOrdered =
+      !(config.isUseFollowerSlidingWindow() && config.isEnableWeakAcceptance());
+
+  public int bindingThreadNum;
+  public static int maxBatchSize = 10;
+
+  public LogDispatcher(RaftMember member, RaftConfig config) {
+    this.member = member;
+    this.config = config;
+    bindingThreadNum = config.getDispatcherBindingThreadNum();
+    createQueueAndBindingThreads();
+  }
+
+  public void updateRateLimiter() {
+    logger.info("TEndPoint rates: {}", nodesRate);
+    for (Entry<TEndPoint, Double> nodeDoubleEntry : nodesRate.entrySet()) {
+      nodesRateLimiter.get(nodeDoubleEntry.getKey()).setRate(nodeDoubleEntry.getValue());
+    }
+  }
+
+  void createQueueAndBindingThreads() {
+    for (TEndPoint node : member.getAllNodes()) {
+      if (!node.equals(member.getThisNode())) {
+        BlockingQueue<VotingLog> logBlockingQueue;
+        logBlockingQueue = new ArrayBlockingQueue<>(config.getMaxNumOfLogsInMem());
+        nodesLogQueuesMap.put(node, logBlockingQueue);
+        FlowMonitorManager.INSTANCE.register(node);
+        nodesRateLimiter.put(node, RateLimiter.create(Double.MAX_VALUE));
+      }
+    }
+    updateRateLimiter();
+
+    for (int i = 0; i < bindingThreadNum; i++) {
+      for (Entry<TEndPoint, BlockingQueue<VotingLog>> pair : nodesLogQueuesMap.entrySet()) {
+        executorServices
+            .computeIfAbsent(
+                pair.getKey(),
+                n ->
+                    IoTDBThreadPoolFactory.newCachedThreadPool(
+                        "LogDispatcher-" + member.getName() + "-" + pair.getKey()))
+            .submit(newDispatcherThread(pair.getKey(), pair.getValue()));
+      }
+    }
+  }
+
+  @TestOnly
+  public void close() throws InterruptedException {
+    for (Entry<TEndPoint, ExecutorService> entry : executorServices.entrySet()) {
+      ExecutorService pool = entry.getValue();
+      pool.shutdownNow();
+      boolean closeSucceeded = pool.awaitTermination(10, TimeUnit.SECONDS);
+      if (!closeSucceeded) {
+        logger.warn("Cannot shut down dispatcher pool of {}-{}", member.getName(), entry.getKey());
+      }
+    }
+    resultHandlerThread.shutdownNow();
+  }
+
+  protected boolean addToQueue(BlockingQueue<VotingLog> nodeLogQueue, VotingLog request) {
+    return nodeLogQueue.add(request);
+  }
+
+  public void offer(VotingLog request) {
+
+    for (Entry<TEndPoint, BlockingQueue<VotingLog>> entry : nodesLogQueuesMap.entrySet()) {
+      if (nodesEnabled != null && !this.nodesEnabled.getOrDefault(entry.getKey(), false)) {
+        continue;
+      }
+
+      BlockingQueue<VotingLog> nodeLogQueue = entry.getValue();
+      try {
+        boolean addSucceeded = addToQueue(nodeLogQueue, request);
+
+        if (!addSucceeded) {
+          logger.debug(
+              "Log queue[{}] of {} is full, ignore the request to this node",
+              entry.getKey(),
+              member.getName());
+        }
+      } catch (IllegalStateException e) {
+        logger.debug(
+            "Log queue[{}] of {} is full, ignore the request to this node",
+            entry.getKey(),
+            member.getName());
+      }
+    }
+  }
+
+  DispatcherThread newDispatcherThread(TEndPoint node, BlockingQueue<VotingLog> logBlockingQueue) {
+    return new DispatcherThread(node, logBlockingQueue);
+  }
+
+  protected class DispatcherThread implements Runnable {
+
+    TEndPoint receiver;
+    private final BlockingQueue<VotingLog> logBlockingDeque;
+    protected List<VotingLog> currBatch = new ArrayList<>();
+    private final String baseName;
+
+    protected DispatcherThread(TEndPoint receiver, BlockingQueue<VotingLog> logBlockingDeque) {
+      this.receiver = receiver;
+      this.logBlockingDeque = logBlockingDeque;
+      baseName = "LogDispatcher-" + member.getName() + "-" + receiver;
+    }
+
+    @Override
+    public void run() {
+      if (logger.isDebugEnabled()) {
+        Thread.currentThread().setName(baseName);
+      }
+      try {
+        while (!Thread.interrupted()) {
+          synchronized (logBlockingDeque) {
+            VotingLog poll = logBlockingDeque.take();
+            currBatch.add(poll);
+            if (maxBatchSize > 1) {
+              while (!logBlockingDeque.isEmpty() && currBatch.size() < maxBatchSize) {
+                currBatch.add(logBlockingDeque.take());
+              }
+            }
+          }
+          if (logger.isDebugEnabled()) {
+            logger.debug("Sending {} logs to {}", currBatch.size(), receiver);
+          }
+          serializeEntries();
+          if (!queueOrdered) {
+            currBatch.sort(Comparator.comparingLong(s -> s.getEntry().getCurrLogIndex()));
+          }
+          sendLogs(currBatch);
+          currBatch.clear();
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      } catch (Exception e) {
+        logger.error("Unexpected error in log dispatcher", e);
+      }
+      logger.info("Dispatcher exits");
+    }
+
+    protected void serializeEntries() throws InterruptedException {
+      for (VotingLog request : currBatch) {
+
+        request.getAppendEntryRequest().entry = request.getEntry().serialize();
+        request.getEntry().setByteSize(request.getAppendEntryRequest().entry.limit());
+      }
+    }
+
+    private void appendEntriesAsync(
+        List<ByteBuffer> logList, AppendEntriesRequest request, List<VotingLog> currBatch)
+        throws TException {
+      AsyncMethodCallback<AppendEntryResult> handler = new AppendEntriesHandler(currBatch);
+      AsyncRaftServiceClient client = member.getClient(receiver);
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "{}: append entries {} with {} logs", member.getName(), receiver, logList.size());
+      }
+      if (client != null) {
+        client.appendEntries(request, handler);
+      }
+    }
+
+    protected AppendEntriesRequest prepareRequest(
+        List<ByteBuffer> logList, List<VotingLog> currBatch, int firstIndex) {
+      AppendEntriesRequest request = new AppendEntriesRequest();
+
+      request.setGroupId(member.getRaftGroupId().convertToTConsensusGroupId());
+      request.setLeader(member.getThisNode());
+      request.setLeaderCommit(member.getLogManager().getCommitLogIndex());
+
+      request.setTerm(member.getStatus().getTerm().get());
+
+      request.setEntries(logList);
+      // set index for raft
+      request.setPrevLogIndex(currBatch.get(firstIndex).getEntry().getCurrLogIndex() - 1);
+      try {
+        request.setPrevLogTerm(currBatch.get(firstIndex).getAppendEntryRequest().prevLogTerm);
+      } catch (Exception e) {
+        logger.error("getTerm failed for newly append entries", e);
+      }
+      return request;
+    }
+
+    private void sendLogs(List<VotingLog> currBatch) throws TException {
+      int logIndex = 0;
+      logger.debug(
+          "send logs from index {} to {}",
+          currBatch.get(0).getEntry().getCurrLogIndex(),
+          currBatch.get(currBatch.size() - 1).getEntry().getCurrLogIndex());
+      while (logIndex < currBatch.size()) {
+        long logSize = 0;
+        long logSizeLimit = config.getThriftMaxFrameSize();
+        List<ByteBuffer> logList = new ArrayList<>();
+        int prevIndex = logIndex;
+
+        for (; logIndex < currBatch.size(); logIndex++) {
+          long curSize = currBatch.get(logIndex).getAppendEntryRequest().entry.array().length;
+          if (logSizeLimit - curSize - logSize <= IoTDBConstant.LEFT_SIZE_IN_REQUEST) {
+            break;
+          }
+          logSize += curSize;
+          logList.add(currBatch.get(logIndex).getAppendEntryRequest().entry);
+        }
+
+        AppendEntriesRequest appendEntriesRequest = prepareRequest(logList, currBatch, prevIndex);
+        FlowMonitorManager.INSTANCE.report(receiver, logSize);
+        nodesRateLimiter.get(receiver).acquire((int) logSize);
+
+        appendEntriesAsync(logList, appendEntriesRequest, currBatch.subList(prevIndex, logIndex));
+      }
+    }
+
+    public AppendNodeEntryHandler getAppendNodeEntryHandler(
+        VotingLog log, TEndPoint node, int quorumSize) {
+      AppendNodeEntryHandler handler = new AppendNodeEntryHandler();
+      handler.setDirectReceiver(node);
+      handler.setLog(log);
+      handler.setMember(member);
+      handler.setQuorumSize(quorumSize);
+      return handler;
+    }
+
+    class AppendEntriesHandler implements AsyncMethodCallback<AppendEntryResult> {
+
+      private final List<AsyncMethodCallback<AppendEntryResult>> singleEntryHandlers;
+
+      private AppendEntriesHandler(List<VotingLog> batch) {
+        singleEntryHandlers = new ArrayList<>(batch.size());
+        for (VotingLog sendLogRequest : batch) {
+          AppendNodeEntryHandler handler =
+              getAppendNodeEntryHandler(sendLogRequest, receiver, sendLogRequest.getQuorumSize());
+          singleEntryHandlers.add(handler);
+        }
+      }
+
+      @Override
+      public void onComplete(AppendEntryResult aLong) {
+        for (AsyncMethodCallback<AppendEntryResult> singleEntryHandler : singleEntryHandlers) {
+          singleEntryHandler.onComplete(aLong);
+        }
+      }
+
+      @Override
+      public void onError(Exception e) {
+        for (AsyncMethodCallback<AppendEntryResult> singleEntryHandler : singleEntryHandlers) {
+          singleEntryHandler.onError(e);
+        }
+      }
+    }
+  }
+
+  public Map<TEndPoint, Double> getNodesRate() {
+    return nodesRate;
+  }
+
+  public Map<TEndPoint, BlockingQueue<VotingLog>> getNodesLogQueuesMap() {
+    return nodesLogQueuesMap;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/dispatch/VotingLogList.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/dispatch/VotingLogList.java
new file mode 100644
index 0000000000..8669c50d19
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/dispatch/VotingLogList.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.consensus.natraft.protocol.log.dispatch;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.consensus.natraft.exception.LogExecutionException;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.protocol.log.VotingLog;
+import org.apache.iotdb.consensus.natraft.protocol.log.manager.RaftLogManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class VotingLogList {
+
+  private static final Logger logger = LoggerFactory.getLogger(VotingLogList.class);
+  private int quorumSize;
+  private RaftMember member;
+  private Map<TEndPoint, Long> stronglyAcceptedIndices = new ConcurrentHashMap<>();
+  private AtomicLong newCommitIndex = new AtomicLong(-1);
+
+  public VotingLogList(int quorumSize, RaftMember member) {
+    this.quorumSize = quorumSize;
+    this.member = member;
+  }
+
+  private boolean tryCommit() {
+    RaftLogManager logManager = member.getLogManager();
+
+    if (computeNewCommitIndex()
+        && logManager != null
+        && newCommitIndex.get() > logManager.getCommitLogIndex()) {
+      try {
+        logManager.commitTo(newCommitIndex.get());
+      } catch (LogExecutionException e) {
+        logger.error("Fail to commit {}", newCommitIndex, e);
+      }
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  public boolean computeNewCommitIndex() {
+    List<Entry<TEndPoint, Long>> nodeIndices = new ArrayList<>(stronglyAcceptedIndices.entrySet());
+    if (nodeIndices.size() < quorumSize) {
+      return false;
+    }
+    nodeIndices.sort(Entry.comparingByValue());
+    Long value = nodeIndices.get(nodeIndices.size() - quorumSize).getValue();
+    long oldValue = newCommitIndex.getAndUpdate(oldV -> Math.max(value, oldV));
+    return value > oldValue;
+  }
+
+  /**
+   * When an entry of index-term is strongly accepted by a node of acceptingNodeId, record the id in
+   * all entries whose index <= the accepted entry. If any entry is accepted by a quorum, remove it
+   * from the list.
+   *
+   * @param index
+   * @param term
+   * @param acceptingNode
+   * @return the lastly removed entry if any.
+   */
+  public void onStronglyAccept(long index, long term, TEndPoint acceptingNode) {
+    logger.debug("{}-{} is strongly accepted by {}", index, term, acceptingNode);
+
+    Long newIndex =
+        stronglyAcceptedIndices.compute(
+            acceptingNode,
+            (nid, oldIndex) -> {
+              if (oldIndex == null) {
+                return index;
+              } else {
+                if (index > oldIndex) {
+                  return index;
+                }
+                return oldIndex;
+              }
+            });
+    if (newIndex == index) {
+      tryCommit();
+    }
+  }
+
+  public int totalAcceptedNodeNum(VotingLog log) {
+    long index = log.getEntry().getCurrLogIndex();
+    int num = log.getWeaklyAcceptedNodes().size();
+    for (Entry<TEndPoint, Long> entry : stronglyAcceptedIndices.entrySet()) {
+      if (entry.getValue() >= index) {
+        num++;
+      }
+    }
+    return num;
+  }
+
+  public String report() {
+    return String.format(
+        "Nodes accepted indices: %s, new commitIndex: %d",
+        stronglyAcceptedIndices, newCommitIndex.get());
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/flowcontrol/FlowBalancer.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/flowcontrol/FlowBalancer.java
new file mode 100644
index 0000000000..1411d98e49
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/flowcontrol/FlowBalancer.java
@@ -0,0 +1,140 @@
+/*
+ * 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.natraft.protocol.log.flowcontrol;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.protocol.RaftRole;
+import org.apache.iotdb.consensus.natraft.protocol.log.VotingLog;
+import org.apache.iotdb.consensus.natraft.protocol.log.dispatch.LogDispatcher;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class FlowBalancer {
+
+  private static final Logger logger = LoggerFactory.getLogger(FlowBalancer.class);
+  private double maxFlow = 900_000_000;
+  private double minFlow = 10_000_000;
+  private int windowsToUse;
+  private double overestimateFactor;
+  private int flowBalanceIntervalMS = 1000;
+  private FlowMonitorManager flowMonitorManager = FlowMonitorManager.INSTANCE;
+  private LogDispatcher logDispatcher;
+  private RaftMember member;
+
+  private ScheduledExecutorService scheduledExecutorService;
+
+  public FlowBalancer(LogDispatcher logDispatcher, RaftMember member, RaftConfig config) {
+    this.logDispatcher = logDispatcher;
+    this.member = member;
+    windowsToUse = config.getFollowerLoadBalanceWindowsToUse();
+    overestimateFactor = config.getFollowerLoadBalanceOverestimateFactor();
+  }
+
+  public void start() {
+    scheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
+    ScheduledExecutorUtil.safelyScheduleAtFixedRate(
+        scheduledExecutorService,
+        this::rebalance,
+        flowBalanceIntervalMS,
+        flowBalanceIntervalMS,
+        TimeUnit.MILLISECONDS);
+  }
+
+  public void stop() {
+    scheduledExecutorService.shutdownNow();
+  }
+
+  private void rebalance() {
+    if (!member.getStatus().getRole().equals(RaftRole.LEADER)) {
+      return;
+    }
+
+    List<TEndPoint> followers = new ArrayList<>(member.getAllNodes());
+    followers.remove(member.getThisNode());
+
+    int nodeNum = member.getAllNodes().size();
+    int followerNum = nodeNum - 1;
+
+    double thisNodeFlow = flowMonitorManager.averageFlow(member.getThisNode(), windowsToUse);
+    double assumedFlow = thisNodeFlow * overestimateFactor;
+    logger.info("Flow of this node: {}", thisNodeFlow);
+    Map<TEndPoint, BlockingQueue<VotingLog>> nodesLogQueuesMap =
+        logDispatcher.getNodesLogQueuesMap();
+    Map<TEndPoint, Double> nodesRate = logDispatcher.getNodesRate();
+
+    // sort followers according to their queue length
+    followers.sort(Comparator.comparing(node -> nodesLogQueuesMap.get(node).size()));
+    if (assumedFlow * followerNum > maxFlow) {
+      enterBurst(nodesRate, nodeNum, assumedFlow, followers);
+    } else {
+      exitBurst(followerNum, nodesRate, followers);
+    }
+    logDispatcher.updateRateLimiter();
+  }
+
+  private void enterBurst(
+      Map<TEndPoint, Double> nodesRate,
+      int nodeNum,
+      double assumedFlow,
+      List<TEndPoint> followers) {
+    int followerNum = nodeNum - 1;
+    int quorumFollowerNum = nodeNum / 2;
+    double remainingFlow = maxFlow;
+    double quorumMaxFlow = maxFlow / quorumFollowerNum;
+    // distribute flow to quorum followers with the shortest queues
+    double flowToQuorum = Math.min(assumedFlow, quorumMaxFlow);
+    int i = 0;
+    for (; i < quorumFollowerNum; i++) {
+      TEndPoint node = followers.get(i);
+      nodesRate.put(node, maxFlow);
+      remainingFlow -= flowToQuorum;
+    }
+    double flowToRemaining = remainingFlow / (followerNum - quorumFollowerNum);
+    if (flowToRemaining < minFlow) {
+      flowToRemaining = minFlow;
+    }
+    for (; i < followerNum; i++) {
+      TEndPoint node = followers.get(i);
+      nodesRate.put(node, flowToRemaining);
+    }
+  }
+
+  private void exitBurst(
+      int followerNum, Map<TEndPoint, Double> nodesRate, List<TEndPoint> followers) {
+    // lift flow limits
+    for (int i = 0; i < followerNum; i++) {
+      TEndPoint node = followers.get(i);
+      nodesRate.put(node, maxFlow);
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/flowcontrol/FlowMonitor.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/flowcontrol/FlowMonitor.java
new file mode 100644
index 0000000000..11dd8f7991
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/flowcontrol/FlowMonitor.java
@@ -0,0 +1,144 @@
+/*
+ * 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.natraft.protocol.log.flowcontrol;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayDeque;
+import java.util.Date;
+import java.util.Iterator;
+
+public class FlowMonitor {
+
+  private static final Logger logger = LoggerFactory.getLogger(FlowMonitor.class);
+  private static final String FILE_SUFFIX = ".flow";
+  private ArrayDeque<Pair<Long, Long>> windows;
+  private long currWindowStart;
+  private long currWindowSum;
+  private long windowInterval;
+  private TEndPoint node;
+  private int maxWindowSize;
+  private BufferedWriter writer;
+  private DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+  private RaftConfig config;
+
+  public FlowMonitor(TEndPoint node, RaftConfig config) throws IOException {
+    this.maxWindowSize = config.getFlowMonitorMaxWindowSize();
+    this.windows = new ArrayDeque<>(maxWindowSize);
+    this.windowInterval = config.getFlowMonitorWindowInterval();
+    this.node = node;
+    this.config = config;
+    initSerializer();
+  }
+
+  private void initSerializer() throws IOException {
+    String path =
+        config.getStorageDir() + File.separator + node.getIp() + "-" + node.getPort() + FILE_SUFFIX;
+    File file = new File(path);
+    file.delete();
+    writer = new BufferedWriter(new FileWriter(file));
+    writer.write("Time,FlowSum");
+    writer.newLine();
+  }
+
+  public void close() {
+    saveWindow();
+    while (windows.size() > 0) {
+      serializeWindow();
+    }
+    try {
+      writer.close();
+      logger.info("Flow monitor {} is closed", node);
+    } catch (IOException e) {
+      logger.warn("Cannot close serializer of {}", node, e);
+    }
+  }
+
+  private void resetWindow(long newWindowStart) {
+    currWindowStart = newWindowStart;
+    currWindowSum = 0;
+  }
+
+  private void serializeWindow() {
+    Pair<Long, Long> window = windows.removeFirst();
+    try {
+      String windowString =
+          String.format("%s,%d", dateFormat.format(new Date(window.left)), window.right);
+      logger.debug("New window {} serialized by {}", windowString, node);
+      writer.write(windowString);
+      writer.newLine();
+    } catch (IOException e) {
+      logger.warn("Cannot serialize window {} of {}", window, node, e);
+    }
+  }
+
+  private void checkSize() {
+    if (windows.size() == maxWindowSize) {
+      serializeWindow();
+    }
+  }
+
+  private void saveWindow() {
+    if (currWindowSum != 0) {
+      checkSize();
+      windows.add(new Pair<>(currWindowStart, currWindowSum));
+      logger.debug("New window {},{} generated by {}", currWindowStart, currWindowSum, node);
+    }
+  }
+
+  public synchronized void report(long val) {
+    long currTime = System.currentTimeMillis();
+    long targetWindowStart = currTime - currTime % windowInterval;
+    if (targetWindowStart != currWindowStart) {
+      // save the current window and start a new window
+      saveWindow();
+      resetWindow(targetWindowStart);
+    }
+    // update the current window
+    currWindowSum += val;
+  }
+
+  public double averageFlow(int windowsToUse) {
+    long flowSum = currWindowSum;
+    long intervalSum = System.currentTimeMillis() - currWindowStart;
+    Iterator<Pair<Long, Long>> windowIterator = windows.descendingIterator();
+    for (int i = 1; i < windowsToUse; i++) {
+      if (windowIterator.hasNext()) {
+        Pair<Long, Long> window = windowIterator.next();
+        flowSum += window.right;
+        intervalSum += windowInterval;
+      } else {
+        break;
+      }
+    }
+    return flowSum * 1.0 / intervalSum * 1000;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/flowcontrol/FlowMonitorManager.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/flowcontrol/FlowMonitorManager.java
new file mode 100644
index 0000000000..f4f7af79e7
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/flowcontrol/FlowMonitorManager.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.consensus.natraft.protocol.log.flowcontrol;
+
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class FlowMonitorManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(FlowMonitorManager.class);
+  public static final FlowMonitorManager INSTANCE = new FlowMonitorManager();
+
+  private Map<TEndPoint, FlowMonitor> monitorMap = new ConcurrentHashMap<>();
+  private RaftConfig config;
+
+  private FlowMonitorManager() {}
+
+  public void setConfig(RaftConfig config) {
+    this.config = config;
+  }
+
+  public void close() {
+    for (FlowMonitor flowMonitor : monitorMap.values()) {
+      flowMonitor.close();
+    }
+    monitorMap.clear();
+  }
+
+  public void register(TEndPoint node) {
+    logger.info("Registering flow monitor {}", node);
+    monitorMap.computeIfAbsent(
+        node,
+        n -> {
+          try {
+            return new FlowMonitor(n, config);
+          } catch (IOException e) {
+            logger.warn("Cannot register flow monitor for {}", node, e);
+            return null;
+          }
+        });
+  }
+
+  public void report(TEndPoint node, long val) {
+    FlowMonitor flowMonitor = monitorMap.get(node);
+    if (flowMonitor != null) {
+      flowMonitor.report(val);
+    } else {
+      logger.warn("Flow monitor {} is not registered", node);
+    }
+  }
+
+  public double averageFlow(TEndPoint node, int windowsToUse) {
+    return monitorMap.get(node).averageFlow(windowsToUse);
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/logtype/EmptyEntry.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/logtype/EmptyEntry.java
new file mode 100644
index 0000000000..28a8724115
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/logtype/EmptyEntry.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.consensus.natraft.protocol.log.logtype;
+
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class EmptyEntry extends Entry {
+
+  public EmptyEntry() {}
+
+  public EmptyEntry(long index, long term) {
+    this.setCurrLogIndex(index);
+    this.setCurrLogTerm(term);
+  }
+
+  @Override
+  public ByteBuffer serialize() {
+    ByteArrayOutputStream byteArrayOutputStream =
+        new ByteArrayOutputStream(getDefaultSerializationBufferSize());
+    try (DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
+      dataOutputStream.writeByte((byte) Types.EMPTY.ordinal());
+      dataOutputStream.writeLong(getCurrLogIndex());
+      dataOutputStream.writeLong(getCurrLogTerm());
+    } catch (IOException e) {
+      // unreachable
+    }
+    return ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
+  }
+
+  @Override
+  public void deserialize(ByteBuffer buffer) {
+    setCurrLogIndex(buffer.getLong());
+    setCurrLogTerm(buffer.getLong());
+  }
+
+  @Override
+  public long estimateSize() {
+    return 0;
+  }
+
+  @Override
+  public String toString() {
+    return "term:" + getCurrLogTerm() + ",index:" + getCurrLogIndex();
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/logtype/RequestEntry.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/logtype/RequestEntry.java
new file mode 100644
index 0000000000..e7b1b4b57e
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/logtype/RequestEntry.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.consensus.natraft.protocol.log.logtype;
+
+import org.apache.iotdb.consensus.common.request.ByteBufferConsensusRequest;
+import org.apache.iotdb.consensus.common.request.IConsensusRequest;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+import static org.apache.iotdb.consensus.natraft.protocol.log.Entry.Types.CLIENT_REQUEST;
+
+/** RequestLog contains a non-partitioned request like set storage group. */
+public class RequestEntry extends Entry {
+
+  private static final Logger logger = LoggerFactory.getLogger(RequestEntry.class);
+  private IConsensusRequest request;
+
+  public RequestEntry() {}
+
+  public RequestEntry(IConsensusRequest request) {
+    this.request = request;
+  }
+
+  @Override
+  public ByteBuffer serialize() {
+    PublicBAOS byteArrayOutputStream = new PublicBAOS(getDefaultSerializationBufferSize());
+    try (DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
+      dataOutputStream.writeByte((byte) CLIENT_REQUEST.ordinal());
+
+      dataOutputStream.writeLong(getCurrLogIndex());
+      dataOutputStream.writeLong(getCurrLogTerm());
+
+      ByteBuffer byteBuffer = request.serializeToByteBuffer();
+      dataOutputStream.write(
+          byteBuffer.array(), byteBuffer.arrayOffset(), byteBuffer.limit() - byteBuffer.position());
+    } catch (IOException e) {
+      // unreachable
+    }
+
+    return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size());
+  }
+
+  @Override
+  public void serialize(ByteBuffer buffer) {
+    buffer.put((byte) CLIENT_REQUEST.ordinal());
+    buffer.putLong(getCurrLogIndex());
+    buffer.putLong(getCurrLogTerm());
+    buffer.put(request.serializeToByteBuffer());
+  }
+
+  @Override
+  public void deserialize(ByteBuffer buffer) {
+    setCurrLogIndex(buffer.getLong());
+    setCurrLogTerm(buffer.getLong());
+
+    request = new ByteBufferConsensusRequest(buffer);
+  }
+
+  public IConsensusRequest getRequest() {
+    return request;
+  }
+
+  public void setRequest(IConsensusRequest request) {
+    this.request = request;
+  }
+
+  @Override
+  public String toString() {
+    return request + ",term:" + getCurrLogTerm() + ",index:" + getCurrLogIndex();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    if (!super.equals(o)) {
+      return false;
+    }
+    RequestEntry that = (RequestEntry) o;
+    return Objects.equals(request, that.request);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), request);
+  }
+
+  @Override
+  public long estimateSize() {
+    return request.estimateSize();
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/manager/RaftLogManager.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/manager/RaftLogManager.java
new file mode 100644
index 0000000000..6a15693a77
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/manager/RaftLogManager.java
@@ -0,0 +1,913 @@
+/*
+ * 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.natraft.protocol.log.manager;
+
+import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil;
+import org.apache.iotdb.consensus.IStateMachine;
+import org.apache.iotdb.consensus.natraft.exception.LogExecutionException;
+import org.apache.iotdb.consensus.natraft.protocol.HardState;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+import org.apache.iotdb.consensus.natraft.protocol.log.applier.LogApplier;
+import org.apache.iotdb.consensus.natraft.protocol.log.logtype.EmptyEntry;
+import org.apache.iotdb.consensus.natraft.protocol.log.serialization.LogManagerMeta;
+import org.apache.iotdb.consensus.natraft.protocol.log.serialization.StableEntryManager;
+import org.apache.iotdb.consensus.natraft.protocol.log.snapshot.Snapshot;
+
+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.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public abstract class RaftLogManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RaftLogManager.class);
+
+  private RaftConfig config;
+
+  /** manage uncommitted entries */
+  private List<Entry> entries;
+
+  /** manage committed entries in disk for safety */
+  private StableEntryManager stableEntryManager;
+
+  private volatile long commitIndex;
+
+  /**
+   * The committed logs whose index is smaller than this are all have been applied, for example,
+   * suppose there are 5 committed logs, whose log index is 1,2,3,4,5; if the applied sequence is
+   * 1,3,2,5,4, then the maxHaveAppliedCommitIndex according is 1,1,3,3,5. This attributed is only
+   * used for asyncLogApplier
+   */
+  private volatile long appliedIndex;
+
+  private volatile long appliedTerm;
+
+  private final Object changeApplyCommitIndexCond = new Object();
+
+  protected LogApplier logApplier;
+
+  /** to distinguish managers of different members */
+  private String name;
+
+  private ScheduledExecutorService deleteLogExecutorService;
+  private ScheduledFuture<?> deleteLogFuture;
+
+  private ExecutorService checkLogApplierExecutorService;
+  private Future<?> checkLogApplierFuture;
+
+  /** minimum number of committed logs in memory */
+  private int minNumOfLogsInMem;
+
+  /** maximum number of committed logs in memory */
+  private int maxNumOfLogsInMem;
+
+  private long maxLogMemSize;
+
+  /**
+   * Each time new logs are appended, this condition will be notified so logs that have larger
+   * indices but arrived earlier can proceed.
+   */
+  private final Object[] logUpdateConditions = new Object[1024];
+
+  protected List<Entry> blockedUnappliedLogList;
+
+  protected IStateMachine stateMachine;
+
+  protected ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+  private long committedEntrySize;
+
+  protected RaftLogManager(
+      StableEntryManager stableEntryManager,
+      LogApplier applier,
+      String name,
+      IStateMachine stateMachine) {
+    this.logApplier = applier;
+    this.name = name;
+    this.stateMachine = stateMachine;
+    this.setStableEntryManager(stableEntryManager);
+
+    initConf();
+    initEntries();
+
+    this.blockedUnappliedLogList = new CopyOnWriteArrayList<>();
+
+    this.deleteLogExecutorService =
+        IoTDBThreadPoolFactory.newScheduledThreadPoolWithDaemon(1, "raft-log-delete-" + name);
+
+    this.checkLogApplierExecutorService =
+        IoTDBThreadPoolFactory.newSingleThreadExecutorWithDaemon("check-log-applier-" + name);
+
+    /** deletion check period of the submitted log */
+    int logDeleteCheckIntervalSecond = config.getLogDeleteCheckIntervalSecond();
+
+    if (logDeleteCheckIntervalSecond > 0) {
+      this.deleteLogFuture =
+          ScheduledExecutorUtil.safelyScheduleAtFixedRate(
+              deleteLogExecutorService,
+              this::checkDeleteLog,
+              logDeleteCheckIntervalSecond,
+              logDeleteCheckIntervalSecond,
+              TimeUnit.SECONDS);
+    }
+
+    this.checkLogApplierFuture = checkLogApplierExecutorService.submit(this::checkAppliedLogIndex);
+
+    /** flush log to file periodically */
+    if (config.isEnableRaftLogPersistence()) {
+      this.applyAllCommittedLogWhenStartUp();
+    }
+
+    for (int i = 0; i < logUpdateConditions.length; i++) {
+      logUpdateConditions[i] = new Object();
+    }
+  }
+
+  private void initEntries() {
+    LogManagerMeta meta = stableEntryManager.getMeta();
+    List<Entry> allEntriesAfterAppliedIndex = stableEntryManager.getAllEntriesAfterAppliedIndex();
+    if (!allEntriesAfterAppliedIndex.isEmpty()) {
+      entries.addAll(allEntriesAfterAppliedIndex);
+    } else {
+      entries.add(
+          new EmptyEntry(
+              meta.getLastAppliedIndex() == -1 ? -1 : meta.getLastAppliedIndex() - 1,
+              meta.getLastAppliedTerm()));
+    }
+
+    this.commitIndex = meta.getCommitLogIndex();
+    this.appliedIndex = meta.getLastAppliedIndex();
+    this.appliedTerm = meta.getLastAppliedTerm();
+
+    for (Entry entry : entries) {
+      if (entry.getCurrLogIndex() <= commitIndex) {
+        committedEntrySize += entry.estimateSize();
+      }
+    }
+  }
+
+  private void initConf() {
+    minNumOfLogsInMem = config.getMinNumOfLogsInMem();
+    maxNumOfLogsInMem = config.getMaxNumOfLogsInMem();
+    maxLogMemSize = config.getMaxMemorySizeForRaftLog();
+  }
+
+  public Snapshot getSnapshot() {
+    return getSnapshot(-1);
+  }
+
+  public abstract Snapshot getSnapshot(long minLogIndex);
+
+  /**
+   * IMPORTANT!!!
+   *
+   * <p>The subclass's takeSnapshot() must call this method to insure that all logs have been
+   * applied before take snapshot
+   *
+   * <p>
+   *
+   * @throws IOException timeout exception
+   */
+  public abstract void takeSnapshot();
+
+  /**
+   * Update the raftNode's hardState(currentTerm,voteFor) and flush to disk.
+   *
+   * @param state
+   */
+  public void updateHardState(HardState state) {
+    getStableEntryManager().setHardStateAndFlush(state);
+  }
+
+  /**
+   * Return the raftNode's hardState(currentTerm,voteFor).
+   *
+   * @return state
+   */
+  public HardState getHardState() {
+    return getStableEntryManager().getHardState();
+  }
+
+  /**
+   * Return the raftNode's commitIndex.
+   *
+   * @return commitIndex
+   */
+  public long getCommitLogIndex() {
+    return commitIndex;
+  }
+
+  /**
+   * Return the first entry's index which have not been compacted.
+   *
+   * @return firstIndex
+   */
+  public long getFirstIndex() {
+    return entries.get(0).getCurrLogIndex();
+  }
+
+  /**
+   * Return the last entry's index which have been added into log module.
+   *
+   * @return lastIndex
+   */
+  public long getLastLogIndex() {
+    try {
+      lock.readLock().lock();
+      return entries.get(entries.size() - 1).getCurrLogIndex();
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  public Entry getLastEntry() {
+    try {
+      lock.readLock().lock();
+      return entries.get(entries.size() - 1);
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  /**
+   * Returns the term for given index.
+   *
+   * @param index request entry index
+   * @return throw EntryCompactedException if index < dummyIndex, -1 if index > lastIndex or the
+   *     entry is compacted, otherwise return the entry's term for given index
+   */
+  public long getTerm(long index) {
+    try {
+      lock.readLock().lock();
+      long firstIndex = getFirstIndex();
+      if (index < firstIndex) {
+        // search in disk
+        if (config.isEnableRaftLogPersistence()) {
+          List<Entry> logsInDisk = getStableEntryManager().getEntries(index, index);
+          if (logsInDisk.isEmpty()) {
+            return -1;
+          } else {
+            return logsInDisk.get(0).getCurrLogTerm();
+          }
+        }
+        return -1;
+      }
+
+      long lastIndex = getLastLogIndex();
+      if (index > lastIndex) {
+        return -1;
+      }
+
+      firstIndex = getFirstIndex();
+      return entries.get((int) (index - firstIndex)).getCurrLogTerm();
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  /**
+   * Return the last entry's term. If it goes wrong, there must be an unexpected exception.
+   *
+   * @return last entry's term
+   */
+  public long getLastLogTerm() {
+    try {
+      lock.readLock().lock();
+      return entries.get(entries.size() - 1).getCurrLogTerm();
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  /**
+   * Return the commitIndex's term. If it goes wrong, there must be an unexpected exception.
+   *
+   * @return commitIndex's term
+   */
+  public long getCommitLogTerm() {
+    return getTerm(commitIndex);
+  }
+
+  /**
+   * Used by follower node to support leader's complicated log replication rpc parameters and try to
+   * commit entries.
+   *
+   * @param lastIndex leader's matchIndex for this follower node
+   * @param lastTerm the entry's term which index is leader's matchIndex for this follower node
+   * @param leaderCommit leader's commitIndex
+   * @param entries entries sent from the leader node Note that the leader must ensure
+   *     entries[0].index = lastIndex + 1
+   * @return -1 if the entries cannot be appended, otherwise the last index of new entries
+   */
+  public long maybeAppend(long lastIndex, long lastTerm, long leaderCommit, List<Entry> entries) {
+    try {
+      lock.writeLock().lock();
+      if (matchTerm(lastTerm, lastIndex)) {
+        long newLastIndex = lastIndex + entries.size();
+        long ci = findConflict(entries);
+        if (ci <= commitIndex) {
+          if (ci != -1) {
+            logger.error(
+                "{}: entry {} conflict with committed entry [commitIndex({})]",
+                name,
+                ci,
+                commitIndex);
+          } else {
+            if (logger.isDebugEnabled() && !entries.isEmpty()) {
+              logger.debug(
+                  "{}: Appending entries [{} and other {} logs] all exist locally",
+                  name,
+                  entries.get(0),
+                  entries.size() - 1);
+            }
+          }
+
+        } else {
+          long offset = lastIndex + 1;
+          append(entries.subList((int) (ci - offset), entries.size()));
+        }
+        try {
+          commitTo(Math.min(leaderCommit, newLastIndex));
+        } catch (LogExecutionException e) {
+          // exceptions are ignored on follower side
+        }
+        return newLastIndex;
+      }
+      return -1;
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  /**
+   * Used by leader node or MaybeAppend to directly append to unCommittedEntryManager. Note that the
+   * caller should ensure entries[0].index > committed.
+   *
+   * @param appendingEntries appendingEntries
+   * @return the newly generated lastIndex
+   */
+  public long append(List<Entry> appendingEntries) {
+    if (entries.isEmpty()) {
+      return getLastLogIndex();
+    }
+
+    Entry firstAppendingEntry = appendingEntries.get(0);
+    Entry lastAppendingEntry = appendingEntries.get(appendingEntries.size() - 1);
+    if (getTerm(firstAppendingEntry.getCurrLogIndex()) == firstAppendingEntry.getCurrLogTerm()
+        && getTerm(lastAppendingEntry.getCurrLogIndex()) == lastAppendingEntry.getCurrLogTerm()) {
+      // skip existing entry
+      return getLastLogIndex();
+    }
+
+    long after = appendingEntries.get(0).getCurrLogIndex();
+    long len = after - getFirstIndex();
+    if (len < 0) {
+      // the logs are being truncated to before our current offset portion, which is committed
+      // entries
+      logger.error("The logs which first index is {} are going to truncate committed logs", after);
+    } else if (len == entries.size()) {
+      // after is the next index in the entries
+      // directly append
+      entries.addAll(appendingEntries);
+    } else {
+      // clear conflict entries
+      // then append
+      logger.info("truncate the entries after index {}", after);
+      int truncateIndex = (int) (after - getFirstIndex());
+      if (truncateIndex < entries.size()) {
+        entries.subList(truncateIndex, entries.size()).clear();
+      }
+      entries.addAll(appendingEntries);
+    }
+
+    Object logUpdateCondition =
+        getLogUpdateCondition(entries.get(entries.size() - 1).getCurrLogIndex());
+    synchronized (logUpdateCondition) {
+      logUpdateCondition.notifyAll();
+    }
+    return getLastLogIndex();
+  }
+
+  /**
+   * Used by leader node to try to commit entries.
+   *
+   * @param leaderCommit leader's commitIndex
+   * @param term the entry's term which index is leaderCommit in leader's log module
+   * @return true or false
+   */
+  public synchronized boolean maybeCommit(long leaderCommit, long term) {
+    if (leaderCommit > commitIndex && matchTerm(term, leaderCommit)) {
+      try {
+        commitTo(leaderCommit);
+      } catch (LogExecutionException e) {
+        // exceptions are ignored on follower side
+      }
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Overwrites the contents of this object with those of the given snapshot.
+   *
+   * @param snapshot leader's snapshot
+   */
+  public void applySnapshot(Snapshot snapshot) {
+    logger.info(
+        "{}: log module starts to restore snapshot [index: {}, term: {}]",
+        name,
+        snapshot.getLastLogIndex(),
+        snapshot.getLastLogTerm());
+    try {
+      lock.writeLock().lock();
+
+      long localIndex = commitIndex;
+      long snapIndex = snapshot.getLastLogIndex();
+      if (localIndex >= snapIndex) {
+        logger.info("requested snapshot is older than the existing snapshot");
+        return;
+      }
+
+      entries.subList(1, entries.size()).clear();
+      entries.set(0, new EmptyEntry(snapshot.getLastLogIndex(), snapshot.getLastLogTerm()));
+
+      this.commitIndex = snapshot.getLastLogIndex();
+
+      // as the follower receives a snapshot, the logs persisted is not complete, so remove them
+      if (config.isEnableRaftLogPersistence()) {
+        getStableEntryManager().clearAllLogs(commitIndex);
+      }
+
+      synchronized (changeApplyCommitIndexCond) {
+        this.appliedIndex = snapshot.getLastLogIndex();
+      }
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  /**
+   * Determines if the given (lastTerm, lastIndex) log is more up-to-date by comparing the index and
+   * term of the last entries in the existing logs. If the logs have last entries with different
+   * terms, then the log with the later term is more up-to-date. If the logs end with the same term,
+   * then whichever log has the larger lastIndex is more up-to-date. If the logs are the same, the
+   * given log is up-to-date.
+   *
+   * @param lastTerm candidate's lastTerm
+   * @param lastIndex candidate's lastIndex
+   * @return true or false
+   */
+  public boolean isLogUpToDate(long lastTerm, long lastIndex) {
+    return lastTerm > getLastLogTerm()
+        || (lastTerm == getLastLogTerm() && lastIndex >= getLastLogIndex());
+  }
+
+  /**
+   * Pack entries from low through high - 1, just like slice (entries[low:high]). firstIndex <= low
+   * <= high <= lastIndex.
+   *
+   * @param low request index low bound
+   * @param high request index upper bound
+   */
+  public List<Entry> getEntries(long low, long high) {
+    if (low >= high) {
+      return Collections.emptyList();
+    }
+    try {
+      lock.readLock().lock();
+      long localFirst = getFirstIndex();
+      long localLast = getLastLogIndex();
+      low = Math.max(low, localFirst);
+      high = Math.min(high, localLast);
+      return new ArrayList<>(entries.subList((int) (low - localFirst), (int) (high - localFirst)));
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  public Entry getEntryUnsafe(long index) {
+    return entries.get((int) (index - getFirstIndex()));
+  }
+
+  private long entrySize(long low, long hi) {
+    long entryMemory = 0;
+    for (Entry entry : getEntries(low, hi)) {
+      entryMemory += entry.estimateSize();
+    }
+    return entryMemory;
+  }
+
+  private int maxLogNumShouldReserve(long maxMemSize) {
+    long totalSize = 0;
+    for (int i = entries.size() - 1; i >= 1; i--) {
+      if (totalSize + entries.get(i).estimateSize() > maxMemSize) {
+        return entries.size() - 1 - i;
+      }
+      totalSize += entries.get(i).estimateSize();
+    }
+    return entries.size() - 1;
+  }
+
+  private void checkCompaction(List<Entry> entries) {
+    boolean needToCompactLog = false;
+    // calculate the number of old committed entries to be reserved by entry number
+    int numToReserveForNew = minNumOfLogsInMem;
+    if (entries.size() > maxNumOfLogsInMem) {
+      needToCompactLog = true;
+      numToReserveForNew = maxNumOfLogsInMem - entries.size();
+    }
+
+    // calculate the number of old committed entries to be reserved by entry size
+    long newEntryMemSize = 0;
+    for (Entry entry : entries) {
+      newEntryMemSize += entry.estimateSize();
+    }
+
+    int sizeToReserveForNew = minNumOfLogsInMem;
+    if (newEntryMemSize + committedEntrySize > maxLogMemSize) {
+      needToCompactLog = true;
+      sizeToReserveForNew = maxLogNumShouldReserve(maxLogMemSize - newEntryMemSize);
+    }
+
+    // reserve old committed entries with the minimum number
+    if (needToCompactLog) {
+      int numForNew = Math.min(numToReserveForNew, sizeToReserveForNew);
+      int sizeToReserveForConfig = minNumOfLogsInMem;
+      innerDeleteLog(Math.min(sizeToReserveForConfig, numForNew));
+    }
+  }
+
+  private void removedCommitted(List<Entry> entries) {
+    long commitLogIndex = getCommitLogIndex();
+    long firstLogIndex = entries.get(0).getCurrLogIndex();
+    if (commitLogIndex >= firstLogIndex) {
+      logger.warn(
+          "Committing logs that has already been committed: {} >= {}",
+          commitLogIndex,
+          firstLogIndex);
+      entries
+          .subList(0, (int) (getCommitLogIndex() - entries.get(0).getCurrLogIndex() + 1))
+          .clear();
+    }
+  }
+
+  private void commitEntries(List<Entry> entries) throws LogExecutionException {
+    try {
+      // Operations here are so simple that the execution could be thought
+      // success or fail together approximately.
+      Entry lastLog = entries.get(entries.size() - 1);
+      commitIndex = lastLog.getCurrLogIndex();
+
+      if (config.isEnableRaftLogPersistence()) {
+        // Cluster could continue provide service when exception is thrown here
+        getStableEntryManager().append(entries, appliedIndex);
+      }
+    } catch (IOException e) {
+      // The exception will block the raft service continue accept log.
+      // TODO: Notify user that the persisted logs before these entries(include) are corrupted.
+      // TODO: An idea is that we can degrade the service by disable raft log persistent for
+      // TODO: the group. It needs fine-grained control for the config of Raft log persistence.
+      logger.error("{}: persistent raft log error:", name, e);
+      throw new LogExecutionException(e);
+    }
+  }
+
+  /**
+   * Used by MaybeCommit or MaybeAppend or follower to commit newly committed entries.
+   *
+   * @param newCommitIndex request commitIndex
+   */
+  public void commitTo(long newCommitIndex) throws LogExecutionException {
+    if (commitIndex >= newCommitIndex) {
+      return;
+    }
+
+    try {
+      lock.writeLock().lock();
+      long lo = commitIndex + 1;
+      long hi = newCommitIndex + 1;
+      List<Entry> entries = new ArrayList<>(getEntries(lo, hi));
+
+      if (entries.isEmpty()) {
+        return;
+      }
+
+      removedCommitted(entries);
+      checkCompaction(entries);
+      commitEntries(entries);
+      applyEntries(entries);
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  /**
+   * Returns whether the index and term passed in match.
+   *
+   * @param term request entry term
+   * @param index request entry index
+   * @return true or false
+   */
+  public boolean matchTerm(long term, long index) {
+    long t;
+    try {
+      t = getTerm(index);
+    } catch (Exception e) {
+      return false;
+    }
+    return t == term;
+  }
+
+  /**
+   * Used by commitTo to apply newly committed entries
+   *
+   * @param entries applying entries
+   */
+  void applyEntries(List<Entry> entries) {
+    for (Entry entry : entries) {
+      applyEntry(entry);
+    }
+
+    long unappliedLogSize = getCommitLogIndex() - appliedIndex;
+    if (unappliedLogSize > config.getMaxNumOfLogsInMem()) {
+      logger.info(
+          "There are too many unapplied logs [{}], wait for a while to avoid memory overflow",
+          unappliedLogSize);
+      try {
+        synchronized (changeApplyCommitIndexCond) {
+          changeApplyCommitIndexCond.wait(
+              Math.min((unappliedLogSize - config.getMaxNumOfLogsInMem()) / 10 + 1, 1000));
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      }
+    }
+  }
+
+  public void applyEntry(Entry entry) {
+    // For add/remove logs in data groups, this log will be applied immediately when it is
+    // appended to the raft log.
+    // In this case, it will apply a log that has been applied.
+    if (entry.isApplied()) {
+      return;
+    }
+    try {
+      logApplier.apply(entry);
+    } catch (Exception e) {
+      entry.setException(e);
+      entry.setApplied(true);
+    }
+  }
+
+  /**
+   * findConflict finds the index of the conflict. It returns the first pair of conflicting entries
+   * between the existing entries and the given entries, if there are any. If there is no
+   * conflicting entries, and the existing entries contains all the given entries, -1 will be
+   * returned. If there is no conflicting entries, but the given entries contains new entries, the
+   * index of the first new entry will be returned. An entry is considered to be conflicting if it
+   * has the same index but a different term. The index of the given entries MUST be continuously
+   * increasing.
+   *
+   * @param entries request entries
+   * @return -1 or conflictIndex
+   */
+  long findConflict(List<Entry> entries) {
+    for (Entry entry : entries) {
+      if (!matchTerm(entry.getCurrLogTerm(), entry.getCurrLogIndex())) {
+        if (entry.getCurrLogIndex() <= getLastLogIndex()) {
+          logger.info("found conflict at index {}", entry.getCurrLogIndex());
+        }
+        return entry.getCurrLogIndex();
+      }
+    }
+    return -1;
+  }
+
+  public void close() {
+    getStableEntryManager().close();
+    if (deleteLogExecutorService != null) {
+      deleteLogExecutorService.shutdownNow();
+      if (deleteLogFuture != null) {
+        deleteLogFuture.cancel(true);
+      }
+
+      try {
+        deleteLogExecutorService.awaitTermination(20, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        logger.warn("Close delete log thread interrupted");
+      }
+      deleteLogExecutorService = null;
+    }
+
+    if (checkLogApplierExecutorService != null) {
+      checkLogApplierExecutorService.shutdownNow();
+      checkLogApplierFuture.cancel(true);
+      try {
+        checkLogApplierExecutorService.awaitTermination(20, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        logger.warn("Close check log applier thread interrupted");
+      }
+      checkLogApplierExecutorService = null;
+    }
+
+    if (logApplier != null) {
+      logApplier.close();
+    }
+  }
+
+  public StableEntryManager getStableEntryManager() {
+    return stableEntryManager;
+  }
+
+  private void setStableEntryManager(StableEntryManager stableEntryManager) {
+    this.stableEntryManager = stableEntryManager;
+  }
+
+  public long getAppliedIndex() {
+    return appliedIndex;
+  }
+
+  /** check whether delete the committed log */
+  void checkDeleteLog() {
+    try {
+      lock.writeLock().lock();
+      if (appliedIndex - getFirstIndex() <= minNumOfLogsInMem) {
+        return;
+      }
+      innerDeleteLog(minNumOfLogsInMem);
+    } catch (Exception e) {
+      logger.error("{}, error occurred when checking delete log", name, e);
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  private void innerDeleteLog(int sizeToReserve) {
+    long appliedLogNum = appliedIndex - getFirstIndex();
+    long removeSize = appliedLogNum - sizeToReserve;
+    if (removeSize <= 0) {
+      return;
+    }
+
+    long compactIndex = getFirstIndex() + removeSize;
+    logger.debug(
+        "{}: Before compaction index {}-{}, compactIndex {}, removeSize {}, committedLogSize "
+            + "{}, maxAppliedLog {}",
+        name,
+        getFirstIndex(),
+        getLastLogIndex(),
+        compactIndex,
+        removeSize,
+        commitIndex - getFirstIndex(),
+        appliedIndex);
+    compactEntries(compactIndex);
+    if (config.isEnableRaftLogPersistence()) {
+      getStableEntryManager().removeCompactedEntries(compactIndex);
+    }
+    logger.debug(
+        "{}: After compaction index {}-{}, committedLogSize {}",
+        name,
+        getFirstIndex(),
+        getLastLogIndex(),
+        commitIndex - getFirstIndex());
+  }
+
+  void compactEntries(long compactIndex) {
+    long firstIndex = getFirstIndex();
+    if (compactIndex < firstIndex) {
+      logger.info(
+          "entries before request index ({}) have been compacted, and the compactIndex is ({})",
+          firstIndex,
+          compactIndex);
+      return;
+    }
+    long lastLogIndex = getLastLogIndex();
+    if (compactIndex >= lastLogIndex) {
+      logger.info("compact ({}) is out of bound lastIndex ({})", compactIndex, lastLogIndex);
+      compactIndex = lastLogIndex - 1;
+    }
+    int index = (int) (compactIndex - firstIndex);
+    for (int i = 0; i < index; i++) {
+      committedEntrySize -= entries.get(0).estimateSize();
+    }
+    if (index > 0) {
+      entries.subList(0, index).clear();
+    }
+  }
+
+  public Object getLogUpdateCondition(long logIndex) {
+    return logUpdateConditions[(int) (logIndex % logUpdateConditions.length)];
+  }
+
+  void applyAllCommittedLogWhenStartUp() {
+    long lo = appliedIndex;
+    long hi = commitIndex + 1;
+    if (lo >= hi) {
+      logger.info(
+          "{}: the maxHaveAppliedCommitIndex={}, lastIndex={}, no need to reapply",
+          name,
+          appliedIndex,
+          hi);
+      return;
+    }
+
+    List<Entry> entries = new ArrayList<>(getEntries(lo, hi));
+    applyEntries(entries);
+  }
+
+  public void checkAppliedLogIndex() {
+    while (!Thread.interrupted()) {
+      try {
+        doCheckAppliedLogIndex();
+      } catch (IndexOutOfBoundsException e) {
+        // ignore
+      } catch (Exception e) {
+        logger.error("{}, an exception occurred when checking the applied log index", name, e);
+      }
+    }
+    logger.info(
+        "{}, the check-log-applier thread {} is interrupted",
+        name,
+        Thread.currentThread().getName());
+  }
+
+  void doCheckAppliedLogIndex() {
+    long nextToCheckIndex = appliedIndex + 1;
+    try {
+      if (nextToCheckIndex > commitIndex) {
+        // avoid spinning
+        Thread.sleep(100);
+        return;
+      }
+      Entry log = getEntryUnsafe(nextToCheckIndex);
+      if (log == null || log.getCurrLogIndex() != nextToCheckIndex) {
+        logger.debug(
+            "{}, get log error when checking the applied log index, log={}, nextToCheckIndex={}",
+            name,
+            log,
+            nextToCheckIndex);
+        return;
+      }
+      if (!log.isApplied() && appliedIndex < log.getCurrLogIndex()) {
+        synchronized (log) {
+          while (!log.isApplied() && appliedIndex < log.getCurrLogIndex()) {
+            // wait until the log is applied or a newer snapshot is installed
+            log.wait(10);
+          }
+        }
+      }
+      synchronized (changeApplyCommitIndexCond) {
+        // maxHaveAppliedCommitIndex may change if a snapshot is applied concurrently
+        appliedIndex = Math.max(appliedIndex, nextToCheckIndex);
+      }
+      logger.debug(
+          "{}: log={} is applied, nextToCheckIndex={}, commitIndex={}, maxHaveAppliedCommitIndex={}",
+          name,
+          log,
+          nextToCheckIndex,
+          commitIndex,
+          appliedIndex);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      logger.info("{}: do check applied log index is interrupt", name);
+    }
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public ReentrantReadWriteLock getLock() {
+    return lock;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/manager/SnapshotRaftLogManager.java
similarity index 50%
copy from consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
copy to consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/manager/SnapshotRaftLogManager.java
index daa2a7d7d0..32f71531a1 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/manager/SnapshotRaftLogManager.java
@@ -17,22 +17,29 @@
  * under the License.
  */
 
-package org.apache.iotdb.consensus.common.request;
+package org.apache.iotdb.consensus.natraft.protocol.log.manager;
 
-import java.nio.ByteBuffer;
+import org.apache.iotdb.consensus.IStateMachine;
+import org.apache.iotdb.consensus.natraft.protocol.log.applier.LogApplier;
+import org.apache.iotdb.consensus.natraft.protocol.log.serialization.StableEntryManager;
+import org.apache.iotdb.consensus.natraft.protocol.log.snapshot.Snapshot;
 
-public interface IConsensusRequest {
-  /**
-   * Serialize all the data to a ByteBuffer.
-   *
-   * <p>In a specific implementation, ByteBuf or PublicBAOS can be used to reduce the number of
-   * memory copies.
-   *
-   * <p>To improve efficiency, a specific implementation could return a DirectByteBuffer to reduce
-   * the memory copy required to send an RPC
-   *
-   * <p>Note: The implementation needs to ensure that the data in the returned Bytebuffer cannot be
-   * changed or an error may occur
-   */
-  ByteBuffer serializeToByteBuffer();
+public class SnapshotRaftLogManager extends RaftLogManager {
+
+  public SnapshotRaftLogManager(
+      StableEntryManager stableEntryManager,
+      LogApplier applier,
+      String name,
+      IStateMachine stateMachine) {
+    super(stableEntryManager, applier, name, stateMachine);
+  }
+
+  @Override
+  public Snapshot getSnapshot(long minLogIndex) {
+    // TODO-Raft: implement
+    return null;
+  }
+
+  @Override
+  public void takeSnapshot() {}
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/sequencing/LogSequencer.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/sequencing/LogSequencer.java
new file mode 100644
index 0000000000..328b74099a
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/sequencing/LogSequencer.java
@@ -0,0 +1,44 @@
+/*
+ * 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.natraft.protocol.log.sequencing;
+
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+import org.apache.iotdb.consensus.natraft.protocol.log.VotingLog;
+import org.apache.iotdb.consensus.natraft.protocol.log.manager.RaftLogManager;
+
+/**
+ * LogSequencer assigns a unique index and associated term to a log entry and offers the entry to a
+ * LogDispatcher which will send the entry to followers.
+ */
+public interface LogSequencer {
+
+  /**
+   * assigns a unique index and associated term to a log entry and offers the entry to a
+   * LogDispatcher which will send the entry to followers.
+   *
+   * @param e a log entry that is not yet indexed.
+   * @return A SendLogRequest through which the caller can monitor the status of the sending entry.
+   */
+  VotingLog sequence(Entry e);
+
+  void setLogManager(RaftLogManager logManager);
+
+  void close();
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/sequencing/LogSequencerFactory.java
similarity index 56%
copy from consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
copy to consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/sequencing/LogSequencerFactory.java
index daa2a7d7d0..62c73f6955 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/request/IConsensusRequest.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/sequencing/LogSequencerFactory.java
@@ -17,22 +17,13 @@
  * under the License.
  */
 
-package org.apache.iotdb.consensus.common.request;
+package org.apache.iotdb.consensus.natraft.protocol.log.sequencing;
 
-import java.nio.ByteBuffer;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.protocol.log.manager.RaftLogManager;
 
-public interface IConsensusRequest {
-  /**
-   * Serialize all the data to a ByteBuffer.
-   *
-   * <p>In a specific implementation, ByteBuf or PublicBAOS can be used to reduce the number of
-   * memory copies.
-   *
-   * <p>To improve efficiency, a specific implementation could return a DirectByteBuffer to reduce
-   * the memory copy required to send an RPC
-   *
-   * <p>Note: The implementation needs to ensure that the data in the returned Bytebuffer cannot be
-   * changed or an error may occur
-   */
-  ByteBuffer serializeToByteBuffer();
+public interface LogSequencerFactory {
+
+  LogSequencer create(RaftMember member, RaftLogManager logManager, RaftConfig config);
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/sequencing/SynchronousSequencer.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/sequencing/SynchronousSequencer.java
new file mode 100644
index 0000000000..617085a9ee
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/sequencing/SynchronousSequencer.java
@@ -0,0 +1,179 @@
+/*
+ * 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.natraft.protocol.log.sequencing;
+
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.RaftMember;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+import org.apache.iotdb.consensus.natraft.protocol.log.VotingLog;
+import org.apache.iotdb.consensus.natraft.protocol.log.manager.RaftLogManager;
+import org.apache.iotdb.consensus.raft.thrift.AppendEntryRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * SynchronizedSequencer performs sequencing by taking the monitor of a LogManager within the caller
+ * thread.
+ */
+public class SynchronousSequencer implements LogSequencer {
+
+  private static final Logger logger = LoggerFactory.getLogger(SynchronousSequencer.class);
+  private RaftMember member;
+  private RaftLogManager logManager;
+  private RaftConfig config;
+
+  public SynchronousSequencer(RaftMember member, RaftLogManager logManager, RaftConfig config) {
+    this.member = member;
+    this.logManager = logManager;
+    this.config = config;
+  }
+
+  private VotingLog enqueueEntry(VotingLog sendLogRequest) {
+
+    if (member.getAllNodes().size() > 1) {
+      member.getLogDispatcher().offer(sendLogRequest);
+    }
+    return sendLogRequest;
+  }
+
+  private static AtomicLong indexBlockCounter = new AtomicLong();
+
+  @Override
+  public VotingLog sequence(Entry e) {
+    VotingLog sendLogRequest = null;
+
+    long startWaitingTime = System.currentTimeMillis();
+
+    while (true) {
+      try {
+        logManager.getLock().writeLock().lock();
+        indexBlockCounter.decrementAndGet();
+        Entry lastEntry = logManager.getLastEntry();
+        long lastIndex = lastEntry.getCurrLogIndex();
+        long lastTerm = lastEntry.getCurrLogTerm();
+        if ((lastEntry.getCurrLogIndex() - logManager.getCommitLogIndex()
+            <= config.getUncommittedRaftLogNumForRejectThreshold())) {
+          // if the log contains a physical plan which is not a LogPlan, assign the same index to
+          // the plan so the state machine can be bridged with the consensus
+          e.setCurrLogTerm(member.getStatus().getTerm().get());
+          e.setCurrLogIndex(lastIndex + 1);
+          e.setPrevTerm(lastTerm);
+
+          // logDispatcher will serialize log, and set log size, and we will use the size after it
+          logManager.append(Collections.singletonList(e));
+
+          sendLogRequest = buildSendLogRequest(e);
+
+          if (!(config.isUseFollowerSlidingWindow() && config.isEnableWeakAcceptance())) {
+            sendLogRequest = enqueueEntry(sendLogRequest);
+          }
+          break;
+        }
+      } finally {
+        logManager.getLock().writeLock().unlock();
+      }
+
+      try {
+        TimeUnit.MILLISECONDS.sleep(config.getCheckPeriodWhenInsertBlocked());
+        if (System.currentTimeMillis() - startWaitingTime
+            > config.getMaxWaitingTimeWhenInsertBlocked()) {
+          return null;
+        }
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+      }
+    }
+
+    if (config.isUseFollowerSlidingWindow() && config.isEnableWeakAcceptance()) {
+      sendLogRequest = enqueueEntry(sendLogRequest);
+    }
+
+    return sendLogRequest;
+  }
+
+  @Override
+  public void setLogManager(RaftLogManager logManager) {
+    this.logManager = logManager;
+  }
+
+  private VotingLog buildSendLogRequest(Entry e) {
+    VotingLog votingLog = member.buildVotingLog(e);
+
+    AppendEntryRequest appendEntryRequest = buildAppendEntryRequest(e, false);
+    votingLog.setAppendEntryRequest(appendEntryRequest);
+
+    return votingLog;
+  }
+
+  public AppendEntryRequest buildAppendEntryRequest(Entry e, boolean serializeNow) {
+    AppendEntryRequest request = buildAppendEntryRequestBasic(e, serializeNow);
+    request = buildAppendEntryRequestExtended(request, e, serializeNow);
+    return request;
+  }
+
+  protected AppendEntryRequest buildAppendEntryRequestBasic(Entry entry, boolean serializeNow) {
+    AppendEntryRequest request = new AppendEntryRequest();
+    request.setTerm(member.getStatus().getTerm().get());
+    if (serializeNow) {
+      ByteBuffer byteBuffer = entry.serialize();
+      entry.setByteSize(byteBuffer.array().length);
+      request.entry = byteBuffer;
+    }
+    try {
+      if (entry.getPrevTerm() != -1) {
+        request.setPrevLogTerm(entry.getPrevTerm());
+      } else {
+        request.setPrevLogTerm(logManager.getTerm(entry.getCurrLogIndex() - 1));
+      }
+    } catch (Exception e) {
+      logger.error("getTerm failed for newly append entries", e);
+    }
+    request.setLeader(member.getThisNode());
+    // don't need lock because even if it's larger than the commitIndex when appending this log to
+    // logManager, the follower can handle the larger commitIndex with no effect
+    request.setLeaderCommit(logManager.getCommitLogIndex());
+    request.setPrevLogIndex(entry.getCurrLogIndex() - 1);
+    request.setGroupId(member.getRaftGroupId().convertToTConsensusGroupId());
+
+    return request;
+  }
+
+  protected AppendEntryRequest buildAppendEntryRequestExtended(
+      AppendEntryRequest request, Entry e, boolean serializeNow) {
+    return request;
+  }
+
+  public static class Factory implements LogSequencerFactory {
+
+    @Override
+    public LogSequencer create(RaftMember member, RaftLogManager logManager, RaftConfig config) {
+      return new SynchronousSequencer(member, logManager, config);
+    }
+  }
+
+  @Override
+  public void close() {}
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/serialization/LogManagerMeta.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/serialization/LogManagerMeta.java
new file mode 100644
index 0000000000..17a55f9133
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/serialization/LogManagerMeta.java
@@ -0,0 +1,155 @@
+/*
+ * 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.natraft.protocol.log.serialization;
+
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+import java.nio.ByteBuffer;
+
+public class LogManagerMeta {
+
+  private long commitLogTerm = -1;
+  private long commitLogIndex = -1;
+  private long lastLogIndex = -1;
+  private long lastLogTerm = -1;
+  private long lastAppliedIndex = -1;
+  private long lastAppliedTerm = -1;
+
+  public static LogManagerMeta deserialize(ByteBuffer buffer) {
+    LogManagerMeta res = new LogManagerMeta();
+    res.commitLogTerm = ReadWriteIOUtils.readLong(buffer);
+    res.commitLogIndex = ReadWriteIOUtils.readLong(buffer);
+    res.lastLogIndex = ReadWriteIOUtils.readLong(buffer);
+    res.lastLogTerm = ReadWriteIOUtils.readLong(buffer);
+    res.lastAppliedIndex = ReadWriteIOUtils.readLong(buffer);
+    res.lastAppliedTerm = ReadWriteIOUtils.readLong(buffer);
+
+    return res;
+  }
+
+  public long getCommitLogIndex() {
+    return commitLogIndex;
+  }
+
+  void setCommitLogIndex(long commitLogIndex) {
+    this.commitLogIndex = commitLogIndex;
+  }
+
+  public ByteBuffer serialize() {
+    // 5 is the number of attributes in class LogManagerMeta
+    ByteBuffer byteBuffer = ByteBuffer.allocate(Long.BYTES * 5);
+    byteBuffer.putLong(commitLogTerm);
+    byteBuffer.putLong(commitLogIndex);
+    byteBuffer.putLong(lastLogIndex);
+    byteBuffer.putLong(lastLogTerm);
+    byteBuffer.putLong(lastAppliedIndex);
+    byteBuffer.putLong(lastAppliedTerm);
+
+    byteBuffer.flip();
+    return byteBuffer;
+  }
+
+  @Override
+  public String toString() {
+    return "LogManagerMeta{"
+        + " commitLogTerm="
+        + commitLogTerm
+        + ", commitLogIndex="
+        + commitLogIndex
+        + ", lastLogIndex="
+        + lastLogIndex
+        + ", lastLogTerm="
+        + lastLogTerm
+        + ", maxHaveAppliedCommitIndex="
+        + lastAppliedIndex
+        + "}";
+  }
+
+  public long getLastLogIndex() {
+    return lastLogIndex;
+  }
+
+  public void setLastLogIndex(long lastLogIndex) {
+    this.lastLogIndex = lastLogIndex;
+  }
+
+  public long getLastLogTerm() {
+    return lastLogTerm;
+  }
+
+  public void setLastLogTerm(long lastLogTerm) {
+    this.lastLogTerm = lastLogTerm;
+  }
+
+  public void setCommitLogTerm(long commitLogTerm) {
+    this.commitLogTerm = commitLogTerm;
+  }
+
+  public long getLastAppliedIndex() {
+    return lastAppliedIndex;
+  }
+
+  public void setLastAppliedIndex(long lastAppliedIndex) {
+    this.lastAppliedIndex = lastAppliedIndex;
+  }
+
+  public long getLastAppliedTerm() {
+    return lastAppliedTerm;
+  }
+
+  public void setLastAppliedTerm(long lastAppliedTerm) {
+    this.lastAppliedTerm = lastAppliedTerm;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (!(o instanceof LogManagerMeta)) {
+      return false;
+    }
+
+    LogManagerMeta that = (LogManagerMeta) o;
+
+    return new EqualsBuilder()
+        .append(commitLogIndex, that.commitLogIndex)
+        .append(lastLogIndex, that.lastLogIndex)
+        .append(lastLogTerm, that.lastLogTerm)
+        .append(commitLogTerm, that.commitLogTerm)
+        .append(lastAppliedIndex, that.lastAppliedIndex)
+        .append(lastAppliedTerm, that.lastAppliedTerm)
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+        .append(commitLogIndex)
+        .append(lastLogIndex)
+        .append(lastLogTerm)
+        .append(commitLogTerm)
+        .append(lastAppliedIndex)
+        .toHashCode();
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/serialization/StableEntryManager.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/serialization/StableEntryManager.java
new file mode 100644
index 0000000000..37ed1eb050
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/serialization/StableEntryManager.java
@@ -0,0 +1,62 @@
+/*
+ * 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.natraft.protocol.log.serialization;
+
+import org.apache.iotdb.consensus.natraft.protocol.HardState;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+
+import java.io.IOException;
+import java.util.List;
+
+public interface StableEntryManager {
+
+  List<Entry> getAllEntriesAfterAppliedIndex();
+
+  List<Entry> getAllEntriesAfterCommittedIndex();
+
+  void append(List<Entry> entries, long maxHaveAppliedCommitIndex) throws IOException;
+
+  void flushLogBuffer();
+
+  void forceFlushLogBuffer();
+
+  void removeCompactedEntries(long index);
+
+  void setHardStateAndFlush(HardState state);
+
+  HardState getHardState();
+
+  LogManagerMeta getMeta();
+
+  /**
+   * @param startIndex (inclusive) the log start index
+   * @param endIndex (inclusive) the log end index
+   * @return the raft log which index between [startIndex, endIndex] or empty if not found
+   */
+  List<Entry> getEntries(long startIndex, long endIndex);
+
+  void close();
+
+  /**
+   * clear all logs, this method mainly used for after a follower accept a snapshot, all the logs
+   * should be cleaned
+   */
+  void clearAllLogs(long commitIndex);
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/serialization/SyncLogDequeSerializer.java b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/serialization/SyncLogDequeSerializer.java
new file mode 100644
index 0000000000..68335c6c25
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/natraft/protocol/log/serialization/SyncLogDequeSerializer.java
@@ -0,0 +1,1556 @@
+/*
+ * 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.natraft.protocol.log.serialization;
+
+import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil;
+import org.apache.iotdb.commons.consensus.ConsensusGroupId;
+import org.apache.iotdb.commons.file.SystemFileFactory;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.consensus.natraft.exception.UnknownLogTypeException;
+import org.apache.iotdb.consensus.natraft.protocol.HardState;
+import org.apache.iotdb.consensus.natraft.protocol.RaftConfig;
+import org.apache.iotdb.consensus.natraft.protocol.log.Entry;
+import org.apache.iotdb.consensus.natraft.protocol.log.LogParser;
+import org.apache.iotdb.consensus.natraft.protocol.log.serialization.SyncLogDequeSerializer.VersionController.SimpleFileVersionController;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.concurrent.BasicThreadFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.file.Files;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static org.apache.iotdb.commons.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
+
+public class SyncLogDequeSerializer implements StableEntryManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(SyncLogDequeSerializer.class);
+  private static final String LOG_DATA_FILE_SUFFIX = "data";
+  private static final String LOG_INDEX_FILE_SUFFIX = "idx";
+
+  /** the log data files */
+  private List<File> logDataFileList;
+
+  /** the log index files */
+  private List<File> logIndexFileList;
+
+  private LogParser parser = LogParser.getINSTANCE();
+  private File metaFile;
+  private FileOutputStream currentLogDataOutputStream;
+  private FileOutputStream currentLogIndexOutputStream;
+  private LogManagerMeta meta;
+  private HardState state;
+
+  /** min version of available log */
+  private long minAvailableVersion = 0;
+
+  /** max version of available log */
+  private long maxAvailableVersion = Long.MAX_VALUE;
+
+  private String logDir;
+
+  private VersionController versionController;
+
+  private ByteBuffer logDataBuffer;
+  private ByteBuffer logIndexBuffer;
+
+  private long offsetOfTheCurrentLogDataOutputStream = 0;
+
+  private int maxNumberOfLogsPerFetchOnDisk;
+
+  private static final String LOG_META = "logMeta";
+  private static final String LOG_META_TMP = "logMeta.tmp";
+
+  /**
+   * file name pattern:
+   *
+   * <p>for log data file: ${startLogIndex}-${endLogIndex}-{version}-data
+   *
+   * <p>for log index file: ${startLogIndex}-${endLogIndex}-{version}-idx
+   */
+  private static final int FILE_NAME_PART_LENGTH = 4;
+
+  private int maxRaftLogIndexSizeInMemory;
+
+  private int maxRaftLogPersistDataSizePerFile;
+
+  private int maxNumberOfPersistRaftLogFiles;
+
+  private int maxPersistRaftLogNumberOnDisk;
+
+  private ScheduledExecutorService persistLogDeleteExecutorService;
+  private ScheduledFuture<?> persistLogDeleteLogFuture;
+
+  /**
+   * indicate the first raft log's index of {@link SyncLogDequeSerializer#logIndexOffsetList}, for
+   * example, if firstLogIndex=1000, then the offset of the log index 1000 equals
+   * logIndexOffsetList[0], the offset of the log index 1001 equals logIndexOffsetList[1], and so
+   * on.
+   */
+  private long firstLogIndex = 0;
+
+  /**
+   * the offset of the log's index, for example, the first value is the offset of index
+   * ${firstLogIndex}, the second value is the offset of index ${firstLogIndex+1}
+   */
+  private List<Long> logIndexOffsetList;
+
+  private static final int LOG_DELETE_CHECK_INTERVAL_SECOND = 5;
+
+  /** the lock uses when change the log data files or log index files */
+  private final Lock lock = new ReentrantLock();
+
+  private volatile boolean isClosed = false;
+  private RaftConfig config;
+
+  private void initCommonProperties() {
+    logDataBuffer = ByteBuffer.allocate(config.getRaftLogBufferSize());
+    logIndexBuffer = ByteBuffer.allocate(config.getRaftLogBufferSize());
+    maxNumberOfLogsPerFetchOnDisk = config.getMaxNumberOfLogsPerFetchOnDisk();
+    maxRaftLogIndexSizeInMemory = config.getMaxRaftLogIndexSizeInMemory();
+    maxNumberOfPersistRaftLogFiles = config.getMaxNumberOfPersistRaftLogFiles();
+    maxPersistRaftLogNumberOnDisk = config.getMaxPersistRaftLogNumberOnDisk();
+
+    this.logDataFileList = new ArrayList<>();
+    this.logIndexFileList = new ArrayList<>();
+    this.logIndexOffsetList = new ArrayList<>(maxRaftLogIndexSizeInMemory);
+    try {
+      versionController = new SimpleFileVersionController(logDir);
+    } catch (IOException e) {
+      logger.error("log serializer build version controller failed", e);
+    }
+    this.persistLogDeleteExecutorService =
+        new ScheduledThreadPoolExecutor(
+            1,
+            new BasicThreadFactory.Builder()
+                .namingPattern("persist-log-delete-" + logDir)
+                .daemon(true)
+                .build());
+
+    this.persistLogDeleteLogFuture =
+        ScheduledExecutorUtil.safelyScheduleAtFixedRate(
+            persistLogDeleteExecutorService,
+            this::checkDeletePersistRaftLog,
+            LOG_DELETE_CHECK_INTERVAL_SECOND,
+            LOG_DELETE_CHECK_INTERVAL_SECOND,
+            TimeUnit.SECONDS);
+  }
+
+  /**
+   * for log tools
+   *
+   * @param logPath log dir path
+   */
+  public SyncLogDequeSerializer(String logPath, RaftConfig config) {
+    this.config = config;
+    logDir = logPath + File.separator;
+    initCommonProperties();
+    initMetaAndLogFiles();
+  }
+
+  /**
+   * log in disk is [size of log1 | log1 buffer] [size of log2 | log2 buffer]
+   *
+   * <p>build serializer with node id
+   */
+  public SyncLogDequeSerializer(ConsensusGroupId groupId, RaftConfig config) {
+    this.config = config;
+    logDir = getLogDir(groupId);
+    initCommonProperties();
+    initMetaAndLogFiles();
+  }
+
+  public String getLogDir(ConsensusGroupId groupId) {
+    String systemDir = config.getStorageDir();
+    return systemDir + File.separator + groupId + File.separator + "raftLog" + File.separator;
+  }
+
+  @TestOnly
+  String getLogDir() {
+    return logDir;
+  }
+
+  @TestOnly
+  File getMetaFile() {
+    return metaFile;
+  }
+
+  /** for log tools */
+  @Override
+  public LogManagerMeta getMeta() {
+    return meta;
+  }
+
+  /** Recover all the logs in disk. This function will be called once this instance is created. */
+  @Override
+  public List<Entry> getAllEntriesAfterAppliedIndex() {
+    logger.debug(
+        "getAllEntriesBeforeAppliedIndex, maxHaveAppliedCommitIndex={}, commitLogIndex={}",
+        meta.getLastAppliedIndex(),
+        meta.getCommitLogIndex());
+    if (meta.getLastAppliedIndex() >= meta.getCommitLogIndex()) {
+      return Collections.emptyList();
+    }
+    return getEntries(meta.getLastAppliedIndex(), meta.getCommitLogIndex());
+  }
+
+  /**
+   * When raft log files flushed,meta would not be flushed synchronously.So data has flushed to disk
+   * is uncommitted for persistent LogManagerMeta(meta's info is stale).We need to recover these
+   * already persistent logs.
+   *
+   * <p>For example,commitIndex is 5 in persistent LogManagerMeta,But the log file has actually been
+   * flushed to 7,when we restart cluster,we need to recover 6 and 7.
+   *
+   * <p>Maybe,we can extract getAllEntriesAfterAppliedIndex and getAllEntriesAfterCommittedIndex
+   * into getAllEntriesByIndex,but now there are too many test cases using it.
+   */
+  @Override
+  public List<Entry> getAllEntriesAfterCommittedIndex() {
+    long lastIndex = firstLogIndex + logIndexOffsetList.size() - 1;
+    logger.debug(
+        "getAllEntriesAfterCommittedIndex, firstUnCommitIndex={}, lastIndexBeforeStart={}",
+        meta.getCommitLogIndex() + 1,
+        lastIndex);
+    if (meta.getCommitLogIndex() >= lastIndex) {
+      return Collections.emptyList();
+    }
+    return getEntries(meta.getCommitLogIndex() + 1, lastIndex);
+  }
+
+  @Override
+  public void append(List<Entry> entries, long maxHaveAppliedCommitIndex) throws IOException {
+    lock.lock();
+    try {
+      putLogs(entries);
+      Entry entry = entries.get(entries.size() - 1);
+      meta.setCommitLogIndex(entry.getCurrLogIndex());
+      meta.setCommitLogTerm(entry.getCurrLogTerm());
+      meta.setLastLogIndex(entry.getCurrLogIndex());
+      meta.setLastLogTerm(entry.getCurrLogTerm());
+      meta.setLastAppliedIndex(maxHaveAppliedCommitIndex);
+      logger.debug(
+          "maxHaveAppliedCommitIndex={}, commitLogIndex={},lastLogIndex={}",
+          maxHaveAppliedCommitIndex,
+          meta.getCommitLogIndex(),
+          meta.getLastLogIndex());
+    } catch (BufferOverflowException e) {
+      throw new IOException(
+          "Log cannot fit into buffer, please increase raft_log_buffer_size;"
+              + "otherwise, please increase the JVM memory",
+          e);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Put each log in entries to local buffer. If the buffer overflows, flush the buffer to the disk,
+   * and try to push the log again.
+   *
+   * @param entries logs to put to buffer
+   */
+  private void putLogs(List<Entry> entries) {
+    for (Entry log : entries) {
+      logDataBuffer.mark();
+      logIndexBuffer.mark();
+      ByteBuffer logData = log.serialize();
+      int size = logData.capacity() + Integer.BYTES;
+      try {
+        logDataBuffer.putInt(logData.capacity());
+        logDataBuffer.put(logData);
+        logIndexBuffer.putLong(offsetOfTheCurrentLogDataOutputStream);
+        logIndexOffsetList.add(offsetOfTheCurrentLogDataOutputStream);
+        offsetOfTheCurrentLogDataOutputStream += size;
+      } catch (BufferOverflowException e) {
+        logger.info("Raft log buffer overflow!");
+        logDataBuffer.reset();
+        logIndexBuffer.reset();
+        flushLogBuffer();
+        checkCloseCurrentFile(log.getCurrLogIndex() - 1);
+        logDataBuffer.putInt(logData.capacity());
+        logDataBuffer.put(logData);
+        logIndexBuffer.putLong(offsetOfTheCurrentLogDataOutputStream);
+        logIndexOffsetList.add(offsetOfTheCurrentLogDataOutputStream);
+        offsetOfTheCurrentLogDataOutputStream += size;
+      }
+    }
+  }
+
+  private void checkCloseCurrentFile(long commitIndex) {
+    if (offsetOfTheCurrentLogDataOutputStream > maxRaftLogPersistDataSizePerFile) {
+      try {
+        closeCurrentFile(commitIndex);
+        serializeMeta(meta);
+        createNewLogFile(logDir, commitIndex + 1);
+      } catch (IOException e) {
+        logger.error("check close current file failed", e);
+      }
+    }
+  }
+
+  private void closeCurrentFile(long commitIndex) throws IOException {
+    if (currentLogDataOutputStream != null) {
+      currentLogDataOutputStream.close();
+      logger.info("{}: Closed a log data file {}", this, getCurrentLogDataFile());
+      currentLogDataOutputStream = null;
+
+      File currentLogDataFile = getCurrentLogDataFile();
+      String newDataFileName =
+          currentLogDataFile
+              .getName()
+              .replaceAll(String.valueOf(Long.MAX_VALUE), String.valueOf(commitIndex));
+      File newCurrentLogDatFile =
+          SystemFileFactory.INSTANCE.getFile(
+              currentLogDataFile.getParent() + File.separator + newDataFileName);
+      if (!currentLogDataFile.renameTo(newCurrentLogDatFile)) {
+        logger.error(
+            "rename log data file={} to {} failed",
+            currentLogDataFile.getAbsoluteFile(),
+            newCurrentLogDatFile);
+      }
+      logDataFileList.set(logDataFileList.size() - 1, newCurrentLogDatFile);
+
+      logger.debug(
+          "rename data file={} to file={}",
+          currentLogDataFile.getAbsoluteFile(),
+          newCurrentLogDatFile.getAbsoluteFile());
+    }
+
+    if (currentLogIndexOutputStream != null) {
+      currentLogIndexOutputStream.close();
+      logger.info("{}: Closed a log index file {}", this, getCurrentLogIndexFile());
+      currentLogIndexOutputStream = null;
+
+      File currentLogIndexFile = getCurrentLogIndexFile();
+      String newIndexFileName =
+          currentLogIndexFile
+              .getName()
+              .replaceAll(String.valueOf(Long.MAX_VALUE), String.valueOf(commitIndex));
+      File newCurrentLogIndexFile =
+          SystemFileFactory.INSTANCE.getFile(
+              currentLogIndexFile.getParent() + File.separator + newIndexFileName);
+      if (!currentLogIndexFile.renameTo(newCurrentLogIndexFile)) {
+        logger.error("rename log index file={} failed", currentLogIndexFile.getAbsoluteFile());
+      }
+      logger.debug(
+          "rename index file={} to file={}",
+          currentLogIndexFile.getAbsoluteFile(),
+          newCurrentLogIndexFile.getAbsoluteFile());
+
+      logIndexFileList.set(logIndexFileList.size() - 1, newCurrentLogIndexFile);
+    }
+
+    offsetOfTheCurrentLogDataOutputStream = 0;
+  }
+
+  @Override
+  public void flushLogBuffer() {
+    if (isClosed || logDataBuffer.position() == 0) {
+      return;
+    }
+    lock.lock();
+    try {
+      // write into disk
+      try {
+        checkStream();
+        // 1. write to the log data file
+        ReadWriteIOUtils.writeWithoutSize(
+            logDataBuffer, 0, logDataBuffer.position(), currentLogDataOutputStream);
+        ReadWriteIOUtils.writeWithoutSize(
+            logIndexBuffer, 0, logIndexBuffer.position(), currentLogIndexOutputStream);
+        if (config.getFlushRaftLogThreshold() == 0) {
+          currentLogDataOutputStream.getChannel().force(true);
+          currentLogIndexOutputStream.getChannel().force(true);
+        }
+      } catch (IOException e) {
+        logger.error("Error in logs serialization: ", e);
+        return;
+      }
+      logDataBuffer.clear();
+      logIndexBuffer.clear();
+      logger.debug("End flushing log buffer.");
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  private void forceFlushLogBufferWithoutCloseFile() {
+    if (isClosed) {
+      return;
+    }
+    lock.lock();
+    flushLogBuffer();
+    serializeMeta(meta);
+    try {
+      if (currentLogDataOutputStream != null) {
+        currentLogDataOutputStream.getChannel().force(true);
+      }
+      if (currentLogIndexOutputStream != null) {
+        currentLogIndexOutputStream.getChannel().force(true);
+      }
+    } catch (ClosedByInterruptException e) {
+      // ignore
+    } catch (IOException e) {
... 1806 lines suppressed ...