You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by as...@apache.org on 2021/06/09 08:30:41 UTC

[ignite-3] branch main updated: IGNITE-13885 jraft fork - Fixes #167.

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

ascherbakov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 509802a  IGNITE-13885 jraft fork - Fixes #167.
509802a is described below

commit 509802aec06c1faaedf985730fa331e3aa0d001a
Author: Alexey Scherbakov <al...@gmail.com>
AuthorDate: Wed Jun 9 11:25:43 2021 +0300

    IGNITE-13885 jraft fork - Fixes #167.
    
    Signed-off-by: Alexey Scherbakov <al...@gmail.com>
---
 modules/bytecode/pom.xml                           |    2 +-
 .../client/ITMetaStorageServiceTest.java           |    4 +-
 .../ignite/network/internal/netty/NettyServer.java |    3 +
 modules/raft/pom.xml                               |   57 +-
 .../ignite/raft/jraft/core/ITCliServiceTest.java   |  558 +++
 .../apache/ignite/raft/jraft/core/ITNodeTest.java  | 3580 ++++++++++++++++++++
 .../apache/ignite/raft/server/CounterListener.java |   59 +-
 .../ignite/raft/server/CounterSnapshotFile.java    |   81 +
 .../apache/ignite/raft/server/GetValueCommand.java |    4 +-
 .../raft/server/ITJRaftCounterServerTest.java      |  565 +++
 ...verTest.java => ITSimpleCounterServerTest.java} |   54 +-
 .../ignite/raft/server/IncrementAndGetCommand.java |    8 +-
 ...stractTest.java => RaftServerAbstractTest.java} |   48 +-
 .../java/org/apache/ignite/internal/raft/Loza.java |    4 +-
 .../{ => internal}/raft/server/RaftServer.java     |   15 +-
 .../internal/raft/server/impl/JRaftServerImpl.java |  314 ++
 .../raft/server/impl/RaftServerImpl.java           |   14 +-
 .../org/apache/ignite/raft/jraft/CliService.java   |  175 +
 .../org/apache/ignite/raft/jraft/Closure.java}     |   16 +-
 .../org/apache/ignite/raft/jraft/FSMCaller.java    |  116 +
 .../org/apache/ignite/raft/jraft/Iterator.java     |   66 +
 .../ignite/raft/jraft/JRaftServiceFactory.java     |   71 +
 .../org/apache/ignite/raft/jraft/JRaftUtils.java   |  205 ++
 .../org/apache/ignite/raft/jraft/Lifecycle.java}   |   19 +-
 .../java/org/apache/ignite/raft/jraft/Node.java    |  301 ++
 .../org/apache/ignite/raft/jraft/NodeManager.java  |   99 +
 .../apache/ignite/raft/jraft/RaftGroupService.java |  278 ++
 .../apache/ignite/raft/jraft/ReadOnlyService.java  |   50 +
 .../apache/ignite/raft/jraft/ReplicatorGroup.java  |  210 ++
 .../org/apache/ignite/raft/jraft/StateMachine.java |  123 +
 .../java/org/apache/ignite/raft/jraft/Status.java  |  235 ++
 .../raft/jraft/ThreadPoolMetricsSignalHandler.java |   58 +
 .../ignite/raft/jraft/closure/CatchUpClosure.java  |   67 +
 .../ignite/raft/jraft/closure/ClosureQueue.java    |   65 +
 .../raft/jraft/closure/ClosureQueueImpl.java       |  147 +
 .../ignite/raft/jraft/closure/JoinableClosure.java |   57 +
 .../raft/jraft/closure/LoadSnapshotClosure.java}   |   17 +-
 .../raft/jraft/closure/ReadIndexClosure.java       |  101 +
 .../raft/jraft/closure/SaveSnapshotClosure.java}   |   31 +-
 .../raft/jraft/closure/SynchronizedClosure.java    |   77 +
 .../ignite/raft/jraft/closure/TaskClosure.java}    |   29 +-
 .../ignite/raft/jraft/conf/Configuration.java      |  321 ++
 .../ignite/raft/jraft/conf/ConfigurationEntry.java |  124 +
 .../raft/jraft/conf/ConfigurationManager.java      |  106 +
 .../apache/ignite/raft/jraft/core/BallotBox.java   |  281 ++
 .../ignite/raft/jraft/core/CliServiceImpl.java     |  702 ++++
 .../jraft/core/DefaultJRaftServiceFactory.java     |   60 +
 .../ignite/raft/jraft/core/ElectionPriority.java}  |   28 +-
 .../ignite/raft/jraft/core/FSMCallerImpl.java      |  720 ++++
 .../ignite/raft/jraft/core/IteratorImpl.java       |  160 +
 .../ignite/raft/jraft/core/IteratorWrapper.java    |   74 +
 .../apache/ignite/raft/jraft/core/NodeImpl.java    | 3515 +++++++++++++++++++
 .../apache/ignite/raft/jraft/core/NodeMetrics.java |  106 +
 .../raft/jraft/core/ReadOnlyServiceImpl.java       |  423 +++
 .../apache/ignite/raft/jraft/core/Replicator.java  | 1806 ++++++++++
 .../raft/jraft/core/ReplicatorGroupImpl.java       |  330 ++
 .../ignite/raft/jraft/core/ReplicatorType.java}    |   18 +-
 .../apache/ignite/raft/jraft/core/Scheduler.java   |   72 +
 .../org/apache/ignite/raft/jraft/core/State.java}  |   36 +-
 .../raft/jraft/core/StateMachineAdapter.java       |  104 +
 .../ignite/raft/jraft/core/TimerManager.java       |   65 +
 .../apache/ignite/raft/jraft/entity/Ballot.java    |  135 +
 .../apache/ignite/raft/jraft/entity/Checksum.java} |   31 +-
 .../ignite/raft/jraft/entity/EnumOutter.java       |  150 +
 .../raft/jraft/entity/LeaderChangeContext.java     |  113 +
 .../raft/jraft/entity/LocalFileMetaOutter.java     |   94 +
 .../raft/jraft/entity/LocalStorageOutter.java      |  112 +
 .../apache/ignite/raft/jraft/entity/LogEntry.java  |  255 ++
 .../org/apache/ignite/raft/jraft/entity/LogId.java |  121 +
 .../apache/ignite/raft/jraft/entity/NodeId.java    |  100 +
 .../apache/ignite/raft/jraft/entity/PeerId.java    |  288 ++
 .../ignite/raft/jraft/entity/RaftOutter.java       |  133 +
 .../ignite/raft/jraft/entity/ReadIndexState.java   |   63 +
 .../ignite/raft/jraft/entity/ReadIndexStatus.java  |   54 +
 .../org/apache/ignite/raft/jraft/entity/Task.java  |  169 +
 .../apache/ignite/raft/jraft/entity/UserLog.java   |   62 +
 .../jraft/entity/codec/AutoDetectDecoder.java}     |   40 +-
 .../entity/codec/DefaultLogEntryCodecFactory.java} |   39 +-
 .../jraft/entity/codec/LogEntryCodecFactory.java}  |   30 +-
 .../raft/jraft/entity/codec/LogEntryDecoder.java}  |   18 +-
 .../raft/jraft/entity/codec/LogEntryEncoder.java}  |   18 +-
 .../entity/codec/v1/LogEntryV1CodecFactory.java    |   55 +
 .../raft/jraft/entity/codec/v1/V1Decoder.java      |  148 +
 .../raft/jraft/entity/codec/v1/V1Encoder.java      |  175 +
 .../raft/jraft/error/InvokeTimeoutException.java}  |   37 +-
 .../ignite/raft/jraft/error/JRaftException.java}   |   36 +-
 .../jraft/error/LogEntryCorruptedException.java    |   51 +
 .../jraft/error/LogIndexOutOfBoundsException.java  |   50 +
 .../raft/jraft/error/LogNotFoundException.java}    |   33 +-
 .../error/MessageClassNotFoundException.java}      |   38 +-
 .../apache/ignite/raft/jraft/error/RaftError.java  |  269 ++
 .../ignite/raft/jraft/error/RaftException.java     |   80 +
 .../raft/jraft/error/RemotingException.java}       |   36 +-
 .../raft/jraft/error/RetryAgainException.java}     |   42 +-
 .../raft/jraft/option/BallotBoxOptions.java}       |   37 +-
 .../ignite/raft/jraft/option/BootstrapOptions.java |  133 +
 .../ignite/raft/jraft/option/CliOptions.java}      |   33 +-
 .../ignite/raft/jraft/option/CopyOptions.java}     |   42 +-
 .../ignite/raft/jraft/option/FSMCallerOptions.java |   96 +
 .../raft/jraft/option/LogManagerOptions.java       |  103 +
 .../raft/jraft/option/LogStorageOptions.java       |   47 +
 .../ignite/raft/jraft/option/NodeOptions.java      |  538 +++
 .../raft/jraft/option/RaftMetaStorageOptions.java} |   18 +-
 .../ignite/raft/jraft/option/RaftOptions.java      |  302 ++
 .../ignite/raft/jraft/option/ReadOnlyOption.java}  |   19 +-
 .../raft/jraft/option/ReadOnlyServiceOptions.java} |   46 +-
 .../raft/jraft/option/ReplicatorGroupOptions.java  |  120 +
 .../raft/jraft/option/ReplicatorOptions.java       |  224 ++
 .../ignite/raft/jraft/option/RpcOptions.java       |  136 +
 .../raft/jraft/option/SnapshotCopierOptions.java   |   76 +
 .../raft/jraft/option/SnapshotExecutorOptions.java |  103 +
 .../ignite/raft/jraft/rpc/CliClientService.java    |  147 +
 .../apache/ignite/raft/jraft/rpc/CliRequests.java  |  461 +++
 .../ignite/raft/jraft/rpc/ClientService.java       |   47 +
 .../ignite/raft/jraft/rpc/InvokeCallback.java}     |   15 +-
 .../ignite/raft/jraft/rpc/InvokeContext.java}      |   41 +-
 .../org/apache/ignite/raft/jraft/rpc/Message.java} |   29 +-
 .../raft/jraft/rpc/MessageBuilderFactory.java      |  102 +
 .../ignite/raft/jraft/rpc/RaftClientService.java   |  106 +
 .../ignite/raft/jraft/rpc/RaftRpcFactory.java}     |   13 +-
 .../ignite/raft/jraft/rpc/RaftServerService.java   |   81 +
 .../apache/ignite/raft/jraft/rpc/RpcClient.java    |   95 +
 .../apache/ignite/raft/jraft/rpc/RpcClientEx.java} |   21 +-
 .../apache/ignite/raft/jraft/rpc/RpcContext.java}  |   38 +-
 .../apache/ignite/raft/jraft/rpc/RpcProcessor.java |   72 +
 .../ignite/raft/jraft/rpc/RpcRequestClosure.java   |   68 +
 .../ignite/raft/jraft/rpc/RpcRequestProcessor.java |   68 +
 .../apache/ignite/raft/jraft/rpc/RpcRequests.java  |  471 +++
 .../ignite/raft/jraft/rpc/RpcResponseClosure.java} |   21 +-
 .../raft/jraft/rpc/RpcResponseClosureAdapter.java} |   33 +-
 .../ignite/raft/jraft/rpc/RpcResponseFactory.java  |   76 +
 .../apache/ignite/raft/jraft/rpc/RpcServer.java}   |   33 +-
 .../raft/jraft/rpc/impl/AbstractClientService.java |  276 ++
 .../rpc/impl/ConnectionClosedEventListener.java}   |   12 +-
 .../ignite/raft/jraft/rpc/impl/FutureImpl.java}    |   13 +-
 .../raft/jraft/rpc/impl/IgniteRpcClient.java       |  222 ++
 .../raft/jraft/rpc/impl/IgniteRpcServer.java       |  218 ++
 .../raft/jraft/rpc/impl/PingRequestProcessor.java  |   55 +
 .../rpc/impl/cli/AddLearnersRequestProcessor.java  |   97 +
 .../rpc/impl/cli/AddPeerRequestProcessor.java      |   91 +
 .../rpc/impl/cli/BaseCliRequestProcessor.java      |  145 +
 .../rpc/impl/cli/ChangePeersRequestProcessor.java  |   89 +
 .../jraft/rpc/impl/cli/CliClientServiceImpl.java   |  126 +
 .../rpc/impl/cli/GetLeaderRequestProcessor.java    |  100 +
 .../rpc/impl/cli/GetPeersRequestProcessor.java     |   73 +
 .../impl/cli/RemoveLearnersRequestProcessor.java   |   92 +
 .../rpc/impl/cli/RemovePeerRequestProcessor.java   |   85 +
 .../impl/cli/ResetLearnersRequestProcessor.java    |   93 +
 .../rpc/impl/cli/ResetPeerRequestProcessor.java    |   75 +
 .../rpc/impl/cli/SnapshotRequestProcessor.java     |   57 +
 .../impl/cli/TransferLeaderRequestProcessor.java   |   68 +
 .../rpc/impl/client/ActionRequestProcessor.java    |  207 ++
 .../rpc/impl/client/GetLeaderRequestProcessor.java |   74 +
 .../rpc/impl/client/SnapshotRequestProcessor.java  |   75 +
 .../impl/core/AppendEntriesRequestProcessor.java   |  468 +++
 .../rpc/impl/core/DefaultRaftClientService.java    |  154 +
 .../rpc/impl/core/GetFileRequestProcessor.java     |   45 +
 .../impl/core/InstallSnapshotRequestProcessor.java |   57 +
 .../jraft/rpc/impl/core/NodeRequestProcessor.java  |   68 +
 .../rpc/impl/core/ReadIndexRequestProcessor.java   |   72 +
 .../rpc/impl/core/RequestVoteRequestProcessor.java |   60 +
 .../rpc/impl/core/TimeoutNowRequestProcessor.java  |   57 +
 .../jraft/rpc/message/AddLearnersRequestImpl.java  |   69 +
 .../raft/jraft/rpc/message/AddPeerRequestImpl.java |   59 +
 .../jraft/rpc/message/AddPeerResponseImpl.java     |   66 +
 .../rpc/message/AppendEntriesRequestImpl.java      |  186 +
 .../rpc/message/AppendEntriesResponseImpl.java     |   59 +
 .../jraft/rpc/message/ChangePeerRequestImpl.java   |   69 +
 .../jraft/rpc/message/ChangePeersResponseImpl.java |   66 +
 .../rpc/message/DefaultMessageBuilderFactory.java  |  174 +
 .../raft/jraft/rpc/message/EntryMetaImpl.java      |  186 +
 .../raft/jraft/rpc/message/ErrorResponseImpl.java} |   40 +-
 .../raft/jraft/rpc/message/GetFileRequestImpl.java |  109 +
 .../jraft/rpc/message/GetFileResponseImpl.java     |   60 +
 .../jraft/rpc/message/GetLeaderRequestImpl.java}   |   44 +-
 .../jraft/rpc/message/GetLeaderResponseImpl.java}  |   29 +-
 .../jraft/rpc/message/GetPeersRequestImpl.java     |   59 +
 .../jraft/rpc/message/GetPeersResponseImpl.java    |   66 +
 .../rpc/message/InstallSnapshotRequestImpl.java    |  124 +
 .../rpc/message/InstallSnapshotResponseImpl.java}  |   40 +-
 .../jraft/rpc/message/LearnersOpResponseImpl.java  |   66 +
 .../raft/jraft/rpc/message/LocalFileMetaImpl.java  |   83 +
 .../rpc/message/LocalSnapshotMetaFileImpl.java     |   49 +
 .../jraft/rpc/message/LocalSnapshotMetaImpl.java   |   68 +
 .../raft/jraft/rpc/message/PingRequestImpl.java}   |   29 +-
 .../raft/jraft/rpc/message/PreVoteRequestImpl.java |  103 +
 .../jraft/rpc/message/ReadIndexRequestImpl.java    |   92 +
 .../jraft/rpc/message/ReadIndexResponseImpl.java}  |   40 +-
 .../rpc/message/RemoveLearnersRequestImpl.java     |   69 +
 .../jraft/rpc/message/RemovePeerRequestImpl.java   |   59 +
 .../jraft/rpc/message/RemovePeerResponseImpl.java  |   66 +
 .../rpc/message/RequestVoteResponseImpl.java}      |   40 +-
 .../rpc/message/ResetLearnersRequestImpl.java      |   69 +
 .../jraft/rpc/message/ResetPeerRequestImpl.java    |   82 +
 .../raft/jraft/rpc/message/SnapshotMetaImpl.java   |  157 +
 .../jraft/rpc/message/SnapshotRequestImpl.java}    |   40 +-
 .../ignite/raft/jraft/rpc/message/StableMeta.java} |   40 +-
 .../jraft/rpc/message/TimeoutNowRequestImpl.java   |   95 +
 .../jraft/rpc/message/TimeoutNowResponseImpl.java} |   40 +-
 .../rpc/message/TransferLeaderRequestImpl.java     |   63 +
 .../ignite/raft/jraft/storage/FileService.java     |  150 +
 .../ignite/raft/jraft/storage/LogManager.java      |  224 ++
 .../ignite/raft/jraft/storage/LogStorage.java      |   77 +
 .../raft/jraft/storage/RaftMetaStorage.java}       |   40 +-
 .../raft/jraft/storage/SnapshotExecutor.java       |   84 +
 .../ignite/raft/jraft/storage/SnapshotStorage.java |   68 +
 .../raft/jraft/storage/SnapshotThrottle.java}      |   17 +-
 .../apache/ignite/raft/jraft/storage/Storage.java} |   14 +-
 .../raft/jraft/storage/impl/LocalLogStorage.java   |  278 ++
 .../jraft/storage/impl/LocalRaftMetaStorage.java   |  194 ++
 .../raft/jraft/storage/impl/LogManagerImpl.java    | 1227 +++++++
 .../raft/jraft/storage/impl/RocksDBLogStorage.java |  768 +++++
 .../ignite/raft/jraft/storage/io/FileReader.java   |   52 +
 .../raft/jraft/storage/io/LocalDirReader.java      |   94 +
 .../ignite/raft/jraft/storage/io/MessageFile.java  |   99 +
 .../raft/jraft/storage/snapshot/Snapshot.java      |   53 +
 .../jraft/storage/snapshot/SnapshotCopier.java}    |   35 +-
 .../storage/snapshot/SnapshotExecutorImpl.java     |  736 ++++
 .../jraft/storage/snapshot/SnapshotReader.java}    |   27 +-
 .../jraft/storage/snapshot/SnapshotWriter.java     |   72 +
 .../snapshot/ThroughputSnapshotThrottle.java       |   82 +
 .../storage/snapshot/local/LocalSnapshot.java      |   58 +
 .../snapshot/local/LocalSnapshotCopier.java        |  472 +++
 .../snapshot/local/LocalSnapshotMetaTable.java     |  180 +
 .../snapshot/local/LocalSnapshotReader.java        |  162 +
 .../snapshot/local/LocalSnapshotStorage.java       |  347 ++
 .../snapshot/local/LocalSnapshotWriter.java        |  133 +
 .../storage/snapshot/local/SnapshotFileReader.java |   87 +
 .../jraft/storage/snapshot/remote/CopySession.java |  298 ++
 .../storage/snapshot/remote/RemoteFileCopier.java  |  194 ++
 .../jraft/storage/snapshot/remote/Session.java}    |   31 +-
 .../raft/jraft/util/AdaptiveBufAllocator.java      |  206 ++
 .../apache/ignite/raft/jraft/util/ArrayDeque.java  |  103 +
 .../ignite/raft/jraft/util/AsciiStringUtil.java    |   60 +
 .../org/apache/ignite/raft/jraft/util/Bits.java    |   49 +
 .../raft/jraft/util/ByteBufferCollector.java       |  148 +
 .../apache/ignite/raft/jraft/util/ByteString.java  |  103 +
 .../org/apache/ignite/raft/jraft/util/Bytes.java   |  140 +
 .../apache/ignite/raft/jraft/util/BytesUtil.java   |  179 +
 .../org/apache/ignite/raft/jraft/util/CRC64.java   |  127 +
 .../apache/ignite/raft/jraft/util/Copiable.java}   |   19 +-
 .../ignite/raft/jraft/util/CountDownEvent.java     |   74 +
 .../org/apache/ignite/raft/jraft/util/CrcUtil.java |   82 +
 .../ignite/raft/jraft/util/DebugStatistics.java}   |   19 +-
 .../apache/ignite/raft/jraft/util/Describer.java   |   67 +
 .../ignite/raft/jraft/util/DirectExecutor.java}    |   20 +-
 .../ignite/raft/jraft/util/DisruptorBuilder.java   |   95 +
 .../ignite/raft/jraft/util/DisruptorMetricSet.java |   50 +
 .../apache/ignite/raft/jraft/util/Endpoint.java    |   94 +
 .../raft/jraft/util/ExecutorServiceHelper.java     |   70 +
 .../raft/jraft/util/FileOutputSignalHandler.java   |   50 +
 .../ignite/raft/jraft/util/HeapByteBufUtil.java    |  133 +
 .../org/apache/ignite/raft/jraft/util/Ints.java    |   82 +
 .../ignite/raft/jraft/util/JDKMarshaller.java      |   57 +
 .../raft/jraft/util/JRaftSignalHandler.java}       |   11 +-
 .../raft/jraft/util/LogExceptionHandler.java       |   65 +
 .../jraft/util/LogScheduledThreadPoolExecutor.java |   92 +
 .../raft/jraft/util/LogThreadPoolExecutor.java     |   97 +
 .../apache/ignite/raft/jraft/util/Marshaller.java} |    9 +-
 .../ignite/raft/jraft/util/MetricReporter.java     |  435 +++
 .../util/MetricScheduledThreadPoolExecutor.java    |   72 +
 .../raft/jraft/util/MetricThreadPoolExecutor.java  |   79 +
 .../org/apache/ignite/raft/jraft/util/Mpsc.java}   |   32 +-
 .../ignite/raft/jraft/util/NamedThreadFactory.java |   62 +
 .../ignite/raft/jraft/util/NonReentrantLock.java   |   92 +
 .../ignite/raft/jraft/util/OnlyForTest.java}       |   19 +-
 .../apache/ignite/raft/jraft/util/Platform.java    |   67 +
 .../apache/ignite/raft/jraft/util/Recyclable.java} |   14 +-
 .../raft/jraft/util/RecyclableByteBufferList.java  |  149 +
 .../ignite/raft/jraft/util/RecycleUtil.java}       |   17 +-
 .../apache/ignite/raft/jraft/util/Recyclers.java   |  417 +++
 .../ignite/raft/jraft/util/RepeatedTimer.java      |  296 ++
 .../apache/ignite/raft/jraft/util/Requires.java    |   98 +
 .../apache/ignite/raft/jraft/util/SegmentList.java |  371 ++
 .../raft/jraft/util/StorageOptionsFactory.java     |  342 ++
 .../apache/ignite/raft/jraft/util/StringUtils.java |  361 ++
 .../ignite/raft/jraft/util/SystemPropertyUtil.java |  178 +
 .../ignite/raft/jraft/util/ThreadHelper.java}      |   34 +-
 .../apache/ignite/raft/jraft/util/ThreadId.java    |  173 +
 .../raft/jraft/util/ThreadPoolMetricRegistry.java} |   28 +-
 .../raft/jraft/util/ThreadPoolMetricSet.java       |   52 +
 .../ignite/raft/jraft/util/ThreadPoolUtil.java     |  263 ++
 .../org/apache/ignite/raft/jraft/util/Utils.java   |  540 +++
 .../jraft/util/concurrent/AdjustableSemaphore.java |  126 +
 .../jraft/util/concurrent/ConcurrentHashSet.java   |   93 +
 .../concurrent/DefaultExecutorChooserFactory.java  |   80 +
 .../DefaultFixedThreadsExecutorGroup.java          |  121 +
 .../DefaultFixedThreadsExecutorGroupFactory.java   |   79 +
 .../concurrent/DefaultSingleThreadExecutor.java    |  115 +
 .../util/concurrent/ExecutorChooserFactory.java}   |   35 +-
 .../util/concurrent/FixedThreadsExecutorGroup.java |   63 +
 .../FixedThreadsExecutorGroupFactory.java          |   42 +
 .../concurrent/LongHeldDetectingReadWriteLock.java |  149 +
 .../util/concurrent/MpscSingleThreadExecutor.java  |  404 +++
 .../util/concurrent/RejectedExecutionHandler.java} |   29 +-
 .../concurrent/RejectedExecutionHandlers.java}     |   29 +-
 .../util/concurrent/SingleThreadExecutor.java}     |   32 +-
 .../jraft/util/internal/IntegerFieldUpdater.java}  |   11 +-
 .../jraft/util/internal/LongFieldUpdater.java}     |   11 +-
 .../util/internal/ReferenceFieldUpdater.java}      |   11 +-
 .../internal/ReflectionIntegerFieldUpdater.java    |   52 +
 .../util/internal/ReflectionLongFieldUpdater.java  |   52 +
 .../internal/ReflectionReferenceFieldUpdater.java  |   51 +
 .../ignite/raft/jraft/util/internal/ThrowUtil.java |   64 +
 .../raft/jraft/util/internal/UnsafeUtil.java       |  618 ++++
 .../ignite/raft/jraft/util/internal/Updaters.java  |   88 +
 .../jraft/util/timer/DefaultRaftTimerFactory.java  |   57 +
 .../ignite/raft/jraft/util/timer/DefaultTimer.java |  121 +
 .../raft/jraft/util/timer/HashedWheelTimer.java    |  736 ++++
 .../raft/jraft/util/timer/RaftTimerFactory.java}   |   32 +-
 .../ignite/raft/jraft/util/timer/Timeout.java      |   51 +
 .../apache/ignite/raft/jraft/util/timer/Timer.java |   47 +
 .../ignite/raft/jraft/util/timer/TimerTask.java    |   29 +
 .../org/apache/ignite/raft/jraft/StatusTest.java   |   92 +
 .../raft/jraft/closure/ClosureQueueTest.java       |  115 +
 .../jraft/closure/SynchronizedClosureTest.java     |   78 +
 .../raft/jraft/conf/ConfigurationEntryTest.java    |   87 +
 .../raft/jraft/conf/ConfigurationManagerTest.java  |  109 +
 .../ignite/raft/jraft/conf/ConfigurationTest.java  |  171 +
 .../ignite/raft/jraft/core/BallotBoxTest.java      |  158 +
 .../ignite/raft/jraft/core/ExpectClosure.java      |   65 +
 .../ignite/raft/jraft/core/FSMCallerTest.java      |  293 ++
 .../ignite/raft/jraft/core/IteratorImplTest.java   |  144 +
 .../ignite/raft/jraft/core/IteratorTest.java       |  113 +
 .../ignite/raft/jraft/core/MockClosure.java}       |   16 +-
 .../ignite/raft/jraft/core/MockStateMachine.java   |  235 ++
 .../raft/jraft/core/ReadOnlyServiceTest.java       |  268 ++
 .../raft/jraft/core/ReplicatorGroupTest.java       |  290 ++
 .../ignite/raft/jraft/core/ReplicatorTest.java     |  820 +++++
 .../apache/ignite/raft/jraft/core/TestCluster.java |  460 +++
 .../raft/jraft/core/TestJRaftServiceFactory.java}  |   27 +-
 .../ignite/raft/jraft/entity/BallotTest.java       |   50 +
 .../ignite/raft/jraft/entity/LogEntryTest.java     |  124 +
 .../apache/ignite/raft/jraft/entity/LogIdTest.java |   51 +
 .../ignite/raft/jraft/entity/PeerIdTest.java       |  144 +
 .../entity/codec/BaseLogEntryCodecFactoryTest.java |  143 +
 .../jraft/entity/codec/LogEntryCodecPerfTest.java  |  120 +
 .../codec/v1/LogEntryV1CodecFactoryTest.java}      |   27 +-
 .../raft/jraft/rpc/AbstractClientServiceTest.java  |  242 ++
 .../ignite/raft/jraft/rpc/AbstractRpcTest.java     |  370 ++
 .../raft/jraft/rpc/AppendEntriesBenchmark.java     |  244 ++
 .../ignite/raft/jraft/rpc/IgniteRpcTest.java}      |   87 +-
 .../raft/jraft/rpc/RpcResponseFactoryTest.java     |   65 +
 .../ignite/raft/jraft/rpc/TestIgniteRpcServer.java |   80 +
 .../ignite/raft/jraft/rpc/impl/FutureTest.java     |  140 +
 .../jraft/rpc/impl/PingRequestProcessorTest.java}  |   33 +-
 .../impl/cli/AbstractCliRequestProcessorTest.java  |   93 +
 .../impl/cli/AddLearnersRequestProcessorTest.java  |   66 +
 .../rpc/impl/cli/AddPeerRequestProcessorTest.java  |   61 +
 .../rpc/impl/cli/BaseCliRequestProcessorTest.java  |  192 ++
 .../impl/cli/ChangePeersRequestProcessorTest.java  |   63 +
 .../rpc/impl/cli/GetPeersRequestProcessorTest.java |   53 +
 .../cli/RemoveLearnersRequestProcessorTest.java    |   62 +
 .../impl/cli/RemovePeerRequestProcessorTest.java   |   61 +
 .../cli/ResetLearnersRequestProcessorTest.java     |   66 +
 .../impl/cli/ResetPeersRequestProcessorTest.java   |   52 +
 .../rpc/impl/cli/SnapshotRequestProcessorTest.java |   49 +
 .../TransferLeadershipRequestProcessorTest.java    |   50 +
 .../core/AppendEntriesRequestProcessorTest.java    |  189 ++
 .../impl/core/BaseNodeRequestProcessorTest.java    |   86 +
 .../core/InstallSnapshotRequestProcessorTest.java  |   54 +
 .../rpc/impl/core/NodeRequestProcessorTest.java    |  123 +
 .../rpc/impl/core/PreVoteRequestProcessorTest.java |   53 +
 .../impl/core/ReadIndexRequestProcessorTest.java   |   50 +
 .../impl/core/RequestVoteRequestProcessorTest.java |   53 +
 .../impl/core/TimeoutNowRequestProcessorTest.java  |   50 +
 .../raft/jraft/storage/BaseStorageTest.java}       |   38 +-
 .../ignite/raft/jraft/storage/FileServiceTest.java |  164 +
 .../raft/jraft/storage/SnapshotExecutorTest.java   |  302 ++
 .../jraft/storage/impl/BaseLogStorageTest.java     |  242 ++
 .../jraft/storage/impl/LocalLogStorageTest.java}   |   13 +-
 .../storage/impl/LocalRaftMetaStorageTest.java     |   95 +
 .../raft/jraft/storage/impl/LogManagerTest.java    |  429 +++
 .../jraft/storage/impl/LogStorageBenchmark.java    |  142 +
 .../jraft/storage/impl/RocksDBLogManagerTest.java} |   12 +-
 .../jraft/storage/impl/RocksDBLogStorageTest.java} |   13 +-
 .../raft/jraft/storage/io/LocalFileReaderTest.java |  105 +
 .../raft/jraft/storage/io/MessageFileTest.java     |   49 +
 .../snapshot/ThroughputSnapshotThrottleTest.java   |   43 +
 .../snapshot/local/LocalSnapshotCopierTest.java    |  209 ++
 .../snapshot/local/LocalSnapshotMetaTableTest.java |  116 +
 .../snapshot/local/LocalSnapshotReaderTest.java    |   83 +
 .../snapshot/local/LocalSnapshotStorageTest.java   |   92 +
 .../snapshot/local/LocalSnapshotWriterTest.java    |   90 +
 .../snapshot/local/SnapshotFileReaderTest.java     |   90 +
 .../storage/snapshot/remote/CopySessionTest.java   |  181 +
 .../snapshot/remote/RemoteFileCopierTest.java      |   66 +
 .../ignite/raft/jraft/test/MockAsyncContext.java   |   57 +
 .../apache/ignite/raft/jraft/test/TestUtils.java   |  195 ++
 .../raft/jraft/util/AdaptiveBufAllocatorTest.java  |   75 +
 .../ignite/raft/jraft/util/ArrayDequeTest.java     |   73 +
 .../raft/jraft/util/AsciiCodecBenchmark.java       |   92 +
 .../raft/jraft/util/AsciiStringUtilTest.java}      |   34 +-
 .../apache/ignite/raft/jraft/util/BitsTest.java}   |   33 +-
 .../raft/jraft/util/ByteBufferCollectorTest.java   |   68 +
 .../ignite/raft/jraft/util/BytesUtilTest.java      |  124 +
 .../ignite/raft/jraft/util/CountDownEventTest.java |   83 +
 .../ignite/raft/jraft/util/CrcUtilTest.java}       |   36 +-
 .../ignite/raft/jraft/util/EndpointTest.java       |   47 +
 .../jraft/util/FileOutputSignalHandlerTest.java    |   61 +
 .../jraft/util/RecyclableByteBufferListTest.java   |   53 +
 .../ignite/raft/jraft/util/RecyclersTest.java      |  145 +
 .../ignite/raft/jraft/util/RepeatedTimerTest.java  |  135 +
 .../ignite/raft/jraft/util/SegmentListTest.java    |  299 ++
 .../ignite/raft/jraft/util/ThreadIdTest.java       |  107 +
 .../ignite/raft/jraft/util/Utf8CodecBenchmark.java |   98 +
 .../apache/ignite/raft/jraft/util/UtilsTest.java   |  202 ++
 .../util/concurrent/AdjustableSemaphoreTest.java   |   68 +
 .../LongHeldDetectingReadWriteLockTest.java        |  141 +
 .../concurrent/MpscSingleThreadExecutorTest.java   |  167 +
 .../concurrent/SingleThreadExecutorBenchmark.java  |  166 +
 .../ignite/distributed/ITDistributedTableTest.java |    4 +-
 parent/pom.xml                                     |   43 +-
 413 files changed, 57722 insertions(+), 1117 deletions(-)

diff --git a/modules/bytecode/pom.xml b/modules/bytecode/pom.xml
index a437a7d..cac4253 100644
--- a/modules/bytecode/pom.xml
+++ b/modules/bytecode/pom.xml
@@ -65,7 +65,7 @@
     <build>
         <plugins>
             <!--
-               Disable javadoc validataion for forked foreign module.
+               Disable javadoc validation for forked foreign module.
            -->
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
diff --git a/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ITMetaStorageServiceTest.java b/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ITMetaStorageServiceTest.java
index b8573d2..441e6b9 100644
--- a/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ITMetaStorageServiceTest.java
+++ b/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ITMetaStorageServiceTest.java
@@ -46,8 +46,8 @@ import org.apache.ignite.raft.client.Peer;
 import org.apache.ignite.raft.client.message.RaftClientMessagesFactory;
 import org.apache.ignite.raft.client.service.RaftGroupService;
 import org.apache.ignite.raft.client.service.impl.RaftGroupServiceImpl;
-import org.apache.ignite.raft.server.RaftServer;
-import org.apache.ignite.raft.server.impl.RaftServerImpl;
+import org.apache.ignite.internal.raft.server.RaftServer;
+import org.apache.ignite.internal.raft.server.impl.RaftServerImpl;
 import org.jetbrains.annotations.NotNull;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
diff --git a/modules/network/src/main/java/org/apache/ignite/network/internal/netty/NettyServer.java b/modules/network/src/main/java/org/apache/ignite/network/internal/netty/NettyServer.java
index d68a370..9993b74 100644
--- a/modules/network/src/main/java/org/apache/ignite/network/internal/netty/NettyServer.java
+++ b/modules/network/src/main/java/org/apache/ignite/network/internal/netty/NettyServer.java
@@ -180,6 +180,9 @@ public class NettyServer {
                  * the connection is refused.
                  */
                 .option(ChannelOption.SO_BACKLOG, 128)
+                .option(ChannelOption.SO_REUSEADDR, true)
+                .childOption(ChannelOption.SO_LINGER, 0)
+                .childOption(ChannelOption.TCP_NODELAY, true)
                 /*
                  * When the keepalive option is set for a TCP socket and no data has been exchanged across the socket
                  * in either direction for 2 hours (NOTE: the actual value is implementation dependent),
diff --git a/modules/raft/pom.xml b/modules/raft/pom.xml
index a83c2c8..4d69098 100644
--- a/modules/raft/pom.xml
+++ b/modules/raft/pom.xml
@@ -38,6 +38,26 @@
             <artifactId>ignite-raft-client</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>com.lmax</groupId>
+            <artifactId>disruptor</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.rocksdb</groupId>
+            <artifactId>rocksdbjni</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>io.dropwizard.metrics</groupId>
+            <artifactId>metrics-core</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.jctools</groupId>
+            <artifactId>jctools-core</artifactId>
+        </dependency>
+
         <!-- Test dependencies -->
         <dependency>
             <groupId>org.junit.jupiter</groupId>
@@ -46,17 +66,43 @@
         </dependency>
 
         <dependency>
+            <groupId>org.junit.vintage</groupId>
+            <artifactId>junit-vintage-engine</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.mockito</groupId>
             <artifactId>mockito-junit-jupiter</artifactId>
             <scope>test</scope>
         </dependency>
-        
+
         <dependency>
             <groupId>org.mockito</groupId>
             <artifactId>mockito-core</artifactId>
             <scope>test</scope>
         </dependency>
 
+        <!-- Logging in tests -->
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- benchmark -->
+        <dependency>
+            <groupId>org.openjdk.jmh</groupId>
+            <artifactId>jmh-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.openjdk.jmh</groupId>
+            <artifactId>jmh-generator-annprocess</artifactId>
+            <scope>test</scope>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-network</artifactId>
@@ -77,6 +123,15 @@
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-compiler-plugin</artifactId>
             </plugin>
+            <!-- Disable javadoc validataion for forked foreign module. -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-javadoc-plugin</artifactId>
+                <configuration>
+                    <!-- TODO asch disable validation only for jraft package IGNITE-14832 -->
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
 </project>
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ITCliServiceTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ITCliServiceTest.java
new file mode 100644
index 0000000..5a01039
--- /dev/null
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ITCliServiceTest.java
@@ -0,0 +1,558 @@
+/*
+ * 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.ignite.raft.jraft.core;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BooleanSupplier;
+import java.util.stream.Collectors;
+import org.apache.ignite.network.ClusterLocalConfiguration;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessageSerializationRegistryImpl;
+import org.apache.ignite.network.scalecube.ScaleCubeClusterServiceFactory;
+import org.apache.ignite.network.scalecube.TestScaleCubeClusterServiceFactory;
+import org.apache.ignite.network.serialization.MessageSerializationRegistry;
+import org.apache.ignite.raft.jraft.CliService;
+import org.apache.ignite.raft.jraft.JRaftUtils;
+import org.apache.ignite.raft.jraft.Node;
+import org.apache.ignite.raft.jraft.Status;
+import org.apache.ignite.raft.jraft.conf.Configuration;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.apache.ignite.raft.jraft.entity.Task;
+import org.apache.ignite.raft.jraft.option.CliOptions;
+import org.apache.ignite.raft.jraft.rpc.impl.IgniteRpcClient;
+import org.apache.ignite.raft.jraft.test.TestUtils;
+import org.apache.ignite.raft.jraft.util.Utils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Thread.sleep;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Jraft cli tests.
+ */
+public class ITCliServiceTest {
+    /**
+     * The logger.
+     */
+    static final Logger LOG = LoggerFactory.getLogger(ITCliServiceTest.class);
+
+    /**
+     * The registry.
+     */
+    private static final MessageSerializationRegistry SERIALIZATION_REGISTRY = new MessageSerializationRegistryImpl();
+
+    /**
+     * The message factory.
+     */
+    private final static ScaleCubeClusterServiceFactory factory = new TestScaleCubeClusterServiceFactory();
+
+    private String dataPath;
+
+    private TestCluster cluster;
+    private final String groupId = "CliServiceTest";
+
+    private CliService cliService;
+
+    private Configuration conf;
+
+    @Rule
+    public TestName testName = new TestName();
+
+    private static final int LEARNER_PORT_STEP = 100;
+
+    @Before
+    public void setup() throws Exception {
+        LOG.info(">>>>>>>>>>>>>>> Start test method: " + this.testName.getMethodName());
+        this.dataPath = TestUtils.mkTempDir();
+        new File(this.dataPath).mkdirs();
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        final LinkedHashSet<PeerId> learners = new LinkedHashSet<>();
+
+        // 2 learners
+        for (int i = 0; i < 2; i++) {
+            learners.add(new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + LEARNER_PORT_STEP + i));
+        }
+
+        this.cluster = new TestCluster(this.groupId, this.dataPath, peers, learners, 300);
+        for (final PeerId peer : peers) {
+            this.cluster.start(peer.getEndpoint());
+        }
+
+        for (final PeerId peer : learners) {
+            this.cluster.startLearner(peer);
+        }
+
+        this.cluster.waitLeader();
+
+        for (Node follower : cluster.getFollowers()) {
+            assertTrue(waitForCondition(() -> follower.getLeaderId() != null, 3_000));
+        }
+
+        for (PeerId learner : cluster.getLearners()) {
+            Node node = cluster.getNode(learner.getEndpoint());
+
+            assertTrue(waitForCondition(() -> node.getLeaderId() != null, 3_000));
+        }
+
+        this.cliService = new CliServiceImpl();
+        this.conf = new Configuration(peers, learners);
+
+        CliOptions opts = new CliOptions();
+        opts.setClientExecutor(JRaftUtils.createClientExecutor(opts, "client"));
+
+        ClusterService clientSvc = factory.createClusterService(new ClusterLocalConfiguration("client",
+            TestUtils.INIT_PORT - 1, peers.stream().map(p -> p.getEndpoint().toString()).collect(Collectors.toList()),
+            SERIALIZATION_REGISTRY));
+
+        IgniteRpcClient rpcClient = new IgniteRpcClient(clientSvc, false);
+        opts.setRpcClient(rpcClient);
+        assertTrue(this.cliService.init(opts));
+    }
+
+    @After
+    public void teardown() throws Exception {
+        this.cliService.shutdown();
+        this.cluster.stopAll();
+        Utils.delete(new File(this.dataPath));
+        LOG.info(">>>>>>>>>>>>>>> End test method: " + this.testName.getMethodName());
+    }
+
+    @Test
+    public void testTransferLeader() throws Exception {
+        final PeerId leader = this.cluster.getLeader().getNodeId().getPeerId().copy();
+        assertNotNull(leader);
+
+        final Set<PeerId> peers = this.conf.getPeerSet();
+        PeerId targetPeer = null;
+        for (final PeerId peer : peers) {
+            if (!peer.equals(leader)) {
+                targetPeer = peer;
+                break;
+            }
+        }
+        assertNotNull(targetPeer);
+        assertTrue(this.cliService.transferLeader(this.groupId, this.conf, targetPeer).isOk());
+        this.cluster.waitLeader();
+        assertEquals(targetPeer, this.cluster.getLeader().getNodeId().getPeerId());
+    }
+
+    @SuppressWarnings("SameParameterValue")
+    private void sendTestTaskAndWait(final Node node, final int code) throws InterruptedException {
+        final CountDownLatch latch = new CountDownLatch(10);
+        for (int i = 0; i < 10; i++) {
+            final ByteBuffer data = ByteBuffer.wrap(("hello" + i).getBytes());
+            final Task task = new Task(data, new ExpectClosure(code, null, latch));
+            node.apply(task);
+        }
+        assertTrue(latch.await(10, TimeUnit.SECONDS));
+    }
+
+    @Test
+    public void testLearnerServices() throws Exception {
+        final PeerId learner3 = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + LEARNER_PORT_STEP + 3);
+        assertTrue(this.cluster.startLearner(learner3));
+        sendTestTaskAndWait(this.cluster.getLeader(), 0);
+        Thread.sleep(500);
+        for (final MockStateMachine fsm : this.cluster.getFsms()) {
+            if (!fsm.getAddress().equals(learner3.getEndpoint())) {
+                assertEquals(10, fsm.getLogs().size());
+            }
+        }
+        assertEquals(0, this.cluster.getFsmByPeer(learner3).getLogs().size());
+        List<PeerId> oldLearners = new ArrayList<PeerId>(this.conf.getLearners());
+        assertEquals(oldLearners, this.cliService.getLearners(this.groupId, this.conf));
+        assertEquals(oldLearners, this.cliService.getAliveLearners(this.groupId, this.conf));
+
+        // Add learner3
+        this.cliService.addLearners(this.groupId, this.conf, Arrays.asList(learner3));
+        Thread.sleep(1000);
+        assertEquals(10, this.cluster.getFsmByPeer(learner3).getLogs().size());
+
+        sendTestTaskAndWait(this.cluster.getLeader(), 0);
+        Thread.sleep(1000);
+        for (final MockStateMachine fsm : this.cluster.getFsms()) {
+            assertEquals(20, fsm.getLogs().size());
+
+        }
+        List<PeerId> newLearners = new ArrayList<>(oldLearners);
+        newLearners.add(learner3);
+        assertEquals(newLearners, this.cliService.getLearners(this.groupId, this.conf));
+        assertEquals(newLearners, this.cliService.getAliveLearners(this.groupId, this.conf));
+
+        // Remove  3
+        this.cliService.removeLearners(this.groupId, this.conf, Arrays.asList(learner3));
+        sendTestTaskAndWait(this.cluster.getLeader(), 0);
+        Thread.sleep(1000);
+        for (final MockStateMachine fsm : this.cluster.getFsms()) {
+            if (!fsm.getAddress().equals(learner3.getEndpoint())) {
+                assertEquals(30, fsm.getLogs().size());
+            }
+        }
+        // Latest 10 logs are not replicated to learner3, because it's removed.
+        assertEquals(20, this.cluster.getFsmByPeer(learner3).getLogs().size());
+        assertEquals(oldLearners, this.cliService.getLearners(this.groupId, this.conf));
+        assertEquals(oldLearners, this.cliService.getAliveLearners(this.groupId, this.conf));
+
+        // Set learners into [learner3]
+        this.cliService.resetLearners(this.groupId, this.conf, Arrays.asList(learner3));
+        Thread.sleep(100);
+        assertEquals(30, this.cluster.getFsmByPeer(learner3).getLogs().size());
+
+        sendTestTaskAndWait(this.cluster.getLeader(), 0);
+        Thread.sleep(1000);
+        // Latest 10 logs are not replicated to learner1 and learner2, because they were removed by resetting learners set.
+        for (final MockStateMachine fsm : this.cluster.getFsms()) {
+            if (!oldLearners.contains(new PeerId(fsm.getAddress(), 0))) {
+                assertEquals(40, fsm.getLogs().size());
+            }
+            else {
+                assertEquals(30, fsm.getLogs().size());
+            }
+        }
+        assertEquals(Arrays.asList(learner3), this.cliService.getLearners(this.groupId, this.conf));
+        assertEquals(Arrays.asList(learner3), this.cliService.getAliveLearners(this.groupId, this.conf));
+
+        // Stop learner3
+        this.cluster.stop(learner3.getEndpoint());
+        Thread.sleep(1000);
+        assertEquals(Arrays.asList(learner3), this.cliService.getLearners(this.groupId, this.conf));
+        assertTrue(this.cliService.getAliveLearners(this.groupId, this.conf).isEmpty());
+    }
+
+    @Test
+    public void testAddPeerRemovePeer() throws Exception {
+        final PeerId peer3 = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + 3);
+        assertTrue(this.cluster.start(peer3.getEndpoint()));
+        sendTestTaskAndWait(this.cluster.getLeader(), 0);
+        Thread.sleep(100);
+        assertEquals(0, this.cluster.getFsmByPeer(peer3).getLogs().size());
+
+        assertTrue(this.cliService.addPeer(this.groupId, this.conf, peer3).isOk());
+        Thread.sleep(100);
+        assertEquals(10, this.cluster.getFsmByPeer(peer3).getLogs().size());
+        sendTestTaskAndWait(this.cluster.getLeader(), 0);
+        Thread.sleep(100);
+        assertEquals(6, this.cluster.getFsms().size());
+        for (final MockStateMachine fsm : this.cluster.getFsms()) {
+            assertEquals(20, fsm.getLogs().size());
+        }
+
+        //remove peer3
+        assertTrue(this.cliService.removePeer(this.groupId, this.conf, peer3).isOk());
+        Thread.sleep(200);
+        sendTestTaskAndWait(this.cluster.getLeader(), 0);
+        Thread.sleep(1000);
+        assertEquals(6, this.cluster.getFsms().size());
+        for (final MockStateMachine fsm : this.cluster.getFsms()) {
+            if (fsm.getAddress().equals(peer3.getEndpoint())) {
+                assertEquals(20, fsm.getLogs().size());
+            }
+            else {
+                assertEquals(30, fsm.getLogs().size());
+            }
+        }
+    }
+
+    @Test
+    public void testChangePeers() throws Exception {
+        final List<PeerId> newPeers = TestUtils.generatePeers(10);
+        newPeers.removeAll(this.conf.getPeerSet());
+        for (final PeerId peer : newPeers) {
+            assertTrue(this.cluster.start(peer.getEndpoint()));
+        }
+        this.cluster.waitLeader();
+        final Node oldLeaderNode = this.cluster.getLeader();
+        assertNotNull(oldLeaderNode);
+        final PeerId oldLeader = oldLeaderNode.getNodeId().getPeerId();
+        assertNotNull(oldLeader);
+        assertTrue(this.cliService.changePeers(this.groupId, this.conf, new Configuration(newPeers)).isOk());
+        this.cluster.waitLeader();
+        final PeerId newLeader = this.cluster.getLeader().getNodeId().getPeerId();
+        assertNotEquals(oldLeader, newLeader);
+        assertTrue(newPeers.contains(newLeader));
+    }
+
+    @Test
+    public void testSnapshot() throws Exception {
+        sendTestTaskAndWait(this.cluster.getLeader(), 0);
+        assertEquals(5, this.cluster.getFsms().size());
+        for (final MockStateMachine fsm : this.cluster.getFsms()) {
+            assertEquals(0, fsm.getSaveSnapshotTimes());
+        }
+
+        for (final PeerId peer : this.conf) {
+            assertTrue(this.cliService.snapshot(this.groupId, peer).isOk());
+        }
+
+        for (final PeerId peer : this.conf.getLearners()) {
+            assertTrue(this.cliService.snapshot(this.groupId, peer).isOk());
+        }
+
+        for (final MockStateMachine fsm : this.cluster.getFsms()) {
+            assertEquals(1, fsm.getSaveSnapshotTimes());
+        }
+    }
+
+    @Test
+    public void testGetPeers() throws Exception {
+        PeerId leader = this.cluster.getLeader().getNodeId().getPeerId();
+        assertNotNull(leader);
+        assertArrayEquals(this.conf.getPeerSet().toArray(),
+            new HashSet<>(this.cliService.getPeers(this.groupId, this.conf)).toArray());
+
+        // stop one peer
+        final List<PeerId> peers = this.conf.getPeers();
+        this.cluster.stop(peers.get(0).getEndpoint());
+
+        this.cluster.waitLeader();
+
+        leader = this.cluster.getLeader().getNodeId().getPeerId();
+        assertNotNull(leader);
+        assertArrayEquals(this.conf.getPeerSet().toArray(),
+            new HashSet<>(this.cliService.getPeers(this.groupId, this.conf)).toArray());
+
+        this.cluster.stopAll();
+
+        try {
+            this.cliService.getPeers(this.groupId, this.conf);
+            fail();
+        }
+        catch (final IllegalStateException e) {
+            assertTrue(e.getMessage(), e.getMessage().startsWith("Fail to get leader of group " + this.groupId));
+        }
+    }
+
+    @Test
+    public void testGetAlivePeers() throws Exception {
+        PeerId leader = this.cluster.getLeader().getNodeId().getPeerId();
+        assertNotNull(leader);
+        assertArrayEquals(this.conf.getPeerSet().toArray(),
+            new HashSet<>(this.cliService.getAlivePeers(this.groupId, this.conf)).toArray());
+
+        // stop one peer
+        final List<PeerId> peers = this.conf.getPeers();
+        this.cluster.stop(peers.get(0).getEndpoint());
+        peers.remove(0);
+
+        this.cluster.waitLeader();
+
+        Thread.sleep(1000);
+
+        leader = this.cluster.getLeader().getNodeId().getPeerId();
+        assertNotNull(leader);
+        assertArrayEquals(new HashSet<>(peers).toArray(),
+            new HashSet<>(this.cliService.getAlivePeers(this.groupId, this.conf)).toArray());
+
+        this.cluster.stopAll();
+
+        try {
+            this.cliService.getAlivePeers(this.groupId, this.conf);
+            fail();
+        }
+        catch (final IllegalStateException e) {
+            assertTrue(e.getMessage(), e.getMessage().startsWith("Fail to get leader of group " + this.groupId));
+        }
+    }
+
+    @Test
+    public void testRebalance() {
+        final Set<String> groupIds = new TreeSet<>();
+        groupIds.add("group_1");
+        groupIds.add("group_2");
+        groupIds.add("group_3");
+        groupIds.add("group_4");
+        groupIds.add("group_5");
+        groupIds.add("group_6");
+        groupIds.add("group_7");
+        groupIds.add("group_8");
+        final Configuration conf = new Configuration();
+        conf.addPeer(new PeerId("host_1", 8080));
+        conf.addPeer(new PeerId("host_2", 8080));
+        conf.addPeer(new PeerId("host_3", 8080));
+
+        final Map<String, PeerId> rebalancedLeaderIds = new HashMap<>();
+
+        final CliService cliService = new MockCliService(rebalancedLeaderIds, new PeerId("host_1", 8080));
+
+        assertTrue(cliService.rebalance(groupIds, conf, rebalancedLeaderIds).isOk());
+        assertEquals(groupIds.size(), rebalancedLeaderIds.size());
+
+        final Map<PeerId, Integer> ret = new HashMap<>();
+        for (Map.Entry<String, PeerId> entry : rebalancedLeaderIds.entrySet()) {
+            ret.compute(entry.getValue(), (ignored, num) -> num == null ? 1 : num + 1);
+        }
+        final int expectedAvgLeaderNum = (int) Math.ceil((double) groupIds.size() / conf.size());
+        for (Map.Entry<PeerId, Integer> entry : ret.entrySet()) {
+            System.out.println(entry);
+            assertTrue(entry.getValue() <= expectedAvgLeaderNum);
+        }
+    }
+
+    @Test
+    public void testRebalanceOnLeaderFail() {
+        final Set<String> groupIds = new TreeSet<>();
+        groupIds.add("group_1");
+        groupIds.add("group_2");
+        groupIds.add("group_3");
+        groupIds.add("group_4");
+        final Configuration conf = new Configuration();
+        conf.addPeer(new PeerId("host_1", 8080));
+        conf.addPeer(new PeerId("host_2", 8080));
+        conf.addPeer(new PeerId("host_3", 8080));
+
+        final Map<String, PeerId> rebalancedLeaderIds = new HashMap<>();
+
+        final CliService cliService = new MockLeaderFailCliService();
+
+        assertEquals("Fail to get leader", cliService.rebalance(groupIds, conf, rebalancedLeaderIds).getErrorMsg());
+    }
+
+    @Test
+    public void testRelalanceOnTransferLeaderFail() {
+        final Set<String> groupIds = new TreeSet<>();
+        groupIds.add("group_1");
+        groupIds.add("group_2");
+        groupIds.add("group_3");
+        groupIds.add("group_4");
+        groupIds.add("group_5");
+        groupIds.add("group_6");
+        groupIds.add("group_7");
+        final Configuration conf = new Configuration();
+        conf.addPeer(new PeerId("host_1", 8080));
+        conf.addPeer(new PeerId("host_2", 8080));
+        conf.addPeer(new PeerId("host_3", 8080));
+
+        final Map<String, PeerId> rebalancedLeaderIds = new HashMap<>();
+
+        final CliService cliService = new MockTransferLeaderFailCliService(rebalancedLeaderIds,
+            new PeerId("host_1", 8080));
+
+        assertEquals("Fail to transfer leader",
+            cliService.rebalance(groupIds, conf, rebalancedLeaderIds).getErrorMsg());
+        assertTrue(groupIds.size() >= rebalancedLeaderIds.size());
+
+        final Map<PeerId, Integer> ret = new HashMap<>();
+        for (Map.Entry<String, PeerId> entry : rebalancedLeaderIds.entrySet()) {
+            ret.compute(entry.getValue(), (ignored, num) -> num == null ? 1 : num + 1);
+        }
+        for (Map.Entry<PeerId, Integer> entry : ret.entrySet()) {
+            System.out.println(entry);
+            assertEquals(new PeerId("host_1", 8080), entry.getKey());
+        }
+    }
+
+    static class MockCliService extends CliServiceImpl {
+        private final Map<String, PeerId> rebalancedLeaderIds;
+        private final PeerId initialLeaderId;
+
+        MockCliService(final Map<String, PeerId> rebalancedLeaderIds, final PeerId initialLeaderId) {
+            this.rebalancedLeaderIds = rebalancedLeaderIds;
+            this.initialLeaderId = initialLeaderId;
+        }
+
+        @Override
+        public Status getLeader(final String groupId, final Configuration conf, final PeerId leaderId) {
+            final PeerId ret = this.rebalancedLeaderIds.get(groupId);
+            if (ret != null) {
+                leaderId.parse(ret.toString());
+            }
+            else {
+                leaderId.parse(this.initialLeaderId.toString());
+            }
+            return Status.OK();
+        }
+
+        @Override
+        public List<PeerId> getAlivePeers(final String groupId, final Configuration conf) {
+            return conf.getPeers();
+        }
+
+        @Override
+        public Status transferLeader(final String groupId, final Configuration conf, final PeerId peer) {
+            return Status.OK();
+        }
+    }
+
+    static class MockLeaderFailCliService extends MockCliService {
+        MockLeaderFailCliService() {
+            super(null, null);
+        }
+
+        @Override
+        public Status getLeader(final String groupId, final Configuration conf, final PeerId leaderId) {
+            return new Status(-1, "Fail to get leader");
+        }
+    }
+
+    static class MockTransferLeaderFailCliService extends MockCliService {
+        MockTransferLeaderFailCliService(final Map<String, PeerId> rebalancedLeaderIds, final PeerId initialLeaderId) {
+            super(rebalancedLeaderIds, initialLeaderId);
+        }
+
+        @Override
+        public Status transferLeader(final String groupId, final Configuration conf, final PeerId peer) {
+            return new Status(-1, "Fail to transfer leader");
+        }
+    }
+
+    /**
+     * @param cond The condition.
+     * @param timeout The timeout.
+     * @return {@code True} if condition has happened within the timeout.
+     */
+    @SuppressWarnings("BusyWait") protected boolean waitForCondition(BooleanSupplier cond, long timeout) {
+        long stop = System.currentTimeMillis() + timeout;
+
+        while (System.currentTimeMillis() < stop) {
+            if (cond.getAsBoolean())
+                return true;
+
+            try {
+                sleep(50);
+            }
+            catch (InterruptedException e) {
+                return false;
+            }
+        }
+
+        return false;
+    }
+}
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ITNodeTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ITNodeTest.java
new file mode 100644
index 0000000..d168942
--- /dev/null
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ITNodeTest.java
@@ -0,0 +1,3580 @@
+/*
+ * 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.ignite.raft.jraft.core;
+
+import com.codahale.metrics.ConsoleReporter;
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.Vector;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiPredicate;
+import java.util.function.BooleanSupplier;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.raft.jraft.Iterator;
+import org.apache.ignite.raft.jraft.JRaftUtils;
+import org.apache.ignite.raft.jraft.Node;
+import org.apache.ignite.raft.jraft.NodeManager;
+import org.apache.ignite.raft.jraft.RaftGroupService;
+import org.apache.ignite.raft.jraft.StateMachine;
+import org.apache.ignite.raft.jraft.Status;
+import org.apache.ignite.raft.jraft.closure.JoinableClosure;
+import org.apache.ignite.raft.jraft.closure.ReadIndexClosure;
+import org.apache.ignite.raft.jraft.closure.SynchronizedClosure;
+import org.apache.ignite.raft.jraft.closure.TaskClosure;
+import org.apache.ignite.raft.jraft.conf.Configuration;
+import org.apache.ignite.raft.jraft.entity.EnumOutter;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.apache.ignite.raft.jraft.entity.Task;
+import org.apache.ignite.raft.jraft.entity.UserLog;
+import org.apache.ignite.raft.jraft.error.LogIndexOutOfBoundsException;
+import org.apache.ignite.raft.jraft.error.LogNotFoundException;
+import org.apache.ignite.raft.jraft.error.RaftError;
+import org.apache.ignite.raft.jraft.error.RaftException;
+import org.apache.ignite.raft.jraft.option.BootstrapOptions;
+import org.apache.ignite.raft.jraft.option.NodeOptions;
+import org.apache.ignite.raft.jraft.option.RaftOptions;
+import org.apache.ignite.raft.jraft.rpc.RpcClientEx;
+import org.apache.ignite.raft.jraft.rpc.RpcRequests;
+import org.apache.ignite.raft.jraft.rpc.RpcServer;
+import org.apache.ignite.raft.jraft.rpc.TestIgniteRpcServer;
+import org.apache.ignite.raft.jraft.rpc.impl.IgniteRpcClient;
+import org.apache.ignite.raft.jraft.rpc.impl.IgniteRpcServer;
+import org.apache.ignite.raft.jraft.rpc.impl.core.DefaultRaftClientService;
+import org.apache.ignite.raft.jraft.storage.SnapshotThrottle;
+import org.apache.ignite.raft.jraft.storage.snapshot.SnapshotReader;
+import org.apache.ignite.raft.jraft.storage.snapshot.ThroughputSnapshotThrottle;
+import org.apache.ignite.raft.jraft.test.TestUtils;
+import org.apache.ignite.raft.jraft.util.Bits;
+import org.apache.ignite.raft.jraft.util.Endpoint;
+import org.apache.ignite.raft.jraft.util.Utils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Integration tests for raft cluster.
+ */
+public class ITNodeTest {
+    static final Logger LOG = LoggerFactory.getLogger(ITNodeTest.class);
+
+    private String dataPath;
+
+    private final AtomicInteger startedCounter = new AtomicInteger(0);
+    private final AtomicInteger stoppedCounter = new AtomicInteger(0);
+
+    @Rule
+    public TestName testName = new TestName();
+
+    private long testStartMs;
+
+    private static DumpThread dumpThread;
+
+    private TestCluster cluster;
+
+    static class DumpThread extends Thread {
+        private static long DUMP_TIMEOUT_MS = 5 * 60 * 1000;
+        private volatile boolean stopped = false;
+
+        @SuppressWarnings("BusyWait") @Override
+        public void run() {
+            while (!this.stopped) {
+                try {
+                    Thread.sleep(DUMP_TIMEOUT_MS);
+                    LOG.info("Test hang too long, dump threads");
+                    TestUtils.dumpThreads();
+                }
+                catch (InterruptedException e) {
+                    // reset request, continue
+                    continue;
+                }
+            }
+        }
+    }
+
+    @BeforeClass
+    public static void setupNodeTest() {
+        dumpThread = new DumpThread();
+        dumpThread.setName("NodeTest-DumpThread");
+        dumpThread.setDaemon(true);
+        dumpThread.start();
+    }
+
+    @AfterClass
+    public static void tearNodeTest() throws Exception {
+        dumpThread.stopped = true;
+        dumpThread.interrupt();
+        dumpThread.join(100);
+    }
+
+    @Before
+    public void setup() throws Exception {
+        LOG.info(">>>>>>>>>>>>>>> Start test method: " + this.testName.getMethodName());
+        this.dataPath = TestUtils.mkTempDir();
+
+        File dataFile = new File(this.dataPath);
+
+        if (dataFile.exists())
+            assertTrue(Utils.delete(dataFile));
+
+        dataFile.mkdirs();
+        this.testStartMs = Utils.monotonicMs();
+        dumpThread.interrupt(); // reset dump timeout
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (cluster != null)
+            cluster.stopAll();
+
+        assertTrue(Utils.delete(new File(this.dataPath)));
+        this.startedCounter.set(0);
+        this.stoppedCounter.set(0);
+        LOG.info(">>>>>>>>>>>>>>> End test method: " + this.testName.getMethodName() + ", cost:"
+            + (Utils.monotonicMs() - this.testStartMs) + " ms.");
+    }
+
+    @Test
+    public void testInitShutdown() throws Exception {
+        final Endpoint addr = new Endpoint(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        final NodeOptions nodeOptions = createNodeOptions();
+
+        nodeOptions.setFsm(new MockStateMachine(addr));
+        nodeOptions.setLogUri(this.dataPath + File.separator + "log");
+        nodeOptions.setRaftMetaUri(this.dataPath + File.separator + "meta");
+        nodeOptions.setSnapshotUri(this.dataPath + File.separator + "snapshot");
+
+        RaftGroupService service = createService("unittest", new PeerId(addr, 0), nodeOptions);
+
+        service.start(true);
+
+        service.shutdown();
+    }
+
+    @Test
+    public void testNodeTaskOverload() throws Exception {
+        final Endpoint addr = new Endpoint(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        final PeerId peer = new PeerId(addr, 0);
+
+        final NodeOptions nodeOptions = createNodeOptions();
+        final RaftOptions raftOptions = new RaftOptions();
+        raftOptions.setDisruptorBufferSize(2);
+        nodeOptions.setRaftOptions(raftOptions);
+        final MockStateMachine fsm = new MockStateMachine(addr);
+        nodeOptions.setFsm(fsm);
+        nodeOptions.setLogUri(this.dataPath + File.separator + "log");
+        nodeOptions.setRaftMetaUri(this.dataPath + File.separator + "meta");
+        nodeOptions.setSnapshotUri(this.dataPath + File.separator + "snapshot");
+        nodeOptions.setInitialConf(new Configuration(Collections.singletonList(peer)));
+
+        RaftGroupService service = createService("unittest", new PeerId(addr, 0), nodeOptions);
+
+        final Node node = service.start(true);
+
+        assertEquals(1, node.listPeers().size());
+        assertTrue(node.listPeers().contains(peer));
+
+        while (!node.isLeader()) {
+            ;
+        }
+
+        final List<Task> tasks = new ArrayList<>();
+        final AtomicInteger c = new AtomicInteger(0);
+        for (int i = 0; i < 10; i++) {
+            final ByteBuffer data = ByteBuffer.wrap(("hello" + i).getBytes());
+            final Task task = new Task(data, new JoinableClosure(status -> {
+                System.out.println(status);
+                if (!status.isOk()) {
+                    assertTrue(
+                        status.getRaftError() == RaftError.EBUSY || status.getRaftError() == RaftError.EPERM);
+                }
+                c.incrementAndGet();
+            }));
+            node.apply(task);
+            tasks.add(task);
+        }
+        try {
+            Task.joinAll(tasks, TimeUnit.SECONDS.toMillis(30));
+            assertEquals(10, c.get());
+        }
+        finally {
+            service.shutdown();
+        }
+    }
+
+    /**
+     * Test rollback stateMachine with readIndex for issue 317: https://github.com/sofastack/sofa-jraft/issues/317
+     */
+    @Test
+    public void testRollbackStateMachineWithReadIndex_Issue317() throws Exception {
+        final Endpoint addr = new Endpoint(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        final PeerId peer = new PeerId(addr, 0);
+
+        final NodeOptions nodeOptions = createNodeOptions();
+        final CountDownLatch applyCompleteLatch = new CountDownLatch(1);
+        final CountDownLatch applyLatch = new CountDownLatch(1);
+        final CountDownLatch readIndexLatch = new CountDownLatch(1);
+        final AtomicInteger currentValue = new AtomicInteger(-1);
+        final String errorMsg = this.testName.getMethodName();
+        final StateMachine fsm = new StateMachineAdapter() {
+
+            @Override
+            public void onApply(final Iterator iter) {
+                // Notify that the #onApply is preparing to go.
+                readIndexLatch.countDown();
+                // Wait for submitting a read-index request
+                try {
+                    applyLatch.await();
+                }
+                catch (InterruptedException e) {
+                    fail();
+                }
+                int i = 0;
+                while (iter.hasNext()) {
+                    byte[] data = iter.next().array();
+                    int v = Bits.getInt(data, 0);
+                    assertEquals(i++, v);
+                    currentValue.set(v);
+                }
+                if (i > 0) {
+                    // rollback
+                    currentValue.set(i - 1);
+                    iter.setErrorAndRollback(1, new Status(-1, errorMsg));
+                    applyCompleteLatch.countDown();
+                }
+            }
+        };
+        nodeOptions.setFsm(fsm);
+        nodeOptions.setLogUri(this.dataPath + File.separator + "log");
+        nodeOptions.setRaftMetaUri(this.dataPath + File.separator + "meta");
+        nodeOptions.setSnapshotUri(this.dataPath + File.separator + "snapshot");
+        nodeOptions.setInitialConf(new Configuration(Collections.singletonList(peer)));
+
+        RaftGroupService service = createService("unittest", peer, nodeOptions);
+
+        final Node node = service.start(true);
+
+        assertEquals(1, node.listPeers().size());
+        assertTrue(node.listPeers().contains(peer));
+
+        while (!node.isLeader()) {
+            ;
+        }
+
+        int n = 5;
+        {
+            // apply tasks
+            for (int i = 0; i < n; i++) {
+                byte[] b = new byte[4];
+                Bits.putInt(b, 0, i);
+                node.apply(new Task(ByteBuffer.wrap(b), null));
+            }
+        }
+
+        final AtomicInteger readIndexSuccesses = new AtomicInteger(0);
+        {
+            // Submit a read-index, wait for #onApply
+            readIndexLatch.await();
+            final CountDownLatch latch = new CountDownLatch(1);
+            node.readIndex(null, new ReadIndexClosure() {
+
+                @Override
+                public void run(final Status status, final long index, final byte[] reqCtx) {
+                    try {
+                        if (status.isOk()) {
+                            readIndexSuccesses.incrementAndGet();
+                        }
+                        else {
+                            assertTrue("Unexpected status: " + status,
+                                status.getErrorMsg().contains(errorMsg) || status.getRaftError() == RaftError.ETIMEDOUT
+                                    || status.getErrorMsg().contains("Invalid state for readIndex: STATE_ERROR"));
+                        }
+                    }
+                    finally {
+                        latch.countDown();
+                    }
+                }
+            });
+            // We have already submit a read-index request,
+            // notify #onApply can go right now
+            applyLatch.countDown();
+
+            // The state machine is in error state, the node should step down.
+            while (node.isLeader()) {
+                Thread.sleep(10);
+            }
+            latch.await();
+            applyCompleteLatch.await();
+        }
+        // No read-index request succeed.
+        assertEquals(0, readIndexSuccesses.get());
+        assertTrue(n - 1 >= currentValue.get());
+
+        service.shutdown();
+    }
+
+    @Test
+    public void testSingleNode() throws Exception {
+        final Endpoint addr = new Endpoint(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        final PeerId peer = new PeerId(addr, 0);
+
+        final NodeOptions nodeOptions = createNodeOptions();
+        final MockStateMachine fsm = new MockStateMachine(addr);
+        nodeOptions.setFsm(fsm);
+        nodeOptions.setLogUri(this.dataPath + File.separator + "log");
+        nodeOptions.setRaftMetaUri(this.dataPath + File.separator + "meta");
+        nodeOptions.setSnapshotUri(this.dataPath + File.separator + "snapshot");
+        nodeOptions.setInitialConf(new Configuration(Collections.singletonList(peer)));
+        RaftGroupService service = createService("unittest", peer, nodeOptions);
+
+        Node node = service.start();
+
+        assertEquals(1, node.listPeers().size());
+        assertTrue(node.listPeers().contains(peer));
+
+        while (!node.isLeader()) {
+            ;
+        }
+
+        sendTestTaskAndWait(node);
+        assertEquals(10, fsm.getLogs().size());
+        int i = 0;
+        for (final ByteBuffer data : fsm.getLogs()) {
+            assertEquals("hello" + i++, new String(data.array()));
+        }
+        service.shutdown();
+    }
+
+    @Test
+    public void testNoLeader() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+
+        assertTrue(cluster.start(peers.get(0).getEndpoint()));
+
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(1, followers.size());
+
+        final Node follower = followers.get(0);
+        sendTestTaskAndWait(follower, 0, RaftError.EPERM);
+
+        // adds a peer3
+        final PeerId peer3 = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + 3);
+        CountDownLatch latch = new CountDownLatch(1);
+        follower.addPeer(peer3, new ExpectClosure(RaftError.EPERM, latch));
+        waitLatch(latch);
+
+        // remove the peer0
+        final PeerId peer0 = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        latch = new CountDownLatch(1);
+        follower.removePeer(peer0, new ExpectClosure(RaftError.EPERM, latch));
+        waitLatch(latch);
+    }
+
+    private void sendTestTaskAndWait(final Node node) throws InterruptedException {
+        this.sendTestTaskAndWait(node, 0, 10, RaftError.SUCCESS);
+    }
+
+    private void sendTestTaskAndWait(final Node node, int amount) throws InterruptedException {
+        this.sendTestTaskAndWait(node, 0, amount, RaftError.SUCCESS);
+    }
+
+    private void sendTestTaskAndWait(final Node node, final RaftError err) throws InterruptedException {
+        this.sendTestTaskAndWait(node, 0, 10, err);
+    }
+
+    private void sendTestTaskAndWait(final Node node, final int start, int amount,
+        final RaftError err) throws InterruptedException {
+        final CountDownLatch latch = new CountDownLatch(amount);
+        for (int i = start; i < start + amount; i++) {
+            final ByteBuffer data = ByteBuffer.wrap(("hello" + i).getBytes());
+            final Task task = new Task(data, new ExpectClosure(err, latch));
+            node.apply(task);
+        }
+        waitLatch(latch);
+    }
+
+    private void sendTestTaskAndWait(final Node node, final int start,
+        final RaftError err) throws InterruptedException {
+        sendTestTaskAndWait(node, start, 10, err);
+    }
+
+    @SuppressWarnings("SameParameterValue")
+    private void sendTestTaskAndWait(final String prefix, final Node node, final int code) throws InterruptedException {
+        sendTestTaskAndWait(prefix, node, 10, code);
+    }
+
+    @SuppressWarnings("SameParameterValue")
+    private void sendTestTaskAndWait(final String prefix, final Node node, int amount,
+        final int code) throws InterruptedException {
+        final CountDownLatch latch = new CountDownLatch(10);
+        for (int i = 0; i < amount; i++) {
+            final ByteBuffer data = ByteBuffer.wrap((prefix + i).getBytes());
+            final Task task = new Task(data, new ExpectClosure(code, null, latch));
+            node.apply(task);
+        }
+        waitLatch(latch);
+    }
+
+    @Test
+    public void testTripleNodesWithReplicatorStateListener() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+        //final TestCluster cluster = new TestCluster("unittest", this.dataPath, peers);
+
+        final UserReplicatorStateListener listener1 = new UserReplicatorStateListener();
+        final UserReplicatorStateListener listener2 = new UserReplicatorStateListener();
+
+        cluster = new TestCluster("unitest", this.dataPath, peers, new LinkedHashSet<>(), 300,
+            opts -> opts.setReplicationStateListeners(List.of(listener1, listener2)));
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        for (Node follower : cluster.getFollowers())
+            waitForCondition(() -> follower.getLeaderId() != null, 5_000);
+
+        assertEquals(4, this.startedCounter.get());
+        assertEquals(2, cluster.getLeader().getReplicatorStateListeners().size());
+        assertEquals(2, cluster.getFollowers().get(0).getReplicatorStateListeners().size());
+        assertEquals(2, cluster.getFollowers().get(1).getReplicatorStateListeners().size());
+
+        for (Node node : cluster.getNodes()) {
+            node.removeReplicatorStateListener(listener1);
+        }
+        assertEquals(1, cluster.getLeader().getReplicatorStateListeners().size());
+        assertEquals(1, cluster.getFollowers().get(0).getReplicatorStateListeners().size());
+        assertEquals(1, cluster.getFollowers().get(1).getReplicatorStateListeners().size());
+    }
+
+    // TODO asch Broken then using volatile log. A follower with empty log can become a leader IGNITE-14832.
+    @Test
+    @Ignore
+    public void testVoteTimedoutStepDown() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(3, leader.listPeers().size());
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        // Stop all followers
+        List<Node> followers = cluster.getFollowers();
+        assertFalse(followers.isEmpty());
+        for (Node node : followers) {
+            assertTrue(cluster.stop(node.getNodeId().getPeerId().getEndpoint()));
+        }
+
+        // Wait leader to step down.
+        while (leader.isLeader()) {
+            Thread.sleep(10);
+        }
+
+        // old leader try to elect self, it should fail.
+        ((NodeImpl) leader).tryElectSelf();
+        Thread.sleep(1500);
+
+        assertNull(cluster.getLeader());
+
+        // Start followers
+        for (Node node : followers) {
+            assertTrue(cluster.start(node.getNodeId().getPeerId().getEndpoint()));
+        }
+
+        cluster.ensureSame();
+    }
+
+    class UserReplicatorStateListener implements Replicator.ReplicatorStateListener {
+        @Override
+        public void onCreated(final PeerId peer) {
+            int val = ITNodeTest.this.startedCounter.incrementAndGet();
+
+            LOG.info("Replicator has been created {} {}", peer, val);
+        }
+
+        @Override
+        public void onError(final PeerId peer, final Status status) {
+            LOG.info("Replicator has errors {} {}", peer, status);
+        }
+
+        @Override
+        public void onDestroyed(final PeerId peer) {
+            int val = ITNodeTest.this.stoppedCounter.incrementAndGet();
+
+            LOG.info("Replicator has been destroyed {} {}", peer, val);
+        }
+    }
+
+    @Test
+    public void testLeaderTransferWithReplicatorStateListener() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers, new LinkedHashSet<>(), 300,
+            opts -> opts.setReplicationStateListeners(List.of(new UserReplicatorStateListener())));
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+
+        Node leader = cluster.getLeader();
+        this.sendTestTaskAndWait(leader);
+        Thread.sleep(100);
+        final List<Node> followers = cluster.getFollowers();
+
+        assertTrue(this.startedCounter.get() + "", waitForCondition(() -> this.startedCounter.get() == 2, 5_000));
+
+        final PeerId targetPeer = followers.get(0).getNodeId().getPeerId().copy();
+        LOG.info("Transfer leadership from {} to {}", leader, targetPeer);
+        assertTrue(leader.transferLeadershipTo(targetPeer).isOk());
+        Thread.sleep(1000);
+        cluster.waitLeader();
+
+        assertTrue(this.startedCounter.get() + "", waitForCondition(() -> this.startedCounter.get() == 4, 5_000));
+
+        for (Node node : cluster.getNodes()) {
+            node.clearReplicatorStateListeners();
+        }
+        assertEquals(0, cluster.getLeader().getReplicatorStateListeners().size());
+        assertEquals(0, cluster.getFollowers().get(0).getReplicatorStateListeners().size());
+        assertEquals(0, cluster.getFollowers().get(1).getReplicatorStateListeners().size());
+    }
+
+    @Test
+    public void testTripleNodes() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(3, leader.listPeers().size());
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        {
+            final ByteBuffer data = ByteBuffer.wrap("no closure".getBytes());
+            final Task task = new Task(data, null);
+            leader.apply(task);
+        }
+
+        {
+            // task with TaskClosure
+            final ByteBuffer data = ByteBuffer.wrap("task closure".getBytes());
+            final Vector<String> cbs = new Vector<>();
+            final CountDownLatch latch = new CountDownLatch(1);
+            final Task task = new Task(data, new TaskClosure() {
+
+                @Override
+                public void run(final Status status) {
+                    cbs.add("apply");
+                    latch.countDown();
+                }
+
+                @Override
+                public void onCommitted() {
+                    cbs.add("commit");
+
+                }
+            });
+            leader.apply(task);
+            latch.await();
+            assertEquals(2, cbs.size());
+            assertEquals("commit", cbs.get(0));
+            assertEquals("apply", cbs.get(1));
+        }
+
+        cluster.ensureSame();
+        assertEquals(2, cluster.getFollowers().size());
+    }
+
+    @Test
+    public void testSingleNodeWithLearner() throws Exception {
+        final Endpoint addr = new Endpoint(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        final PeerId peer = new PeerId(addr, 0);
+
+        final Endpoint learnerAddr = new Endpoint(TestUtils.getMyIp(), TestUtils.INIT_PORT + 1);
+        final PeerId learnerPeer = new PeerId(learnerAddr, 0);
+
+        final int cnt = 10;
+        MockStateMachine learnerFsm;
+        RaftGroupService learnerServer;
+        {
+            // Start learner
+            final NodeOptions nodeOptions = createNodeOptions();
+            learnerFsm = new MockStateMachine(learnerAddr);
+            nodeOptions.setFsm(learnerFsm);
+            nodeOptions.setLogUri(this.dataPath + File.separator + "log1");
+            nodeOptions.setRaftMetaUri(this.dataPath + File.separator + "meta1");
+            nodeOptions.setSnapshotUri(this.dataPath + File.separator + "snapshot1");
+            nodeOptions.setInitialConf(new Configuration(Collections.singletonList(peer), Collections
+                .singletonList(learnerPeer)));
+
+            learnerServer = createService("unittest", new PeerId(learnerAddr, 0), nodeOptions);
+            learnerServer.start(true);
+        }
+
+        {
+            // Start leader
+            final NodeOptions nodeOptions = createNodeOptions();
+            final MockStateMachine fsm = new MockStateMachine(addr);
+            nodeOptions.setFsm(fsm);
+            nodeOptions.setLogUri(this.dataPath + File.separator + "log");
+            nodeOptions.setRaftMetaUri(this.dataPath + File.separator + "meta");
+            nodeOptions.setSnapshotUri(this.dataPath + File.separator + "snapshot");
+            nodeOptions.setInitialConf(new Configuration(Collections.singletonList(peer), Collections
+                .singletonList(learnerPeer)));
+
+            RaftGroupService server = createService("unittest", new PeerId(addr, 0), nodeOptions);
+            Node node = server.start(true);
+
+            assertEquals(1, node.listPeers().size());
+            assertTrue(node.listPeers().contains(peer));
+            assertTrue(waitForCondition(() -> node.isLeader(), 1_000));
+
+            sendTestTaskAndWait(node, cnt);
+            assertEquals(cnt, fsm.getLogs().size());
+            int i = 0;
+            for (final ByteBuffer data : fsm.getLogs()) {
+                assertEquals("hello" + i++, new String(data.array()));
+            }
+            Thread.sleep(1000); //wait for entries to be replicated to learner.
+            server.shutdown();
+        }
+        {
+            // assert learner fsm
+            assertEquals(cnt, learnerFsm.getLogs().size());
+            int i = 0;
+            for (final ByteBuffer data : learnerFsm.getLogs()) {
+                assertEquals("hello" + i++, new String(data.array()));
+            }
+            learnerServer.shutdown();
+        }
+    }
+
+    @Test
+    public void testResetLearners() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        final LinkedHashSet<PeerId> learners = new LinkedHashSet<>();
+
+        for (int i = 0; i < 3; i++) {
+            learners.add(new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + 3 + i));
+        }
+
+        cluster = new TestCluster("unittest", this.dataPath, peers, learners, 300);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+        for (final PeerId peer : learners) {
+            assertTrue(cluster.startLearner(peer));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        Node leader = cluster.getLeader();
+
+        waitForCondition(() -> leader.listAlivePeers().size() == 3, 5_000);
+        waitForCondition(() -> leader.listAliveLearners().size() == 3, 5_000);
+
+        this.sendTestTaskAndWait(leader);
+        Thread.sleep(500);
+        List<MockStateMachine> fsms = cluster.getFsms();
+        assertEquals(6, fsms.size());
+        cluster.ensureSame();
+
+        {
+            // Reset learners to 2 nodes
+            PeerId learnerPeer = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + 3);
+            learners.remove(learnerPeer);
+            assertEquals(2, learners.size());
+
+            SynchronizedClosure done = new SynchronizedClosure();
+            leader.resetLearners(new ArrayList<>(learners), done);
+            assertTrue(done.await().isOk());
+            assertEquals(2, leader.listAliveLearners().size());
+            assertEquals(2, leader.listLearners().size());
+            this.sendTestTaskAndWait(leader);
+            Thread.sleep(500);
+
+            assertEquals(6, fsms.size());
+
+            MockStateMachine fsm = fsms.remove(3); // get the removed learner's fsm
+            assertEquals(fsm.getAddress(), learnerPeer.getEndpoint());
+            // Ensure no more logs replicated to the removed learner.
+            assertTrue(cluster.getLeaderFsm().getLogs().size() > fsm.getLogs().size());
+            assertEquals(cluster.getLeaderFsm().getLogs().size(), 2 * fsm.getLogs().size());
+        }
+        {
+            // remove another learner
+            PeerId learnerPeer = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + 4);
+            SynchronizedClosure done = new SynchronizedClosure();
+            leader.removeLearners(Arrays.asList(learnerPeer), done);
+            assertTrue(done.await().isOk());
+
+            this.sendTestTaskAndWait(leader);
+            Thread.sleep(500);
+            MockStateMachine fsm = fsms.remove(3); // get the removed learner's fsm
+            assertEquals(fsm.getAddress(), learnerPeer.getEndpoint());
+            // Ensure no more logs replicated to the removed learner.
+            assertTrue(cluster.getLeaderFsm().getLogs().size() > fsm.getLogs().size());
+            assertEquals(cluster.getLeaderFsm().getLogs().size(), fsm.getLogs().size() / 2 * 3);
+        }
+
+        assertEquals(3, leader.listAlivePeers().size());
+        assertEquals(1, leader.listAliveLearners().size());
+        assertEquals(1, leader.listLearners().size());
+    }
+
+    @Test
+    public void testTripleNodesWithStaticLearners() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        LinkedHashSet<PeerId> learners = new LinkedHashSet<>();
+        PeerId learnerPeer = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + 3);
+        learners.add(learnerPeer);
+        cluster.setLearners(learners);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+        final Node leader = cluster.getLeader();
+
+        assertEquals(3, leader.listPeers().size());
+        assertEquals(leader.listLearners().size(), 1);
+        assertTrue(leader.listLearners().contains(learnerPeer));
+        assertTrue(leader.listAliveLearners().isEmpty());
+
+        // start learner after cluster setup.
+        assertTrue(cluster.start(learnerPeer.getEndpoint()));
+
+        Thread.sleep(1000);
+
+        assertEquals(3, leader.listPeers().size());
+        assertEquals(leader.listLearners().size(), 1);
+        assertEquals(leader.listAliveLearners().size(), 1);
+
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        cluster.ensureSame();
+        assertEquals(4, cluster.getFsms().size());
+    }
+
+    @Test
+    public void testTripleNodesWithLearners() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(3, leader.listPeers().size());
+        assertTrue(leader.listLearners().isEmpty());
+        assertTrue(leader.listAliveLearners().isEmpty());
+
+        {
+            // Adds a learner
+            SynchronizedClosure done = new SynchronizedClosure();
+            PeerId learnerPeer = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + 3);
+            // Start learner
+            assertTrue(cluster.startLearner(learnerPeer));
+            leader.addLearners(Arrays.asList(learnerPeer), done);
+            assertTrue(done.await().isOk());
+            assertEquals(1, leader.listAliveLearners().size());
+            assertEquals(1, leader.listLearners().size());
+        }
+
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        {
+            final ByteBuffer data = ByteBuffer.wrap("no closure".getBytes());
+            final Task task = new Task(data, null);
+            leader.apply(task);
+        }
+
+        {
+            // task with TaskClosure
+            final ByteBuffer data = ByteBuffer.wrap("task closure".getBytes());
+            final Vector<String> cbs = new Vector<>();
+            final CountDownLatch latch = new CountDownLatch(1);
+            final Task task = new Task(data, new TaskClosure() {
+
+                @Override
+                public void run(final Status status) {
+                    cbs.add("apply");
+                    latch.countDown();
+                }
+
+                @Override
+                public void onCommitted() {
+                    cbs.add("commit");
+
+                }
+            });
+            leader.apply(task);
+            latch.await();
+            assertEquals(2, cbs.size());
+            assertEquals("commit", cbs.get(0));
+            assertEquals("apply", cbs.get(1));
+        }
+
+        assertEquals(4, cluster.getFsms().size());
+        assertEquals(2, cluster.getFollowers().size());
+        assertEquals(1, cluster.getLearners().size());
+        cluster.ensureSame();
+
+        {
+            // Adds another learner
+            SynchronizedClosure done = new SynchronizedClosure();
+            PeerId learnerPeer = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + 4);
+            // Start learner
+            assertTrue(cluster.startLearner(learnerPeer));
+            leader.addLearners(Arrays.asList(learnerPeer), done);
+            assertTrue(done.await().isOk());
+            assertEquals(2, leader.listAliveLearners().size());
+            assertEquals(2, leader.listLearners().size());
+        }
+        {
+            // stop two followers
+            for (Node follower : cluster.getFollowers()) {
+                assertTrue(cluster.stop(follower.getNodeId().getPeerId().getEndpoint()));
+            }
+            // send a new task
+            final ByteBuffer data = ByteBuffer.wrap("task closure".getBytes());
+            SynchronizedClosure done = new SynchronizedClosure();
+            leader.apply(new Task(data, done));
+            // should fail
+            assertFalse(done.await().isOk());
+            assertEquals(RaftError.EPERM, done.getStatus().getRaftError());
+            // One peer with two learners.
+            assertEquals(3, cluster.getFsms().size());
+            cluster.ensureSame();
+        }
+    }
+
+    @Test
+    public void testNodesWithPriorityElection() throws Exception {
+
+        List<Integer> priorities = new ArrayList<>();
+        priorities.add(100);
+        priorities.add(40);
+        priorities.add(40);
+
+        final List<PeerId> peers = TestUtils.generatePriorityPeers(3, priorities);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(3, leader.listPeers().size());
+        assertEquals(100, leader.getNodeTargetPriority());
+        assertEquals(100, leader.getLeaderId().getPriority());
+        assertEquals(2, cluster.getFollowers().size());
+    }
+
+    @Test
+    public void testNodesWithPartPriorityElection() throws Exception {
+
+        List<Integer> priorities = new ArrayList<>();
+        priorities.add(100);
+        priorities.add(40);
+        priorities.add(-1);
+
+        final List<PeerId> peers = TestUtils.generatePriorityPeers(3, priorities);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(3, leader.listPeers().size());
+        assertEquals(2, cluster.getFollowers().size());
+    }
+
+    @Test
+    public void testNodesWithSpecialPriorityElection() throws Exception {
+
+        List<Integer> priorities = new ArrayList<Integer>();
+        priorities.add(0);
+        priorities.add(0);
+        priorities.add(-1);
+
+        final List<PeerId> peers = TestUtils.generatePriorityPeers(3, priorities);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(3, leader.listPeers().size());
+        assertEquals(2, cluster.getFollowers().size());
+    }
+
+    @Test
+    public void testNodesWithZeroValPriorityElection() throws Exception {
+
+        List<Integer> priorities = new ArrayList<Integer>();
+        priorities.add(50);
+        priorities.add(0);
+        priorities.add(0);
+
+        final List<PeerId> peers = TestUtils.generatePriorityPeers(3, priorities);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(3, leader.listPeers().size());
+        assertEquals(2, cluster.getFollowers().size());
+        assertEquals(50, leader.getNodeTargetPriority());
+        assertEquals(50, leader.getLeaderId().getPriority());
+    }
+
+    @Test
+    public void testNoLeaderWithZeroValPriorityElection() throws Exception {
+        List<Integer> priorities = new ArrayList<>();
+        priorities.add(0);
+        priorities.add(0);
+        priorities.add(0);
+
+        final List<PeerId> peers = TestUtils.generatePriorityPeers(3, priorities);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority()));
+        }
+
+        Thread.sleep(200);
+
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(3, followers.size());
+
+        for (Node follower : followers) {
+            assertEquals(0, follower.getNodeId().getPeerId().getPriority());
+        }
+    }
+
+    @Test
+    public void testLeaderStopAndReElectWithPriority() throws Exception {
+        final List<Integer> priorities = new ArrayList<>();
+        priorities.add(100);
+        priorities.add(60);
+        priorities.add(10);
+
+        final List<PeerId> peers = TestUtils.generatePriorityPeers(3, priorities);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority()));
+        }
+
+        cluster.waitLeader();
+        Node leader = cluster.getLeader();
+
+        assertNotNull(leader);
+        assertEquals(100, leader.getNodeId().getPeerId().getPriority());
+        assertEquals(100, leader.getNodeTargetPriority());
+
+        // apply tasks to leader
+        sendTestTaskAndWait(leader);
+
+        // stop leader
+        assertTrue(cluster.stop(leader.getNodeId().getPeerId().getEndpoint()));
+
+        // elect new leader
+        cluster.waitLeader();
+        leader = cluster.getLeader();
+
+        assertNotNull(leader);
+
+        // get current leader priority value
+        int leaderPriority = leader.getNodeId().getPeerId().getPriority();
+
+        // get current leader log size
+        int peer1LogSize = cluster.getFsmByPeer(peers.get(1)).getLogs().size();
+        int peer2LogSize = cluster.getFsmByPeer(peers.get(2)).getLogs().size();
+
+        // if the leader is lower priority value
+        if (leaderPriority == 10) {
+            // we just compare the two peers' log size value;
+            assertTrue(peer2LogSize > peer1LogSize);
+        }
+        else {
+            assertEquals(60, leader.getNodeId().getPeerId().getPriority());
+            assertEquals(100, leader.getNodeTargetPriority());
+        }
+    }
+
+    @Test
+    public void testRemoveLeaderWithPriority() throws Exception {
+        final List<Integer> priorities = new ArrayList<Integer>();
+        priorities.add(100);
+        priorities.add(60);
+        priorities.add(10);
+
+        final List<PeerId> peers = TestUtils.generatePriorityPeers(3, priorities);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(100, leader.getNodeTargetPriority());
+        assertEquals(100, leader.getNodeId().getPeerId().getPriority());
+
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+
+        final PeerId oldLeader = leader.getNodeId().getPeerId().copy();
+        final Endpoint oldLeaderAddr = oldLeader.getEndpoint();
+
+        // remove old leader
+        LOG.info("Remove old leader {}", oldLeader);
+        CountDownLatch latch = new CountDownLatch(1);
+        leader.removePeer(oldLeader, new ExpectClosure(latch));
+        waitLatch(latch);
+        assertEquals(60, leader.getNodeTargetPriority());
+
+        // stop and clean old leader
+        LOG.info("Stop and clean old leader {}", oldLeader);
+        assertTrue(cluster.stop(oldLeaderAddr));
+        cluster.clean(oldLeaderAddr);
+
+        // elect new leader
+        cluster.waitLeader();
+        leader = cluster.getLeader();
+        LOG.info("New leader is {}", leader);
+        assertNotNull(leader);
+        assertNotSame(leader, oldLeader);
+    }
+
+    @Test
+    @Ignore // TODO asch https://issues.apache.org/jira/browse/IGNITE-14833
+    public void testChecksum() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        // start with checksum validation
+        {
+            final TestCluster cluster = new TestCluster("unittest", this.dataPath, peers);
+            try {
+                final RaftOptions raftOptions = new RaftOptions();
+                raftOptions.setEnableLogEntryChecksum(true);
+                for (final PeerId peer : peers) {
+                    assertTrue(cluster.start(peer.getEndpoint(), false, 300, true, null, raftOptions));
+                }
+
+                cluster.waitLeader();
+                final Node leader = cluster.getLeader();
+                assertNotNull(leader);
+                assertEquals(3, leader.listPeers().size());
+                this.sendTestTaskAndWait(leader);
+                cluster.ensureSame();
+            }
+            finally {
+                cluster.stopAll();
+            }
+        }
+
+        // restart with peer3 enable checksum validation
+        {
+            final TestCluster cluster = new TestCluster("unittest", this.dataPath, peers);
+            try {
+                RaftOptions raftOptions = new RaftOptions();
+                raftOptions.setEnableLogEntryChecksum(false);
+                for (final PeerId peer : peers) {
+                    if (peer.equals(peers.get(2))) {
+                        raftOptions = new RaftOptions();
+                        raftOptions.setEnableLogEntryChecksum(true);
+                    }
+                    assertTrue(cluster.start(peer.getEndpoint(), false, 300, true, null, raftOptions));
+                }
+
+                cluster.waitLeader();
+                final Node leader = cluster.getLeader();
+                assertNotNull(leader);
+                assertEquals(3, leader.listPeers().size());
+                this.sendTestTaskAndWait(leader);
+                cluster.ensureSame();
+            }
+            finally {
+                cluster.stopAll();
+            }
+        }
+
+        // restart with no checksum validation
+        {
+            final TestCluster cluster = new TestCluster("unittest", this.dataPath, peers);
+            try {
+                final RaftOptions raftOptions = new RaftOptions();
+                raftOptions.setEnableLogEntryChecksum(false);
+                for (final PeerId peer : peers) {
+                    assertTrue(cluster.start(peer.getEndpoint(), false, 300, true, null, raftOptions));
+                }
+
+                cluster.waitLeader();
+                final Node leader = cluster.getLeader();
+                assertNotNull(leader);
+                assertEquals(3, leader.listPeers().size());
+                this.sendTestTaskAndWait(leader);
+                cluster.ensureSame();
+            }
+            finally {
+                cluster.stopAll();
+            }
+        }
+
+        // restart with all peers enable checksum validation
+        {
+            final TestCluster cluster = new TestCluster("unittest", this.dataPath, peers);
+            try {
+                final RaftOptions raftOptions = new RaftOptions();
+                raftOptions.setEnableLogEntryChecksum(true);
+                for (final PeerId peer : peers) {
+                    assertTrue(cluster.start(peer.getEndpoint(), false, 300, true, null, raftOptions));
+                }
+
+                cluster.waitLeader();
+                final Node leader = cluster.getLeader();
+                assertNotNull(leader);
+                assertEquals(3, leader.listPeers().size());
+                this.sendTestTaskAndWait(leader);
+                cluster.ensureSame();
+            }
+            finally {
+                cluster.stopAll();
+            }
+        }
+
+    }
+
+    @Test
+    public void testReadIndex() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), false, 300, true));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(3, leader.listPeers().size());
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        // first call will fail-fast when no connection
+        if (!assertReadIndex(leader, 11)) {
+            assertTrue(assertReadIndex(leader, 11));
+        }
+
+        // read from follower
+        for (final Node follower : cluster.getFollowers()) {
+            assertNotNull(follower);
+
+            assertTrue(waitForCondition(() -> leader.getNodeId().getPeerId().equals(follower.getLeaderId()), 5_000));
+
+            assertReadIndex(follower, 11);
+        }
+
+        // read with null request context
+        final CountDownLatch latch = new CountDownLatch(1);
+        leader.readIndex(null, new ReadIndexClosure() {
+
+            @Override
+            public void run(final Status status, final long index, final byte[] reqCtx) {
+                assertNull(reqCtx);
+                assertTrue(status.isOk());
+                latch.countDown();
+            }
+        });
+        latch.await();
+    }
+
+    @Test // TODO asch do we need read index timeout ? https://issues.apache.org/jira/browse/IGNITE-14832
+    public void testReadIndexTimeout() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), false, 300, true));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(3, leader.listPeers().size());
+        // apply tasks to leader
+        sendTestTaskAndWait(leader);
+
+        // first call will fail-fast when no connection
+        if (!assertReadIndex(leader, 11)) {
+            assertTrue(assertReadIndex(leader, 11));
+        }
+
+        // read from follower
+        for (final Node follower : cluster.getFollowers()) {
+            assertNotNull(follower);
+
+            assertTrue(waitForCondition(() -> leader.getNodeId().getPeerId().equals(follower.getLeaderId()), 5_000));
+
+            assertReadIndex(follower, 11);
+        }
+
+        // read with null request context
+        final CountDownLatch latch = new CountDownLatch(1);
+        final long start = System.currentTimeMillis();
+        leader.readIndex(null, new ReadIndexClosure() {
+
+            @Override
+            public void run(final Status status, final long index, final byte[] reqCtx) {
+                assertNull(reqCtx);
+                if (status.isOk()) {
+                    System.err.println("Read-index so fast: " + (System.currentTimeMillis() - start) + "ms");
+                }
+                else {
+                    assertEquals(status, new Status(RaftError.ETIMEDOUT, "read-index request timeout"));
+                    assertEquals(index, -1);
+                }
+                latch.countDown();
+            }
+        });
+        latch.await();
+    }
+
+    @Test
+    public void testReadIndexFromLearner() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), false, 300, true));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(3, leader.listPeers().size());
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        {
+            // Adds a learner
+            SynchronizedClosure done = new SynchronizedClosure();
+            PeerId learnerPeer = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + 3);
+            // Start learner
+            assertTrue(cluster.startLearner(learnerPeer));
+            leader.addLearners(Arrays.asList(learnerPeer), done);
+            assertTrue(done.await().isOk());
+            assertEquals(1, leader.listAliveLearners().size());
+            assertEquals(1, leader.listLearners().size());
+        }
+
+        Thread.sleep(100);
+        // read from learner
+        Node learner = cluster.getNodes().get(3);
+        assertNotNull(leader);
+        assertReadIndex(learner, 12);
+        assertReadIndex(learner, 12);
+    }
+
+    @Test
+    public void testReadIndexChaos() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), false, 300, true));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(3, leader.listPeers().size());
+
+        final CountDownLatch latch = new CountDownLatch(10);
+        for (int i = 0; i < 10; i++) {
+            new Thread() {
+                @Override
+                public void run() {
+                    try {
+                        for (int i = 0; i < 100; i++) {
+                            try {
+                                sendTestTaskAndWait(leader);
+                            }
+                            catch (final InterruptedException e) {
+                                Thread.currentThread().interrupt();
+                            }
+                            readIndexRandom(cluster);
+                        }
+                    }
+                    finally {
+                        latch.countDown();
+                    }
+                }
+
+                private void readIndexRandom(final TestCluster cluster) {
+                    final CountDownLatch readLatch = new CountDownLatch(1);
+                    final byte[] requestContext = TestUtils.getRandomBytes();
+                    cluster.getNodes().get(ThreadLocalRandom.current().nextInt(3))
+                        .readIndex(requestContext, new ReadIndexClosure() {
+
+                            @Override
+                            public void run(final Status status, final long index, final byte[] reqCtx) {
+                                if (status.isOk()) {
+                                    assertTrue(status.toString(), status.isOk());
+                                    assertTrue(index > 0);
+                                    assertArrayEquals(requestContext, reqCtx);
+                                }
+                                readLatch.countDown();
+                            }
+                        });
+                    try {
+                        readLatch.await();
+                    }
+                    catch (final InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                    }
+                }
+            }.start();
+        }
+
+        latch.await();
+
+        cluster.ensureSame();
+
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(10000, fsm.getLogs().size());
+        }
+    }
+
+    @SuppressWarnings({"unused", "SameParameterValue"})
+    private boolean assertReadIndex(final Node node, final int index) throws InterruptedException {
+        final CountDownLatch latch = new CountDownLatch(1);
+        final byte[] requestContext = TestUtils.getRandomBytes();
+        final AtomicBoolean success = new AtomicBoolean(false);
+        node.readIndex(requestContext, new ReadIndexClosure() {
+
+            @Override
+            public void run(final Status status, final long theIndex, final byte[] reqCtx) {
+                if (status.isOk()) {
+                    assertEquals(index, theIndex);
+                    assertArrayEquals(requestContext, reqCtx);
+                    success.set(true);
+                }
+                else {
+                    assertTrue(status.getErrorMsg(), status.getErrorMsg().contains("RPC exception:Check connection["));
+                    assertTrue(status.getErrorMsg(), status.getErrorMsg().contains("] fail and try to create new one"));
+                }
+                latch.countDown();
+            }
+        });
+        latch.await();
+        return success.get();
+    }
+
+    @Test
+    public void testNodeMetrics() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), false, 300, true));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertEquals(3, leader.listPeers().size());
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        {
+            final ByteBuffer data = ByteBuffer.wrap("no closure".getBytes());
+            final Task task = new Task(data, null);
+            leader.apply(task);
+        }
+
+        cluster.ensureSame();
+        for (final Node node : cluster.getNodes()) {
+            System.out.println("-------------" + node.getNodeId() + "-------------");
+            final ConsoleReporter reporter = ConsoleReporter.forRegistry(node.getNodeMetrics().getMetricRegistry())
+                .build();
+            reporter.report();
+            reporter.close();
+            System.out.println();
+        }
+        // TODO check http status https://issues.apache.org/jira/browse/IGNITE-14832
+        assertEquals(2, cluster.getFollowers().size());
+    }
+
+    @Test
+    public void testLeaderFail() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        LOG.info("Current leader is {}", leader.getLeaderId());
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        List<Node> followers = cluster.getFollowers();
+
+        for (Node follower : followers) {
+            NodeImpl follower0 = (NodeImpl) follower;
+            DefaultRaftClientService rpcService = (DefaultRaftClientService) follower0.getRpcClientService();
+            RpcClientEx rpcClientEx = (RpcClientEx) rpcService.getRpcClient();
+            rpcClientEx.blockMessages(new BiPredicate<Object, String>() {
+                @Override public boolean test(Object msg, String nodeId) {
+                    if (msg instanceof RpcRequests.RequestVoteRequest) {
+                        RpcRequests.RequestVoteRequest msg0 = (RpcRequests.RequestVoteRequest) msg;
+
+                        return !msg0.getPreVote();
+                    }
+
+                    return false;
+                }
+            });
+        }
+
+        // stop leader
+        LOG.warn("Stop leader {}", leader.getNodeId().getPeerId());
+        final PeerId oldLeader = leader.getNodeId().getPeerId();
+        assertTrue(cluster.stop(leader.getNodeId().getPeerId().getEndpoint()));
+
+        // apply something when follower
+        //final List<Node> followers = cluster.getFollowers();
+        assertFalse(followers.isEmpty());
+        this.sendTestTaskAndWait("follower apply ", followers.get(0), -1);
+
+        for (Node follower : followers) {
+            NodeImpl follower0 = (NodeImpl) follower;
+            DefaultRaftClientService rpcService = (DefaultRaftClientService) follower0.getRpcClientService();
+            RpcClientEx rpcClientEx = (RpcClientEx) rpcService.getRpcClient();
+            rpcClientEx.stopBlock();
+        }
+
+        // elect new leader
+        cluster.waitLeader();
+        leader = cluster.getLeader();
+        LOG.info("Elect new leader is {}", leader.getLeaderId());
+        // apply tasks to new leader
+        CountDownLatch latch = new CountDownLatch(10);
+        for (int i = 10; i < 20; i++) {
+            final ByteBuffer data = ByteBuffer.wrap(("hello" + i).getBytes());
+            final Task task = new Task(data, new ExpectClosure(latch));
+            leader.apply(task);
+        }
+        waitLatch(latch);
+
+        // restart old leader
+        LOG.info("restart old leader {}", oldLeader);
+        assertTrue(cluster.start(oldLeader.getEndpoint()));
+        // apply something
+        latch = new CountDownLatch(10);
+        for (int i = 20; i < 30; i++) {
+            final ByteBuffer data = ByteBuffer.wrap(("hello" + i).getBytes());
+            final Task task = new Task(data, new ExpectClosure(latch));
+            leader.apply(task);
+        }
+        waitLatch(latch);
+
+        // stop and clean old leader
+        cluster.stop(oldLeader.getEndpoint());
+        cluster.clean(oldLeader.getEndpoint());
+
+        // restart old leader
+        LOG.info("restart old leader {}", oldLeader);
+        assertTrue(cluster.start(oldLeader.getEndpoint()));
+        cluster.ensureSame();
+
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(30, fsm.getLogs().size());
+        }
+    }
+
+    @Test
+    public void testJoinNodes() throws Exception {
+        final PeerId peer0 = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        final PeerId peer1 = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + 1);
+        final PeerId peer2 = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + 2);
+        final PeerId peer3 = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + 3);
+
+        final ArrayList<PeerId> peers = new ArrayList<>();
+        peers.add(peer0);
+
+        // start single cluster
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        assertTrue(cluster.start(peer0.getEndpoint()));
+
+        cluster.waitLeader();
+
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        Assert.assertEquals(leader.getNodeId().getPeerId(), peer0);
+        this.sendTestTaskAndWait(leader);
+
+        // start peer1
+        assertTrue(cluster.start(peer1.getEndpoint(), false, 300));
+
+        // add peer1
+        CountDownLatch latch = new CountDownLatch(1);
+        peers.add(peer1);
+        leader.addPeer(peer1, new ExpectClosure(latch));
+        waitLatch(latch);
+
+        cluster.ensureSame();
+        assertEquals(2, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(10, fsm.getLogs().size());
+        }
+
+        // add peer2 but not start
+        peers.add(peer2);
+        latch = new CountDownLatch(1);
+        leader.addPeer(peer2, new ExpectClosure(RaftError.ECATCHUP, latch));
+        waitLatch(latch);
+
+        // start peer2 after 2 seconds
+        Thread.sleep(2000);
+        assertTrue(cluster.start(peer2.getEndpoint(), false, 300));
+
+        // re-add peer2
+        latch = new CountDownLatch(2);
+        leader.addPeer(peer2, new ExpectClosure(latch));
+        // concurrent configuration change
+        leader.addPeer(peer3, new ExpectClosure(RaftError.EBUSY, latch));
+        waitLatch(latch);
+
+        // re-add peer2 directly
+
+        try {
+            leader.addPeer(peer2, new ExpectClosure(latch));
+            fail();
+        }
+        catch (final IllegalArgumentException e) {
+            assertEquals("Peer already exists in current configuration", e.getMessage());
+        }
+
+        cluster.ensureSame();
+        assertEquals(3, cluster.getFsms().size());
+        assertEquals(2, cluster.getFollowers().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(10, fsm.getLogs().size());
+        }
+    }
+
+    private void waitLatch(final CountDownLatch latch) throws InterruptedException {
+        assertTrue(latch.await(30, TimeUnit.SECONDS));
+    }
+
+    @Test
+    public void testRemoveFollower() throws Exception {
+        List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        cluster.ensureSame();
+
+        List<Node> followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+
+        final PeerId followerPeer = followers.get(0).getNodeId().getPeerId();
+        final Endpoint followerAddr = followerPeer.getEndpoint();
+
+        // stop and clean follower
+        LOG.info("Stop and clean follower {}", followerPeer);
+        assertTrue(cluster.stop(followerAddr));
+        cluster.clean(followerAddr);
+
+        // remove follower
+        LOG.info("Remove follower {}", followerPeer);
+        CountDownLatch latch = new CountDownLatch(1);
+        leader.removePeer(followerPeer, new ExpectClosure(latch));
+        waitLatch(latch);
+
+        this.sendTestTaskAndWait(leader, 10, RaftError.SUCCESS);
+        followers = cluster.getFollowers();
+        assertEquals(1, followers.size());
+
+        peers = TestUtils.generatePeers(3);
+        assertTrue(peers.remove(followerPeer));
+
+        // start follower
+        LOG.info("Start and add follower {}", followerPeer);
+        assertTrue(cluster.start(followerAddr));
+        // re-add follower
+        latch = new CountDownLatch(1);
+        leader.addPeer(followerPeer, new ExpectClosure(latch));
+        waitLatch(latch);
+
+        followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+
+        cluster.ensureSame();
+        assertEquals(3, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(20, fsm.getLogs().size());
+        }
+    }
+
+    @Test
+    public void testRemoveLeader() throws Exception {
+        List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        // elect leader
+        cluster.waitLeader();
+
+        // get leader
+        Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        cluster.ensureSame();
+
+        List<Node> followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+
+        final PeerId oldLeader = leader.getNodeId().getPeerId().copy();
+        final Endpoint oldLeaderAddr = oldLeader.getEndpoint();
+
+        // remove old leader
+        LOG.info("Remove old leader {}", oldLeader);
+        CountDownLatch latch = new CountDownLatch(1);
+        leader.removePeer(oldLeader, new ExpectClosure(latch));
+        waitLatch(latch);
+        Thread.sleep(100);
+
+        // elect new leader
+        cluster.waitLeader();
+        leader = cluster.getLeader();
+        LOG.info("New leader is {}", leader);
+        assertNotNull(leader);
+        // apply tasks to new leader
+        this.sendTestTaskAndWait(leader, 10, RaftError.SUCCESS);
+
+        // stop and clean old leader
+        LOG.info("Stop and clean old leader {}", oldLeader);
+        assertTrue(cluster.stop(oldLeaderAddr));
+        cluster.clean(oldLeaderAddr);
+
+        // Add and start old leader
+        LOG.info("Start and add old leader {}", oldLeader);
+        assertTrue(cluster.start(oldLeaderAddr));
+
+        peers = TestUtils.generatePeers(3);
+        assertTrue(peers.remove(oldLeader));
+        latch = new CountDownLatch(1);
+        leader.addPeer(oldLeader, new ExpectClosure(latch));
+        waitLatch(latch);
+
+        followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+        cluster.ensureSame();
+        assertEquals(3, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(20, fsm.getLogs().size());
+        }
+    }
+
+    @Test
+    public void testPreVote() throws Exception {
+        List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+        // get leader
+        Node leader = cluster.getLeader();
+        final long savedTerm = ((NodeImpl) leader).getCurrentTerm();
+        assertNotNull(leader);
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        cluster.ensureSame();
+
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+
+        final PeerId followerPeer = followers.get(0).getNodeId().getPeerId();
+        final Endpoint followerAddr = followerPeer.getEndpoint();
+
+        // remove follower
+        LOG.info("Remove follower {}", followerPeer);
+        CountDownLatch latch = new CountDownLatch(1);
+        leader.removePeer(followerPeer, new ExpectClosure(latch));
+        waitLatch(latch);
+
+        this.sendTestTaskAndWait(leader, 10, RaftError.SUCCESS);
+
+        Thread.sleep(2000);
+
+        // add follower
+        LOG.info("Add follower {}", followerAddr);
+        peers = TestUtils.generatePeers(3);
+        assertTrue(peers.remove(followerPeer));
+        latch = new CountDownLatch(1);
+        leader.addPeer(followerPeer, new ExpectClosure(latch));
+        waitLatch(latch);
+        leader = cluster.getLeader();
+        assertNotNull(leader);
+        // leader term should not be changed.
+        assertEquals(savedTerm, ((NodeImpl) leader).getCurrentTerm());
+    }
+
+    @Test
+    public void testSetPeer1() throws Exception {
+        cluster = new TestCluster("testSetPeer1", this.dataPath, new ArrayList<>());
+
+        final PeerId bootPeer = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        assertTrue(cluster.start(bootPeer.getEndpoint()));
+        final List<Node> nodes = cluster.getFollowers();
+        assertEquals(1, nodes.size());
+
+        final List<PeerId> peers = new ArrayList<>();
+        peers.add(bootPeer);
+        // reset peers from empty
+        assertTrue(nodes.get(0).resetPeers(new Configuration(peers)).isOk());
+        cluster.waitLeader();
+        assertNotNull(cluster.getLeader());
+    }
+
+    @Test
+    public void testSetPeer2() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+        // get leader
+        Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        cluster.ensureSame();
+
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+
+        final PeerId followerPeer1 = followers.get(0).getNodeId().getPeerId();
+        final Endpoint followerAddr1 = followerPeer1.getEndpoint();
+        final PeerId followerPeer2 = followers.get(1).getNodeId().getPeerId();
+        final Endpoint followerAddr2 = followerPeer2.getEndpoint();
+
+        LOG.info("Stop and clean follower {}", followerPeer1);
+        assertTrue(cluster.stop(followerAddr1));
+        cluster.clean(followerAddr1);
+
+        // apply tasks to leader again
+        this.sendTestTaskAndWait(leader, 10, RaftError.SUCCESS);
+        // set peer when no quorum die
+        final Endpoint leaderAddr = leader.getLeaderId().getEndpoint().copy();
+        LOG.info("Set peers to {}", leaderAddr);
+
+        LOG.info("Stop and clean follower {}", followerPeer2);
+        assertTrue(cluster.stop(followerAddr2));
+        cluster.clean(followerAddr2);
+
+        assertTrue(waitForTopology(cluster, leaderAddr, 1, 5_000));
+
+        // leader will step-down, become follower
+        Thread.sleep(2000);
+        List<PeerId> newPeers = new ArrayList<>();
+        newPeers.add(new PeerId(leaderAddr, 0));
+
+        // new peers equal to current conf
+        assertTrue(leader.resetPeers(new Configuration(peers)).isOk());
+        // set peer when quorum die
+        LOG.warn("Set peers to {}", leaderAddr);
+        assertTrue(leader.resetPeers(new Configuration(newPeers)).isOk());
+
+        cluster.waitLeader();
+        leader = cluster.getLeader();
+        assertNotNull(leader);
+        Assert.assertEquals(leaderAddr, leader.getNodeId().getPeerId().getEndpoint());
+
+        LOG.info("start follower {}", followerAddr1);
+        assertTrue(cluster.start(followerAddr1, true, 300));
+        LOG.info("start follower {}", followerAddr2);
+        assertTrue(cluster.start(followerAddr2, true, 300));
+
+        // Make sure the leader has discovered new nodes.
+        assertTrue(waitForTopology(cluster, leaderAddr, 3, 30_000)); // Discovery may take a while sometimes.
+
+        CountDownLatch latch = new CountDownLatch(1);
+        LOG.info("Add old follower {}", followerAddr1);
+        leader.addPeer(followerPeer1, new ExpectClosure(latch));
+        waitLatch(latch);
+
+        latch = new CountDownLatch(1);
+        LOG.info("Add old follower {}", followerAddr2);
+        leader.addPeer(followerPeer2, new ExpectClosure(latch));
+        waitLatch(latch);
+
+        newPeers.add(followerPeer1);
+        newPeers.add(followerPeer2);
+
+        cluster.ensureSame();
+        assertEquals(3, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(20, fsm.getLogs().size());
+        }
+    }
+
+    /**
+     * @throws Exception
+     */
+    @Test
+    public void testRestoreSnasphot() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+        // get leader
+        final Node leader = cluster.getLeader();
+
+        LOG.info("Leader: " + leader);
+
+        assertNotNull(leader);
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        cluster.ensureSame();
+        triggerLeaderSnapshot(cluster, leader);
+
+        // stop leader
+        final Endpoint leaderAddr = leader.getNodeId().getPeerId().getEndpoint().copy();
+        assertTrue(cluster.stop(leaderAddr));
+
+        // restart leader
+        cluster.waitLeader();
+        assertEquals(0, cluster.getLeaderFsm().getLoadSnapshotTimes());
+        assertTrue(cluster.start(leaderAddr));
+        cluster.ensureSame();
+        assertEquals(0, cluster.getLeaderFsm().getLoadSnapshotTimes());
+    }
+
+    /**
+     * @throws Exception
+     */
+    @Test
+    public void testRestoreSnapshotWithDelta() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+        // get leader
+        final Node leader = cluster.getLeader();
+
+        LOG.info("Leader: " + leader);
+
+        assertNotNull(leader);
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        cluster.ensureSame();
+        triggerLeaderSnapshot(cluster, leader);
+
+        // stop leader
+        final Endpoint leaderAddr = leader.getNodeId().getPeerId().getEndpoint().copy();
+        assertTrue(cluster.stop(leaderAddr));
+
+        // restart leader
+        cluster.waitLeader();
+
+        sendTestTaskAndWait(cluster.getLeader(), 10, RaftError.SUCCESS);
+
+        assertEquals(0, cluster.getLeaderFsm().getLoadSnapshotTimes());
+        assertTrue(cluster.start(leaderAddr));
+
+        Node oldLeader = cluster.getNode(leaderAddr);
+
+        cluster.ensureSame();
+        assertEquals(0, cluster.getLeaderFsm().getLoadSnapshotTimes());
+
+        MockStateMachine fsm = (MockStateMachine) oldLeader.getOptions().getFsm();
+        assertEquals(1, fsm.getLoadSnapshotTimes());
+    }
+
+    private void triggerLeaderSnapshot(final TestCluster cluster, final Node leader) throws InterruptedException {
+        this.triggerLeaderSnapshot(cluster, leader, 1);
+    }
+
+    private void triggerLeaderSnapshot(final TestCluster cluster, final Node leader, final int times)
+        throws InterruptedException {
+        // trigger leader snapshot
+        // first snapshot will be triggered randomly
+        int snapshotTimes = cluster.getLeaderFsm().getSaveSnapshotTimes();
+        assertTrue("snapshotTimes=" + snapshotTimes + ", times=" + times, snapshotTimes == times - 1
+            || snapshotTimes == times);
+        final CountDownLatch latch = new CountDownLatch(1);
+        leader.snapshot(new ExpectClosure(latch));
+        waitLatch(latch);
+        assertEquals(snapshotTimes + 1, cluster.getLeaderFsm().getSaveSnapshotTimes());
+    }
+
+    @Test
+    public void testInstallSnapshotWithThrottle() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), false, 200, false, new ThroughputSnapshotThrottle(1024, 1)));
+        }
+
+        cluster.waitLeader();
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        cluster.ensureSame();
+
+        // stop follower1
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+
+        final Endpoint followerAddr = followers.get(0).getNodeId().getPeerId().getEndpoint();
+        assertTrue(cluster.stop(followerAddr));
+
+        cluster.waitLeader();
+
+        // apply something more
+        this.sendTestTaskAndWait(leader, 10, RaftError.SUCCESS);
+
+        Thread.sleep(1000);
+
+        // trigger leader snapshot
+        triggerLeaderSnapshot(cluster, leader);
+        // apply something more
+        this.sendTestTaskAndWait(leader, 20, RaftError.SUCCESS);
+        // trigger leader snapshot
+        triggerLeaderSnapshot(cluster, leader, 2);
+
+        // wait leader to compact logs
+        Thread.sleep(1000);
+
+        // restart follower.
+        cluster.clean(followerAddr);
+        assertTrue(cluster.start(followerAddr, true, 300, false, new ThroughputSnapshotThrottle(1024, 1)));
+
+        Thread.sleep(2000);
+        cluster.ensureSame();
+
+        assertEquals(3, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(30, fsm.getLogs().size());
+        }
+    }
+
+    @Test // TODO add test for timeout on snapshot install https://issues.apache.org/jira/browse/IGNITE-14832
+    public void testInstallLargeSnapshotWithThrottle() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(4);
+        cluster = new TestCluster("unitest", this.dataPath, peers.subList(0, 3));
+        for (int i = 0; i < peers.size() - 1; i++) {
+            final PeerId peer = peers.get(i);
+            final boolean started = cluster.start(peer.getEndpoint(), false, 200, false);
+            assertTrue(started);
+        }
+        cluster.waitLeader();
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        // apply tasks to leader
+        sendTestTaskAndWait(leader, 0, RaftError.SUCCESS);
+
+        cluster.ensureSame();
+
+        // apply something more
+        for (int i = 1; i < 100; i++) {
+            sendTestTaskAndWait(leader, i * 10, RaftError.SUCCESS);
+        }
+
+        Thread.sleep(1000);
+
+        // trigger leader snapshot
+        triggerLeaderSnapshot(cluster, leader);
+
+        // apply something more
+        for (int i = 100; i < 200; i++) {
+            sendTestTaskAndWait(leader, i * 10, RaftError.SUCCESS);
+        }
+        // trigger leader snapshot
+        triggerLeaderSnapshot(cluster, leader, 2);
+
+        // wait leader to compact logs
+        Thread.sleep(1000);
+
+        // add follower
+        final PeerId newPeer = peers.get(3);
+        final SnapshotThrottle snapshotThrottle = new ThroughputSnapshotThrottle(128, 1);
+        final boolean started = cluster.start(newPeer.getEndpoint(), false, 300, false, snapshotThrottle);
+        assertTrue(started);
+
+        final CountDownLatch latch = new CountDownLatch(1);
+        leader.addPeer(newPeer, status -> {
+            assertTrue(status.toString(), status.isOk());
+            latch.countDown();
+        });
+        waitLatch(latch);
+
+        cluster.ensureSame();
+
+        assertEquals(4, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(2000, fsm.getLogs().size());
+        }
+    }
+
+    @Test
+    public void testInstallLargeSnapshot() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(4);
+        cluster = new TestCluster("unitest", this.dataPath, peers.subList(0, 3));
+        for (int i = 0; i < peers.size() - 1; i++) {
+            final PeerId peer = peers.get(i);
+            final boolean started = cluster.start(peer.getEndpoint(), false, 200, false);
+            assertTrue(started);
+        }
+        cluster.waitLeader();
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        // apply tasks to leader
+        sendTestTaskAndWait(leader, 0, RaftError.SUCCESS);
+
+        cluster.ensureSame();
+
+        // apply something more
+        for (int i = 1; i < 100; i++) {
+            sendTestTaskAndWait(leader, i * 10, RaftError.SUCCESS);
+        }
+
+        Thread.sleep(1000);
+
+        // trigger leader snapshot
+        triggerLeaderSnapshot(cluster, leader);
+
+        // apply something more
+        for (int i = 100; i < 200; i++) {
+            sendTestTaskAndWait(leader, i * 10, RaftError.SUCCESS);
+        }
+        // trigger leader snapshot
+        triggerLeaderSnapshot(cluster, leader, 2);
+
+        // wait leader to compact logs
+        Thread.sleep(1000);
+
+        // add follower
+        final PeerId newPeer = peers.get(3);
+        final RaftOptions raftOptions = new RaftOptions();
+        raftOptions.setMaxByteCountPerRpc(128);
+        final boolean started = cluster.start(newPeer.getEndpoint(), false, 300, false, null, raftOptions);
+        assertTrue(started);
+
+        final CountDownLatch latch = new CountDownLatch(1);
+        leader.addPeer(newPeer, status -> {
+            assertTrue(status.toString(), status.isOk());
+            latch.countDown();
+        });
+        waitLatch(latch);
+
+        cluster.ensureSame();
+
+        assertEquals(4, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(2000, fsm.getLogs().size());
+        }
+    }
+
+    @Test
+    @Ignore("https://issues.apache.org/jira/browse/IGNITE-14853")
+    public void testInstallSnapshot() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+        // get leader
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        // apply tasks to leader
+        this.sendTestTaskAndWait(leader);
+
+        cluster.ensureSame();
+
+        // stop follower1
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+
+        final Endpoint followerAddr = followers.get(0).getNodeId().getPeerId().getEndpoint();
+        assertTrue(cluster.stop(followerAddr));
+
+        // apply something more
+        sendTestTaskAndWait(leader, 10, RaftError.SUCCESS);
+
+        // trigger leader snapshot
+        triggerLeaderSnapshot(cluster, leader);
+        // apply something more
+        sendTestTaskAndWait(leader, 20, RaftError.SUCCESS);
+        triggerLeaderSnapshot(cluster, leader, 2);
+
+        // wait leader to compact logs
+        Thread.sleep(50);
+
+        //restart follower.
+        cluster.clean(followerAddr);
+        assertTrue(cluster.start(followerAddr, false, 300));
+
+        cluster.ensureSame();
+
+        assertEquals(3, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(fsm.getAddress().toString(), 30, fsm.getLogs().size());
+        }
+    }
+
+    @Test
+    public void testNoSnapshot() throws Exception {
+        final Endpoint addr = new Endpoint(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        final NodeOptions nodeOptions = createNodeOptions();
+        final MockStateMachine fsm = new MockStateMachine(addr);
+        nodeOptions.setFsm(fsm);
+        nodeOptions.setLogUri(this.dataPath + File.separator + "log");
+        nodeOptions.setRaftMetaUri(this.dataPath + File.separator + "meta");
+        nodeOptions.setInitialConf(new Configuration(Collections.singletonList(new PeerId(addr, 0))));
+
+        RaftGroupService service = createService("unittest", new PeerId(addr, 0), nodeOptions);
+        final Node node = service.start();
+        // wait node elect self as leader
+
+        Thread.sleep(2000);
+
+        this.sendTestTaskAndWait(node);
+
+        assertEquals(0, fsm.getSaveSnapshotTimes());
+        // do snapshot but returns error
+        CountDownLatch latch = new CountDownLatch(1);
+        node.snapshot(new ExpectClosure(RaftError.EINVAL, "Snapshot is not supported", latch));
+        waitLatch(latch);
+        assertEquals(0, fsm.getSaveSnapshotTimes());
+
+        service.shutdown();
+    }
+
+    @Test
+    public void testAutoSnapshot() throws Exception {
+        final Endpoint addr = new Endpoint(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        final NodeOptions nodeOptions = createNodeOptions();
+        final MockStateMachine fsm = new MockStateMachine(addr);
+        nodeOptions.setFsm(fsm);
+        nodeOptions.setLogUri(this.dataPath + File.separator + "log");
+        nodeOptions.setSnapshotUri(this.dataPath + File.separator + "snapshot");
+        nodeOptions.setRaftMetaUri(this.dataPath + File.separator + "meta");
+        nodeOptions.setSnapshotIntervalSecs(10);
+        nodeOptions.setInitialConf(new Configuration(Collections.singletonList(new PeerId(addr, 0))));
+
+        RaftGroupService service = createService("unittest", new PeerId(addr, 0), nodeOptions);
+        final Node node = service.start();
+        // wait node elect self as leader
+        Thread.sleep(2000);
+
+        sendTestTaskAndWait(node);
+
+        // wait for auto snapshot
+        Thread.sleep(10000);
+        // first snapshot will be triggered randomly
+        final int times = fsm.getSaveSnapshotTimes();
+        assertTrue("snapshotTimes=" + times, times >= 1);
+        assertTrue(fsm.getSnapshotIndex() > 0);
+
+        service.shutdown();
+    }
+
+    @Test
+    public void testLeaderShouldNotChange() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+        // get leader
+        final Node leader0 = cluster.getLeader();
+        assertNotNull(leader0);
+        final long savedTerm = ((NodeImpl) leader0).getCurrentTerm();
+        LOG.info("Current leader is {}, term is {}", leader0, savedTerm);
+        Thread.sleep(5000);
+        cluster.waitLeader();
+        final Node leader1 = cluster.getLeader();
+        assertNotNull(leader1);
+        LOG.info("Current leader is {}", leader1);
+        assertEquals(savedTerm, ((NodeImpl) leader1).getCurrentTerm());
+    }
+
+    @Test
+    public void testRecoverFollower() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+
+        // Ensure the quorum before stopping a follower, otherwise leader can step down.
+        assertTrue(waitForCondition(() -> followers.get(1).getLeaderId() != null, 5_000));
+
+        final Endpoint followerAddr = followers.get(0).getNodeId().getPeerId().getEndpoint().copy();
+        assertTrue(cluster.stop(followerAddr));
+
+        this.sendTestTaskAndWait(leader);
+
+        for (int i = 10; i < 30; i++) {
+            final ByteBuffer data = ByteBuffer.wrap(("no cluster" + i).getBytes());
+            final Task task = new Task(data, null);
+            leader.apply(task);
+        }
+        // wait leader to compact logs
+        Thread.sleep(5000);
+        // restart follower
+        assertTrue(cluster.start(followerAddr));
+        cluster.ensureSame();
+        assertEquals(3, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(30, fsm.getLogs().size());
+        }
+    }
+
+    @Test
+    public void testLeaderTransfer() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers, 300);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+
+        Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        this.sendTestTaskAndWait(leader);
+
+        Thread.sleep(100);
+
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+
+        final PeerId targetPeer = followers.get(0).getNodeId().getPeerId().copy();
+        LOG.info("Transfer leadership from {} to {}", leader, targetPeer);
+        assertTrue(leader.transferLeadershipTo(targetPeer).isOk());
+        Thread.sleep(1000);
+        cluster.waitLeader();
+        leader = cluster.getLeader();
+        Assert.assertEquals(leader.getNodeId().getPeerId(), targetPeer);
+    }
+
+    @Test
+    public void testLeaderTransferBeforeLogIsCompleted() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers, 300);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), false, 1));
+        }
+
+        cluster.waitLeader();
+
+        Node leader = cluster.getLeader();
+        assertNotNull(leader);
+
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+
+        // Ensure the quorum before stopping a follower, otherwise leader can step down.
+        assertTrue(waitForCondition(() -> followers.get(1).getLeaderId() != null, 5_000));
+
+        final PeerId targetPeer = followers.get(0).getNodeId().getPeerId().copy();
+        assertTrue(cluster.stop(targetPeer.getEndpoint()));
+
+        this.sendTestTaskAndWait(leader);
+        LOG.info("Transfer leadership from {} to {}", leader, targetPeer);
+        assertTrue(leader.transferLeadershipTo(targetPeer).isOk());
+
+        final CountDownLatch latch = new CountDownLatch(1);
+        final Task task = new Task(ByteBuffer.wrap("aaaaa".getBytes()), new ExpectClosure(RaftError.EBUSY, latch));
+        leader.apply(task);
+        waitLatch(latch);
+
+        cluster.waitLeader();
+
+        assertTrue(cluster.start(targetPeer.getEndpoint()));
+
+        leader = cluster.getLeader();
+
+        Assert.assertNotEquals(targetPeer, leader.getNodeId().getPeerId());
+        cluster.ensureSame();
+    }
+
+    @Test
+    public void testLeaderTransferResumeOnFailure() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers, 300);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint(), false, 1));
+        }
+
+        cluster.waitLeader();
+
+        Node leader = cluster.getLeader();
+        assertNotNull(leader);
+
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+
+        final PeerId targetPeer = followers.get(0).getNodeId().getPeerId().copy();
+        assertTrue(cluster.stop(targetPeer.getEndpoint()));
+
+        this.sendTestTaskAndWait(leader);
+
+        assertTrue(leader.transferLeadershipTo(targetPeer).isOk());
+        final Node savedLeader = leader;
+        //try to apply task when transferring leadership
+        CountDownLatch latch = new CountDownLatch(1);
+        Task task = new Task(ByteBuffer.wrap("aaaaa".getBytes()), new ExpectClosure(RaftError.EBUSY, latch));
+        leader.apply(task);
+        waitLatch(latch);
+
+        Thread.sleep(100);
+        cluster.waitLeader();
+        leader = cluster.getLeader();
+        assertSame(leader, savedLeader);
+
+        // restart target peer
+        assertTrue(cluster.start(targetPeer.getEndpoint()));
+        Thread.sleep(100);
+        // retry apply task
+        latch = new CountDownLatch(1);
+        task = new Task(ByteBuffer.wrap("aaaaa".getBytes()), new ExpectClosure(latch));
+        leader.apply(task);
+        waitLatch(latch);
+
+        cluster.ensureSame();
+    }
+
+    /**
+     * mock state machine that fails to load snapshot.
+     */
+    static class MockFSM1 extends MockStateMachine {
+
+        MockFSM1() {
+            this(new Endpoint(Utils.IP_ANY, 0));
+        }
+
+        MockFSM1(final Endpoint address) {
+            super(address);
+        }
+
+        @Override
+        public boolean onSnapshotLoad(final SnapshotReader reader) {
+            return false;
+        }
+    }
+
+    @Test
+    public void testShutdownAndJoinWorkAfterInitFails() throws Exception {
+        final Endpoint addr = new Endpoint(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        {
+            final NodeOptions nodeOptions = createNodeOptions();
+            final MockStateMachine fsm = new MockStateMachine(addr);
+            nodeOptions.setFsm(fsm);
+            nodeOptions.setLogUri(this.dataPath + File.separator + "log");
+            nodeOptions.setSnapshotUri(this.dataPath + File.separator + "snapshot");
+            nodeOptions.setRaftMetaUri(this.dataPath + File.separator + "meta");
+            nodeOptions.setSnapshotIntervalSecs(10);
+            nodeOptions.setInitialConf(new Configuration(Collections.singletonList(new PeerId(addr, 0))));
+
+            RaftGroupService service = createService("unittest", new PeerId(addr, 0), nodeOptions);
+            final Node node = service.start(true);
+
+            Thread.sleep(1000);
+            this.sendTestTaskAndWait(node);
+
+            // save snapshot
+            final CountDownLatch latch = new CountDownLatch(1);
+            node.snapshot(new ExpectClosure(latch));
+            waitLatch(latch);
+            service.shutdown();
+        }
+        {
+            final NodeOptions nodeOptions = createNodeOptions();
+            final MockStateMachine fsm = new MockFSM1(addr);
+            nodeOptions.setFsm(fsm);
+            nodeOptions.setLogUri(this.dataPath + File.separator + "log");
+            nodeOptions.setSnapshotUri(this.dataPath + File.separator + "snapshot");
+            nodeOptions.setRaftMetaUri(this.dataPath + File.separator + "meta");
+            nodeOptions.setSnapshotIntervalSecs(10);
+            nodeOptions.setInitialConf(new Configuration(Collections.singletonList(new PeerId(addr, 0))));
+
+            RaftGroupService service = createService("unittest", new PeerId(addr, 0), nodeOptions);
+            try {
+                service.start(true);
+
+                fail();
+            }
+            catch (Exception e) {
+                // Expected.
+            }
+            finally {
+                service.shutdown();
+            }
+
+        }
+    }
+
+    /**
+     * 4.2.2 Removing the current leader
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testShuttingDownLeaderTriggerTimeoutNow() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers, 300);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+
+        Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        final Node oldLeader = leader;
+
+        LOG.info("Shutdown leader {}", leader);
+        leader.shutdown();
+        leader.join();
+
+        cluster.waitLeader();
+        leader = cluster.getLeader();
+
+        assertNotNull(leader);
+        assertNotSame(leader, oldLeader);
+    }
+
+    @Test
+    public void testRemovingLeaderTriggerTimeoutNow() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers, 300);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+
+        // Ensure the quorum before removing a leader, otherwise removePeer can be rejected.
+        for (Node follower : cluster.getFollowers()) {
+            assertTrue(waitForCondition(() -> follower.getLeaderId() != null, 5_000));
+        }
+
+        Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        final Node oldLeader = leader;
+
+        final CountDownLatch latch = new CountDownLatch(1);
+        oldLeader.removePeer(oldLeader.getNodeId().getPeerId(), new ExpectClosure(latch));
+        waitLatch(latch);
+
+        cluster.waitLeader();
+        leader = cluster.getLeader();
+        assertNotNull(leader);
+        assertNotSame(leader, oldLeader);
+    }
+
+    @Test
+    public void testTransferShouldWorkAfterInstallSnapshot() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers, 1000);
+
+        for (int i = 0; i < peers.size() - 1; i++) {
+            assertTrue(cluster.start(peers.get(i).getEndpoint()));
+        }
+
+        cluster.waitLeader();
+
+        Node leader = cluster.getLeader();
+        assertNotNull(leader);
+
+        this.sendTestTaskAndWait(leader);
+
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(1, followers.size());
+
+        final PeerId follower = followers.get(0).getNodeId().getPeerId();
+        assertTrue(leader.transferLeadershipTo(follower).isOk());
+        Thread.sleep(2000);
+        leader = cluster.getLeader();
+        Assert.assertEquals(follower, leader.getNodeId().getPeerId());
+
+        CountDownLatch latch = new CountDownLatch(1);
+        leader.snapshot(new ExpectClosure(latch));
+        waitLatch(latch);
+        latch = new CountDownLatch(1);
+        leader.snapshot(new ExpectClosure(latch));
+        waitLatch(latch);
+
+        // start the last peer which should be recover with snapshot.
+        final PeerId lastPeer = peers.get(2);
+        assertTrue(cluster.start(lastPeer.getEndpoint()));
+        Thread.sleep(5000);
+        assertTrue(leader.transferLeadershipTo(lastPeer).isOk());
+        Thread.sleep(2000);
+        leader = cluster.getLeader();
+        Assert.assertEquals(lastPeer, leader.getNodeId().getPeerId());
+        assertEquals(3, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(10, fsm.getLogs().size());
+        }
+    }
+
+    @Test
+    public void testAppendEntriesWhenFollowerIsInErrorState() throws Exception {
+        // start five nodes
+        final List<PeerId> peers = TestUtils.generatePeers(5);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers, 1000);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+        final Node oldLeader = cluster.getLeader();
+        assertNotNull(oldLeader);
+        // apply something
+        this.sendTestTaskAndWait(oldLeader);
+
+        // set one follower into error state
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(4, followers.size());
+        final Node errorNode = followers.get(0);
+        final PeerId errorPeer = errorNode.getNodeId().getPeerId().copy();
+        final Endpoint errorFollowerAddr = errorPeer.getEndpoint();
+        LOG.info("Set follower {} into error state", errorNode);
+        ((NodeImpl) errorNode).onError(new RaftException(EnumOutter.ErrorType.ERROR_TYPE_STATE_MACHINE, new Status(-1,
+            "Follower has something wrong.")));
+
+        // increase term  by stopping leader and electing a new leader again
+        final Endpoint oldLeaderAddr = oldLeader.getNodeId().getPeerId().getEndpoint().copy();
+        assertTrue(cluster.stop(oldLeaderAddr));
+        cluster.waitLeader();
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        LOG.info("Elect a new leader {}", leader);
+        // apply something again
+        this.sendTestTaskAndWait(leader, 10, RaftError.SUCCESS);
+
+        // stop error follower
+        Thread.sleep(20);
+        LOG.info("Stop error follower {}", errorNode);
+        assertTrue(cluster.stop(errorFollowerAddr));
+        // restart error and old leader
+        LOG.info("Restart error follower {} and old leader {}", errorFollowerAddr, oldLeaderAddr);
+
+        assertTrue(cluster.start(errorFollowerAddr));
+        assertTrue(cluster.start(oldLeaderAddr));
+        cluster.ensureSame();
+        assertEquals(5, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(20, fsm.getLogs().size());
+        }
+    }
+
+    @Test
+    public void testFollowerStartStopFollowing() throws Exception {
+        // start five nodes
+        final List<PeerId> peers = TestUtils.generatePeers(5);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers, 1000);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+        cluster.waitLeader();
+        final Node firstLeader = cluster.getLeader();
+        assertNotNull(firstLeader);
+        // apply something
+        this.sendTestTaskAndWait(firstLeader);
+
+        // assert follow times
+        final List<Node> firstFollowers = cluster.getFollowers();
+        assertEquals(4, firstFollowers.size());
+        for (final Node node : firstFollowers) {
+            assertTrue(waitForCondition(() -> ((MockStateMachine) node.getOptions().getFsm()).getOnStartFollowingTimes() == 1, 5_000));
+            assertEquals(0, ((MockStateMachine) node.getOptions().getFsm()).getOnStopFollowingTimes());
+        }
+
+        // stop leader and elect new one
+        final Endpoint fstLeaderAddr = firstLeader.getNodeId().getPeerId().getEndpoint();
+        assertTrue(cluster.stop(fstLeaderAddr));
+        cluster.waitLeader();
+        final Node secondLeader = cluster.getLeader();
+        assertNotNull(secondLeader);
+        this.sendTestTaskAndWait(secondLeader, 10, RaftError.SUCCESS);
+
+        // ensure start/stop following times
+        final List<Node> secondFollowers = cluster.getFollowers();
+        assertEquals(3, secondFollowers.size());
+        for (final Node node : secondFollowers) {
+            assertEquals(2, ((MockStateMachine) node.getOptions().getFsm()).getOnStartFollowingTimes());
+            assertEquals(1, ((MockStateMachine) node.getOptions().getFsm()).getOnStopFollowingTimes());
+        }
+
+        // transfer leadership to a follower
+        final PeerId targetPeer = secondFollowers.get(0).getNodeId().getPeerId().copy();
+        assertTrue(secondLeader.transferLeadershipTo(targetPeer).isOk());
+        Thread.sleep(100);
+        cluster.waitLeader();
+        final Node thirdLeader = cluster.getLeader();
+        Assert.assertEquals(targetPeer, thirdLeader.getNodeId().getPeerId());
+        this.sendTestTaskAndWait(thirdLeader, 20, RaftError.SUCCESS);
+
+        final List<Node> thirdFollowers = cluster.getFollowers();
+        assertEquals(3, thirdFollowers.size());
+        for (int i = 0; i < 3; i++) {
+            if (thirdFollowers.get(i).getNodeId().getPeerId().equals(secondLeader.getNodeId().getPeerId())) {
+                assertEquals(2,
+                    ((MockStateMachine) thirdFollowers.get(i).getOptions().getFsm()).getOnStartFollowingTimes());
+                assertEquals(1,
+                    ((MockStateMachine) thirdFollowers.get(i).getOptions().getFsm()).getOnStopFollowingTimes());
+                continue;
+            }
+            assertEquals(3, ((MockStateMachine) thirdFollowers.get(i).getOptions().getFsm()).getOnStartFollowingTimes());
+            assertEquals(2, ((MockStateMachine) thirdFollowers.get(i).getOptions().getFsm()).getOnStopFollowingTimes());
+        }
+
+        cluster.ensureSame();
+    }
+
+    @Test
+    public void readCommittedUserLog() throws Exception {
+        // setup cluster
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+
+        cluster = new TestCluster("unitest", this.dataPath, peers, 1000);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+        cluster.waitLeader();
+
+        final Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        this.sendTestTaskAndWait(leader);
+
+        // index == 1 is a CONFIGURATION log, so real_index will be 2 when returned.
+        UserLog userLog = leader.readCommittedUserLog(1);
+        assertNotNull(userLog);
+        assertEquals(2, userLog.getIndex());
+        assertEquals("hello0", new String(userLog.getData().array()));
+
+        // index == 5 is a DATA log(a user log)
+        userLog = leader.readCommittedUserLog(5);
+        assertNotNull(userLog);
+        assertEquals(5, userLog.getIndex());
+        assertEquals("hello3", new String(userLog.getData().array()));
+
+        // index == 15 is greater than last_committed_index
+        try {
+            assertNull(leader.readCommittedUserLog(15));
+            fail();
+        }
+        catch (final LogIndexOutOfBoundsException e) {
+            assertEquals(e.getMessage(), "Request index 15 is greater than lastAppliedIndex: 11");
+        }
+
+        // index == 0 invalid request
+        try {
+            assertNull(leader.readCommittedUserLog(0));
+            fail();
+        }
+        catch (final LogIndexOutOfBoundsException e) {
+            assertEquals(e.getMessage(), "Request index is invalid: 0");
+        }
+        LOG.info("Trigger leader snapshot");
+        CountDownLatch latch = new CountDownLatch(1);
+        leader.snapshot(new ExpectClosure(latch));
+        waitLatch(latch);
+
+        // remove and add a peer to add two CONFIGURATION logs
+        final List<Node> followers = cluster.getFollowers();
+        assertEquals(2, followers.size());
+        final Node testFollower = followers.get(0);
+        latch = new CountDownLatch(1);
+        leader.removePeer(testFollower.getNodeId().getPeerId(), new ExpectClosure(latch));
+        waitLatch(latch);
+        latch = new CountDownLatch(1);
+        leader.addPeer(testFollower.getNodeId().getPeerId(), new ExpectClosure(latch));
+        waitLatch(latch);
+
+        this.sendTestTaskAndWait(leader, 10, RaftError.SUCCESS);
+
+        // trigger leader snapshot for the second time, after this the log of index 1~11 will be deleted.
+        LOG.info("Trigger leader snapshot");
+        latch = new CountDownLatch(1);
+        leader.snapshot(new ExpectClosure(latch));
+        waitLatch(latch);
+        Thread.sleep(100);
+
+        // index == 5 log has been deleted in log_storage.
+        try {
+            leader.readCommittedUserLog(5);
+            fail();
+        }
+        catch (final LogNotFoundException e) {
+            assertEquals("User log is deleted at index: 5", e.getMessage());
+        }
+
+        // index == 12、index == 13、index=14、index=15 are 4 CONFIGURATION logs(joint consensus), so real_index will be 16 when returned.
+        userLog = leader.readCommittedUserLog(12);
+        assertNotNull(userLog);
+        assertEquals(16, userLog.getIndex());
+        assertEquals("hello10", new String(userLog.getData().array()));
+
+        // now index == 17 is a user log
+        userLog = leader.readCommittedUserLog(17);
+        assertNotNull(userLog);
+        assertEquals(17, userLog.getIndex());
+        assertEquals("hello11", new String(userLog.getData().array()));
+
+        cluster.ensureSame();
+        assertEquals(3, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(20, fsm.getLogs().size());
+            for (int i = 0; i < 20; i++) {
+                assertEquals("hello" + i, new String(fsm.getLogs().get(i).array()));
+            }
+        }
+    }
+
+    @Test
+    public void testBootStrapWithSnapshot() throws Exception {
+        final Endpoint addr = JRaftUtils.getEndPoint("127.0.0.1:5006");
+        final MockStateMachine fsm = new MockStateMachine(addr);
+
+        for (char ch = 'a'; ch <= 'z'; ch++) {
+            fsm.getLogs().add(ByteBuffer.wrap(new byte[] {(byte) ch}));
+        }
+
+        final BootstrapOptions opts = new BootstrapOptions();
+        opts.setServiceFactory(new DefaultJRaftServiceFactory());
+        opts.setLastLogIndex(fsm.getLogs().size());
+        opts.setRaftMetaUri(this.dataPath + File.separator + "meta");
+        opts.setLogUri(this.dataPath + File.separator + "log");
+        opts.setSnapshotUri(this.dataPath + File.separator + "snapshot");
+        opts.setGroupConf(JRaftUtils.getConfiguration("127.0.0.1:5006"));
+        opts.setFsm(fsm);
+
+        final NodeOptions nodeOpts = createNodeOptions();
+        opts.setNodeOptions(nodeOpts);
+
+        assertTrue(JRaftUtils.bootstrap(opts));
+
+        nodeOpts.setRaftMetaUri(this.dataPath + File.separator + "meta");
+        nodeOpts.setLogUri(this.dataPath + File.separator + "log");
+        nodeOpts.setSnapshotUri(this.dataPath + File.separator + "snapshot");
+        nodeOpts.setFsm(fsm);
+
+        RaftGroupService service = createService("test", new PeerId(addr, 0), nodeOpts);
+        try {
+            Node node = service.start(true);
+            assertEquals(26, fsm.getLogs().size());
+
+            for (int i = 0; i < 26; i++) {
+                assertEquals('a' + i, fsm.getLogs().get(i).get());
+            }
+
+            // Group configuration will be restored from snapshot meta.
+            while (!node.isLeader()) {
+                Thread.sleep(20);
+            }
+            this.sendTestTaskAndWait(node);
+            assertEquals(36, fsm.getLogs().size());
+        }
+        finally {
+            service.shutdown();
+        }
+    }
+
+    @Test
+    public void testBootStrapWithoutSnapshot() throws Exception {
+        final Endpoint addr = JRaftUtils.getEndPoint("127.0.0.1:5006");
+        final MockStateMachine fsm = new MockStateMachine(addr);
+
+        final BootstrapOptions opts = new BootstrapOptions();
+        opts.setServiceFactory(new DefaultJRaftServiceFactory());
+        opts.setLastLogIndex(0);
+        opts.setRaftMetaUri(this.dataPath + File.separator + "meta");
+        opts.setLogUri(this.dataPath + File.separator + "log");
+        opts.setSnapshotUri(this.dataPath + File.separator + "snapshot");
+        opts.setGroupConf(JRaftUtils.getConfiguration("127.0.0.1:5006"));
+        opts.setFsm(fsm);
+        final NodeOptions nodeOpts = createNodeOptions();
+        opts.setNodeOptions(nodeOpts);
+
+        assertTrue(JRaftUtils.bootstrap(opts));
+
+        nodeOpts.setRaftMetaUri(this.dataPath + File.separator + "meta");
+        nodeOpts.setLogUri(this.dataPath + File.separator + "log");
+        nodeOpts.setSnapshotUri(this.dataPath + File.separator + "snapshot");
+        nodeOpts.setFsm(fsm);
+
+        RaftGroupService service = createService("test", new PeerId(addr, 0), nodeOpts);
+        try {
+            Node node = service.start(true);
+            while (!node.isLeader()) {
+                Thread.sleep(20);
+            }
+            this.sendTestTaskAndWait(node);
+            assertEquals(10, fsm.getLogs().size());
+        }
+        finally {
+            service.shutdown();
+        }
+    }
+
+    @Test
+    @Ignore("https://issues.apache.org/jira/browse/IGNITE-14852")
+    public void testChangePeers() throws Exception {
+        final PeerId peer0 = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        cluster = new TestCluster("testChangePeers", this.dataPath, Collections.singletonList(peer0));
+        assertTrue(cluster.start(peer0.getEndpoint()));
+
+        cluster.waitLeader();
+        Node leader = cluster.getLeader();
+        this.sendTestTaskAndWait(leader);
+
+        for (int i = 1; i < 10; i++) {
+            final PeerId peer = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + i);
+            assertTrue(cluster.start(peer.getEndpoint(), false, 300));
+        }
+        for (int i = 0; i < 9; i++) {
+            cluster.waitLeader();
+            leader = cluster.getLeader();
+            assertNotNull(leader);
+            PeerId peer = new PeerId(TestUtils.getMyIp(), peer0.getEndpoint().getPort() + i);
+            Assert.assertEquals(peer, leader.getNodeId().getPeerId());
+            peer = new PeerId(TestUtils.getMyIp(), peer0.getEndpoint().getPort() + i + 1);
+            final SynchronizedClosure done = new SynchronizedClosure();
+            leader.changePeers(new Configuration(Collections.singletonList(peer)), done);
+            Status status = done.await();
+            assertTrue(status.getRaftError().toString(), status.isOk());
+        }
+        cluster.ensureSame();
+    }
+
+    @Test
+    public void testChangePeersAddMultiNodes() throws Exception {
+        final PeerId peer0 = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT);
+        cluster = new TestCluster("testChangePeers", this.dataPath, Collections.singletonList(peer0));
+        assertTrue(cluster.start(peer0.getEndpoint()));
+
+        cluster.waitLeader();
+        final Node leader = cluster.getLeader();
+        this.sendTestTaskAndWait(leader);
+
+        final Configuration conf = new Configuration();
+        for (int i = 0; i < 3; i++) {
+            final PeerId peer = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + i);
+            conf.addPeer(peer);
+        }
+
+        PeerId peer = new PeerId(TestUtils.getMyIp(), peer0.getEndpoint().getPort() + 1);
+        // fail, because the peers are not started.
+        final SynchronizedClosure done = new SynchronizedClosure();
+        leader.changePeers(new Configuration(Collections.singletonList(peer)), done);
+        Assert.assertEquals(RaftError.ECATCHUP, done.await().getRaftError());
+
+        // start peer1
+        assertTrue(cluster.start(peer.getEndpoint()));
+        // still fail, because peer2 is not started
+        done.reset();
+        leader.changePeers(conf, done);
+        Assert.assertEquals(RaftError.ECATCHUP, done.await().getRaftError());
+        // start peer2
+        peer = new PeerId(TestUtils.getMyIp(), peer0.getEndpoint().getPort() + 2);
+        assertTrue(cluster.start(peer.getEndpoint()));
+        done.reset();
+        // works
+        leader.changePeers(conf, done);
+        assertTrue(done.await().isOk());
+
+        cluster.ensureSame();
+        assertEquals(3, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertEquals(10, fsm.getLogs().size());
+        }
+    }
+
+    @Test
+    public void testChangePeersStepsDownInJointConsensus() throws Exception {
+        final List<PeerId> peers = new ArrayList<>();
+
+        final PeerId peer0 = JRaftUtils.getPeerId(TestUtils.getMyIp() + ":5006");
+        final PeerId peer1 = JRaftUtils.getPeerId(TestUtils.getMyIp() + ":5007");
+        final PeerId peer2 = JRaftUtils.getPeerId(TestUtils.getMyIp() + ":5008");
+        final PeerId peer3 = JRaftUtils.getPeerId(TestUtils.getMyIp() + ":5009");
+
+        // start single cluster
+        peers.add(peer0);
+        cluster = new TestCluster("testChangePeersStepsDownInJointConsensus", this.dataPath, peers);
+        assertTrue(cluster.start(peer0.getEndpoint()));
+
+        cluster.waitLeader();
+        Node leader = cluster.getLeader();
+        assertNotNull(leader);
+        this.sendTestTaskAndWait(leader);
+
+        // start peer1-3
+        assertTrue(cluster.start(peer1.getEndpoint()));
+        assertTrue(cluster.start(peer2.getEndpoint()));
+        assertTrue(cluster.start(peer3.getEndpoint()));
+
+        // Make sure the topology is ready before adding peers.
+        assertTrue(waitForTopology(cluster, leader.getNodeId().getPeerId().getEndpoint(), 4, 3_000));
+
+        final Configuration conf = new Configuration();
+        conf.addPeer(peer0);
+        conf.addPeer(peer1);
+        conf.addPeer(peer2);
+        conf.addPeer(peer3);
+
+        // change peers
+        final SynchronizedClosure done = new SynchronizedClosure();
+        leader.changePeers(conf, done);
+        assertTrue(done.await().isOk());
+
+        // stop peer3
+        assertTrue(cluster.stop(peer3.getEndpoint()));
+
+        conf.removePeer(peer0);
+        conf.removePeer(peer1);
+
+        // Change peers to [peer2, peer3], which must fail since peer3 is stopped
+        done.reset();
+        leader.changePeers(conf, done);
+        Assert.assertEquals(RaftError.EPERM, done.await().getRaftError());
+        LOG.info(done.getStatus().toString());
+
+        assertFalse(((NodeImpl) leader).getConf().isStable());
+
+        leader = cluster.getLeader();
+        assertNull(leader);
+
+        assertTrue(cluster.start(peer3.getEndpoint()));
+        Thread.sleep(1000);
+        cluster.waitLeader();
+        leader = cluster.getLeader();
+        final List<PeerId> thePeers = leader.listPeers();
+        assertTrue(!thePeers.isEmpty());
+        assertEquals(conf.getPeerSet(), new HashSet<>(thePeers));
+    }
+
+    static class ChangeArg {
+        TestCluster c;
+        List<PeerId> peers;
+        volatile boolean stop;
+        boolean dontRemoveFirstPeer;
+
+        ChangeArg(final TestCluster c, final List<PeerId> peers, final boolean stop,
+            final boolean dontRemoveFirstPeer) {
+            super();
+            this.c = c;
+            this.peers = peers;
+            this.stop = stop;
+            this.dontRemoveFirstPeer = dontRemoveFirstPeer;
+        }
+
+    }
+
+    private Future<?> startChangePeersThread(final ChangeArg arg) {
+        final Set<RaftError> expectedErrors = new HashSet<>();
+        expectedErrors.add(RaftError.EBUSY);
+        expectedErrors.add(RaftError.EPERM);
+        expectedErrors.add(RaftError.ECATCHUP);
+
+        ExecutorService executor = Executors.newSingleThreadExecutor();
+
+        return Utils.runInThread(executor, () -> {
+            try {
+                while (!arg.stop) {
+                    arg.c.waitLeader();
+                    final Node leader = arg.c.getLeader();
+                    if (leader == null) {
+                        continue;
+                    }
+                    // select peers in random
+                    final Configuration conf = new Configuration();
+                    if (arg.dontRemoveFirstPeer) {
+                        conf.addPeer(arg.peers.get(0));
+                    }
+                    for (int i = 0; i < arg.peers.size(); i++) {
+                        final boolean select = ThreadLocalRandom.current().nextInt(64) < 32;
+                        if (select && !conf.contains(arg.peers.get(i))) {
+                            conf.addPeer(arg.peers.get(i));
+                        }
+                    }
+                    if (conf.isEmpty()) {
+                        LOG.warn("No peer has been selected");
+                        continue;
+                    }
+                    final SynchronizedClosure done = new SynchronizedClosure();
+                    leader.changePeers(conf, done);
+                    done.await();
+                    assertTrue(done.getStatus().toString(),
+                        done.getStatus().isOk() || expectedErrors.contains(done.getStatus().getRaftError()));
+                }
+            }
+            catch (final InterruptedException e) {
+                LOG.error("ChangePeersThread is interrupted", e);
+            }
+        });
+    }
+
+    @Test
+    public void testChangePeersChaosWithSnapshot() throws Exception {
+        // start cluster
+        final List<PeerId> peers = new ArrayList<>();
+        peers.add(new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT));
+        cluster = new TestCluster("unittest", this.dataPath, peers, 1000);
+        assertTrue(cluster.start(peers.get(0).getEndpoint(), false, 2));
+        // start other peers
+        for (int i = 1; i < 10; i++) {
+            final PeerId peer = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + i);
+            peers.add(peer);
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        final ChangeArg arg = new ChangeArg(cluster, peers, false, false);
+
+        final Future<?> future = startChangePeersThread(arg);
+        for (int i = 0; i < 5000; ) {
+            cluster.waitLeader();
+            final Node leader = cluster.getLeader();
+            if (leader == null) {
+                continue;
+            }
+            final SynchronizedClosure done = new SynchronizedClosure();
+            final Task task = new Task(ByteBuffer.wrap(("hello" + i).getBytes()), done);
+            leader.apply(task);
+            final Status status = done.await();
+            if (status.isOk()) {
+                if (++i % 100 == 0) {
+                    System.out.println("Progress:" + i);
+                }
+            }
+            else {
+                assertEquals(RaftError.EPERM, status.getRaftError());
+            }
+        }
+        arg.stop = true;
+        future.get();
+        cluster.waitLeader();
+        final SynchronizedClosure done = new SynchronizedClosure();
+        final Node leader = cluster.getLeader();
+        leader.changePeers(new Configuration(peers), done);
+        final Status st = done.await();
+        assertTrue(st.getErrorMsg(), st.isOk());
+        cluster.ensureSame();
+        assertEquals(10, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertTrue(fsm.getLogs().size() >= 5000);
+        }
+    }
+
+    @Test
+    public void testChangePeersChaosWithoutSnapshot() throws Exception {
+        // start cluster
+        final List<PeerId> peers = new ArrayList<>();
+        peers.add(new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT));
+        cluster = new TestCluster("unittest", this.dataPath, peers, 1000);
+        assertTrue(cluster.start(peers.get(0).getEndpoint(), false, 100000));
+        // start other peers
+        for (int i = 1; i < 10; i++) {
+            final PeerId peer = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + i);
+            peers.add(peer);
+            assertTrue(cluster.start(peer.getEndpoint(), false, 10000));
+        }
+
+        final ChangeArg arg = new ChangeArg(cluster, peers, false, true);
+
+        final Future<?> future = startChangePeersThread(arg);
+        final int tasks = 5000;
+        for (int i = 0; i < tasks; ) {
+            cluster.waitLeader();
+            final Node leader = cluster.getLeader();
+            if (leader == null) {
+                continue;
+            }
+            final SynchronizedClosure done = new SynchronizedClosure();
+            final Task task = new Task(ByteBuffer.wrap(("hello" + i).getBytes()), done);
+            leader.apply(task);
+            final Status status = done.await();
+            if (status.isOk()) {
+                if (++i % 100 == 0) {
+                    System.out.println("Progress:" + i);
+                }
+            }
+            else {
+                assertEquals(RaftError.EPERM, status.getRaftError());
+            }
+        }
+        arg.stop = true;
+        future.get();
+        cluster.waitLeader();
+        final SynchronizedClosure done = new SynchronizedClosure();
+        final Node leader = cluster.getLeader();
+        leader.changePeers(new Configuration(peers), done);
+        assertTrue(done.await().isOk());
+        cluster.ensureSame();
+        assertEquals(10, cluster.getFsms().size());
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            assertTrue(fsm.getLogs().size() >= tasks);
+            assertTrue(fsm.getLogs().size() - tasks < 100);
+        }
+    }
+
+    @Test
+    public void testChangePeersChaosApplyTasks() throws Exception {
+        // start cluster
+        final List<PeerId> peers = new ArrayList<>();
+        peers.add(new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT));
+        cluster = new TestCluster("unittest", this.dataPath, peers, 1000);
+        assertTrue(cluster.start(peers.get(0).getEndpoint(), false, 100000));
+        // start other peers
+        for (int i = 1; i < 10; i++) {
+            final PeerId peer = new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + i);
+            peers.add(peer);
+            assertTrue(cluster.start(peer.getEndpoint(), false, 100000));
+        }
+
+        final int threads = 3;
+        final List<ChangeArg> args = new ArrayList<>();
+        final List<Future<?>> futures = new ArrayList<>();
+        final CountDownLatch latch = new CountDownLatch(threads);
+
+        Executor executor = Executors.newFixedThreadPool(threads);
+
+        for (int t = 0; t < threads; t++) {
+            final ChangeArg arg = new ChangeArg(cluster, peers, false, true);
+            args.add(arg);
+            futures.add(startChangePeersThread(arg));
+
+            Utils.runInThread(executor, () -> {
+                try {
+                    for (int i = 0; i < 5000; ) {
+                        cluster.waitLeader();
+                        final Node leader = cluster.getLeader();
+                        if (leader == null) {
+                            continue;
+                        }
+                        final SynchronizedClosure done = new SynchronizedClosure();
+                        final Task task = new Task(ByteBuffer.wrap(("hello" + i).getBytes()), done);
+                        leader.apply(task);
+                        final Status status = done.await();
+                        if (status.isOk()) {
+                            if (++i % 100 == 0) {
+                                System.out.println("Progress:" + i);
+                            }
+                        }
+                        else {
+                            assertEquals(RaftError.EPERM, status.getRaftError());
+                        }
+                    }
+                }
+                catch (final Exception e) {
+                    LOG.error("Failed to run tasks", e);
+                }
+                finally {
+                    latch.countDown();
+                }
+            });
+        }
+
+        latch.await();
+        for (final ChangeArg arg : args) {
+            arg.stop = true;
+        }
+        for (final Future<?> future : futures) {
+            future.get();
+        }
+
+        cluster.waitLeader();
+        final SynchronizedClosure done = new SynchronizedClosure();
+        final Node leader = cluster.getLeader();
+        leader.changePeers(new Configuration(peers), done);
+        assertTrue(done.await().isOk());
+        cluster.ensureSame();
+        assertEquals(10, cluster.getFsms().size());
+
+        for (final MockStateMachine fsm : cluster.getFsms()) {
+            final int logSize = fsm.getLogs().size();
+            assertTrue("logSize= " + logSize, logSize >= 5000 * threads);
+            assertTrue("logSize= " + logSize, logSize - 5000 * threads < 100);
+        }
+    }
+
+    @Test
+    public void testBlockedElection() throws Exception {
+        final List<PeerId> peers = TestUtils.generatePeers(3);
+        cluster = new TestCluster("unittest", this.dataPath, peers);
+
+        for (final PeerId peer : peers) {
+            assertTrue(cluster.start(peer.getEndpoint()));
+        }
+
+        cluster.waitLeader();
+
+        Node leader = cluster.getLeader();
+
+        LOG.warn("Current leader {}, electTimeout={}", leader.getNodeId().getPeerId(), ((NodeImpl) leader).getOptions().getElectionTimeoutMs());
+
+        List<Node> followers = cluster.getFollowers();
+
+        for (Node follower : followers) {
+            NodeImpl follower0 = (NodeImpl) follower;
+            DefaultRaftClientService rpcService = (DefaultRaftClientService) follower0.getRpcClientService();
+            RpcClientEx rpcClientEx = (RpcClientEx) rpcService.getRpcClient();
+            rpcClientEx.blockMessages(new BiPredicate<Object, String>() {
+                @Override public boolean test(Object msg, String nodeId) {
+                    if (msg instanceof RpcRequests.RequestVoteRequest) {
+                        RpcRequests.RequestVoteRequest msg0 = (RpcRequests.RequestVoteRequest) msg;
+
+                        return !msg0.getPreVote();
+                    }
+
+                    return false;
+                }
+            });
+        }
+
+        LOG.warn("Stop leader {}, curTerm={}", leader.getNodeId().getPeerId(), ((NodeImpl) leader).getCurrentTerm());
+
+        assertTrue(cluster.stop(leader.getNodeId().getPeerId().getEndpoint()));
+
+        assertNull(cluster.getLeader());
+
+        Thread.sleep(3000);
+
+        assertNull(cluster.getLeader());
+
+        for (Node follower : followers) {
+            NodeImpl follower0 = (NodeImpl) follower;
+            DefaultRaftClientService rpcService = (DefaultRaftClientService) follower0.getRpcClientService();
+            RpcClientEx rpcClientEx = (RpcClientEx) rpcService.getRpcClient();
+            rpcClientEx.stopBlock();
+        }
+
+        // elect new leader
+        cluster.waitLeader();
+        leader = cluster.getLeader();
+        LOG.info("Elect new leader is {}, curTerm={}", leader.getLeaderId(), ((NodeImpl) leader).getCurrentTerm());
+    }
+
+    private NodeOptions createNodeOptions() {
+        final NodeOptions options = new NodeOptions();
+
+        options.setCommonExecutor(JRaftUtils.createCommonExecutor(options));
+        options.setStripedExecutor(JRaftUtils.createAppendEntriesExecutor(options));
+
+        return options;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    private boolean waitForTopology(TestCluster cluster, Endpoint addr, int expected, long timeout) {
+        RaftGroupService grp = cluster.getServer(addr);
+
+        if (grp == null) {
+            LOG.warn("Node has not been found {}", addr);
+
+            return false;
+        }
+
+        RpcServer rpcServer = grp.getRpcServer();
+
+        if (!(rpcServer instanceof IgniteRpcServer))
+            return true;
+
+        ClusterService service = ((IgniteRpcServer) grp.getRpcServer()).clusterService();
+
+        long stop = System.currentTimeMillis() + timeout;
+
+        while (System.currentTimeMillis() < stop) {
+            if (service.topologyService().allMembers().size() >= expected)
+                return true;
+
+            try {
+                Thread.sleep(50);
+            }
+            catch (InterruptedException e) {
+                return false;
+            }
+        }
+
+        return false;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    private boolean waitForCondition(BooleanSupplier cond, long timeout) {
+        long stop = System.currentTimeMillis() + timeout;
+
+        while (System.currentTimeMillis() < stop) {
+            if (cond.getAsBoolean())
+                return true;
+
+            try {
+                Thread.sleep(50);
+            }
+            catch (InterruptedException e) {
+                return false;
+            }
+        }
+
+        return false;
+    }
+
+    /**
+     * @param groupId Group id.
+     * @param peerId Peer id.
+     * @param nodeOptions Node options.
+     * @return Raft group service.
+     */
+    private RaftGroupService createService(String groupId, PeerId peerId, NodeOptions nodeOptions) {
+        NodeManager nodeManager = new NodeManager();
+
+        List<String> servers = new ArrayList<>();
+
+        Configuration initialConf = nodeOptions.getInitialConf();
+
+        if (initialConf != null) {
+            for (PeerId id : initialConf.getPeers()) {
+                servers.add(id.getEndpoint().toString());
+            }
+
+            for (PeerId id : initialConf.getLearners()) {
+                servers.add(id.getEndpoint().toString());
+            }
+        }
+
+        final IgniteRpcServer rpcServer = new TestIgniteRpcServer(peerId.getEndpoint(), servers, nodeManager);
+        nodeOptions.setRpcClient(new IgniteRpcClient(rpcServer.clusterService(), true));
+
+        return new RaftGroupService(groupId, peerId, nodeOptions, rpcServer, nodeManager);
+    }
+}
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/CounterListener.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/CounterListener.java
index ffa7021..c692a66 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/CounterListener.java
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/CounterListener.java
@@ -17,19 +17,39 @@
 
 package org.apache.ignite.raft.server;
 
+import java.io.File;
+import java.io.IOException;
 import java.util.Iterator;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.raft.client.ReadCommand;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.apache.ignite.raft.client.service.CommandClosure;
 import org.apache.ignite.raft.client.service.RaftGroupListener;
+import org.apache.ignite.raft.jraft.util.Utils;
 
-/** */
+/**
+ * The counter listener implementation.
+ */
 public class CounterListener implements RaftGroupListener {
-    /** */
+    /**
+     * The logger.
+     */
+    private static final IgniteLogger LOG = IgniteLogger.forClass(CounterListener.class);
+
+    /**
+     * The counter.
+     */
     private AtomicLong counter = new AtomicLong();
 
+    /**
+     * Snapshot executor.
+     */
+    private Executor executor = Executors.newSingleThreadExecutor();
+
     /** {@inheritDoc} */
     @Override public void onRead(Iterator<CommandClosure<ReadCommand>> iterator) {
         while (iterator.hasNext()) {
@@ -52,12 +72,41 @@ public class CounterListener implements RaftGroupListener {
         }
     }
 
+    /** {@inheritDoc} */
     @Override public void onSnapshotSave(String path, Consumer<Throwable> doneClo) {
-        // Not implemented.
+        final long currVal = this.counter.get();
+
+        Utils.runInThread(executor, () -> {
+            final CounterSnapshotFile snapshot = new CounterSnapshotFile(path + File.separator + "data");
+
+            try {
+                snapshot.save(currVal);
+
+                doneClo.accept(null);
+            }
+            catch (Throwable e) {
+                doneClo.accept(e);
+            }
+        });
     }
 
+    /** {@inheritDoc} */
     @Override public boolean onSnapshotLoad(String path) {
-        // Not implemented.
-        return false;
+        final CounterSnapshotFile snapshot = new CounterSnapshotFile(path + File.separator + "data");
+        try {
+            this.counter.set(snapshot.load());
+            return true;
+        }
+        catch (final IOException e) {
+            LOG.error("Fail to load snapshot from {}", snapshot.getPath());
+            return false;
+        }
+    }
+
+    /**
+     * @return Current value.
+     */
+    public long value() {
+        return counter.get();
     }
 }
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/CounterSnapshotFile.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/CounterSnapshotFile.java
new file mode 100644
index 0000000..4cac0f1
--- /dev/null
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/CounterSnapshotFile.java
@@ -0,0 +1,81 @@
+/*
+ * 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.ignite.raft.server;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.raft.jraft.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Counter snapshot file.
+ */
+class CounterSnapshotFile {
+    /**
+     * The logger.
+     */
+    private static final Logger LOG = LoggerFactory.getLogger(CounterSnapshotFile.class);
+
+    /**
+     * The path.
+     */
+    private String path;
+
+    /**
+     * @param path The path.
+     */
+    CounterSnapshotFile(String path) {
+        super();
+        this.path = path;
+    }
+
+    /**
+     * @return The path.
+     */
+    public String getPath() {
+        return this.path;
+    }
+
+    /**
+     * Save value to snapshot file.
+     * @param value The value.
+     */
+    public void save(final long value) throws IOException {
+        try {
+            Files.writeString(new File(path).toPath(), String.valueOf(value));
+        }
+        catch (IOException e) {
+            LOG.error("Fail to save snapshot", e);
+
+            throw e;
+        }
+    }
+
+    /**
+     * @return The loaded counter value.
+     * @throws IOException If failed.
+     */
+    public long load() throws IOException {
+        final String s = Files.readString(new File(path).toPath());
+        if (!StringUtils.isBlank(s)) {
+            return Long.parseLong(s);
+        }
+        throw new IOException("Fail to load snapshot from " + path + ",content: " + s);
+    }
+}
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java
index 96948a3..38f8226 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java
@@ -19,6 +19,8 @@ package org.apache.ignite.raft.server;
 
 import org.apache.ignite.raft.client.ReadCommand;
 
-/** */
+/**
+ * Get a value command.
+ */
 public class GetValueCommand implements ReadCommand {
 }
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITJRaftCounterServerTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITJRaftCounterServerTest.java
new file mode 100644
index 0000000..b639566
--- /dev/null
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITJRaftCounterServerTest.java
@@ -0,0 +1,565 @@
+/*
+ * 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.ignite.raft.server;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import org.apache.ignite.internal.raft.server.RaftServer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.ClusterServiceFactory;
+import org.apache.ignite.network.scalecube.TestScaleCubeClusterServiceFactory;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.WriteCommand;
+import org.apache.ignite.raft.client.exception.RaftException;
+import org.apache.ignite.raft.client.message.RaftClientMessagesFactory;
+import org.apache.ignite.raft.client.service.CommandClosure;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.apache.ignite.raft.client.service.impl.RaftGroupServiceImpl;
+import org.apache.ignite.raft.jraft.test.TestUtils;
+import org.apache.ignite.raft.jraft.util.Utils;
+import org.apache.ignite.internal.raft.server.impl.JRaftServerImpl;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+
+import static org.apache.ignite.raft.jraft.test.TestUtils.waitForCondition;
+import static org.apache.ignite.raft.jraft.test.TestUtils.waitForTopology;
+import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/**
+ * Jraft server.
+ */
+class ITJRaftCounterServerTest extends RaftServerAbstractTest {
+    /**
+     * The logger.
+     */
+    private static final IgniteLogger LOG = IgniteLogger.forClass(ITJRaftCounterServerTest.class);
+
+    /**
+     * Message factory.
+     */
+    private static final RaftClientMessagesFactory FACTORY = new RaftClientMessagesFactory();
+
+    /** Network factory. */
+    private static final ClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
+
+    /**
+     * Counter group name 0.
+     */
+    private static final String COUNTER_GROUP_0 = "counter0";
+
+    /**
+     * Counter group name 1.
+     */
+    private static final String COUNTER_GROUP_1 = "counter1";
+
+    /**
+     * The server port offset.
+     */
+    private static final int PORT = 5003;
+
+    /**
+     * The client port offset.
+     */
+    private static final int CLIENT_PORT = 6003;
+
+    /**
+     * Initial configuration.
+     */
+    private static final List<Peer> INITIAL_CONF = List.of(
+        new Peer(TestUtils.getMyIp() + ":" + PORT),
+        new Peer(TestUtils.getMyIp() + ":" + (PORT + 1)),
+        new Peer(TestUtils.getMyIp() + ":" + (PORT + 2)));
+
+    /**
+     * Listener factory.
+     */
+    private Supplier<CounterListener> listenerFactory = () -> new CounterListener();
+
+    /**
+     * Servers list.
+     */
+    protected List<JRaftServerImpl> servers = new ArrayList<>();
+
+    /**
+     * Clients list.
+     */
+    protected List<RaftGroupService> clients = new ArrayList<>();
+
+    /**
+     * Data path.
+     */
+    private String dataPath;
+
+    @BeforeEach
+    void before(TestInfo testInfo) {
+        LOG.info(">>>>>>>>>>>>>>> Start test method: " + testInfo.getTestMethod().orElseThrow().getName());
+
+        this.dataPath = TestUtils.mkTempDir();
+    }
+
+    @AfterEach
+    void after(TestInfo testInfo) throws Exception {
+        LOG.info("Start client shutdown");
+
+        for (RaftGroupService client : clients) {
+            client.shutdown();
+        }
+
+        LOG.info("Start server shutdown servers={}", servers.size());
+
+        for (RaftServer server : servers)
+            server.shutdown();
+
+        assertTrue("Failed to delete " + this.dataPath, Utils.delete(new File(this.dataPath)));
+
+        LOG.info(">>>>>>>>>>>>>>> End test method: " + testInfo.getTestMethod().orElseThrow().getName());
+    }
+
+    /**
+     * @param idx The index.
+     * @return Raft server instance.
+     */
+    protected JRaftServerImpl startServer(int idx, Consumer<RaftServer> clo) {
+        ClusterService service = clusterService("server" + idx, PORT + idx,
+            List.of(TestUtils.getMyIp() + ":" + PORT), false);
+
+        JRaftServerImpl server = new JRaftServerImpl(service, dataPath, FACTORY, false);
+
+        clo.accept(server);
+
+        servers.add(server);
+
+        assertTrue(waitForTopology(service, servers.size(), 5_000));
+
+        return server;
+    }
+
+    /**
+     * @param groupId Group id.
+     * @return The client.
+     */
+    protected RaftGroupService startClient(String groupId) {
+        String addr = TestUtils.getMyIp() + ":" + PORT;
+
+        ClusterService clientNode1 = clusterService("client_" + groupId + "_", CLIENT_PORT + clients.size(),
+            List.of(addr), false);
+
+        RaftGroupServiceImpl client = new RaftGroupServiceImpl(groupId, clientNode1, FACTORY, 10_000,
+            List.of(new Peer(addr)), false, 200, false);
+
+        clients.add(client);
+
+        return client;
+    }
+
+    /**
+     * Starts a cluster for the test.
+     */
+    private void startCluster() {
+        for (int i = 0; i < 3; i++) {
+            startServer(i, new Consumer<RaftServer>() {
+                @Override public void accept(RaftServer raftServer) {
+                    raftServer.startRaftGroup(COUNTER_GROUP_0, listenerFactory.get(), INITIAL_CONF);
+                    raftServer.startRaftGroup(COUNTER_GROUP_1, listenerFactory.get(), INITIAL_CONF);
+                }
+            });
+        }
+
+        startClient(COUNTER_GROUP_0);
+        startClient(COUNTER_GROUP_1);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testRefreshLeader() throws Exception {
+        startCluster();
+
+        Peer leader = clients.get(0).leader();
+
+        assertNull(leader);
+
+        clients.get(0).refreshLeader().get();
+
+        assertNotNull(clients.get(0).leader());
+
+        leader = clients.get(1).leader();
+
+        assertNull(leader);
+
+        clients.get(1).refreshLeader().get();
+
+        assertNotNull(clients.get(1).leader());
+    }
+
+    /**
+     * @throws Exception
+     */
+    @Test
+    public void testCounterCommandListener() throws Exception {
+        startCluster();
+
+        RaftGroupService client1 = clients.get(0);
+        RaftGroupService client2 = clients.get(1);
+
+        client1.refreshLeader().get();
+        client2.refreshLeader().get();
+
+        assertNotNull(client1.leader());
+        assertNotNull(client2.leader());
+
+        assertEquals(2, client1.<Long>run(new IncrementAndGetCommand(2)).get());
+        assertEquals(2, client1.<Long>run(new GetValueCommand()).get());
+        assertEquals(3, client1.<Long>run(new IncrementAndGetCommand(1)).get());
+        assertEquals(3, client1.<Long>run(new GetValueCommand()).get());
+
+        assertEquals(4, client2.<Long>run(new IncrementAndGetCommand(4)).get());
+        assertEquals(4, client2.<Long>run(new GetValueCommand()).get());
+        assertEquals(7, client2.<Long>run(new IncrementAndGetCommand(3)).get());
+        assertEquals(7, client2.<Long>run(new GetValueCommand()).get());
+    }
+
+    @Test
+    public void testCreateSnapshot() throws Exception {
+        startCluster();
+
+        RaftGroupService client1 = clients.get(0);
+        RaftGroupService client2 = clients.get(1);
+
+        client1.refreshLeader().get();
+        client2.refreshLeader().get();
+
+        JRaftServerImpl server = servers.get(0);
+
+        long val = applyIncrements(client1, 1, 10);
+
+        assertEquals(sum(10), val);
+
+        client1.snapshot(server.localPeer(COUNTER_GROUP_0)).get();
+
+        long val2 = applyIncrements(client2, 1, 20);
+
+        assertEquals(sum(20), val2);
+
+        client2.snapshot(server.localPeer(COUNTER_GROUP_1)).get();
+
+        String snapshotDir0 = server.getServerDataPath(COUNTER_GROUP_0) + File.separator + "snapshot";
+        assertEquals(1, new File(snapshotDir0).list().length);
+
+        String snapshotDir1 = server.getServerDataPath(COUNTER_GROUP_1) + File.separator + "snapshot";
+        assertEquals(1, new File(snapshotDir1).list().length);
+    }
+
+    @Test
+    public void testCreateSnapshotGracefulFailure() throws Exception {
+        listenerFactory = () -> new CounterListener() {
+            @Override public void onSnapshotSave(String path, Consumer<Throwable> doneClo) {
+                doneClo.accept(new IgniteInternalException("Very bad"));
+            }
+        };
+
+        startCluster();
+
+        RaftGroupService client1 = clients.get(0);
+        RaftGroupService client2 = clients.get(1);
+
+        client1.refreshLeader().get();
+        client2.refreshLeader().get();
+
+        RaftServer server = servers.get(0);
+
+        Peer peer = server.localPeer(COUNTER_GROUP_0);
+
+        long val = applyIncrements(client1, 1, 10);
+
+        assertEquals(sum(10), val);
+
+        try {
+            client1.snapshot(peer).get();
+
+            fail();
+        }
+        catch (Exception e) {
+            Assertions.assertTrue(e.getCause() instanceof RaftException);
+        }
+    }
+
+    @Test
+    public void testCreateSnapshotAbnormalFailure() throws Exception {
+        listenerFactory = () -> new CounterListener() {
+            @Override public void onSnapshotSave(String path, Consumer<Throwable> doneClo) {
+                doneClo.accept(new IgniteInternalException("Very bad"));
+            }
+        };
+
+        startCluster();
+
+        RaftGroupService client1 = clients.get(0);
+        RaftGroupService client2 = clients.get(1);
+
+        client1.refreshLeader().get();
+        client2.refreshLeader().get();
+
+        long val = applyIncrements(client1, 1, 10);
+
+        assertEquals(sum(10), val);
+
+        Peer peer = servers.get(0).localPeer(COUNTER_GROUP_0);
+
+        try {
+            client1.snapshot(peer).get();
+
+            fail();
+        }
+        catch (Exception e) {
+            Assertions.assertTrue(e.getCause() instanceof RaftException);
+        }
+    }
+
+    /** Tests if a raft group become unavaiable in case of a critical error */
+    @Test
+    public void testApplyWithFailure() throws Exception {
+        listenerFactory = () -> new CounterListener() {
+            @Override public void onWrite(Iterator<CommandClosure<WriteCommand>> iterator) {
+                Iterator<CommandClosure<WriteCommand>> wrapper = new Iterator<CommandClosure<WriteCommand>>() {
+                    @Override public boolean hasNext() {
+                        return iterator.hasNext();
+                    }
+
+                    @Override public CommandClosure<WriteCommand> next() {
+                        CommandClosure<WriteCommand> cmd = iterator.next();
+
+                        IncrementAndGetCommand command = (IncrementAndGetCommand) cmd.command();
+
+                        if (command.delta() == 10)
+                            throw new IgniteInternalException("Very bad");
+
+                        return cmd;
+                    }
+                };
+
+                super.onWrite(wrapper);
+            }
+        };
+
+        startCluster();
+
+        RaftGroupService client1 = clients.get(0);
+        RaftGroupService client2 = clients.get(1);
+
+        client1.refreshLeader().get();
+        client2.refreshLeader().get();
+
+        long val1 = applyIncrements(client1, 1, 5);
+        long val2 = applyIncrements(client2, 1, 7);
+
+        assertEquals(sum(5), val1);
+        assertEquals(sum(7), val2);
+
+        long val3 = applyIncrements(client1, 6, 9);
+        assertEquals(sum(9), val3);
+
+        try {
+            client1.<Long>run(new IncrementAndGetCommand(10)).get();
+
+            fail();
+        }
+        catch (Exception e) {
+            // Expected.
+            Throwable cause = e.getCause();
+
+            Assertions.assertTrue(cause instanceof RaftException);
+        }
+
+        try {
+            client1.<Long>run(new IncrementAndGetCommand(11)).get();
+        }
+        catch (Exception e) {
+            Assertions.assertTrue(e.getCause() instanceof TimeoutException, "New leader should not get elected");
+        }
+    }
+
+    /** Tests if a follower is catching up the leader after restarting. */
+    @Test
+    public void testFollowerCatchUpFromLog() throws Exception {
+        doTestFollowerCatchUp(false, true);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFollowerCatchUpFromSnapshot() throws Exception {
+        doTestFollowerCatchUp(true, true);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFollowerCatchUpFromLog2() throws Exception {
+        doTestFollowerCatchUp(false, false);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFollowerCatchUpFromSnapshot2() throws Exception {
+        doTestFollowerCatchUp(true, false);
+    }
+
+    /**
+     * @param snapshot {@code True} to create snapshot on leader and truncate log.
+     * @param cleanDir {@code True} to clean persistent state on follower before restart.
+     * @throws Exception If failed.
+     */
+    private void doTestFollowerCatchUp(boolean snapshot, boolean cleanDir) throws Exception {
+        startCluster();
+
+        RaftGroupService client1 = clients.get(0);
+        RaftGroupService client2 = clients.get(1);
+
+        client1.refreshLeader().get();
+        client2.refreshLeader().get();
+
+        Peer leader1 = client1.leader();
+        assertNotNull(leader1);
+
+        Peer leader2 = client2.leader();
+        assertNotNull(leader2);
+
+        applyIncrements(client1, 0, 10);
+        applyIncrements(client2, 0, 20);
+
+        // First snapshot will not truncate logs.
+        client1.snapshot(leader1).get();
+        client2.snapshot(leader2).get();
+
+        JRaftServerImpl toStop = null;
+
+        // Find the follower for both groups.
+        for (JRaftServerImpl server : servers) {
+            Peer peer = server.localPeer(COUNTER_GROUP_0);
+
+            if (!peer.equals(leader1) && !peer.equals(leader2)) {
+                toStop = server;
+                break;
+            }
+        }
+
+        String serverDataPath0 = toStop.getServerDataPath(COUNTER_GROUP_0);
+        String serverDataPath1 = toStop.getServerDataPath(COUNTER_GROUP_1);
+
+        int stopIdx = servers.indexOf(toStop);
+
+        servers.remove(stopIdx);
+
+        toStop.shutdown();
+
+        applyIncrements(client1, 11, 20);
+        applyIncrements(client2, 21, 30);
+
+        if (snapshot) {
+            client1.snapshot(leader1).get();
+            client2.snapshot(leader2).get();
+        }
+
+        if (cleanDir) {
+            Utils.delete(new File(serverDataPath0));
+            Utils.delete(new File(serverDataPath1));
+        }
+
+        var svc2 = startServer(stopIdx, r -> {
+            r.startRaftGroup(COUNTER_GROUP_0, listenerFactory.get(), INITIAL_CONF);
+            r.startRaftGroup(COUNTER_GROUP_1, listenerFactory.get(), INITIAL_CONF);
+        });
+
+        waitForCondition(() -> validateStateMachine(sum(20), svc2, COUNTER_GROUP_0), 5_000);
+        waitForCondition(() -> validateStateMachine(sum(30), svc2, COUNTER_GROUP_1), 5_000);
+
+        svc2.shutdown();
+
+        var svc3 = startServer(stopIdx, r -> {
+            r.startRaftGroup(COUNTER_GROUP_0, listenerFactory.get(), INITIAL_CONF);
+            r.startRaftGroup(COUNTER_GROUP_1, listenerFactory.get(), INITIAL_CONF);
+        });
+
+        waitForCondition(() -> validateStateMachine(sum(20), svc3, COUNTER_GROUP_0), 5_000);
+        waitForCondition(() -> validateStateMachine(sum(30), svc3, COUNTER_GROUP_1), 5_000);
+    }
+
+    /**
+     * @param client The client
+     * @param start Start element.
+     * @param stop Stop element.
+     * @return The counter value.
+     * @throws Exception If failed.
+     */
+    private long applyIncrements(RaftGroupService client, int start, int stop) throws Exception {
+        long val = 0;
+
+        for (int i = start; i <= stop; i++) {
+            val = client.<Long>run(new IncrementAndGetCommand(i)).get();
+
+            LOG.info("Val=" + val + ", i=" + i);
+        }
+
+        return val;
+    }
+
+    /**
+     * Calculates a progression sum.
+     *
+     * @param until Until value.
+     * @return The sum.
+     */
+    public long sum(long until) {
+        return (1 + until) * until / 2;
+    }
+
+    /**
+     * @param expected Expected value.
+     * @param server The server.
+     * @param groupId Group id.
+     * @return Validation result.
+     */
+    private boolean validateStateMachine(long expected, JRaftServerImpl server, String groupId) {
+        org.apache.ignite.raft.jraft.RaftGroupService svc = server.raftGroupService(groupId);
+
+        JRaftServerImpl.DelegatingStateMachine fsm0 =
+            (JRaftServerImpl.DelegatingStateMachine) svc.getRaftNode().getOptions().getFsm();
+
+        return expected == ((CounterListener) fsm0.getListener()).value();
+    }
+}
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITRaftCounterServerTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITSimpleCounterServerTest.java
similarity index 79%
rename from modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITRaftCounterServerTest.java
rename to modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITSimpleCounterServerTest.java
index 5aa0edf..6296836 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITRaftCounterServerTest.java
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITSimpleCounterServerTest.java
@@ -18,38 +18,52 @@
 package org.apache.ignite.raft.server;
 
 import java.util.List;
+import org.apache.ignite.internal.raft.server.RaftServer;
 import org.apache.ignite.network.ClusterNode;
 import org.apache.ignite.network.ClusterService;
 import org.apache.ignite.raft.client.Peer;
 import org.apache.ignite.raft.client.service.RaftGroupService;
 import org.apache.ignite.raft.client.service.impl.RaftGroupServiceImpl;
-import org.apache.ignite.raft.server.impl.RaftServerImpl;
+import org.apache.ignite.internal.raft.server.impl.RaftServerImpl;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.TestInfo;
 
+import static org.apache.ignite.raft.jraft.test.TestUtils.waitForTopology;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-/** */
-class ITRaftCounterServerTest extends RaftCounterServerAbstractTest {
-    /** */
-    protected RaftServer server;
+/**
+ * Single node raft server.
+ */
+class ITSimpleCounterServerTest extends RaftServerAbstractTest {
+    /**
+     * The server implementation.
+     */
+    private RaftServer server;
 
-    /** */
-    protected static final String COUNTER_GROUP_ID_0 = "counter0";
+    /**
+     * Counter raft group 0.
+     */
+    private static final String COUNTER_GROUP_ID_0 = "counter0";
 
-    /** */
-    protected static final String COUNTER_GROUP_ID_1 = "counter1";
+    /**
+     * Counter raft group 1.
+     */
+    private static final String COUNTER_GROUP_ID_1 = "counter1";
 
-    /** */
-    protected RaftGroupService client1;
+    /**
+     * The client 1.
+     */
+    private RaftGroupService client1;
 
-    /** */
-    protected RaftGroupService client2;
+    /**
+     * The client 2.
+     */
+    private RaftGroupService client2;
 
     /**
      * @param testInfo Test info.
@@ -71,18 +85,22 @@ class ITRaftCounterServerTest extends RaftCounterServerAbstractTest {
 
         ClusterService clientNode1 = clusterService("localhost:" + (PORT + 1), PORT + 1, List.of(id), false);
 
-        client1 = new RaftGroupServiceImpl(COUNTER_GROUP_ID_0, clientNode1, FACTORY, 1000, List.of(new Peer(serverNode.address())), false, 200, false);
+        client1 = new RaftGroupServiceImpl(COUNTER_GROUP_ID_0, clientNode1, FACTORY, 1000,
+            List.of(new Peer(serverNode.address())), false, 200, false);
 
         ClusterService clientNode2 = clusterService("localhost:" + (PORT + 2), PORT + 2, List.of(id), false);
 
-        client2 = new RaftGroupServiceImpl(COUNTER_GROUP_ID_1, clientNode2, FACTORY, 1000, List.of(new Peer(serverNode.address())), false, 200, false);
+        client2 = new RaftGroupServiceImpl(COUNTER_GROUP_ID_1, clientNode2, FACTORY, 1000,
+            List.of(new Peer(serverNode.address())), false, 200, false);
 
         assertTrue(waitForTopology(service, 2, 1000));
         assertTrue(waitForTopology(clientNode1, 2, 1000));
         assertTrue(waitForTopology(clientNode2, 2, 1000));
     }
 
-    /** */
+    /**
+     * @throws Exception
+     */
     @AfterEach
     void after() throws Exception {
         server.shutdown();
@@ -91,7 +109,7 @@ class ITRaftCounterServerTest extends RaftCounterServerAbstractTest {
     }
 
     /**
-     * Tests if a leader is fetched correctly.
+     *
      */
     @Test
     public void testRefreshLeader() throws Exception {
@@ -105,7 +123,7 @@ class ITRaftCounterServerTest extends RaftCounterServerAbstractTest {
     }
 
     /**
-     * Tests raft group listener.
+     * @throws Exception
      */
     @Test
     public void testCounterCommandListener() throws Exception {
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/IncrementAndGetCommand.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/IncrementAndGetCommand.java
index ed17aa3..766ba73 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/IncrementAndGetCommand.java
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/IncrementAndGetCommand.java
@@ -19,9 +19,13 @@ package org.apache.ignite.raft.server;
 
 import org.apache.ignite.raft.client.WriteCommand;
 
-/** */
+/**
+ * Increment and get command.
+ */
 public class IncrementAndGetCommand implements WriteCommand {
-    /** */
+    /**
+     * The delta.
+     */
     private final long delta;
 
     /**
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/RaftCounterServerAbstractTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/RaftServerAbstractTest.java
similarity index 66%
copy from modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/RaftCounterServerAbstractTest.java
copy to modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/RaftServerAbstractTest.java
index de78975..c81ee1d 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/RaftCounterServerAbstractTest.java
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/RaftServerAbstractTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.raft.server;
 
 import java.util.List;
-import java.util.function.BooleanSupplier;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.network.ClusterLocalConfiguration;
 import org.apache.ignite.network.ClusterService;
@@ -28,12 +27,12 @@ import org.apache.ignite.network.scalecube.TestScaleCubeClusterServiceFactory;
 import org.apache.ignite.network.serialization.MessageSerializationRegistry;
 import org.apache.ignite.raft.client.message.RaftClientMessagesFactory;
 
-import static java.lang.Thread.sleep;
-
-/** */
-abstract class RaftCounterServerAbstractTest {
+/**
+ * Abstract test for raft server.
+ */
+abstract class RaftServerAbstractTest {
     /** */
-    protected static final IgniteLogger LOG = IgniteLogger.forClass(RaftCounterServerAbstractTest.class);
+    protected static final IgniteLogger LOG = IgniteLogger.forClass(RaftServerAbstractTest.class);
 
     /** */
     protected static final RaftClientMessagesFactory FACTORY = new RaftClientMessagesFactory();
@@ -41,7 +40,9 @@ abstract class RaftCounterServerAbstractTest {
     /** Network factory. */
     protected static final ClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
 
-    /** */
+    /**
+     * Server port offset.
+     */
     protected static final int PORT = 20010;
 
     /** */
@@ -63,37 +64,4 @@ abstract class RaftCounterServerAbstractTest {
 
         return network;
     }
-
-    /**
-     * @param cluster The cluster.
-     * @param expected Expected count.
-     * @param timeout The timeout in millis.
-     * @return {@code True} if topology size is equal to expected.
-     */
-    protected boolean waitForTopology(ClusterService cluster, int expected, int timeout) {
-        return waitForCondition(() -> cluster.topologyService().allMembers().size() >= expected, timeout);
-    }
-
-    /**
-     * @param cond The condition.
-     * @param timeout The timeout.
-     * @return {@code True} if condition has happened within the timeout.
-     */
-    @SuppressWarnings("BusyWait") protected boolean waitForCondition(BooleanSupplier cond, long timeout) {
-        long stop = System.currentTimeMillis() + timeout;
-
-        while(System.currentTimeMillis() < stop) {
-            if (cond.getAsBoolean())
-                return true;
-
-            try {
-                sleep(50);
-            }
-            catch (InterruptedException e) {
-                return false;
-            }
-        }
-
-        return false;
-    }
 }
diff --git a/modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java b/modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java
index ea81304..c6dc324 100644
--- a/modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java
+++ b/modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java
@@ -26,8 +26,8 @@ import org.apache.ignite.raft.client.message.RaftClientMessagesFactory;
 import org.apache.ignite.raft.client.service.RaftGroupListener;
 import org.apache.ignite.raft.client.service.RaftGroupService;
 import org.apache.ignite.raft.client.service.impl.RaftGroupServiceImpl;
-import org.apache.ignite.raft.server.RaftServer;
-import org.apache.ignite.raft.server.impl.RaftServerImpl;
+import org.apache.ignite.internal.raft.server.RaftServer;
+import org.apache.ignite.internal.raft.server.impl.RaftServerImpl;
 
 /**
  * Best raft manager ever since 1982.
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/server/RaftServer.java b/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftServer.java
similarity index 88%
rename from modules/raft/src/main/java/org/apache/ignite/raft/server/RaftServer.java
rename to modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftServer.java
index 8cab2c2..65477ce 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/server/RaftServer.java
+++ b/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftServer.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.raft.server;
+package org.apache.ignite.internal.raft.server;
 
 import java.util.List;
 import org.apache.ignite.network.ClusterService;
@@ -24,12 +24,9 @@ import org.apache.ignite.raft.client.service.RaftGroupListener;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * The RAFT protocol based replication server.
- * <p>
- * Supports multiple RAFT groups.
- * <p>
- * The server listens for client commands, submits them to a replicated log and calls {@link RaftGroupListener}
- * {@code onRead} and {@code onWrite} methods then after the command was committed to the log.
+ * The RAFT protocol based replication server. Supports multiple RAFT groups.
+ * The server listens for client commands, submits them to a replicated log and calls
+ * {@link RaftGroupListener} {@code onRead} and {@code onWrite} methods after the command was committed to the log.
  */
 public interface RaftServer {
     /**
@@ -39,16 +36,17 @@ public interface RaftServer {
 
     /**
      * Starts a raft group bound to this cluster node.
+     *
      * @param groupId Group id.
      * @param lsnr The listener.
      * @param initialConf Inititial group configuration.
-     *
      * @return {@code True} if a group was successfully started.
      */
     boolean startRaftGroup(String groupId, RaftGroupListener lsnr, List<Peer> initialConf);
 
     /**
      * Synchronously stops a raft group.
+     *
      * @param groupId Group id.
      * @return {@code True} if a group was successfully stopped.
      */
@@ -56,6 +54,7 @@ public interface RaftServer {
 
     /**
      * Returns a local peer.
+     *
      * @param groupId Group id.
      * @return Local peer or null if the group is not started.
      */
diff --git a/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/impl/JRaftServerImpl.java b/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/impl/JRaftServerImpl.java
new file mode 100644
index 0000000..1d43aed
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/impl/JRaftServerImpl.java
@@ -0,0 +1,314 @@
+/*
+ * 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.ignite.internal.raft.server.impl;
+
+import java.io.File;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.raft.server.RaftServer;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.raft.client.ElectionPriority;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.WriteCommand;
+import org.apache.ignite.raft.client.message.RaftClientMessagesFactory;
+import org.apache.ignite.raft.client.service.CommandClosure;
+import org.apache.ignite.raft.client.service.RaftGroupListener;
+import org.apache.ignite.raft.jraft.Closure;
+import org.apache.ignite.raft.jraft.Iterator;
+import org.apache.ignite.raft.jraft.JRaftUtils;
+import org.apache.ignite.raft.jraft.NodeManager;
+import org.apache.ignite.raft.jraft.RaftGroupService;
+import org.apache.ignite.raft.jraft.Status;
+import org.apache.ignite.raft.jraft.conf.Configuration;
+import org.apache.ignite.raft.jraft.core.StateMachineAdapter;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.apache.ignite.raft.jraft.error.RaftError;
+import org.apache.ignite.raft.jraft.option.NodeOptions;
+import org.apache.ignite.raft.jraft.rpc.impl.IgniteRpcClient;
+import org.apache.ignite.raft.jraft.rpc.impl.IgniteRpcServer;
+import org.apache.ignite.raft.jraft.storage.snapshot.SnapshotReader;
+import org.apache.ignite.raft.jraft.storage.snapshot.SnapshotWriter;
+import org.apache.ignite.raft.jraft.util.Endpoint;
+import org.apache.ignite.raft.jraft.util.JDKMarshaller;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Raft server implementation on top of forked JRaft library.
+ */
+public class JRaftServerImpl implements RaftServer {
+    /** Cluster service. */
+    private final ClusterService service;
+
+    /** Data path. */
+    private final String dataPath;
+
+    /** Server instance. */
+    private IgniteRpcServer rpcServer;
+
+    /** Started groups. */
+    private ConcurrentMap<String, RaftGroupService> groups = new ConcurrentHashMap<>();
+
+    /** Node manager. */
+    private final NodeManager nodeManager;
+
+    /** Options. */
+    private final NodeOptions opts;
+
+    /**
+     * @param service Cluster service.
+     * @param dataPath Data path.
+     * @param factory The factory.
+     * @param reuse {@code True} to reuse cluster service (do not manage lifecyle)
+     */
+    public JRaftServerImpl(ClusterService service, String dataPath, RaftClientMessagesFactory factory, boolean reuse) {
+        this(service, dataPath, factory, reuse, new NodeOptions());
+    }
+
+    /**
+     * @param service Cluster service.
+     * @param dataPath Data path.
+     * @param factory The factory.
+     * @param reuse {@code True} to reuse cluster service (do not manage lifecyle)
+     * @param opts Default node options.
+     */
+    public JRaftServerImpl(
+        ClusterService service,
+        String dataPath,
+        RaftClientMessagesFactory factory,
+        boolean reuse,
+        NodeOptions opts
+    ) {
+        this.service = service;
+        this.dataPath = dataPath;
+        this.nodeManager = new NodeManager();
+        this.opts = opts;
+
+        assert !reuse || service.topologyService().localMember() != null;
+
+        if (opts.getServerName() == null)
+            opts.setServerName(service.localConfiguration().getName());
+
+        if (opts.getCommonExecutor() == null)
+            opts.setCommonExecutor(JRaftUtils.createCommonExecutor(opts));
+
+        if (opts.getStripedExecutor() == null)
+            opts.setStripedExecutor(JRaftUtils.createAppendEntriesExecutor(opts));
+
+        if (opts.getScheduler() == null)
+            opts.setScheduler(JRaftUtils.createScheduler(opts));
+
+        if (opts.getClientExecutor() == null)
+            opts.setClientExecutor(JRaftUtils.createClientExecutor(opts, opts.getServerName()));
+
+        rpcServer = new IgniteRpcServer(service, reuse, nodeManager, factory, JRaftUtils.createRequestExecutor(opts));
+
+        rpcServer.init(null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterService clusterService() {
+        return service;
+    }
+
+    /**
+     * @param groupId Group id.
+     * @return The path to persistence folder.
+     */
+    public String getServerDataPath(String groupId) {
+        ClusterNode clusterNode = service.topologyService().localMember();
+
+        Endpoint endpoint = new Endpoint(clusterNode.host(), clusterNode.port());
+
+        return this.dataPath + File.separator + groupId + "_" + endpoint.toString().replace(':', '_');
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized boolean startRaftGroup(String groupId, RaftGroupListener lsnr,
+        @Nullable List<Peer> initialConf) {
+        if (groups.containsKey(groupId))
+            return false;
+
+        // Thread pools are shared by all raft groups.
+        final NodeOptions nodeOptions = opts.copy();
+
+        ClusterNode clusterNode = service.topologyService().localMember();
+        Endpoint endpoint = new Endpoint(clusterNode.host(), clusterNode.port());
+
+        final String serverDataPath = getServerDataPath(groupId);
+        new File(serverDataPath).mkdirs();
+
+        nodeOptions.setLogUri(serverDataPath + File.separator + "logs");
+        nodeOptions.setRaftMetaUri(serverDataPath + File.separator + "meta");
+        nodeOptions.setSnapshotUri(serverDataPath + File.separator + "snapshot");
+
+        nodeOptions.setFsm(new DelegatingStateMachine(lsnr));
+
+        if (initialConf != null) {
+            List<PeerId> mapped = initialConf.stream().map(p -> {
+                return PeerId.fromPeer(p);
+            }).collect(Collectors.toList());
+
+            nodeOptions.setInitialConf(new Configuration(mapped, null));
+        }
+
+        IgniteRpcClient client = new IgniteRpcClient(service, true);
+
+        nodeOptions.setRpcClient(client);
+
+        final RaftGroupService server = new RaftGroupService(groupId, new PeerId(endpoint, 0,
+            ElectionPriority.DISABLED), nodeOptions, rpcServer, nodeManager, true);
+
+        server.start(false);
+
+        groups.put(groupId, server);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean stopRaftGroup(String groupId) {
+        RaftGroupService svc = groups.remove(groupId);
+
+        boolean stopped = svc != null;
+
+        if (stopped)
+            svc.shutdown();
+
+        return stopped;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Peer localPeer(String groupId) {
+        RaftGroupService service = groups.get(groupId);
+
+        if (service == null)
+            return null;
+
+        PeerId peerId = service.getRaftNode().getNodeId().getPeerId();
+
+        return new Peer(peerId.getEndpoint().toString(), peerId.getPriority());
+    }
+
+    /**
+     * @param groupId Group id.
+     * @return Service group.
+     */
+    public RaftGroupService raftGroupService(String groupId) {
+        return groups.get(groupId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void shutdown() throws Exception {
+        for (RaftGroupService groupService : groups.values())
+            groupService.shutdown();
+
+        rpcServer.shutdown();
+    }
+
+    /**
+     *
+     */
+    public static class DelegatingStateMachine extends StateMachineAdapter {
+        private final RaftGroupListener listener;
+
+        /**
+         * @param listener The listener.
+         */
+        DelegatingStateMachine(RaftGroupListener listener) {
+            this.listener = listener;
+        }
+
+        public RaftGroupListener getListener() {
+            return listener;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onApply(Iterator iter) {
+            try {
+                listener.onWrite(new java.util.Iterator<CommandClosure<WriteCommand>>() {
+                    @Override public boolean hasNext() {
+                        return iter.hasNext();
+                    }
+
+                    @Override public CommandClosure<WriteCommand> next() {
+                        @Nullable CommandClosure<WriteCommand> done = (CommandClosure<WriteCommand>) iter.done();
+                        ByteBuffer data = iter.getData();
+
+                        return new CommandClosure<WriteCommand>() {
+                            @Override public WriteCommand command() {
+                                return JDKMarshaller.DEFAULT.unmarshall(data.array());
+                            }
+
+                            @Override public void result(Serializable res) {
+                                if (done != null)
+                                    done.result(res);
+
+                                iter.next();
+                            }
+                        };
+                    }
+                });
+            }
+            catch (Exception err) {
+                Status st = new Status(RaftError.ESTATEMACHINE, err.getMessage());
+
+                if (iter.done() != null)
+                    iter.done().run(st);
+
+                iter.setErrorAndRollback(1, st);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSnapshotSave(SnapshotWriter writer, Closure done) {
+            try {
+                listener.onSnapshotSave(writer.getPath(), new Consumer<Throwable>() {
+                    @Override public void accept(Throwable res) {
+                        if (res == null) {
+                            File file = new File(writer.getPath());
+
+                            for (File file0 : file.listFiles()) {
+                                if (file0.isFile())
+                                    writer.addFile(file0.getName(), null);
+                            }
+
+                            done.run(Status.OK());
+                        }
+                        else {
+                            done.run(new Status(RaftError.EIO, "Fail to save snapshot to %s, reason %s",
+                                writer.getPath(), res.getMessage()));
+                        }
+                    }
+                });
+            }
+            catch (Exception e) {
+                done.run(new Status(RaftError.EIO, "Fail to save snapshot %s", e.getMessage()));
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean onSnapshotLoad(SnapshotReader reader) {
+            return listener.onSnapshotLoad(reader.getPath());
+        }
+    }
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/server/impl/RaftServerImpl.java b/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/impl/RaftServerImpl.java
similarity index 96%
rename from modules/raft/src/main/java/org/apache/ignite/raft/server/impl/RaftServerImpl.java
rename to modules/raft/src/main/java/org/apache/ignite/internal/raft/server/impl/RaftServerImpl.java
index 3f2255d..1ac9972 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/server/impl/RaftServerImpl.java
+++ b/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/impl/RaftServerImpl.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.raft.server.impl;
+package org.apache.ignite.internal.raft.server.impl;
 
 import java.io.Serializable;
 import java.util.Iterator;
@@ -26,6 +26,7 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.function.BiConsumer;
+import org.apache.ignite.internal.raft.server.RaftServer;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.network.ClusterNode;
 import org.apache.ignite.network.ClusterService;
@@ -41,11 +42,11 @@ import org.apache.ignite.raft.client.message.RaftErrorResponse;
 import org.apache.ignite.raft.client.message.RaftClientMessagesFactory;
 import org.apache.ignite.raft.client.service.CommandClosure;
 import org.apache.ignite.raft.client.service.RaftGroupListener;
-import org.apache.ignite.raft.server.RaftServer;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * A single node service implementation.
+ * @deprecated TODO asch Replace with jraft server https://issues.apache.org/jira/browse/IGNITE-14848
  */
 public class RaftServerImpl implements RaftServer {
     /** */
@@ -143,7 +144,8 @@ public class RaftServerImpl implements RaftServer {
     }
 
     /** {@inheritDoc} */
-    @Override public synchronized boolean startRaftGroup(String groupId, RaftGroupListener lsnr, List<Peer> initialConf) {
+    @Override public synchronized boolean startRaftGroup(String groupId, RaftGroupListener lsnr,
+        List<Peer> initialConf) {
         if (listeners.containsKey(groupId))
             return false;
 
@@ -197,7 +199,7 @@ public class RaftServerImpl implements RaftServer {
             }
 
             @Override public T command() {
-                return (T)req.command();
+                return (T) req.command();
             }
 
             @Override public void result(Serializable res) {
@@ -242,7 +244,9 @@ public class RaftServerImpl implements RaftServer {
         service.messagingService().send(sender, resp, corellationId);
     }
 
-    /** */
+    /**
+     *
+     */
     private interface CommandClosureEx<T extends Command> extends CommandClosure<T> {
         /**
          * @return The listener.
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/CliService.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/CliService.java
new file mode 100644
index 0000000..b594c2c
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/CliService.java
@@ -0,0 +1,175 @@
+/*
+ * 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.ignite.raft.jraft;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.raft.jraft.conf.Configuration;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.apache.ignite.raft.jraft.option.CliOptions;
+
+/**
+ * Client command-line service
+ */
+public interface CliService extends Lifecycle<CliOptions> {
+    /**
+     * Add a new peer into the replicating group which consists of |conf|. return OK status when success.
+     *
+     * @param groupId the raft group id
+     * @param conf current configuration
+     * @param peer peer to add
+     * @return operation status
+     */
+    Status addPeer(final String groupId, final Configuration conf, final PeerId peer);
+
+    /**
+     * Remove a peer from the replicating group which consists of |conf|. return OK status when success.
+     *
+     * @param groupId the raft group id
+     * @param conf current configuration
+     * @param peer peer to remove
+     * @return operation status
+     */
+    Status removePeer(final String groupId, final Configuration conf, final PeerId peer);
+
+    /**
+     * Gracefully change the peers of the replication group.
+     *
+     * @param groupId the raft group id
+     * @param conf current configuration
+     * @param newPeers new peers to change
+     * @return operation status
+     */
+    Status changePeers(final String groupId, final Configuration conf, final Configuration newPeers);
+
+    /**
+     * Reset the peer set of the target peer.
+     *
+     * @param groupId the raft group id
+     * @param peer target peer
+     * @param newPeers new peers to reset
+     * @return operation status
+     */
+    Status resetPeer(final String groupId, final PeerId peer, final Configuration newPeers);
+
+    /**
+     * Add some new learners into the replicating group which consists of |conf|. return OK status when success.
+     *
+     * @param groupId the raft group id
+     * @param conf current configuration
+     * @param learners learner peers to add
+     * @return operation status
+     */
+    Status addLearners(final String groupId, final Configuration conf, final List<PeerId> learners);
+
+    /**
+     * Remove some learners from the replicating group which consists of |conf|. return OK status when success.
+     *
+     * @param groupId the raft group id
+     * @param conf current configuration
+     * @param learners learner peers to remove
+     * @return operation status
+     */
+    Status removeLearners(final String groupId, final Configuration conf, final List<PeerId> learners);
+
+    /**
+     * Update learners set in the replicating group which consists of |conf|. return OK status when success.
+     *
+     * @param groupId the raft group id
+     * @param conf current configuration
+     * @param learners learner peers to set
+     * @return operation status
+     */
+    Status resetLearners(final String groupId, final Configuration conf, final List<PeerId> learners);
+
+    /**
+     * Transfer the leader of the replication group to the target peer
+     *
+     * @param groupId the raft group id
+     * @param conf current configuration
+     * @param peer target peer of new leader
+     * @return operation status
+     */
+    Status transferLeader(final String groupId, final Configuration conf, final PeerId peer);
+
+    /**
+     * Ask the peer to dump a snapshot immediately.
+     *
+     * @param groupId the raft group id
+     * @param peer target peer
+     * @return operation status
+     */
+    Status snapshot(final String groupId, final PeerId peer);
+
+    /**
+     * Get the leader of the replication group.
+     *
+     * @param groupId the raft group id
+     * @param conf configuration
+     * @param leaderId id of leader
+     * @return operation status
+     */
+    Status getLeader(final String groupId, final Configuration conf, final PeerId leaderId);
+
+    /**
+     * Ask all peers of the replication group.
+     *
+     * @param groupId the raft group id
+     * @param conf target peers configuration
+     * @return all peers of the replication group
+     */
+    List<PeerId> getPeers(final String groupId, final Configuration conf);
+
+    /**
+     * Ask all alive peers of the replication group.
+     *
+     * @param groupId the raft group id
+     * @param conf target peers configuration
+     * @return all alive peers of the replication group
+     */
+    List<PeerId> getAlivePeers(final String groupId, final Configuration conf);
+
+    /**
+     * Ask all learners of the replication group.
+     *
+     * @param groupId the raft group id
+     * @param conf target peers configuration
+     * @return all learners of the replication group
+     */
+    List<PeerId> getLearners(final String groupId, final Configuration conf);
+
+    /**
+     * Ask all alive learners of the replication group.
+     *
+     * @param groupId the raft group id
+     * @param conf target peers configuration
+     * @return all alive learners of the replication group
+     */
+    List<PeerId> getAliveLearners(final String groupId, final Configuration conf);
+
+    /**
+     * Balance the number of leaders.
+     *
+     * @param balanceGroupIds all raft group ids to balance
+     * @param conf configuration of all nodes
+     * @param balancedLeaderIds the result of all balanced leader ids
+     * @return operation status
+     */
+    Status rebalance(final Set<String> balanceGroupIds, final Configuration conf,
+        final Map<String, PeerId> balancedLeaderIds);
+}
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Closure.java
similarity index 78%
copy from modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java
copy to modules/raft/src/main/java/org/apache/ignite/raft/jraft/Closure.java
index 96948a3..3e8c2b7 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Closure.java
@@ -14,11 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.ignite.raft.jraft;
 
-package org.apache.ignite.raft.server;
-
-import org.apache.ignite.raft.client.ReadCommand;
+/**
+ * Callback closure.
+ */
+public interface Closure {
 
-/** */
-public class GetValueCommand implements ReadCommand {
+    /**
+     * Called when task is done.
+     *
+     * @param status the task status.
+     */
+    void run(final Status status);
 }
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/FSMCaller.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/FSMCaller.java
new file mode 100644
index 0000000..2f8aece
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/FSMCaller.java
@@ -0,0 +1,116 @@
+/*
+ * 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.ignite.raft.jraft;
+
+import org.apache.ignite.raft.jraft.closure.LoadSnapshotClosure;
+import org.apache.ignite.raft.jraft.closure.SaveSnapshotClosure;
+import org.apache.ignite.raft.jraft.entity.LeaderChangeContext;
+import org.apache.ignite.raft.jraft.error.RaftException;
+import org.apache.ignite.raft.jraft.option.FSMCallerOptions;
+import org.apache.ignite.raft.jraft.util.Describer;
+
+/**
+ * Finite state machine caller.
+ */
+public interface FSMCaller extends Lifecycle<FSMCallerOptions>, Describer {
+
+    /**
+     * Listen on lastAppliedLogIndex update events.
+     */
+    interface LastAppliedLogIndexListener {
+
+        /**
+         * Called when lastAppliedLogIndex updated.
+         *
+         * @param lastAppliedLogIndex the log index of last applied
+         */
+        void onApplied(final long lastAppliedLogIndex);
+    }
+
+    /**
+     * Adds a LastAppliedLogIndexListener.
+     */
+    void addLastAppliedLogIndexListener(final LastAppliedLogIndexListener listener);
+
+    /**
+     * Called when log entry committed
+     *
+     * @param committedIndex committed log index
+     */
+    boolean onCommitted(final long committedIndex);
+
+    /**
+     * Called after loading snapshot.
+     *
+     * @param done callback
+     */
+    boolean onSnapshotLoad(final LoadSnapshotClosure done);
+
+    /**
+     * Called after saving snapshot.
+     *
+     * @param done callback
+     */
+    boolean onSnapshotSave(final SaveSnapshotClosure done);
+
+    /**
+     * Called when the leader stops.
+     *
+     * @param status status info
+     */
+    boolean onLeaderStop(final Status status);
+
+    /**
+     * Called when the leader starts.
+     *
+     * @param term current term
+     */
+    boolean onLeaderStart(final long term);
+
+    /**
+     * Called when start following a leader.
+     *
+     * @param ctx context of leader change
+     */
+    boolean onStartFollowing(final LeaderChangeContext ctx);
+
+    /**
+     * Called when stop following a leader.
+     *
+     * @param ctx context of leader change
+     */
+    boolean onStopFollowing(final LeaderChangeContext ctx);
+
+    /**
+     * Called when error happens.
+     *
+     * @param error error info
+     */
+    boolean onError(final RaftException error);
+
+    /**
+     * Returns the last log entry index to apply state machine.
+     */
+    long getLastAppliedIndex();
+
+    /**
+     * Called after shutdown to wait it terminates.
+     *
+     * @throws InterruptedException if the current thread is interrupted while waiting
+     */
+    void join() throws InterruptedException;
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Iterator.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Iterator.java
new file mode 100644
index 0000000..cee66d0
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Iterator.java
@@ -0,0 +1,66 @@
+/*
+ * 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.ignite.raft.jraft;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Iterator over a batch of committed tasks.
+ *
+ * @see StateMachine#onApply(Iterator)
+ */
+public interface Iterator extends java.util.Iterator<ByteBuffer> {
+
+    /**
+     * Return the data whose content is the same as what was passed to Node#apply(Task) in the leader node.
+     */
+    ByteBuffer getData();
+
+    /**
+     * Return a unique and monotonically increasing identifier of the current task: - Uniqueness guarantees that
+     * committed tasks in different peers with the same index are always the same and kept unchanged. - Monotonicity
+     * guarantees that for any index pair i, j (i < j), task at index |i| must be applied before task at index |j| in
+     * all the peers from the group.
+     */
+    long getIndex();
+
+    /**
+     * Returns the term of the leader which to task was applied to.
+     */
+    long getTerm();
+
+    /**
+     * If done() is non-NULL, you must call done()->Run() after applying this task no matter this operation succeeds or
+     * fails, otherwise the corresponding resources would leak.
+     *
+     * If this task is proposed by this Node when it was the leader of this group and the leadership has not changed
+     * before this point, done() is exactly what was passed to Node#apply(Task) which may stand for some continuation
+     * (such as respond to the client) after updating the StateMachine with the given task. Otherwise done() must be
+     * NULL.
+     */
+    Closure done();
+
+    /**
+     * Invoked when some critical error occurred. And we will consider the last |ntail| tasks (starting from the last
+     * iterated one) as not applied. After this point, no further changes on the StateMachine as well as the Node would
+     * be allowed and you should try to repair this replica or just drop it.
+     *
+     * @param ntail the number of tasks (starting from the last iterated one)  considered as not to be applied.
+     * @param st Status to describe the detail of the error.
+     */
+    void setErrorAndRollback(final long ntail, final Status st);
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/JRaftServiceFactory.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/JRaftServiceFactory.java
new file mode 100644
index 0000000..a85f53d
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/JRaftServiceFactory.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 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.ignite.raft.jraft;
+
+import org.apache.ignite.raft.jraft.entity.codec.LogEntryCodecFactory;
+import org.apache.ignite.raft.jraft.option.NodeOptions;
+import org.apache.ignite.raft.jraft.option.RaftOptions;
+import org.apache.ignite.raft.jraft.storage.LogStorage;
+import org.apache.ignite.raft.jraft.storage.RaftMetaStorage;
+import org.apache.ignite.raft.jraft.storage.SnapshotStorage;
+import org.apache.ignite.raft.jraft.util.timer.RaftTimerFactory;
+
+/**
+ * Abstract factory to create services.
+ */
+public interface JRaftServiceFactory {
+    /**
+     * Creates a raft log storage.
+     *
+     * @param uri The log storage uri from {@link NodeOptions#getSnapshotUri()}
+     * @param raftOptions the raft options.
+     * @return storage to store raft log entires.
+     */
+    LogStorage createLogStorage(final String uri, final RaftOptions raftOptions);
+
+    /**
+     * Creates a raft snapshot storage
+     *
+     * @param uri The snapshot storage uri from {@link NodeOptions#getSnapshotUri()}
+     * @param raftOptions the raft options.
+     * @return storage to store state machine snapshot.
+     */
+    SnapshotStorage createSnapshotStorage(final String uri, final RaftOptions raftOptions);
+
+    /**
+     * Creates a raft meta storage.
+     *
+     * @param uri The meta storage uri from {@link NodeOptions#getRaftMetaUri()}
+     * @param raftOptions the raft options.
+     * @return meta storage to store raft meta info.
+     */
+    RaftMetaStorage createRaftMetaStorage(final String uri, final RaftOptions raftOptions);
+
+    /**
+     * Creates a log entry codec factory.
+     *
+     * @return a codec factory to create encoder/decoder for raft log entry.
+     */
+    LogEntryCodecFactory createLogEntryCodecFactory();
+
+    /**
+     * Creates raft timer factory.
+     *
+     * @return The factory.
+     */
+    RaftTimerFactory createRaftTimerFactory();
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/JRaftUtils.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/JRaftUtils.java
new file mode 100644
index 0000000..699b1cd
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/JRaftUtils.java
@@ -0,0 +1,205 @@
+/*
+ * 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.ignite.raft.jraft;
+
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.raft.jraft.conf.Configuration;
+import org.apache.ignite.raft.jraft.core.NodeImpl;
+import org.apache.ignite.raft.jraft.core.Scheduler;
+import org.apache.ignite.raft.jraft.core.TimerManager;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.apache.ignite.raft.jraft.option.BootstrapOptions;
+import org.apache.ignite.raft.jraft.option.NodeOptions;
+import org.apache.ignite.raft.jraft.option.RpcOptions;
+import org.apache.ignite.raft.jraft.util.Endpoint;
+import org.apache.ignite.raft.jraft.util.NamedThreadFactory;
+import org.apache.ignite.raft.jraft.util.StringUtils;
+import org.apache.ignite.raft.jraft.util.ThreadPoolUtil;
+import org.apache.ignite.raft.jraft.util.Utils;
+import org.apache.ignite.raft.jraft.util.concurrent.DefaultFixedThreadsExecutorGroupFactory;
+import org.apache.ignite.raft.jraft.util.concurrent.FixedThreadsExecutorGroup;
+
+/**
+ * Some helper methods for jraft usage.
+ */
+public final class JRaftUtils {
+    /**
+     * Bootstrap a non-empty raft node.
+     *
+     * @param opts options of bootstrap
+     * @return true if bootstrap success
+     */
+    public static boolean bootstrap(final BootstrapOptions opts) throws InterruptedException {
+        final NodeImpl node = new NodeImpl();
+        final boolean ret = node.bootstrap(opts);
+        node.shutdown();
+        node.join();
+        return ret;
+    }
+
+    /**
+     * Create a executor with size.
+     *
+     * @param prefix thread name prefix
+     * @param number thread number
+     * @return a new {@link ThreadPoolExecutor} instance
+     */
+    public static ExecutorService createExecutor(final String prefix, final int number) {
+        if (number <= 0) {
+            return null;
+        }
+        return ThreadPoolUtil.newBuilder() //
+            .poolName(prefix) //
+            .enableMetric(true) //
+            .coreThreads(number) //
+            .maximumThreads(number) //
+            .keepAliveSeconds(60L) //
+            .workQueue(new LinkedBlockingQueue<>()) //
+            .threadFactory(createThreadFactory(prefix)) //
+            .build();
+    }
+
+    /**
+     * @param opts Node options.
+     * @return The executor.
+     */
+    public static ExecutorService createCommonExecutor(NodeOptions opts) {
+        return createExecutor("JRaft-Common-Executor-" + opts.getServerName() + "-", opts.getCommonThreadPollSize());
+    }
+
+    /**
+     * @param opts Node options.
+     * @return The executor.
+     */
+    public static FixedThreadsExecutorGroup createAppendEntriesExecutor(NodeOptions opts) {
+        return createStripedExecutor("JRaft-AppendEntries-Processor-" + opts.getServerName() + "-",
+            Utils.APPEND_ENTRIES_THREADS_POOL_SIZE, Utils.MAX_APPEND_ENTRIES_TASKS_PER_THREAD);
+    }
+
+    /**
+     * @param opts Node options.
+     * @return The executor.
+     */
+    public static ExecutorService createRequestExecutor(NodeOptions opts) {
+        return createExecutor("JRaft-Request-Processor-" + opts.getServerName() + "-",
+            opts.getRaftRpcThreadPoolSize());
+    }
+
+    /**
+     * @param opts Options.
+     * @param name The name.
+     * @return The service.
+     */
+    public static ExecutorService createClientExecutor(RpcOptions opts, String name) {
+        String prefix = "JRaft-Response-Processor-" + name + "-";
+        return ThreadPoolUtil.newBuilder()
+            .poolName(prefix) //
+            .enableMetric(true) //
+            .coreThreads(opts.getRpcProcessorThreadPoolSize() / 3) //
+            .maximumThreads(opts.getRpcProcessorThreadPoolSize()) //
+            .keepAliveSeconds(60L) //
+            .workQueue(new ArrayBlockingQueue<>(10000)) //
+            .threadFactory(new NamedThreadFactory(prefix, true)) //
+            .build();
+    }
+
+    /**
+     * @param opts Options.
+     * @return The scheduler.
+     */
+    public static Scheduler createScheduler(NodeOptions opts) {
+        return new TimerManager(opts.getTimerPoolSize(), "JRaft-Node-Scheduler-" + opts.getServerName() + "-");
+    }
+
+    /**
+     * Create a striped executor.
+     *
+     * @param prefix Thread name prefix.
+     * @param number Thread number.
+     * @param tasksPerThread Max tasks per thread.
+     * @return The executor.
+     */
+    public static FixedThreadsExecutorGroup createStripedExecutor(final String prefix, final int number,
+        final int tasksPerThread) {
+        return DefaultFixedThreadsExecutorGroupFactory.INSTANCE
+            .newExecutorGroup(
+                number,
+                prefix,
+                tasksPerThread,
+                true);
+    }
+
+    /**
+     * Create a thread factory.
+     *
+     * @param prefixName the prefix name of thread
+     * @return a new {@link ThreadFactory} instance
+     */
+    public static ThreadFactory createThreadFactory(final String prefixName) {
+        return new NamedThreadFactory(prefixName, true);
+    }
+
+    /**
+     * Create a configuration from a string in the form of "host1:port1[:idx],host2:port2[:idx]......", returns a empty
+     * configuration when string is blank.
+     */
+    public static Configuration getConfiguration(final String s) {
+        final Configuration conf = new Configuration();
+        if (StringUtils.isBlank(s)) {
+            return conf;
+        }
+        if (conf.parse(s)) {
+            return conf;
+        }
+        throw new IllegalArgumentException("Invalid conf str:" + s);
+    }
+
+    /**
+     * Create a peer from a string in the form of "host:port[:idx]", returns a empty peer when string is blank.
+     */
+    public static PeerId getPeerId(final String s) {
+        final PeerId peer = new PeerId();
+        if (StringUtils.isBlank(s)) {
+            return peer;
+        }
+        if (peer.parse(s)) {
+            return peer;
+        }
+        throw new IllegalArgumentException("Invalid peer str:" + s);
+    }
+
+    /**
+     * Create a Endpoint instance from  a string in the form of "host:port", returns null when string is blank.
+     */
+    public static Endpoint getEndPoint(final String s) {
+        if (StringUtils.isBlank(s)) {
+            return null;
+        }
+        final String[] tmps = StringUtils.split(s, ':');
+        if (tmps.length != 2) {
+            throw new IllegalArgumentException("Invalid endpoint string: " + s);
+        }
+        return new Endpoint(tmps[0], Integer.parseInt(tmps[1]));
+    }
+
+    private JRaftUtils() {
+    }
+}
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Lifecycle.java
similarity index 71%
copy from modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java
copy to modules/raft/src/main/java/org/apache/ignite/raft/jraft/Lifecycle.java
index 96948a3..6aeee21 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Lifecycle.java
@@ -14,11 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.ignite.raft.jraft;
 
-package org.apache.ignite.raft.server;
+/**
+ * Service life cycle mark interface.
+ */
+public interface Lifecycle<T> {
 
-import org.apache.ignite.raft.client.ReadCommand;
+    /**
+     * Initialize the service.
+     *
+     * @return true when successes.
+     */
+    boolean init(final T opts);
 
-/** */
-public class GetValueCommand implements ReadCommand {
+    /**
+     * Dispose the resources for service.
+     */
+    void shutdown();
 }
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Node.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Node.java
new file mode 100644
index 0000000..102e8fe
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Node.java
@@ -0,0 +1,301 @@
+/*
+ * 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.ignite.raft.jraft;
+
+import java.util.List;
+import org.apache.ignite.raft.jraft.closure.ReadIndexClosure;
+import org.apache.ignite.raft.jraft.conf.Configuration;
+import org.apache.ignite.raft.jraft.core.NodeMetrics;
+import org.apache.ignite.raft.jraft.core.Replicator;
+import org.apache.ignite.raft.jraft.entity.NodeId;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.apache.ignite.raft.jraft.entity.Task;
+import org.apache.ignite.raft.jraft.entity.UserLog;
+import org.apache.ignite.raft.jraft.error.LogIndexOutOfBoundsException;
+import org.apache.ignite.raft.jraft.error.LogNotFoundException;
+import org.apache.ignite.raft.jraft.option.NodeOptions;
+import org.apache.ignite.raft.jraft.option.RaftOptions;
+import org.apache.ignite.raft.jraft.util.Describer;
+
+/**
+ * A raft replica node.
+ */
+public interface Node extends Lifecycle<NodeOptions>, Describer {
+    /**
+     * Get the leader peer id for redirect, null if absent.
+     */
+    PeerId getLeaderId();
+
+    /**
+     * Get current node id.
+     */
+    NodeId getNodeId();
+
+    /**
+     * Get the node metrics, only valid when node option {@link NodeOptions#isEnableMetrics()} is true.
+     */
+    NodeMetrics getNodeMetrics();
+
+    /**
+     * Get the raft group id.
+     */
+    String getGroupId();
+
+    /**
+     * Get the node options.
+     */
+    NodeOptions getOptions();
+
+    /**
+     * Get the raft options
+     */
+    RaftOptions getRaftOptions();
+
+    /**
+     * Returns true when the node is leader.
+     */
+    boolean isLeader();
+
+    /**
+     * Returns true when the node is leader.
+     *
+     * @param blocking if true, will be blocked until the node finish it's state change
+     */
+    boolean isLeader(final boolean blocking);
+
+    /**
+     * Shutdown local replica node.
+     *
+     * @param done callback
+     */
+    void shutdown(final Closure done);
+
+    /**
+     * Block the thread until the node is successfully stopped.
+     *
+     * @throws InterruptedException if the current thread is interrupted while waiting
+     */
+    void join() throws InterruptedException;
+
+    /**
+     * [Thread-safe and wait-free]
+     *
+     * Apply task to the replicated-state-machine
+     *
+     * About the ownership: |task.data|: for the performance consideration, we will take away the content. If you want
+     * keep the content, copy it before call this function |task.done|: If the data is successfully committed to the
+     * raft group. We will pass the ownership to #{@link StateMachine#onApply(Iterator)}. Otherwise we will specify the
+     * error and call it.
+     *
+     * @param task task to apply
+     */
+    void apply(final Task task);
+
+    /**
+     * [Thread-safe and wait-free]
+     *
+     * Starts a linearizable read-only query request with request context(optional, such as request id etc.) and
+     * closure.  The closure will be called when the request is completed, and user can read data from state machine if
+     * the result status is OK.
+     *
+     * @param requestContext the context of request
+     * @param done callback
+     */
+    void readIndex(final byte[] requestContext, final ReadIndexClosure done);
+
+    /**
+     * List peers of this raft group, only leader returns.
+     *
+     * [NOTE] <strong>when list_peers concurrency with {@link #addPeer(PeerId, Closure)}/{@link #removePeer(PeerId,
+     * Closure)}, maybe return peers is staled.  Because {@link #addPeer(PeerId, Closure)}/{@link #removePeer(PeerId,
+     * Closure)} immediately modify configuration in memory</strong>
+     *
+     * @return the peer list
+     */
+    List<PeerId> listPeers();
+
+    /**
+     * List all alive peers of this raft group, only leader returns.</p>
+     *
+     * [NOTE] <strong>list_alive_peers is just a transient data (snapshot) and a short-term loss of response by the
+     * follower will cause it to temporarily not exist in this list.</strong>
+     *
+     * @return the alive peer list
+     */
+    List<PeerId> listAlivePeers();
+
+    /**
+     * List all learners of this raft group, only leader returns.</p>
+     *
+     * [NOTE] <strong>when listLearners concurrency with {@link #addLearners(List, Closure)}/{@link
+     * #removeLearners(List, Closure)}/{@link #resetLearners(List, Closure)}, maybe return peers is staled.  Because
+     * {@link #addLearners(List, Closure)}/{@link #removeLearners(List, Closure)}/{@link #resetLearners(List, Closure)}
+     * immediately modify configuration in memory</strong>
+     *
+     * @return the learners set
+     */
+    List<PeerId> listLearners();
+
+    /**
+     * List all alive learners of this raft group, only leader returns.</p>
+     *
+     * [NOTE] <strong>when listAliveLearners concurrency with {@link #addLearners(List, Closure)}/{@link
+     * #removeLearners(List, Closure)}/{@link #resetLearners(List, Closure)}, maybe return peers is staled.  Because
+     * {@link #addLearners(List, Closure)}/{@link #removeLearners(List, Closure)}/{@link #resetLearners(List, Closure)}
+     * immediately modify configuration in memory</strong>
+     *
+     * @return the  alive learners set
+     */
+    List<PeerId> listAliveLearners();
+
+    /**
+     * Add a new peer to the raft group. done.run() would be invoked after this operation finishes, describing the
+     * detailed result.
+     *
+     * @param peer peer to add
+     * @param done callback
+     */
+    void addPeer(final PeerId peer, final Closure done);
+
+    /**
+     * Remove the peer from the raft group. done.run() would be invoked after operation finishes, describing the
+     * detailed result.
+     *
+     * @param peer peer to remove
+     * @param done callback
+     */
+    void removePeer(final PeerId peer, final Closure done);
+
+    /**
+     * Change the configuration of the raft group to |newPeers| , done.run() would be invoked after this operation
+     * finishes, describing the detailed result.
+     *
+     * @param newPeers new peers to change
+     * @param done callback
+     */
+    void changePeers(final Configuration newPeers, final Closure done);
+
+    /**
+     * Reset the configuration of this node individually, without any replication to other peers before this node
+     * becomes the leader. This function is supposed to be invoked when the majority of the replication group are dead
+     * and you'd like to revive the service in the consideration of availability. Notice that neither consistency nor
+     * consensus are guaranteed in this case, BE CAREFULE when dealing with this method.
+     *
+     * @param newPeers new peers
+     */
+    Status resetPeers(final Configuration newPeers);
+
+    /**
+     * Add some new learners to the raft group. done.run() will be invoked after this operation finishes, describing the
+     * detailed result.
+     *
+     * @param learners learners to add
+     * @param done callback
+     */
+    void addLearners(final List<PeerId> learners, final Closure done);
+
+    /**
+     * Remove some learners from the raft group. done.run() will be invoked after this operation finishes, describing
+     * the detailed result.
+     *
+     * @param learners learners to remove
+     * @param done callback
+     */
+    void removeLearners(final List<PeerId> learners, final Closure done);
+
+    /**
+     * Reset learners in the raft group. done.run() will be invoked after this operation finishes, describing the
+     * detailed result.
+     *
+     * @param learners learners to set
+     * @param done callback
+     */
+    void resetLearners(final List<PeerId> learners, final Closure done);
+
+    /**
+     * Start a snapshot immediately if possible. done.run() would be invoked when the snapshot finishes, describing the
+     * detailed result.
+     *
+     * @param done callback
+     */
+    void snapshot(final Closure done);
+
+    /**
+     * Reset the election_timeout for the every node.
+     *
+     * @param electionTimeoutMs the timeout millis of election
+     */
+    void resetElectionTimeoutMs(final int electionTimeoutMs);
+
+    /**
+     * Try transferring leadership to |peer|. If peer is ANY_PEER, a proper follower will be chosen as the leader for
+     * the next term. Returns 0 on success, -1 otherwise.
+     *
+     * @param peer the target peer of new leader
+     * @return operation status
+     */
+    Status transferLeadershipTo(final PeerId peer);
+
+    /**
+     * Read the first committed user log from the given index. Return OK on success and user_log is assigned with the
+     * very data. Be awared that the user_log may be not the exact log at the given index, but the first available user
+     * log from the given index to lastCommittedIndex. Otherwise, appropriate errors are returned: - return ELOGDELETED
+     * when the log has been deleted; - return ENOMOREUSERLOG when we can't get a user log even reaching
+     * lastCommittedIndex. [NOTE] in consideration of safety, we use lastAppliedIndex instead of lastCommittedIndex in
+     * code implementation.
+     *
+     * @param index log index
+     * @return user log entry
+     * @throws LogNotFoundException the user log is deleted at index.
+     * @throws LogIndexOutOfBoundsException the special index is out of bounds.
+     */
+    UserLog readCommittedUserLog(final long index);
+
+    /**
+     * JRaft users can implement the ReplicatorStateListener interface by themselves. So users can do their own
+     * logical operator in this listener when replicator created, destroyed or had some errors.
+     *
+     * @param replicatorStateListener added ReplicatorStateListener which is implemented by users.
+     */
+    void addReplicatorStateListener(final Replicator.ReplicatorStateListener replicatorStateListener);
+
+    /**
+     * End User can remove their implement the ReplicatorStateListener interface by themselves.
+     *
+     * @param replicatorStateListener need to remove the ReplicatorStateListener which has been added by users.
+     */
+    void removeReplicatorStateListener(final Replicator.ReplicatorStateListener replicatorStateListener);
+
+    /**
+     * Remove all the ReplicatorStateListeners which have been added by users.
+     */
+    void clearReplicatorStateListeners();
+
+    /**
+     * Get the ReplicatorStateListeners which have been added by users.
+     *
+     * @return node's replicatorStatueListeners which have been added by users.
+     */
+    List<Replicator.ReplicatorStateListener> getReplicatorStateListeners();
+
+    /**
+     * Get the node's target election priority value.
+     *
+     * @return node's target election priority value.
+     */
+    int getNodeTargetPriority();
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/NodeManager.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/NodeManager.java
new file mode 100644
index 0000000..c66f735
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/NodeManager.java
@@ -0,0 +1,99 @@
+/*
+ * 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.ignite.raft.jraft;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.stream.Collectors;
+import org.apache.ignite.raft.jraft.entity.NodeId;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.apache.ignite.raft.jraft.util.OnlyForTest;
+
+/**
+ * Raft nodes manager.
+ */
+public class NodeManager {
+    private final ConcurrentMap<NodeId, Node> nodeMap = new ConcurrentHashMap<>();
+    private final ConcurrentMap<String, List<Node>> groupMap = new ConcurrentHashMap<>();
+
+    /**
+     * Adds a node.
+     */
+    public boolean add(final Node node) {
+        final NodeId nodeId = node.getNodeId();
+        if (this.nodeMap.putIfAbsent(nodeId, node) == null) {
+            final String groupId = node.getGroupId();
+            List<Node> nodes = this.groupMap.get(groupId);
+            if (nodes == null) {
+                nodes = new CopyOnWriteArrayList<>();
+                List<Node> existsNode = this.groupMap.putIfAbsent(groupId, nodes);
+                if (existsNode != null) {
+                    nodes = existsNode;
+                }
+            }
+            nodes.add(node);
+            return true;
+        }
+        return false;
+    }
+
+    /**
+     * Clear the states, for test
+     */
+    @OnlyForTest
+    public void clear() {
+        this.groupMap.clear();
+        this.nodeMap.clear();
+    }
+
+    /**
+     * Remove a node.
+     */
+    public boolean remove(final Node node) {
+        if (this.nodeMap.remove(node.getNodeId(), node)) {
+            final List<Node> nodes = this.groupMap.get(node.getGroupId());
+            if (nodes != null) {
+                return nodes.remove(node);
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Get node by groupId and peer.
+     */
+    public Node get(final String groupId, final PeerId peerId) {
+        return this.nodeMap.get(new NodeId(groupId, peerId));
+    }
+
+    /**
+     * Get all nodes in a raft group.
+     */
+    public List<Node> getNodesByGroupId(final String groupId) {
+        return this.groupMap.get(groupId);
+    }
+
+    /**
+     * Get all nodes
+     */
+    public List<Node> getAllNodes() {
+        return this.groupMap.values().stream().flatMap(Collection::stream).collect(Collectors.toList());
+    }
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/RaftGroupService.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/RaftGroupService.java
new file mode 100644
index 0000000..846860e
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/RaftGroupService.java
@@ -0,0 +1,278 @@
+/*
+ * 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.ignite.raft.jraft;
+
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.raft.jraft.core.NodeImpl;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.apache.ignite.raft.jraft.option.NodeOptions;
+import org.apache.ignite.raft.jraft.option.RpcOptions;
+import org.apache.ignite.raft.jraft.rpc.RpcServer;
+import org.apache.ignite.raft.jraft.util.Endpoint;
+import org.apache.ignite.raft.jraft.util.StringUtils;
+import org.apache.ignite.raft.jraft.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A raft group service.
+ */
+public class RaftGroupService {
+    private static final Logger LOG = LoggerFactory.getLogger(RaftGroupService.class);
+
+    private volatile boolean started = false;
+
+    /**
+     * This node serverId
+     */
+    private PeerId serverId;
+
+    /**
+     * Node options
+     */
+    private NodeOptions nodeOptions;
+
+    /**
+     * The raft RPC server
+     */
+    private RpcServer rpcServer;
+
+    /**
+     * If we want to share the rpcServer instance, then we can't stop it when shutdown.
+     */
+    private final boolean sharedRpcServer;
+
+    /**
+     * The raft group id
+     */
+    private String groupId;
+
+    /**
+     * The raft node.
+     */
+    private Node node;
+
+    /**
+     * The node manager.
+     */
+    private NodeManager nodeManager;
+
+    /**
+     * @param groupId Group Id.
+     * @param serverId Server id.
+     * @param nodeOptions Node options.
+     * @param rpcServer RPC server.
+     * @param nodeManager Node manager.
+     */
+    public RaftGroupService(final String groupId, final PeerId serverId, final NodeOptions nodeOptions,
+        final RpcServer rpcServer, final NodeManager nodeManager) {
+        this(groupId, serverId, nodeOptions, rpcServer, nodeManager, false);
+    }
+
+    /**
+     * @param groupId Group Id.
+     * @param serverId Server id.
+     * @param nodeOptions Node options.
+     * @param rpcServer RPC server.
+     * @param nodeManager Node manager.
+     * @param sharedRpcServer {@code True} if a shared server.
+     */
+    public RaftGroupService(final String groupId, final PeerId serverId, final NodeOptions nodeOptions,
+        final RpcServer rpcServer, final NodeManager nodeManager, final boolean sharedRpcServer) {
+        super();
+        this.groupId = groupId;
+        this.serverId = serverId;
+        this.nodeOptions = nodeOptions;
+        this.rpcServer = rpcServer;
+        this.nodeManager = nodeManager;
+        this.sharedRpcServer = sharedRpcServer;
+    }
+
+    public synchronized Node getRaftNode() {
+        return this.node;
+    }
+
+    /**
+     * Starts the raft group service, returns the raft node.
+     */
+    public synchronized Node start() {
+        return start(true);
+    }
+
+    /**
+     * Starts the raft group service, returns the raft node.
+     *
+     * @param startRpcServer whether to start RPC server.
+     */
+    public synchronized Node start(final boolean startRpcServer) {
+        if (this.started) {
+            return this.node;
+        }
+        if (this.serverId == null || this.serverId.getEndpoint() == null
+            || this.serverId.getEndpoint().equals(new Endpoint(Utils.IP_ANY, 0))) {
+            throw new IllegalArgumentException("Blank serverId:" + this.serverId);
+        }
+        if (StringUtils.isBlank(this.groupId)) {
+            throw new IllegalArgumentException("Blank group id" + this.groupId);
+        }
+
+        assert this.nodeOptions.getRpcClient() != null;
+
+        this.node = new NodeImpl(groupId, serverId);
+
+        if (!this.node.init(this.nodeOptions)) {
+            LOG.warn("Stopping partially started node [groupId={}, serverId={}]", groupId, serverId);
+            this.node.shutdown();
+
+            try {
+                this.node.join();
+            }
+            catch (InterruptedException e) {
+                throw new IgniteInternalException(e);
+            }
+
+            throw new IgniteInternalException("Fail to init node, please see the logs to find the reason.");
+        }
+
+        if (startRpcServer) {
+            this.rpcServer.init(null);
+        }
+        else {
+            LOG.warn("RPC server is not started in RaftGroupService.");
+        }
+
+        this.nodeManager.add(this.node);
+        this.started = true;
+        LOG.info("Start the RaftGroupService successfully {}", this.node.getNodeId());
+        return this.node;
+    }
+
+    public synchronized void shutdown() {
+        // TODO asch remove handlers before shutting down raft node https://issues.apache.org/jira/browse/IGNITE-14519
+        if (!this.started) {
+            return;
+        }
+        if (this.rpcServer != null) {
+            try {
+                if (!this.sharedRpcServer) {
+                    this.rpcServer.shutdown();
+                }
+            }
+            catch (Exception e) {
+                LOG.error("Failed to shutdown the server", e);
+            }
+            this.rpcServer = null;
+        }
+
+        this.node.shutdown();
+        try {
+            this.node.join();
+        }
+        catch (InterruptedException e) {
+            LOG.error("Interrupted while waiting for the node to shutdown");
+        }
+
+        nodeManager.remove(this.node);
+        this.started = false;
+        LOG.info("Stop the RaftGroupService successfully.");
+    }
+
+    /**
+     * Returns true when service is started.
+     */
+    public boolean isStarted() {
+        return this.started;
+    }
+
+    /**
+     * Returns the raft group id.
+     */
+    public String getGroupId() {
+        return this.groupId;
+    }
+
+    /**
+     * Set the raft group id
+     */
+    public void setGroupId(final String groupId) {
+        if (this.started) {
+            throw new IllegalStateException("Raft group service already started");
+        }
+        this.groupId = groupId;
+    }
+
+    /**
+     * Returns the node serverId
+     */
+    public PeerId getServerId() {
+        return this.serverId;
+    }
+
+    /**
+     * Set the node serverId
+     */
+    public void setServerId(final PeerId serverId) {
+        if (this.started) {
+            throw new IllegalStateException("Raft group service already started");
+        }
+        this.serverId = serverId;
+    }
+
+    /**
+     * Returns the node options.
+     */
+    public RpcOptions getNodeOptions() {
+        return this.nodeOptions;
+    }
+
+    /**
+     * Set node options.
+     */
+    public void setNodeOptions(final NodeOptions nodeOptions) {
+        if (this.started) {
+            throw new IllegalStateException("Raft group service already started");
+        }
+        if (nodeOptions == null) {
+            throw new IllegalArgumentException("Invalid node options.");
+        }
+        nodeOptions.validate();
+        this.nodeOptions = nodeOptions;
+    }
+
+    /**
+     * Returns the rpc server instance.
+     */
+    public RpcServer getRpcServer() {
+        return this.rpcServer;
+    }
+
+    /**
+     * Set rpc server.
+     */
+    public void setRpcServer(final RpcServer rpcServer) {
+        if (this.started) {
+            throw new IllegalStateException("Raft group service already started");
+        }
+        if (this.serverId == null) {
+            throw new IllegalStateException("Please set serverId at first");
+        }
+        if (rpcServer.boundPort() != this.serverId.getPort()) {
+            throw new IllegalArgumentException("RPC server port mismatch");
+        }
+        this.rpcServer = rpcServer;
+    }
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/ReadOnlyService.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/ReadOnlyService.java
new file mode 100644
index 0000000..dcdb6b4
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/ReadOnlyService.java
@@ -0,0 +1,50 @@
+/*
+ * 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.ignite.raft.jraft;
+
+import org.apache.ignite.raft.jraft.closure.ReadIndexClosure;
+import org.apache.ignite.raft.jraft.error.RaftException;
+import org.apache.ignite.raft.jraft.option.ReadOnlyServiceOptions;
+
+/**
+ * The read-only query service.
+ */
+public interface ReadOnlyService extends Lifecycle<ReadOnlyServiceOptions> {
+
+    /**
+     * Adds a ReadIndex request.
+     *
+     * @param reqCtx request context of readIndex
+     * @param closure callback
+     */
+    void addRequest(final byte[] reqCtx, final ReadIndexClosure closure);
+
+    /**
+     * Waits for service shutdown.
+     *
+     * @throws InterruptedException if the current thread is interrupted while waiting
+     */
+    void join() throws InterruptedException;
+
+    /**
+     * Called when the node is turned into error state.
+     *
+     * @param error error with raft info
+     */
+    void setError(final RaftException error);
+
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/ReplicatorGroup.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/ReplicatorGroup.java
new file mode 100644
index 0000000..0f9530f
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/ReplicatorGroup.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.raft.jraft;
+
+import java.util.List;
+import org.apache.ignite.raft.jraft.closure.CatchUpClosure;
+import org.apache.ignite.raft.jraft.conf.ConfigurationEntry;
+import org.apache.ignite.raft.jraft.core.ReplicatorType;
+import org.apache.ignite.raft.jraft.entity.NodeId;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.apache.ignite.raft.jraft.option.ReplicatorGroupOptions;
+import org.apache.ignite.raft.jraft.rpc.RpcRequests.AppendEntriesResponse;
+import org.apache.ignite.raft.jraft.rpc.RpcResponseClosure;
+import org.apache.ignite.raft.jraft.util.Describer;
+import org.apache.ignite.raft.jraft.util.ThreadId;
+
+/**
+ * Replicators in a raft group.
+ */
+public interface ReplicatorGroup extends Describer {
+    /**
+     * Init the replicator group.
+     *
+     * @param nodeId node id
+     * @param opts options of replicator grop
+     * @return true if init success
+     */
+    boolean init(final NodeId nodeId, final ReplicatorGroupOptions opts);
+
+    /**
+     * Adds a replicator for follower({@link ReplicatorType#Follower}).
+     *
+     * @param peer target peer
+     * @return true on success
+     * @see #addReplicator(PeerId, ReplicatorType)
+     */
+    default boolean addReplicator(final PeerId peer) {
+        return addReplicator(peer, ReplicatorType.Follower);
+    }
+
+    /**
+     * Add a replicator attached with |peer| will be a notification when the replicator catches up according to the
+     * arguments. NOTE: when calling this function, the replicators starts to work immediately, and might call
+     * Node#stepDown which might have race with the caller, you should deal with this situation.
+     *
+     * @param peer target peer
+     * @param replicatorType replicator type
+     * @return true on success
+     */
+    default boolean addReplicator(final PeerId peer, ReplicatorType replicatorType) {
+        return addReplicator(peer, replicatorType, true);
+    }
+
+    /**
+     * Try to add a replicator attached with |peer| will be a notification when the replicator catches up according to
+     * the arguments. NOTE: when calling this function, the replicators starts to work immediately, and might call
+     * Node#stepDown which might have race with the caller, you should deal with this situation.
+     *
+     * @param peer target peer
+     * @param replicatorType replicator type
+     * @param sync synchronous
+     * @return true on success
+     */
+    boolean addReplicator(final PeerId peer, ReplicatorType replicatorType, boolean sync);
+
+    /**
+     * Send heartbeat to a peer.
+     *
+     * @param peer target peer
+     * @param closure callback
+     */
+    void sendHeartbeat(final PeerId peer, final RpcResponseClosure<AppendEntriesResponse> closure);
+
+    /**
+     * Get replicator id by peer, null if not found.
+     *
+     * @param peer peer of replicator
+     * @return the replicator id
+     */
+    ThreadId getReplicator(final PeerId peer);
+
+    /**
+     * Check replicator state, if it's not started, start it; if it is blocked, unblock it. It should be called by
+     * leader.
+     *
+     * @param peer peer of replicator
+     * @param lockNode if lock with node
+     */
+    void checkReplicator(final PeerId peer, final boolean lockNode);
+
+    /**
+     * Clear failure to start replicators
+     */
+    void clearFailureReplicators();
+
+    /**
+     * Wait the peer catchup.
+     */
+    boolean waitCaughtUp(final PeerId peer, final long maxMargin, final long dueTime, final CatchUpClosure done);
+
+    /**
+     * Get peer's last rpc send timestamp (monotonic time in milliseconds).
+     *
+     * @param peer the peer of replicator
+     */
+    long getLastRpcSendTimestamp(final PeerId peer);
+
+    /**
+     * Stop all replicators.
+     */
+    boolean stopAll();
+
+    /**
+     * Stop replicator for the peer.
+     *
+     * @param peer the peer of replicator
+     * @return true on success
+     */
+    boolean stopReplicator(final PeerId peer);
+
+    /**
+     * Reset the term of all to-add replicators. This method is supposed to be called when the very candidate becomes
+     * the leader, so we suppose that there are no running replicators. Return true on success, false otherwise
+     *
+     * @param newTerm new term num
+     * @return true on success
+     */
+    boolean resetTerm(final long newTerm);
+
+    /**
+     * Reset the interval of heartbeat, This method is supposed to be called when the very candidate becomes the leader,
+     * so we suppose that there are no running replicators. return true when success, false otherwise.
+     *
+     * @param newIntervalMs new heartbeat interval millis
+     * @return true on success
+     */
+    boolean resetHeartbeatInterval(final int newIntervalMs);
+
+    /**
+     * Reset the interval of electionTimeout for replicator.
+     *
+     * @param newIntervalMs new election timeout millis
+     * @return true on success
+     */
+    boolean resetElectionTimeoutInterval(final int newIntervalMs);
+
+    /**
+     * Returns true if the there's a replicator attached to the given |peer|
+     *
+     * @param peer target peer
+     * @return true on contains
+     */
+    boolean contains(final PeerId peer);
+
+    /**
+     * Transfer leadership to the given |peer|
+     *
+     * @param peer target peer
+     * @param logIndex log index
+     * @return true on success
+     */
+    boolean transferLeadershipTo(final PeerId peer, final long logIndex);
+
+    /**
+     * Stop transferring leadership to the given |peer|
+     *
+     * @param peer target peer
+     * @return true on success
+     */
+    boolean stopTransferLeadership(final PeerId peer);
+
+    /**
+     * Stop all the replicators except for the one that we think can be the candidate of the next leader, which has the
+     * largest `last_log_id' among peers in |current_conf|. |candidate| would be returned if we found one and the caller
+     * is responsible for stopping it, or an invalid value if we found none. Returns candidate replicator id on success
+     * and null otherwise.
+     *
+     * @param conf configuration of all replicators
+     * @return candidate replicator id on success
+     */
+    ThreadId stopAllAndFindTheNextCandidate(final ConfigurationEntry conf);
+
+    /**
+     * Find the follower with the most log entries in this group, which is likely becomes the leader according to the
+     * election algorithm of raft. Returns the follower peerId on success and null otherwise.
+     *
+     * @param conf configuration of all replicators
+     * @return the follower peerId on success
+     */
+    PeerId findTheNextCandidate(final ConfigurationEntry conf);
+
+    /**
+     * Returns all replicators.
+     */
+    List<ThreadId> listReplicators();
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/StateMachine.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/StateMachine.java
new file mode 100644
index 0000000..08269c8
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/StateMachine.java
@@ -0,0 +1,123 @@
+/*
+ * 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.ignite.raft.jraft;
+
+import org.apache.ignite.raft.jraft.conf.Configuration;
+import org.apache.ignite.raft.jraft.entity.LeaderChangeContext;
+import org.apache.ignite.raft.jraft.error.RaftException;
+import org.apache.ignite.raft.jraft.storage.snapshot.SnapshotReader;
+import org.apache.ignite.raft.jraft.storage.snapshot.SnapshotWriter;
+
+/**
+ * |StateMachine| is the sink of all the events of a very raft node. Implement a specific StateMachine for your own
+ * business logic. NOTE: All the interfaces are not guaranteed to be thread safe and they are called sequentially,
+ * saying that every single operation will block all the following ones.
+ */
+public interface StateMachine {
+
+    /**
+     * Update the StateMachine with a batch a tasks that can be accessed through |iterator|.
+     *
+     * Invoked when one or more tasks that were passed to Node#apply(Task) have been committed to the raft group (quorum
+     * of the group peers have received those tasks and stored them on the backing storage).
+     *
+     * Once this function returns to the caller, we will regard all the iterated tasks through |iter| have been
+     * successfully applied. And if you didn't apply all the the given tasks, we would regard this as a critical error
+     * and report a error whose type is ERROR_TYPE_STATE_MACHINE.
+     *
+     * @param iter iterator of states
+     */
+    void onApply(final Iterator iter);
+
+    /**
+     * Invoked once when the raft node was shut down. Default do nothing
+     */
+    void onShutdown();
+
+    /**
+     * User defined snapshot generate function, this method will block StateMachine#onApply(Iterator). user can make
+     * snapshot async when fsm can be cow(copy-on-write). call done.run(status) when snapshot finished. Default: Save
+     * nothing and returns error.
+     *
+     * @param writer snapshot writer
+     * @param done callback
+     */
+    void onSnapshotSave(final SnapshotWriter writer, final Closure done);
+
+    /**
+     * User defined snapshot load function get and load snapshot Default: Load nothing and returns error.
+     *
+     * @param reader snapshot reader
+     * @return true on success
+     */
+    boolean onSnapshotLoad(final SnapshotReader reader);
+
+    /**
+     * Invoked when the belonging node becomes the leader of the group at |term| Default: Do nothing
+     *
+     * @param term new term num
+     */
+    void onLeaderStart(final long term);
+
+    /**
+     * Invoked when this node steps down from the leader of the replication group and |status| describes detailed
+     * information
+     *
+     * @param status status info
+     */
+    void onLeaderStop(final Status status);
+
+    /**
+     * This method is called when a critical error was encountered, after this point, no any further modification is
+     * allowed to applied to this node until the error is fixed and this node restarts.
+     *
+     * @param e raft error message
+     */
+    void onError(final RaftException e);
+
+    /**
+     * Invoked when a configuration has been committed to the group.
+     *
+     * @param conf committed configuration
+     */
+    void onConfigurationCommitted(final Configuration conf);
+
+    /**
+     * This method is called when a follower stops following a leader and its leaderId becomes null, situations
+     * including: 1. handle election timeout and start preVote 2. receive requests with higher term such as VoteRequest
+     * from a candidate or appendEntries request from a new leader 3. receive timeoutNow request from current leader and
+     * start request vote.
+     *
+     * the parameter ctx gives the information(leaderId, term and status) about the very leader whom the follower
+     * followed before. User can reset the node's information as it stops following some leader.
+     *
+     * @param ctx context of leader change
+     */
+    void onStopFollowing(final LeaderChangeContext ctx);
+
+    /**
+     * This method is called when a follower or candidate starts following a leader and its leaderId (should be NULL
+     * before the method is called) is set to the leader's id, situations including: 1. a candidate receives
+     * appendEntries request from a leader 2. a follower(without leader) receives appendEntries from a leader
+     *
+     * the parameter ctx gives the information(leaderId, term and status) about the very leader whom the follower starts
+     * to follow. User can reset the node's information as it starts to follow some leader.
+     *
+     * @param ctx context of leader change
+     */
+    void onStartFollowing(final LeaderChangeContext ctx);
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Status.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Status.java
new file mode 100644
index 0000000..017c6f4
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/Status.java
@@ -0,0 +1,235 @@
+/*
+ * 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.ignite.raft.jraft;
+
+import org.apache.ignite.raft.jraft.error.RaftError;
+import org.apache.ignite.raft.jraft.util.Copiable;
+
+//A Status encapsulates the result of an operation. It may indicate success,
+
+//or it may indicate an error with an associated error message. It's suitable
+//for passing status of functions with richer information than just error_code
+//in exception-forbidden code. This utility is inspired by leveldb::Status.
+//
+//Multiple threads can invoke const methods on a Status without
+//external synchronization, but if any of the threads may call a
+//non-const method, all threads accessing the same Status must use
+//external synchronization.
+//
+//Since failed status needs to allocate memory, you should be careful when
+//failed status is frequent.
+public class Status implements Copiable<Status> {
+
+    /**
+     * Status internal state.
+     */
+    private static class State {
+        /** error code */
+        int code;
+        /** error msg */
+        String msg;
+
+        State(int code, String msg) {
+            super();
+            this.code = code;
+            this.msg = msg;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + this.code;
+            result = prime * result + (this.msg == null ? 0 : this.msg.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) {
+                return true;
+            }
+            if (obj == null) {
+                return false;
+            }
+            if (getClass() != obj.getClass()) {
+                return false;
+            }
+            State other = (State) obj;
+            if (this.code != other.code) {
+                return false;
+            }
+            if (this.msg == null) {
+                return other.msg == null;
+            }
+            else {
+                return this.msg.equals(other.msg);
+            }
+        }
+    }
+
+    private State state;
+
+    public Status() {
+        this.state = null;
+    }
+
+    /**
+     * Creates a OK status instance.
+     */
+    public static Status OK() {
+        return new Status();
+    }
+
+    public Status(Status s) {
+        if (s.state != null) {
+            this.state = new State(s.state.code, s.state.msg);
+        }
+        else {
+            this.state = null;
+        }
+    }
+
+    public Status(RaftError raftError, String fmt, Object... args) {
+        this.state = new State(raftError.getNumber(), String.format(fmt, args));
+    }
+
+    public Status(int code, String fmt, Object... args) {
+        this.state = new State(code, String.format(fmt, args));
+    }
+
+    public Status(int code, String errorMsg) {
+        this.state = new State(code, errorMsg);
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (this.state == null ? 0 : this.state.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+        Status other = (Status) obj;
+        if (this.state == null) {
+            return other.state == null;
+        }
+        else {
+            return this.state.equals(other.state);
+        }
+    }
+
+    /**
+     * Reset status to be OK state.
+     */
+    public void reset() {
+        this.state = null;
+    }
+
+    /**
+     * Returns true when status is in OK state.
+     */
+    public boolean isOk() {
+        return this.state == null || this.state.code == 0;
+    }
+
+    /**
+     * Set error code.
+     */
+    public void setCode(int code) {
+        if (this.state == null) {
+            this.state = new State(code, null);
+        }
+        else {
+            this.state.code = code;
+        }
+    }
+
+    /**
+     * Get error code.
+     */
+    public int getCode() {
+        return this.state == null ? 0 : this.state.code;
+    }
+
+    /**
+     * Get raft error from error code.
+     */
+    public RaftError getRaftError() {
+        return this.state == null ? RaftError.SUCCESS : RaftError.forNumber(this.state.code);
+    }
+
+    /**
+     * Set error msg
+     */
+    public void setErrorMsg(String errMsg) {
+        if (this.state == null) {
+            this.state = new State(0, errMsg);
+        }
+        else {
+            this.state.msg = errMsg;
+        }
+    }
+
+    /**
+     * Set error code and error msg.
+     */
+    public void setError(int code, String fmt, Object... args) {
+        this.state = new State(code, String.format(String.valueOf(fmt), args));
+    }
+
+    /**
+     * Set raft error and error msg.
+     */
+    public void setError(RaftError error, String fmt, Object... args) {
+        this.state = new State(error.getNumber(), String.format(String.valueOf(fmt), args));
+    }
+
+    @Override
+    public String toString() {
+        if (isOk()) {
+            return "Status[OK]";
+        }
+        else {
+            return "Status[" + RaftError.describeCode(this.state.code) + "<" + this.state.code + ">: " + this.state.msg
+                + "]";
+        }
+    }
+
+    @Override
+    public Status copy() {
+        return new Status(this.getCode(), this.getErrorMsg());
+    }
+
+    /**
+     * Get the error msg.
+     */
+    public String getErrorMsg() {
+        return this.state == null ? null : this.state.msg;
+    }
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/ThreadPoolMetricsSignalHandler.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/ThreadPoolMetricsSignalHandler.java
new file mode 100644
index 0000000..e8afe27
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/ThreadPoolMetricsSignalHandler.java
@@ -0,0 +1,58 @@
+/*
+ * 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.ignite.raft.jraft;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import org.apache.ignite.raft.jraft.util.FileOutputSignalHandler;
+import org.apache.ignite.raft.jraft.util.MetricReporter;
+import org.apache.ignite.raft.jraft.util.SystemPropertyUtil;
+import org.apache.ignite.raft.jraft.util.ThreadPoolMetricRegistry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ */
+public class ThreadPoolMetricsSignalHandler extends FileOutputSignalHandler {
+
+    private static Logger LOG = LoggerFactory.getLogger(ThreadPoolMetricsSignalHandler.class);
+
+    private static final String DIR = SystemPropertyUtil.get("jraft.signal.thread.pool.metrics.dir", "");
+    private static final String BASE_NAME = "thread_pool_metrics.log";
+
+    @Override
+    public void handle(final String signalName) {
+        try {
+            final File file = getOutputFile(DIR, BASE_NAME);
+
+            LOG.info("Printing thread pools metrics with signal: {} to file: {}.", signalName, file);
+
+            try (final PrintStream out = new PrintStream(new FileOutputStream(file, true))) {
+                MetricReporter.forRegistry(ThreadPoolMetricRegistry.metricRegistry()) //
+                    .outputTo(out) //
+                    .build() //
+                    .report();
+            }
+        }
+        catch (final IOException e) {
+            LOG.error("Fail to print thread pools metrics.", e);
+        }
+    }
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/CatchUpClosure.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/CatchUpClosure.java
new file mode 100644
index 0000000..fd52c99
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/CatchUpClosure.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.ignite.raft.jraft.closure;
+
+import java.util.concurrent.ScheduledFuture;
+import org.apache.ignite.raft.jraft.Closure;
+import org.apache.ignite.raft.jraft.Status;
+
+/**
+ * A catchup closure for peer to catch up.
+ */
+public abstract class CatchUpClosure implements Closure {
+
+    private long maxMargin;
+    private ScheduledFuture<?> timer;
+    private boolean hasTimer;
+    private boolean errorWasSet;
+
+    private final Status status = Status.OK();
+
+    public Status getStatus() {
+        return this.status;
+    }
+
+    public long getMaxMargin() {
+        return this.maxMargin;
+    }
+
+    public void setMaxMargin(long maxMargin) {
+        this.maxMargin = maxMargin;
+    }
+
+    public ScheduledFuture<?> getTimer() {
+        return this.timer;
+    }
+
+    public void setTimer(ScheduledFuture<?> timer) {
+        this.timer = timer;
+        this.hasTimer = true;
+    }
+
+    public boolean hasTimer() {
+        return this.hasTimer;
+    }
+
+    public boolean isErrorWasSet() {
+        return this.errorWasSet;
+    }
+
+    public void setErrorWasSet(boolean errorWasSet) {
+        this.errorWasSet = errorWasSet;
+    }
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/ClosureQueue.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/ClosureQueue.java
new file mode 100644
index 0000000..ef8140a
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/ClosureQueue.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.ignite.raft.jraft.closure;
+
+import java.util.List;
+import org.apache.ignite.raft.jraft.Closure;
+
+/**
+ * A thread-safe closure queue.
+ */
+public interface ClosureQueue {
+    /**
+     * Clear all closure in queue.
+     */
+    void clear();
+
+    /**
+     * Reset the first index in queue.
+     *
+     * @param firstIndex the first index of queue
+     */
+    void resetFirstIndex(final long firstIndex);
+
+    /**
+     * Append a new closure into queue.
+     *
+     * @param closure the closure to append
+     */
+    void appendPendingClosure(final Closure closure);
+
+    /**
+     * Pop closure from queue until index(inclusion), returns the first popped out index, returns -1 when out of range,
+     * returns index+1 when not found.
+     *
+     * @param endIndex the index of queue
+     * @param closures closure list
+     * @return returns the first popped out index, returns -1 when out of range, returns index+1 when not found.
+     */
+    long popClosureUntil(final long endIndex, final List<Closure> closures);
+
+    /**
+     * Pop closure from queue until index(inclusion), returns the first popped out index, returns -1 when out of range,
+     * returns index+1 when not found.
+     *
+     * @param endIndex the index of queue
+     * @param closures closure list
+     * @param taskClosures task closure list
+     * @return returns the first popped out index, returns -1 when out of range, returns index+1 when not found.
+     */
+    long popClosureUntil(final long endIndex, final List<Closure> closures, final List<TaskClosure> taskClosures);
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/ClosureQueueImpl.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/ClosureQueueImpl.java
new file mode 100644
index 0000000..224e26a
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/ClosureQueueImpl.java
@@ -0,0 +1,147 @@
+/*
+ * 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.ignite.raft.jraft.closure;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.raft.jraft.Closure;
+import org.apache.ignite.raft.jraft.Status;
+import org.apache.ignite.raft.jraft.error.RaftError;
+import org.apache.ignite.raft.jraft.option.NodeOptions;
+import org.apache.ignite.raft.jraft.util.OnlyForTest;
+import org.apache.ignite.raft.jraft.util.Requires;
+import org.apache.ignite.raft.jraft.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Closure queue implementation.
+ */
+public class ClosureQueueImpl implements ClosureQueue {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ClosureQueueImpl.class);
+
+    private final Lock lock;
+    private final NodeOptions options;
+    private long firstIndex;
+    private LinkedList<Closure> queue;
+
+    @OnlyForTest
+    public long getFirstIndex() {
+        return firstIndex;
+    }
+
+    @OnlyForTest
+    public LinkedList<Closure> getQueue() {
+        return queue;
+    }
+
+    public ClosureQueueImpl(NodeOptions options) {
+        super();
+        this.lock = new ReentrantLock();
+        this.firstIndex = 0;
+        this.queue = new LinkedList<>();
+        this.options = options;
+    }
+
+    @Override
+    public void clear() {
+        List<Closure> savedQueue;
+        this.lock.lock();
+        try {
+            this.firstIndex = 0;
+            savedQueue = this.queue;
+            this.queue = new LinkedList<>();
+        }
+        finally {
+            this.lock.unlock();
+        }
+
+        final Status status = new Status(RaftError.EPERM, "Leader stepped down");
+        Utils.runInThread(options.getCommonExecutor(), () -> {
+            for (final Closure done : savedQueue) {
+                if (done != null) {
+                    done.run(status);
+                }
+            }
+        });
+    }
+
+    @Override
+    public void resetFirstIndex(final long firstIndex) {
+        this.lock.lock();
+        try {
+            Requires.requireTrue(this.queue.isEmpty(), "Queue is not empty.");
+            this.firstIndex = firstIndex;
+        }
+        finally {
+            this.lock.unlock();
+        }
+    }
+
+    @Override
+    public void appendPendingClosure(final Closure closure) {
+        this.lock.lock();
+        try {
+            this.queue.add(closure);
+        }
+        finally {
+            this.lock.unlock();
+        }
+    }
+
+    @Override
+    public long popClosureUntil(final long endIndex, final List<Closure> closures) {
+        return popClosureUntil(endIndex, closures, null);
+    }
+
+    @Override
+    public long popClosureUntil(final long endIndex, final List<Closure> closures,
+        final List<TaskClosure> taskClosures) {
+        closures.clear();
+        if (taskClosures != null) {
+            taskClosures.clear();
+        }
+        this.lock.lock();
+        try {
+            final int queueSize = this.queue.size();
+            if (queueSize == 0 || endIndex < this.firstIndex) {
+                return endIndex + 1;
+            }
+            if (endIndex > this.firstIndex + queueSize - 1) {
+                LOG.error("Invalid endIndex={}, firstIndex={}, closureQueueSize={}", endIndex, this.firstIndex,
+                    queueSize);
+                return -1;
+            }
+            final long outFirstIndex = this.firstIndex;
+            for (long i = outFirstIndex; i <= endIndex; i++) {
+                final Closure closure = this.queue.pollFirst();
+                if (taskClosures != null && closure instanceof TaskClosure) {
+                    taskClosures.add((TaskClosure) closure);
+                }
+                closures.add(closure);
+            }
+            this.firstIndex = endIndex + 1;
+            return outFirstIndex;
+        }
+        finally {
+            this.lock.unlock();
+        }
+    }
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/JoinableClosure.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/JoinableClosure.java
new file mode 100644
index 0000000..f8f0aed
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/JoinableClosure.java
@@ -0,0 +1,57 @@
+/*
+ * 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.ignite.raft.jraft.closure;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.ignite.raft.jraft.Closure;
+import org.apache.ignite.raft.jraft.Status;
+import org.apache.ignite.raft.jraft.util.Requires;
+
+/**
+ *
+ */
+public class JoinableClosure implements Closure {
+
+    private final CountDownLatch latch = new CountDownLatch(1);
+    private final Closure closure;
+
+    public JoinableClosure(Closure closure) {
+        this.closure = Requires.requireNonNull(closure, "closure");
+    }
+
+    @Override
+    public void run(final Status status) {
+        this.closure.run(status);
+        latch.countDown();
+    }
+
+    public void join() throws InterruptedException {
+        this.latch.await();
+    }
+
+    public void join(final long timeoutMillis) throws InterruptedException, TimeoutException {
+        if (!this.latch.await(timeoutMillis, TimeUnit.MILLISECONDS)) {
+            throw new TimeoutException("joined timeout");
+        }
+    }
+
+    public Closure getClosure() {
+        return closure;
+    }
+}
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/LoadSnapshotClosure.java
similarity index 67%
copy from modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java
copy to modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/LoadSnapshotClosure.java
index 96948a3..07a2c31 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/GetValueCommand.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/LoadSnapshotClosure.java
@@ -14,11 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.ignite.raft.jraft.closure;
 
-package org.apache.ignite.raft.server;
+import org.apache.ignite.raft.jraft.Closure;
+import org.apache.ignite.raft.jraft.storage.snapshot.SnapshotReader;
 
-import org.apache.ignite.raft.client.ReadCommand;
+/**
+ * Load snapshot closure
+ */
+public interface LoadSnapshotClosure extends Closure {
 
-/** */
-public class GetValueCommand implements ReadCommand {
+    /**
+     * Start to load snapshot, returns a snapshot reader.
+     *
+     * @return a snapshot reader.
+     */
+    SnapshotReader start();
 }
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/ReadIndexClosure.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/ReadIndexClosure.java
new file mode 100644
index 0000000..fa4047d
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/ReadIndexClosure.java
@@ -0,0 +1,101 @@
+/*
+ * 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.ignite.raft.jraft.closure;
+
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+import org.apache.ignite.raft.jraft.Closure;
+import org.apache.ignite.raft.jraft.Node;
+import org.apache.ignite.raft.jraft.Status;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Read index closure
+ */
+public abstract class ReadIndexClosure implements Closure {
+    private static final Logger LOG = LoggerFactory.getLogger(ReadIndexClosure.class);
+
+    private static final AtomicIntegerFieldUpdater<ReadIndexClosure> STATE_UPDATER =
+        AtomicIntegerFieldUpdater.newUpdater(ReadIndexClosure.class, "state");
+
+    private static final int PENDING = 0;
+    private static final int COMPLETE = 1;
+
+    /**
+     * Invalid log index -1.
+     */
+    public static final long INVALID_LOG_INDEX = -1;
+
+    private long index = INVALID_LOG_INDEX;
+    private byte[] requestContext;
+
+    private volatile int state = PENDING; // NOPMD
+
+    /**
+     * Called when ReadIndex can be executed.
+     *
+     * @param status the readIndex status.
+     * @param index the committed index when starts readIndex.
+     * @param reqCtx the request context passed by {@link Node#readIndex(byte[], ReadIndexClosure)}.
+     * @see Node#readIndex(byte[], ReadIndexClosure)
+     */
+    public abstract void run(final Status status, final long index, final byte[] reqCtx);
+
+    /**
+     * Set callback result, called by jraft.
+     *
+     * @param index the committed index.
+     * @param reqCtx the request context passed by {@link Node#readIndex(byte[], ReadIndexClosure)}.
+     */
+    public void setResult(final long index, final byte[] reqCtx) {
+        this.index = index;
+        this.requestContext = reqCtx;
+    }
+
+    /**
+     * The committed log index when starts readIndex request. return -1 if fails.
+     *
+     * @return returns the committed index.  returns -1 if fails.
+     */
+    public long getIndex() {
+        return this.index;
+    }
+
+    /**
+     * Returns the request context.
+     *
+     * @return the request context.
+     */
+    public byte[] getRequestContext() {
+        return this.requestContext;
+    }
+
+    @Override
+    public void run(final Status status) {
+        if (!STATE_UPDATER.compareAndSet(this, PENDING, COMPLETE)) {
+            LOG.warn("A timeout read-index response finally returned: {}.", status);
+            return;
+        }
+
+        try {
+            run(status, this.index, this.requestContext);
+        }
+        catch (final Throwable t) {
+            LOG.error("Fail to run ReadIndexClosure with status: {}.", status, t);
+        }
+    }
+}
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/IncrementAndGetCommand.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/SaveSnapshotClosure.java
similarity index 61%
copy from modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/IncrementAndGetCommand.java
copy to modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/SaveSnapshotClosure.java
index ed17aa3..5f81523 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/IncrementAndGetCommand.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/SaveSnapshotClosure.java
@@ -14,27 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.ignite.raft.jraft.closure;
 
-package org.apache.ignite.raft.server;
+import org.apache.ignite.raft.jraft.Closure;
+import org.apache.ignite.raft.jraft.entity.RaftOutter.SnapshotMeta;
+import org.apache.ignite.raft.jraft.storage.snapshot.SnapshotWriter;
 
-import org.apache.ignite.raft.client.WriteCommand;
-
-/** */
-public class IncrementAndGetCommand implements WriteCommand {
-    /** */
-    private final long delta;
-
-    /**
-     * @param delta The delta.
-     */
-    public IncrementAndGetCommand(long delta) {
-        this.delta = delta;
-    }
+/**
+ * Save snapshot closure
+ */
+public interface SaveSnapshotClosure extends Closure {
 
     /**
-     * @return The delta.
+     * Starts to save snapshot, returns the writer.
+     *
+     * @param meta metadata of snapshot.
+     * @return returns snapshot writer.
      */
-    public long delta() {
-        return delta;
-    }
+    SnapshotWriter start(final SnapshotMeta meta);
 }
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/SynchronizedClosure.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/SynchronizedClosure.java
new file mode 100644
index 0000000..6c049e9
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/SynchronizedClosure.java
@@ -0,0 +1,77 @@
+/*
+ * 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.ignite.raft.jraft.closure;
+
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.raft.jraft.Closure;
+import org.apache.ignite.raft.jraft.Status;
+
+/**
+ * A special Closure which provides synchronization primitives.
+ */
+public class SynchronizedClosure implements Closure {
+
+    private CountDownLatch latch;
+    private volatile Status status;
+    /**
+     * Latch count to reset
+     */
+    private int count;
+
+    public SynchronizedClosure() {
+        this(1);
+    }
+
+    public SynchronizedClosure(final int n) {
+        this.count = n;
+        this.latch = new CountDownLatch(n);
+    }
+
+    /**
+     * Get last ran status
+     *
+     * @return returns the last ran status
+     */
+    public Status getStatus() {
+        return this.status;
+    }
+
+    @Override
+    public void run(final Status status) {
+        this.status = status;
+        this.latch.countDown();
+    }
+
+    /**
+     * Wait for closure run
+     *
+     * @return status
+     * @throws InterruptedException if the current thread is interrupted while waiting
+     */
+    public Status await() throws InterruptedException {
+        this.latch.await();
+        return this.status;
+    }
+
+    /**
+     * Reset the closure
+     */
+    public void reset() {
+        this.status = null;
+        this.latch = new CountDownLatch(this.count);
+    }
+}
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/IncrementAndGetCommand.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/TaskClosure.java
similarity index 65%
copy from modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/IncrementAndGetCommand.java
copy to modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/TaskClosure.java
index ed17aa3..1f203ba 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/IncrementAndGetCommand.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/closure/TaskClosure.java
@@ -14,27 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.ignite.raft.jraft.closure;
 
-package org.apache.ignite.raft.server;
+import org.apache.ignite.raft.jraft.Closure;
 
-import org.apache.ignite.raft.client.WriteCommand;
-
-/** */
-public class IncrementAndGetCommand implements WriteCommand {
-    /** */
-    private final long delta;
-
-    /**
-     * @param delta The delta.
-     */
-    public IncrementAndGetCommand(long delta) {
-        this.delta = delta;
-    }
+/**
+ * Closure for task applying.
+ */
+public interface TaskClosure extends Closure {
 
     /**
-     * @return The delta.
+     * Called when task is committed to majority peers of the RAFT group but before it is applied to state machine.
+     *
+     * <strong>Note: user implementation should not block
+     * this method and throw any exceptions.</strong>
      */
-    public long delta() {
-        return delta;
-    }
+    void onCommitted();
 }
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/conf/Configuration.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/conf/Configuration.java
new file mode 100644
index 0000000..f70173a
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/conf/Configuration.java
@@ -0,0 +1,321 @@
+/*
+ * 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.ignite.raft.jraft.conf;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.apache.ignite.raft.jraft.util.Copiable;
+import org.apache.ignite.raft.jraft.util.Requires;
+import org.apache.ignite.raft.jraft.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A configuration with a set of peers.
+ */
+public class Configuration implements Iterable<PeerId>, Copiable<Configuration> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(Configuration.class);
+
+    private static final String LEARNER_POSTFIX = "/learner";
+
+    private List<PeerId> peers = new ArrayList<>();
+
+    // use LinkedHashSet to keep insertion order.
+    private LinkedHashSet<PeerId> learners = new LinkedHashSet<>();
+
+    public Configuration() {
+        super();
+    }
+
+    /**
+     * Construct a configuration instance with peers.
+     *
+     * @param conf configuration
+     */
+    public Configuration(final Iterable<PeerId> conf) {
+        this(conf, null);
+    }
+
+    /**
+     * Construct a configuration from another conf.
+     *
+     * @param conf configuration
+     */
+    public Configuration(final Configuration conf) {
+        this(conf.getPeers(), conf.getLearners());
+    }
+
+    /**
+     * Construct a Configuration instance with peers and learners.
+     *
+     * @param conf peers configuration
+     * @param learners learners
+     */
+    public Configuration(final Iterable<PeerId> conf, final Iterable<PeerId> learners) {
+        Requires.requireNonNull(conf, "conf");
+        for (final PeerId peer : conf) {
+            this.peers.add(peer.copy());
+        }
+        addLearners(learners);
+    }
+
+    public void setLearners(final LinkedHashSet<PeerId> learners) {
+        this.learners = learners;
+    }
+
+    /**
+     * Add a learner peer.
+     *
+     * @param learner learner to add
+     * @return true when add successfully.
+     */
+    public boolean addLearner(final PeerId learner) {
+        return this.learners.add(learner);
+    }
+
+    /**
+     * Add learners in batch, returns the added count.
+     *
+     * @param learners learners to add
+     * @return the total added count
+     */
+    public int addLearners(final Iterable<PeerId> learners) {
+        int ret = 0;
+        if (learners != null) {
+            for (final PeerId peer : learners) {
+                if (this.learners.add(peer.copy())) {
+                    ret++;
+                }
+            }
+        }
+        return ret;
+    }
+
+    /**
+     * Remove a learner peer.
+     *
+     * @param learner learner to remove
+     * @return true when remove successfully.
+     */
+    public boolean removeLearner(final PeerId learner) {
+        return this.learners.remove(learner);
+    }
+
+    /**
+     * Retrieve the learners set.
+     *
+     * @return learners
+     */
+    public LinkedHashSet<PeerId> getLearners() {
+        return this.learners;
+    }
+
+    /**
+     * Retrieve the learners set copy.
+     *
+     * @return learners
+     */
+    public List<PeerId> listLearners() {
+        return new ArrayList<>(this.learners);
+    }
+
+    @Override
+    public Configuration copy() {
+        return new Configuration(this.peers, this.learners);
+    }
+
+    /**
+     * Returns true when the configuration is valid.
+     *
+     * @return true if the configuration is valid.
+     */
+    public boolean isValid() {
+        final Set<PeerId> intersection = new HashSet<>(this.peers);
+        intersection.retainAll(this.learners);
+        return !this.peers.isEmpty() && intersection.isEmpty();
+    }
+
+    public void reset() {
+        this.peers.clear();
+        this.learners.clear();
+    }
+
+    public boolean isEmpty() {
+        return this.peers.isEmpty();
+    }
+
+    /**
+     * Returns the peers total number.
+     *
+     * @return total num of peers
+     */
+    public int size() {
+        return this.peers.size();
+    }
+
+    @Override
+    public Iterator<PeerId> iterator() {
+        return this.peers.iterator();
+    }
+
+    public Set<PeerId> getPeerSet() {
+        return new HashSet<>(this.peers);
+    }
+
+    public List<PeerId> listPeers() {
+        return new ArrayList<>(this.peers);
+    }
+
+    public List<PeerId> getPeers() {
+        return this.peers;
+    }
+
+    public void setPeers(final List<PeerId> peers) {
+        this.peers.clear();
+        for (final PeerId peer : peers) {
+            this.peers.add(peer.copy());
+        }
+    }
+
+    public void appendPeers(final Collection<PeerId> set) {
+        this.peers.addAll(set);
+    }
+
+    public boolean addPeer(final PeerId peer) {
+        return this.peers.add(peer);
+    }
+
+    public boolean removePeer(final PeerId peer) {
+        return this.peers.remove(peer);
+    }
+
+    public boolean contains(final PeerId peer) {
+        return this.peers.contains(peer);
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((this.learners == null) ? 0 : this.learners.hashCode());
+        result = prime * result + ((this.peers == null) ? 0 : this.peers.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+        Configuration other = (Configuration) obj;
+        if (this.learners == null) {
+            if (other.learners != null) {
+                return false;
+            }
+        }
+        else if (!this.learners.equals(other.learners)) {
+            return false;
+        }
+        if (this.peers == null) {
+            return other.peers == null;
+        }
+        else {
+            return this.peers.equals(other.peers);
+        }
+    }
+
+    @Override
+    public String toString() {
+        final StringBuilder sb = new StringBuilder();
+        final List<PeerId> peers = listPeers();
+        int i = 0;
+        int size = peers.size();
+        for (final PeerId peer : peers) {
+            sb.append(peer);
+            if (i < size - 1 || !this.learners.isEmpty()) {
+                sb.append(",");
+            }
+            i++;
+        }
+
+        size = this.learners.size();
+        i = 0;
+        for (final PeerId peer : this.learners) {
+            sb.append(peer).append(LEARNER_POSTFIX);
+            if (i < size - 1) {
+                sb.append(",");
+            }
+            i++;
+        }
+
+        return sb.toString();
+    }
+
+    public boolean parse(final String conf) {
+        if (StringUtils.isBlank(conf)) {
+            return false;
+        }
+        reset();
+        final String[] peerStrs = StringUtils.split(conf, ',');
+        for (String peerStr : peerStrs) {
+            final PeerId peer = new PeerId();
+            int index;
+            boolean isLearner = false;
+            if ((index = peerStr.indexOf(LEARNER_POSTFIX)) > 0) {
+                // It's a learner
+                peerStr = peerStr.substring(0, index);
+                isLearner = true;
+            }
+            if (peer.parse(peerStr)) {
+                if (isLearner) {
+                    addLearner(peer);
+                }
+                else {
+                    addPeer(peer);
+                }
+            }
+            else {
+                LOG.error("Fail to parse peer {} in {}, ignore it.", peerStr, conf);
+            }
+        }
+        return true;
+    }
+
+    /**
+     * Get the difference between |*this| and |rhs| |included| would be assigned to |*this| - |rhs| |excluded| would be
+     * assigned to |rhs| - |*this|
+     */
+    public void diff(final Configuration rhs, final Configuration included, final Configuration excluded) {
+        included.peers = new ArrayList<>(this.peers);
+        included.peers.removeAll(rhs.peers);
+        excluded.peers = new ArrayList<>(rhs.peers);
+        excluded.peers.removeAll(this.peers);
+    }
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/conf/ConfigurationEntry.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/conf/ConfigurationEntry.java
new file mode 100644
index 0000000..6f8956d
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/conf/ConfigurationEntry.java
@@ -0,0 +1,124 @@
+/*
+ * 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.ignite.raft.jraft.conf;
+
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.ignite.raft.jraft.entity.LogId;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A configuration entry with current peers and old peers.
+ */
+public class ConfigurationEntry {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ConfigurationEntry.class);
+
+    private LogId id = new LogId(0, 0);
+    private Configuration conf = new Configuration();
+    private Configuration oldConf = new Configuration();
+
+    public LogId getId() {
+        return this.id;
+    }
+
+    public void setId(final LogId id) {
+        this.id = id;
+    }
+
+    public Configuration getConf() {
+        return this.conf;
+    }
+
+    public void setConf(final Configuration conf) {
+        this.conf = conf;
+    }
+
+    public Configuration getOldConf() {
+        return this.oldConf;
+    }
+
+    public void setOldConf(final Configuration oldConf) {
+        this.oldConf = oldConf;
+    }
+
+    public ConfigurationEntry() {
+        super();
+    }
+
+    public ConfigurationEntry(final LogId id, final Configuration conf, final Configuration oldConf) {
+        super();
+        this.id = id;
+        this.conf = conf;
+        this.oldConf = oldConf;
+    }
+
+    public boolean isStable() {
+        return this.oldConf.isEmpty();
+    }
+
+    public boolean isEmpty() {
+        return this.conf.isEmpty();
+    }
+
+    public Set<PeerId> listPeers() {
+        final Set<PeerId> ret = new HashSet<>(this.conf.listPeers());
+        ret.addAll(this.oldConf.listPeers());
+        return ret;
+    }
+
+    /**
+     * Returns true when the conf entry is valid.
+     *
+     * @return if the the entry is valid
+     */
+    public boolean isValid() {
+        if (!this.conf.isValid()) {
+            return false;
+        }
+
+        // The peer set and learner set should not have intersection set.
+        final Set<PeerId> intersection = listPeers();
+        intersection.retainAll(listLearners());
+        if (intersection.isEmpty()) {
+            return true;
+        }
+        LOG.error("Invalid conf entry {}, peers and learners have intersection: {}.", this, intersection);
+        return false;
+    }
+
+    public Set<PeerId> listLearners() {
+        final Set<PeerId> ret = new HashSet<>(this.conf.getLearners());
+        ret.addAll(this.oldConf.getLearners());
+        return ret;
+    }
+
+    public boolean containsLearner(final PeerId learner) {
+        return this.conf.getLearners().contains(learner) || this.oldConf.getLearners().contains(learner);
+    }
+
+    public boolean contains(final PeerId peer) {
+        return this.conf.contains(peer) || this.oldConf.contains(peer);
+    }
+
+    @Override
+    public String toString() {
+        return "ConfigurationEntry [id=" + this.id + ", conf=" + this.conf + ", oldConf=" + this.oldConf + "]";
+    }
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/conf/ConfigurationManager.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/conf/ConfigurationManager.java
new file mode 100644
index 0000000..e3a7453
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/conf/ConfigurationManager.java
@@ -0,0 +1,106 @@
+/*
+ * 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.ignite.raft.jraft.conf;
+
+import java.util.LinkedList;
+import java.util.ListIterator;
+import org.apache.ignite.raft.jraft.util.Requires;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Configuration manager
+ */
+public class ConfigurationManager {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ConfigurationManager.class);
+
+    private final LinkedList<ConfigurationEntry> configurations = new LinkedList<>();
+    private ConfigurationEntry snapshot = new ConfigurationEntry();
+
+    /**
+     * Adds a new conf entry.
+     */
+    public boolean add(final ConfigurationEntry entry) {
+        if (!this.configurations.isEmpty()) {
+            if (this.configurations.peekLast().getId().getIndex() >= entry.getId().getIndex()) {
+                LOG.error("Did you forget to call truncateSuffix before the last log index goes back.");
+                return false;
+            }
+        }
+        return this.configurations.add(entry);
+    }
+
+    /**
+     * [1, first_index_kept) are being discarded
+     */
+    public void truncatePrefix(final long firstIndexKept) {
+        while (!this.configurations.isEmpty() && this.configurations.peekFirst().getId().getIndex() < firstIndexKept) {
+            this.configurations.pollFirst();
+        }
+    }
+
+    /**
+     * (last_index_kept, infinity) are being discarded
+     */
+    public void truncateSuffix(final long lastIndexKept) {
+        while (!this.configurations.isEmpty() && this.configurations.peekLast().getId().getIndex() > lastIndexKept) {
+            this.configurations.pollLast();
+        }
+    }
+
+    public ConfigurationEntry getSnapshot() {
+        return this.snapshot;
+    }
+
+    public void setSnapshot(final ConfigurationEntry snapshot) {
+        this.snapshot = snapshot;
+    }
+
+    public ConfigurationEntry getLastConfiguration() {
+        if (this.configurations.isEmpty()) {
+            return snapshot;
+        }
+        else {
+            return this.configurations.peekLast();
+        }
+    }
+
+    public ConfigurationEntry get(final long lastIncludedIndex) {
+        if (this.configurations.isEmpty()) {
+            Requires.requireTrue(lastIncludedIndex >= this.snapshot.getId().getIndex(),
+                "lastIncludedIndex %d is less than snapshot index %d", lastIncludedIndex, this.snapshot.getId()
+                    .getIndex());
+            return this.snapshot;
+        }
+        ListIterator<ConfigurationEntry> it = this.configurations.listIterator();
+        while (it.hasNext()) {
+            if (it.next().getId().getIndex() > lastIncludedIndex) {
+                it.previous();
+                break;
+            }
+        }
+        if (it.hasPrevious()) {
+            // find the first position that is less than or equal to lastIncludedIndex.
+            return it.previous();
+        }
+        else {
+            // position not found position, return snapshot.
+            return this.snapshot;
+        }
+    }
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/BallotBox.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/BallotBox.java
new file mode 100644
index 0000000..499db1b
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/BallotBox.java
@@ -0,0 +1,281 @@
+/*
+ * 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.ignite.raft.jraft.core;
+
+import java.util.concurrent.locks.StampedLock;
+import org.apache.ignite.raft.jraft.Closure;
+import org.apache.ignite.raft.jraft.FSMCaller;
+import org.apache.ignite.raft.jraft.Lifecycle;
+import org.apache.ignite.raft.jraft.closure.ClosureQueue;
+import org.apache.ignite.raft.jraft.conf.Configuration;
+import org.apache.ignite.raft.jraft.entity.Ballot;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.apache.ignite.raft.jraft.option.BallotBoxOptions;
+import org.apache.ignite.raft.jraft.util.Describer;
+import org.apache.ignite.raft.jraft.util.OnlyForTest;
+import org.apache.ignite.raft.jraft.util.Requires;
+import org.apache.ignite.raft.jraft.util.SegmentList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Ballot box for voting.
+ */
+public class BallotBox implements Lifecycle<BallotBoxOptions>, Describer {
+
+    private static final Logger LOG = LoggerFactory.getLogger(BallotBox.class);
+
+    private FSMCaller waiter;
+    private ClosureQueue closureQueue;
+    private final StampedLock stampedLock = new StampedLock();
+    private long lastCommittedIndex = 0;
+    private long pendingIndex; // Par. 3.6 prevent commits from previous terms
+    private final SegmentList<Ballot> pendingMetaQueue = new SegmentList<>(false);
+
+    @OnlyForTest
+    long getPendingIndex() {
+        return this.pendingIndex;
+    }
+
+    @OnlyForTest
+    SegmentList<Ballot> getPendingMetaQueue() {
+        return this.pendingMetaQueue;
+    }
+
+    public long getLastCommittedIndex() {
+        long stamp = this.stampedLock.tryOptimisticRead();
+        final long optimisticVal = this.lastCommittedIndex;
+        if (this.stampedLock.validate(stamp)) {
+            return optimisticVal;
+        }
+        stamp = this.stampedLock.readLock();
+        try {
+            return this.lastCommittedIndex;
+        }
+        finally {
+            this.stampedLock.unlockRead(stamp);
+        }
+    }
+
+    @Override
+    public boolean init(final BallotBoxOptions opts) {
+        if (opts.getWaiter() == null || opts.getClosureQueue() == null) {
+            LOG.error("waiter or closure queue is null.");
+            return false;
+        }
+        this.waiter = opts.getWaiter();
+        this.closureQueue = opts.getClosureQueue();
+        return true;
+    }
+
+    /**
+     * Called by leader, otherwise the behavior is undefined Set logs in [first_log_index, last_log_index] are stable at
+     * |peer|.
+     */
+    public boolean commitAt(final long firstLogIndex, final long lastLogIndex, final PeerId peer) {
+        // TODO use lock-free algorithm here? https://issues.apache.org/jira/browse/IGNITE-14832
+        final long stamp = this.stampedLock.writeLock();
+        long lastCommittedIndex = 0;
+        try {
+            if (this.pendingIndex == 0) {
+                return false;
+            }
+            if (lastLogIndex < this.pendingIndex) {
+                return true;
+            }
+
+            if (lastLogIndex >= this.pendingIndex + this.pendingMetaQueue.size()) {
+                throw new ArrayIndexOutOfBoundsException();
+            }
+
+            final long startAt = Math.max(this.pendingIndex, firstLogIndex);
+
+            Ballot.PosHint hint = new Ballot.PosHint();
+
+            for (long logIndex = startAt; logIndex <= lastLogIndex; logIndex++) {
+                final Ballot bl = this.pendingMetaQueue.get((int) (logIndex - this.pendingIndex));
+                hint = bl.grant(peer, hint);
+                if (bl.isGranted()) {
+                    lastCommittedIndex = logIndex;
+                }
+            }
+
+            if (lastCommittedIndex == 0) {
+                return true;
+            }
+
+            // TODO asch investigate https://issues.apache.org/jira/browse/IGNITE-14832.
+            // When removing a peer off the raft group which contains even number of
+            // peers, the quorum would decrease by 1, e.g. 3 of 4 changes to 2 of 3. In
+            // this case, the log after removal may be committed before some previous
+            // logs, since we use the new configuration to deal the quorum of the
+            // removal request, we think it's safe to commit all the uncommitted
+            // previous logs, which is not well proved right now
+            this.pendingMetaQueue.removeFromFirst((int) (lastCommittedIndex - this.pendingIndex) + 1);
+            LOG.debug("Committed log fromIndex={}, toIndex={}.", this.pendingIndex, lastCommittedIndex);
+            this.pendingIndex = lastCommittedIndex + 1;
+            this.lastCommittedIndex = lastCommittedIndex;
+        }
+        finally {
+            this.stampedLock.unlockWrite(stamp);
+        }
+        this.waiter.onCommitted(lastCommittedIndex);
+        return true;
+    }
+
+    /**
+     * Called when the leader steps down, otherwise the behavior is undefined When a leader steps down, the uncommitted
+     * user applications should fail immediately, which the new leader will deal whether to commit or truncate.
+     */
+    public void clearPendingTasks() {
+        final long stamp = this.stampedLock.writeLock();
+        try {
+            this.pendingMetaQueue.clear();
+            this.pendingIndex = 0;
+            this.closureQueue.clear();
+        }
+        finally {
+            this.stampedLock.unlockWrite(stamp);
+        }
+    }
+
+    /**
+     * Called when a candidate becomes the new leader, otherwise the behavior is undefined. According the the raft
+     * algorithm, the logs from previous terms can't be committed until a log at the new term becomes committed, so
+     * |newPendingIndex| should be |last_log_index| + 1.
+     *
+     * @param newPendingIndex pending index of new leader
+     * @return returns true if reset success
+     */
+    public boolean resetPendingIndex(final long newPendingIndex) {
+        final long stamp = this.stampedLock.writeLock();
+        try {
+            if (!(this.pendingIndex == 0 && this.pendingMetaQueue.isEmpty())) {
+                LOG.error("resetPendingIndex fail, pendingIndex={}, pendingMetaQueueSize={}.", this.pendingIndex,
+                    this.pendingMetaQueue.size());
+                return false;
+            }
+            if (newPendingIndex <= this.lastCommittedIndex) {
+                LOG.error("resetPendingIndex fail, newPendingIndex={}, lastCommittedIndex={}.", newPendingIndex,
+                    this.lastCommittedIndex);
+                return false;
+            }
+            this.pendingIndex = newPendingIndex;
+            this.closureQueue.resetFirstIndex(newPendingIndex);
+            return true;
+        }
+        finally {
+            this.stampedLock.unlockWrite(stamp);
+        }
+    }
+
+    /**
+     * Called by leader, otherwise the behavior is undefined Store application context before replication.
+     *
+     * @param conf current configuration
+     * @param oldConf old configuration
+     * @param done callback
+     * @return returns true on success
+     */
+    public boolean appendPendingTask(final Configuration conf, final Configuration oldConf, final Closure done) {
+        final Ballot bl = new Ballot();
+        bl.init(conf, oldConf);
+
+        final long stamp = this.stampedLock.writeLock();
+        try {
+            if (this.pendingIndex <= 0) {
+                LOG.error("Fail to appendingTask, pendingIndex={}.", this.pendingIndex);
+                return false;
+            }
+            this.pendingMetaQueue.add(bl);
+            this.closureQueue.appendPendingClosure(done);
+            return true;
+        }
+        finally {
+            this.stampedLock.unlockWrite(stamp);
+        }
+    }
+
+    /**
+     * Called by follower, otherwise the behavior is undefined. Set committed index received from leader.
+     *
+     * @param lastCommittedIndex Last committed index.
+     * @return Returns true if set success.
+     */
+    public boolean setLastCommittedIndex(final long lastCommittedIndex) {
+        boolean doUnlock = true;
+        final long stamp = this.stampedLock.writeLock();
+        try {
+            if (this.pendingIndex != 0 || !this.pendingMetaQueue.isEmpty()) {
+                Requires.requireTrue(lastCommittedIndex < this.pendingIndex,
+                    "Node changes to leader, pendingIndex=%d, param lastCommittedIndex=%d", this.pendingIndex,
+                    lastCommittedIndex);
+                return false;
+            }
+            if (lastCommittedIndex < this.lastCommittedIndex) {
+                return false;
+            }
+            if (lastCommittedIndex > this.lastCommittedIndex) {
+                this.lastCommittedIndex = lastCommittedIndex;
+                this.stampedLock.unlockWrite(stamp);
+                doUnlock = false;
+                this.waiter.onCommitted(lastCommittedIndex);
+            }
+        }
+        finally {
+            if (doUnlock) {
+                this.stampedLock.unlockWrite(stamp);
+            }
+        }
+        return true;
+    }
+
+    @Override
+    public void shutdown() {
+        clearPendingTasks();
+    }
+
+    @Override
+    public void describe(final Printer out) {
+        long _lastCommittedIndex;
+        long _pendingIndex;
+        long _pendingMetaQueueSize;
+        long stamp = this.stampedLock.tryOptimisticRead();
+        if (this.stampedLock.validate(stamp)) {
+            _lastCommittedIndex = this.lastCommittedIndex;
+            _pendingIndex = this.pendingIndex;
+            _pendingMetaQueueSize = this.pendingMetaQueue.size();
+        }
+        else {
+            stamp = this.stampedLock.readLock();
+            try {
+                _lastCommittedIndex = this.lastCommittedIndex;
+                _pendingIndex = this.pendingIndex;
+                _pendingMetaQueueSize = this.pendingMetaQueue.size();
+            }
+            finally {
+                this.stampedLock.unlockRead(stamp);
+            }
+        }
+        out.print("  lastCommittedIndex: ") //
+            .println(_lastCommittedIndex);
+        out.print("  pendingIndex: ") //
+            .println(_pendingIndex);
+        out.print("  pendingMetaQueueSize: ") //
+            .println(_pendingMetaQueueSize);
+    }
+}
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/CliServiceImpl.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/CliServiceImpl.java
new file mode 100644
index 0000000..8575f93
--- /dev/null
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/CliServiceImpl.java
@@ -0,0 +1,702 @@
+/*
+ * 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.ignite.raft.jraft.core;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.raft.jraft.CliService;
+import org.apache.ignite.raft.jraft.Status;
+import org.apache.ignite.raft.jraft.conf.Configuration;
+import org.apache.ignite.raft.jraft.entity.PeerId;
+import org.apache.ignite.raft.jraft.error.JRaftException;
+import org.apache.ignite.raft.jraft.error.RaftError;
+import org.apache.ignite.raft.jraft.option.CliOptions;
+import org.apache.ignite.raft.jraft.rpc.CliClientService;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.AddLearnersRequest;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.AddPeerRequest;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.AddPeerResponse;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.ChangePeersRequest;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.ChangePeersResponse;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.GetLeaderRequest;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.GetLeaderResponse;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.GetPeersRequest;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.GetPeersResponse;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.LearnersOpResponse;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.RemoveLearnersRequest;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.RemovePeerRequest;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.RemovePeerResponse;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.ResetLearnersRequest;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.ResetPeerRequest;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.SnapshotRequest;
+import org.apache.ignite.raft.jraft.rpc.CliRequests.TransferLeaderRequest;
+import org.apache.ignite.raft.jraft.rpc.Message;
+import org.apache.ignite.raft.jraft.rpc.RpcRequests.ErrorResponse;
+import org.apache.ignite.raft.jraft.rpc.impl.cli.CliClientServiceImpl;
+import org.apache.ignite.raft.jraft.util.Requires;
+import org.apache.ignite.raft.jraft.util.StringUtils;
+import org.apache.ignite.raft.jraft.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Cli service implementation.
+ */
+public class CliServiceImpl implements CliService {
+    private static final Logger LOG = LoggerFactory.getLogger(CliServiceImpl.class);
+
+    private CliOptions cliOptions;
+    private CliClientService cliClientService;
+
+    @Override
+    public synchronized boolean init(final CliOptions opts) {
+        Requires.requireNonNull(opts, "Null cli options");
+
+        if (this.cliClientService != null) {
+            return true;
+        }
+        this.cliOptions = opts;
+        this.cliClientService = new CliClientServiceImpl();
+        return this.cliClientService.init(this.cliOptions);
+    }
+
+    @Override
+    public synchronized void shutdown() {
+        if (this.cliClientService == null) {
+            return;
+        }
+        this.cliClientService.shutdown();
+        this.cliClientService = null;
+    }
+
+    @Override
+    public Status addPeer(final String groupId, final Configuration conf, final PeerId peer) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireNonNull(conf, "Null configuration");
+        Requires.requireNonNull(peer, "Null peer");
+
+        final PeerId leaderId = new PeerId();
+        final Status st = getLeader(groupId, conf, leaderId);
+        if (!st.isOk()) {
+            return st;
+        }
+
+        if (!this.cliClientService.connect(leaderId.getEndpoint())) {
+            return new Status(-1, "Fail to init channel to leader %s", leaderId);
+        }
+        final AddPeerRequest.Builder rb = AddPeerRequest.newBuilder() //
+            .setGroupId(groupId) //
+            .setLeaderId(leaderId.toString()) //
+            .setPeerId(peer.toString());
+
+        try {
+            final Message result = this.cliClientService.addPeer(leaderId.getEndpoint(), rb.build(), null).get();
+            if (result instanceof AddPeerResponse) {
+                final AddPeerResponse resp = (AddPeerResponse) result;
+                final Configuration oldConf = new Configuration();
+                for (final String peerIdStr : resp.getOldPeersList()) {
+                    final PeerId oldPeer = new PeerId();
+                    oldPeer.parse(peerIdStr);
+                    oldConf.addPeer(oldPeer);
+                }
+                final Configuration newConf = new Configuration();
+                for (final String peerIdStr : resp.getNewPeersList()) {
+                    final PeerId newPeer = new PeerId();
+                    newPeer.parse(peerIdStr);
+                    newConf.addPeer(newPeer);
+                }
+
+                LOG.info("Configuration of replication group {} changed from {} to {}.", groupId, oldConf, newConf);
+                return Status.OK();
+            }
+            else {
+                return statusFromResponse(result);
+            }
+
+        }
+        catch (final Exception e) {
+            return new Status(-1, e.getMessage());
+        }
+    }
+
+    private Status statusFromResponse(final Message result) {
+        final ErrorResponse resp = (ErrorResponse) result;
+        return new Status(resp.getErrorCode(), resp.getErrorMsg());
+    }
+
+    @Override
+    public Status removePeer(final String groupId, final Configuration conf, final PeerId peer) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireNonNull(conf, "Null configuration");
+        Requires.requireNonNull(peer, "Null peer");
+        Requires.requireTrue(!peer.isEmpty(), "Removing peer is blank");
+
+        final PeerId leaderId = new PeerId();
+        final Status st = getLeader(groupId, conf, leaderId);
+        if (!st.isOk()) {
+            return st;
+        }
+
+        if (!this.cliClientService.connect(leaderId.getEndpoint())) {
+            return new Status(-1, "Fail to init channel to leader %s", leaderId);
+        }
+
+        final RemovePeerRequest.Builder rb = RemovePeerRequest.newBuilder() //
+            .setGroupId(groupId) //
+            .setLeaderId(leaderId.toString()) //
+            .setPeerId(peer.toString());
+
+        try {
+            final Message result = this.cliClientService.removePeer(leaderId.getEndpoint(), rb.build(), null).get();
+            if (result instanceof RemovePeerResponse) {
+                final RemovePeerResponse resp = (RemovePeerResponse) result;
+                final Configuration oldConf = new Configuration();
+                for (final String peerIdStr : resp.getOldPeersList()) {
+                    final PeerId oldPeer = new PeerId();
+                    oldPeer.parse(peerIdStr);
+                    oldConf.addPeer(oldPeer);
+                }
... 53020 lines suppressed ...