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 2020/12/29 10:27:28 UTC

[ignite-3] 01/02: IGNITE-13885 wip.

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

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

commit 899f44af160ae8e73dafc482f97b027a086a0e1c
Author: Alexey Scherbakov <al...@gmail.com>
AuthorDate: Mon Dec 28 13:06:10 2020 +0300

    IGNITE-13885 wip.
---
 modules/raft/pom.xml                               |  118 +
 .../java/com/alipay/sofa/jraft/CliService.java     |  192 ++
 .../main/java/com/alipay/sofa/jraft/Closure.java   |   34 +
 .../main/java/com/alipay/sofa/jraft/FSMCaller.java |  123 +
 .../main/java/com/alipay/sofa/jraft/Iterator.java  |   75 +
 .../com/alipay/sofa/jraft/JRaftServiceFactory.java |   62 +
 .../java/com/alipay/sofa/jraft/JRaftUtils.java     |  152 +
 .../main/java/com/alipay/sofa/jraft/Lifecycle.java |   39 +
 .../src/main/java/com/alipay/sofa/jraft/Node.java  |  327 ++
 .../sofa/jraft/NodeDescribeSignalHandler.java      |   68 +
 .../java/com/alipay/sofa/jraft/NodeManager.java    |  148 +
 .../sofa/jraft/NodeMetricsSignalHandler.java       |   77 +
 .../com/alipay/sofa/jraft/RaftGroupService.java    |  257 ++
 .../com/alipay/sofa/jraft/RaftServiceFactory.java  |   70 +
 .../com/alipay/sofa/jraft/ReadOnlyService.java     |   53 +
 .../com/alipay/sofa/jraft/ReplicatorGroup.java     |  229 ++
 .../java/com/alipay/sofa/jraft/RouteTable.java     |  384 +++
 .../java/com/alipay/sofa/jraft/StateMachine.java   |  144 +
 .../main/java/com/alipay/sofa/jraft/Status.java    |  233 ++
 .../sofa/jraft/ThreadPoolMetricsSignalHandler.java |   60 +
 .../alipay/sofa/jraft/closure/CatchUpClosure.java  |   72 +
 .../alipay/sofa/jraft/closure/ClosureQueue.java    |   80 +
 .../sofa/jraft/closure/ClosureQueueImpl.java       |  145 +
 .../alipay/sofa/jraft/closure/JoinableClosure.java |   58 +
 .../sofa/jraft/closure/LoadSnapshotClosure.java    |   37 +
 .../sofa/jraft/closure/ReadIndexClosure.java       |  166 +
 .../sofa/jraft/closure/SaveSnapshotClosure.java    |   39 +
 .../sofa/jraft/closure/SynchronizedClosure.java    |   83 +
 .../com/alipay/sofa/jraft/closure/TaskClosure.java |   35 +
 .../com/alipay/sofa/jraft/conf/Configuration.java  |  324 ++
 .../alipay/sofa/jraft/conf/ConfigurationEntry.java |  129 +
 .../sofa/jraft/conf/ConfigurationManager.java      |  110 +
 .../java/com/alipay/sofa/jraft/core/BallotBox.java |  283 ++
 .../com/alipay/sofa/jraft/core/CliServiceImpl.java |  687 ++++
 .../jraft/core/DefaultJRaftServiceFactory.java     |   69 +
 .../alipay/sofa/jraft/core/ElectionPriority.java   |   40 +
 .../com/alipay/sofa/jraft/core/FSMCallerImpl.java  |  729 ++++
 .../com/alipay/sofa/jraft/core/IteratorImpl.java   |  161 +
 .../alipay/sofa/jraft/core/IteratorWrapper.java    |   75 +
 .../java/com/alipay/sofa/jraft/core/NodeImpl.java  | 3464 ++++++++++++++++++++
 .../com/alipay/sofa/jraft/core/NodeMetrics.java    |  107 +
 .../sofa/jraft/core/ReadOnlyServiceImpl.java       |  423 +++
 .../com/alipay/sofa/jraft/core/Replicator.java     | 1788 ++++++++++
 .../sofa/jraft/core/ReplicatorGroupImpl.java       |  312 ++
 .../com/alipay/sofa/jraft/core/ReplicatorType.java |   34 +
 .../java/com/alipay/sofa/jraft/core/Scheduler.java |   88 +
 .../java/com/alipay/sofa/jraft/core/State.java     |   39 +
 .../sofa/jraft/core/StateMachineAdapter.java       |  109 +
 .../com/alipay/sofa/jraft/core/TimerManager.java   |   71 +
 .../java/com/alipay/sofa/jraft/entity/Ballot.java  |  141 +
 .../com/alipay/sofa/jraft/entity/Checksum.java     |   43 +
 .../com/alipay/sofa/jraft/entity/EnumOutter.java   |  149 +
 .../sofa/jraft/entity/LeaderChangeContext.java     |  114 +
 .../sofa/jraft/entity/LocalFileMetaOutter.java     |   81 +
 .../sofa/jraft/entity/LocalStorageOutter.java      |   62 +
 .../com/alipay/sofa/jraft/entity/LogEntry.java     |  291 ++
 .../java/com/alipay/sofa/jraft/entity/LogId.java   |  125 +
 .../java/com/alipay/sofa/jraft/entity/NodeId.java  |   95 +
 .../java/com/alipay/sofa/jraft/entity/PeerId.java  |  278 ++
 .../com/alipay/sofa/jraft/entity/RaftOutter.java   |  116 +
 .../alipay/sofa/jraft/entity/ReadIndexState.java   |   65 +
 .../alipay/sofa/jraft/entity/ReadIndexStatus.java  |   57 +
 .../java/com/alipay/sofa/jraft/entity/Task.java    |  174 +
 .../java/com/alipay/sofa/jraft/entity/UserLog.java |   66 +
 .../sofa/jraft/entity/codec/AutoDetectDecoder.java |   50 +
 .../entity/codec/DefaultLogEntryCodecFactory.java  |   63 +
 .../jraft/entity/codec/LogEntryCodecFactory.java   |   37 +
 .../sofa/jraft/entity/codec/LogEntryDecoder.java   |   35 +
 .../sofa/jraft/entity/codec/LogEntryEncoder.java   |   35 +
 .../entity/codec/v1/LogEntryV1CodecFactory.java    |   57 +
 .../sofa/jraft/entity/codec/v1/V1Decoder.java      |  115 +
 .../sofa/jraft/entity/codec/v1/V1Encoder.java      |  131 +
 .../sofa/jraft/error/InvokeTimeoutException.java   |   44 +
 .../alipay/sofa/jraft/error/JRaftException.java    |   45 +
 .../jraft/error/LogEntryCorruptedException.java    |   53 +
 .../jraft/error/LogIndexOutOfBoundsException.java  |   56 +
 .../sofa/jraft/error/LogNotFoundException.java     |   44 +
 .../jraft/error/MessageClassNotFoundException.java |   49 +
 .../com/alipay/sofa/jraft/error/RaftError.java     |  284 ++
 .../com/alipay/sofa/jraft/error/RaftException.java |   82 +
 .../alipay/sofa/jraft/error/RemotingException.java |   46 +
 .../sofa/jraft/error/RetryAgainException.java      |   52 +
 .../alipay/sofa/jraft/option/BallotBoxOptions.java |   49 +
 .../alipay/sofa/jraft/option/BootstrapOptions.java |  129 +
 .../com/alipay/sofa/jraft/option/CliOptions.java   |   46 +
 .../com/alipay/sofa/jraft/option/CopyOptions.java  |   55 +
 .../alipay/sofa/jraft/option/FSMCallerOptions.java |  100 +
 .../sofa/jraft/option/LogManagerOptions.java       |   99 +
 .../sofa/jraft/option/LogStorageOptions.java       |   49 +
 .../com/alipay/sofa/jraft/option/NodeOptions.java  |  443 +++
 .../sofa/jraft/option/RaftMetaStorageOptions.java  |   36 +
 .../com/alipay/sofa/jraft/option/RaftOptions.java  |  270 ++
 .../alipay/sofa/jraft/option/ReadOnlyOption.java   |   35 +
 .../sofa/jraft/option/ReadOnlyServiceOptions.java  |   56 +
 .../sofa/jraft/option/ReplicatorGroupOptions.java  |  124 +
 .../sofa/jraft/option/ReplicatorOptions.java       |  218 ++
 .../com/alipay/sofa/jraft/option/RpcOptions.java   |  113 +
 .../sofa/jraft/option/SnapshotCopierOptions.java   |   80 +
 .../sofa/jraft/option/SnapshotExecutorOptions.java |  107 +
 .../alipay/sofa/jraft/rpc/CliClientService.java    |  156 +
 .../com/alipay/sofa/jraft/rpc/CliRequests.java     |  531 +++
 .../com/alipay/sofa/jraft/rpc/ClientService.java   |   78 +
 .../java/com/alipay/sofa/jraft/rpc/Connection.java |   56 +
 .../alipay/sofa/jraft/rpc/HasErrorResponse.java    |    5 +
 .../com/alipay/sofa/jraft/rpc/InvokeCallback.java  |   31 +
 .../com/alipay/sofa/jraft/rpc/InvokeContext.java   |   56 +
 .../java/com/alipay/sofa/jraft/rpc/Message.java    |    4 +
 .../alipay/sofa/jraft/rpc/RaftClientService.java   |  112 +
 .../com/alipay/sofa/jraft/rpc/RaftRpcFactory.java  |  108 +
 .../sofa/jraft/rpc/RaftRpcServerFactory.java       |  146 +
 .../alipay/sofa/jraft/rpc/RaftServerService.java   |   88 +
 .../java/com/alipay/sofa/jraft/rpc/RpcClient.java  |  110 +
 .../java/com/alipay/sofa/jraft/rpc/RpcContext.java |   44 +
 .../com/alipay/sofa/jraft/rpc/RpcProcessor.java    |   75 +
 .../alipay/sofa/jraft/rpc/RpcRequestClosure.java   |   79 +
 .../alipay/sofa/jraft/rpc/RpcRequestProcessor.java |   73 +
 .../com/alipay/sofa/jraft/rpc/RpcRequests.java     |  461 +++
 .../alipay/sofa/jraft/rpc/RpcResponseClosure.java  |   38 +
 .../sofa/jraft/rpc/RpcResponseClosureAdapter.java  |   41 +
 .../alipay/sofa/jraft/rpc/RpcResponseFactory.java  |   82 +
 .../java/com/alipay/sofa/jraft/rpc/RpcServer.java  |   47 +
 .../java/com/alipay/sofa/jraft/rpc/RpcUtils.java   |  134 +
 .../sofa/jraft/rpc/impl/AbstractClientService.java |  297 ++
 .../sofa/jraft/rpc/impl/BoltRaftRpcFactory.java    |   98 +
 .../alipay/sofa/jraft/rpc/impl/BoltRpcClient.java  |  193 ++
 .../alipay/sofa/jraft/rpc/impl/BoltRpcServer.java  |  179 +
 .../rpc/impl/ConnectionClosedEventListener.java    |   28 +
 .../com/alipay/sofa/jraft/rpc/impl/FutureImpl.java |  242 ++
 .../sofa/jraft/rpc/impl/PingRequestProcessor.java  |   45 +
 .../rpc/impl/cli/AddLearnersRequestProcessor.java  |  101 +
 .../rpc/impl/cli/AddPeerRequestProcessor.java      |   93 +
 .../rpc/impl/cli/BaseCliRequestProcessor.java      |  148 +
 .../rpc/impl/cli/ChangePeersRequestProcessor.java  |   91 +
 .../jraft/rpc/impl/cli/CliClientServiceImpl.java   |  129 +
 .../rpc/impl/cli/GetLeaderRequestProcessor.java    |  107 +
 .../rpc/impl/cli/GetPeersRequestProcessor.java     |   75 +
 .../impl/cli/RemoveLearnersRequestProcessor.java   |   96 +
 .../rpc/impl/cli/RemovePeerRequestProcessor.java   |   87 +
 .../impl/cli/ResetLearnersRequestProcessor.java    |   97 +
 .../rpc/impl/cli/ResetPeerRequestProcessor.java    |   80 +
 .../rpc/impl/cli/SnapshotRequestProcessor.java     |   61 +
 .../impl/cli/TransferLeaderRequestProcessor.java   |   74 +
 .../impl/core/AppendEntriesRequestProcessor.java   |  517 +++
 .../ClientServiceConnectionEventProcessor.java     |   56 +
 .../rpc/impl/core/DefaultRaftClientService.java    |  158 +
 .../rpc/impl/core/GetFileRequestProcessor.java     |   50 +
 .../impl/core/InstallSnapshotRequestProcessor.java |   60 +
 .../jraft/rpc/impl/core/NodeRequestProcessor.java  |   73 +
 .../rpc/impl/core/ReadIndexRequestProcessor.java   |   73 +
 .../rpc/impl/core/RequestVoteRequestProcessor.java |   64 +
 .../rpc/impl/core/TimeoutNowRequestProcessor.java  |   59 +
 .../com/alipay/sofa/jraft/storage/FileService.java |  153 +
 .../com/alipay/sofa/jraft/storage/LogManager.java  |  241 ++
 .../com/alipay/sofa/jraft/storage/LogStorage.java  |   83 +
 .../alipay/sofa/jraft/storage/RaftMetaStorage.java |   56 +
 .../sofa/jraft/storage/SnapshotExecutor.java       |   98 +
 .../alipay/sofa/jraft/storage/SnapshotStorage.java |   74 +
 .../sofa/jraft/storage/SnapshotThrottle.java       |   34 +
 .../com/alipay/sofa/jraft/storage/Storage.java     |   27 +
 .../jraft/storage/impl/LocalRaftMetaStorage.java   |  195 ++
 .../sofa/jraft/storage/impl/LogManagerImpl.java    | 1190 +++++++
 .../sofa/jraft/storage/impl/RocksDBLogStorage.java |  743 +++++
 .../alipay/sofa/jraft/storage/io/FileReader.java   |   57 +
 .../sofa/jraft/storage/io/LocalDirReader.java      |   97 +
 .../alipay/sofa/jraft/storage/io/ProtoBufFile.java |  126 +
 .../alipay/sofa/jraft/storage/log/AbortFile.java   |   74 +
 .../sofa/jraft/storage/log/CheckpointFile.java     |  119 +
 .../com/alipay/sofa/jraft/storage/log/LibC.java    |   60 +
 .../storage/log/RocksDBSegmentLogStorage.java      | 1216 +++++++
 .../alipay/sofa/jraft/storage/log/SegmentFile.java |  905 +++++
 .../sofa/jraft/storage/snapshot/Snapshot.java      |   58 +
 .../jraft/storage/snapshot/SnapshotCopier.java     |   53 +
 .../storage/snapshot/SnapshotExecutorImpl.java     |  743 +++++
 .../jraft/storage/snapshot/SnapshotReader.java     |   43 +
 .../jraft/storage/snapshot/SnapshotWriter.java     |   77 +
 .../snapshot/ThroughputSnapshotThrottle.java       |   82 +
 .../storage/snapshot/local/LocalSnapshot.java      |   62 +
 .../snapshot/local/LocalSnapshotCopier.java        |  439 +++
 .../snapshot/local/LocalSnapshotMetaTable.java     |  186 ++
 .../snapshot/local/LocalSnapshotReader.java        |  167 +
 .../snapshot/local/LocalSnapshotStorage.java       |  352 ++
 .../snapshot/local/LocalSnapshotWriter.java        |  140 +
 .../storage/snapshot/local/SnapshotFileReader.java |   92 +
 .../jraft/storage/snapshot/remote/CopySession.java |  302 ++
 .../storage/snapshot/remote/RemoteFileCopier.java  |  192 ++
 .../jraft/storage/snapshot/remote/Session.java     |   49 +
 .../sofa/jraft/util/AdaptiveBufAllocator.java      |  203 ++
 .../com/alipay/sofa/jraft/util/ArrayDeque.java     |  110 +
 .../alipay/sofa/jraft/util/AsciiStringUtil.java    |   59 +
 .../main/java/com/alipay/sofa/jraft/util/Bits.java |   50 +
 .../sofa/jraft/util/ByteBufferCollector.java       |  145 +
 .../com/alipay/sofa/jraft/util/ByteString.java     |   25 +
 .../java/com/alipay/sofa/jraft/util/Bytes.java     |  138 +
 .../java/com/alipay/sofa/jraft/util/BytesUtil.java |  183 ++
 .../java/com/alipay/sofa/jraft/util/CRC64.java     |  128 +
 .../java/com/alipay/sofa/jraft/util/Copiable.java  |   33 +
 .../com/alipay/sofa/jraft/util/CountDownEvent.java |   75 +
 .../java/com/alipay/sofa/jraft/util/CrcUtil.java   |   84 +
 .../alipay/sofa/jraft/util/DebugStatistics.java    |   36 +
 .../java/com/alipay/sofa/jraft/util/Describer.java |   70 +
 .../com/alipay/sofa/jraft/util/DirectExecutor.java |   38 +
 .../alipay/sofa/jraft/util/DisruptorBuilder.java   |   98 +
 .../alipay/sofa/jraft/util/DisruptorMetricSet.java |   50 +
 .../java/com/alipay/sofa/jraft/util/Endpoint.java  |   97 +
 .../sofa/jraft/util/ExecutorServiceHelper.java     |   73 +
 .../sofa/jraft/util/FileOutputSignalHandler.java   |   52 +
 .../alipay/sofa/jraft/util/HeapByteBufUtil.java    |  133 +
 .../main/java/com/alipay/sofa/jraft/util/Ints.java |   84 +
 .../alipay/sofa/jraft/util/JRaftServiceLoader.java |  357 ++
 .../alipay/sofa/jraft/util/JRaftSignalHandler.java |   26 +
 .../sofa/jraft/util/LogExceptionHandler.java       |   70 +
 .../jraft/util/LogScheduledThreadPoolExecutor.java |   94 +
 .../sofa/jraft/util/LogThreadPoolExecutor.java     |   99 +
 .../com/alipay/sofa/jraft/util/MetricReporter.java |  436 +++
 .../util/MetricScheduledThreadPoolExecutor.java    |   75 +
 .../sofa/jraft/util/MetricThreadPoolExecutor.java  |   80 +
 .../main/java/com/alipay/sofa/jraft/util/Mpsc.java |   46 +
 .../alipay/sofa/jraft/util/NamedThreadFactory.java |   70 +
 .../alipay/sofa/jraft/util/NonReentrantLock.java   |   92 +
 .../com/alipay/sofa/jraft/util/OnlyForTest.java    |   37 +
 .../java/com/alipay/sofa/jraft/util/Platform.java  |   69 +
 .../com/alipay/sofa/jraft/util/Recyclable.java     |   29 +
 .../sofa/jraft/util/RecyclableByteBufferList.java  |  136 +
 .../com/alipay/sofa/jraft/util/RecycleUtil.java    |   35 +
 .../java/com/alipay/sofa/jraft/util/Recyclers.java |  414 +++
 .../com/alipay/sofa/jraft/util/RepeatedTimer.java  |  295 ++
 .../java/com/alipay/sofa/jraft/util/Requires.java  |  107 +
 .../alipay/sofa/jraft/util/RpcFactoryHelper.java   |   37 +
 .../main/java/com/alipay/sofa/jraft/util/SPI.java  |   37 +
 .../com/alipay/sofa/jraft/util/SegmentList.java    |  392 +++
 .../com/alipay/sofa/jraft/util/SignalHelper.java   |  114 +
 .../sofa/jraft/util/StorageOptionsFactory.java     |  350 ++
 .../com/alipay/sofa/jraft/util/StringUtils.java    |  105 +
 .../alipay/sofa/jraft/util/SystemPropertyUtil.java |  188 ++
 .../com/alipay/sofa/jraft/util/ThreadHelper.java   |   52 +
 .../java/com/alipay/sofa/jraft/util/ThreadId.java  |  173 +
 .../sofa/jraft/util/ThreadPoolMetricRegistry.java  |   40 +
 .../sofa/jraft/util/ThreadPoolMetricSet.java       |   52 +
 .../com/alipay/sofa/jraft/util/ThreadPoolUtil.java |  283 ++
 .../java/com/alipay/sofa/jraft/util/Utils.java     |  451 +++
 .../jraft/util/concurrent/AdjustableSemaphore.java |  129 +
 .../jraft/util/concurrent/ConcurrentHashSet.java   |   95 +
 .../concurrent/DefaultExecutorChooserFactory.java  |   81 +
 .../DefaultFixedThreadsExecutorGroup.java          |  115 +
 .../DefaultFixedThreadsExecutorGroupFactory.java   |   81 +
 .../concurrent/DefaultSingleThreadExecutor.java    |  117 +
 .../util/concurrent/ExecutorChooserFactory.java    |   43 +
 .../util/concurrent/FixedThreadsExecutorGroup.java |   59 +
 .../FixedThreadsExecutorGroupFactory.java          |   43 +
 .../concurrent/LongHeldDetectingReadWriteLock.java |  153 +
 .../util/concurrent/MpscSingleThreadExecutor.java  |  401 +++
 .../util/concurrent/RejectedExecutionHandler.java  |   33 +
 .../util/concurrent/RejectedExecutionHandlers.java |   45 +
 .../util/concurrent/SingleThreadExecutor.java      |   44 +
 .../jraft/util/internal/IntegerFieldUpdater.java   |   27 +
 .../sofa/jraft/util/internal/LongFieldUpdater.java |   27 +
 .../jraft/util/internal/ReferenceFieldUpdater.java |   27 +
 .../internal/ReflectionIntegerFieldUpdater.java    |   51 +
 .../util/internal/ReflectionLongFieldUpdater.java  |   51 +
 .../internal/ReflectionReferenceFieldUpdater.java  |   52 +
 .../alipay/sofa/jraft/util/internal/ThrowUtil.java |   71 +
 .../util/internal/UnsafeIntegerFieldUpdater.java   |   49 +
 .../util/internal/UnsafeLongFieldUpdater.java      |   49 +
 .../util/internal/UnsafeReferenceFieldUpdater.java |   50 +
 .../sofa/jraft/util/internal/UnsafeUtf8Util.java   |  481 +++
 .../sofa/jraft/util/internal/UnsafeUtil.java       |  629 ++++
 .../alipay/sofa/jraft/util/internal/Updaters.java  |   81 +
 .../jraft/util/timer/DefaultRaftTimerFactory.java  |  242 ++
 .../alipay/sofa/jraft/util/timer/DefaultTimer.java |  123 +
 .../sofa/jraft/util/timer/HashedWheelTimer.java    |  758 +++++
 .../sofa/jraft/util/timer/RaftTimerFactory.java    |   39 +
 .../com/alipay/sofa/jraft/util/timer/Timeout.java  |   55 +
 .../com/alipay/sofa/jraft/util/timer/Timer.java    |   51 +
 .../alipay/sofa/jraft/util/timer/TimerTask.java    |   32 +
 pom.xml                                            |    1 +
 275 files changed, 43507 insertions(+)

diff --git a/modules/raft/pom.xml b/modules/raft/pom.xml
new file mode 100644
index 0000000..a01aefb
--- /dev/null
+++ b/modules/raft/pom.xml
@@ -0,0 +1,118 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    POM file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <groupId>org.apache.ignite</groupId>
+    <artifactId>raft</artifactId>
+    <version>3.0-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
+
+    <properties>
+        <maven.compiler.source>1.8</maven.compiler.source>
+        <maven.compiler.target>1.8</maven.compiler.target>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <version>3.4.6</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.lmax</groupId>
+            <artifactId>disruptor</artifactId>
+            <version>3.3.7</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>4.12</version>
+            <scope>test</scope>
+        </dependency>
+        <!-- logging -->
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>1.7.21</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-api</artifactId>
+            <version>2.13.2</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-core</artifactId>
+            <version>2.13.2</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-slf4j-impl</artifactId>
+            <version>2.13.2</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-jcl</artifactId>
+            <version>2.13.2</version>
+            <scope>test</scope>
+        </dependency>
+        <!-- commons -->
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+            <version>2.4</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+            <version>2.6</version>
+        </dependency>
+        <dependency>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+            <version>3.0.2</version>
+        </dependency>
+        <dependency>
+            <groupId>io.dropwizard.metrics</groupId>
+            <artifactId>metrics-core</artifactId>
+            <version>4.0.2</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>3.8.1</version>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/CliService.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/CliService.java
new file mode 100644
index 0000000..ca8461d
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/CliService.java
@@ -0,0 +1,192 @@
+/*
+ * 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 com.alipay.sofa.jraft;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.alipay.sofa.jraft.conf.Configuration;
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.option.CliOptions;
+
+/**
+ * Client command-line service
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-09 4:05:35 PM
+ */
+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
+     * @since 1.3.0
+     *
+     */
+    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
+     * @since 1.3.0
+     *
+     */
+    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
+     * @since 1.3.0
+     *
+     */
+    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
+     * @since 1.3.0
+     */
+    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/main/java/com/alipay/sofa/jraft/Closure.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/Closure.java
new file mode 100644
index 0000000..7d51264
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/Closure.java
@@ -0,0 +1,34 @@
+/*
+ * 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 com.alipay.sofa.jraft;
+
+/**
+ * Callback closure.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-03 11:07:05 AM
+ */
+public interface Closure {
+
+    /**
+     * Called when task is done.
+     *
+     * @param status the task status.
+     */
+    void run(final Status status);
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/FSMCaller.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/FSMCaller.java
new file mode 100644
index 0000000..71b003e
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/FSMCaller.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 com.alipay.sofa.jraft;
+
+import com.alipay.sofa.jraft.closure.LoadSnapshotClosure;
+import com.alipay.sofa.jraft.closure.SaveSnapshotClosure;
+import com.alipay.sofa.jraft.entity.LeaderChangeContext;
+import com.alipay.sofa.jraft.error.RaftException;
+import com.alipay.sofa.jraft.option.FSMCallerOptions;
+import com.alipay.sofa.jraft.util.Describer;
+
+/**
+ * Finite state machine caller.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-03 11:07:52 AM
+ */
+public interface FSMCaller extends Lifecycle<FSMCallerOptions>, Describer {
+
+    /**
+     * Listen on lastAppliedLogIndex update events.
+     *
+     * @author dennis
+     */
+    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/com/alipay/sofa/jraft/Iterator.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/Iterator.java
new file mode 100644
index 0000000..c80de1f
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/Iterator.java
@@ -0,0 +1,75 @@
+/*
+ * 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 com.alipay.sofa.jraft;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Iterator over a batch of committed tasks.
+ * @see StateMachine#onApply(Iterator)
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-03 3:20:15 PM
+ */
+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/com/alipay/sofa/jraft/JRaftServiceFactory.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/JRaftServiceFactory.java
new file mode 100644
index 0000000..74fa168
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/JRaftServiceFactory.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.alipay.sofa.jraft;
+
+import com.alipay.sofa.jraft.entity.codec.LogEntryCodecFactory;
+import com.alipay.sofa.jraft.option.NodeOptions;
+import com.alipay.sofa.jraft.option.RaftOptions;
+import com.alipay.sofa.jraft.storage.LogStorage;
+import com.alipay.sofa.jraft.storage.RaftMetaStorage;
+import com.alipay.sofa.jraft.storage.SnapshotStorage;
+
+/**
+ * Abstract factory to create services for SOFAJRaft.
+ * @author boyan(boyan@antfin.com)
+ * @since  1.2.6
+ */
+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();
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/JRaftUtils.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/JRaftUtils.java
new file mode 100644
index 0000000..020137f
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/JRaftUtils.java
@@ -0,0 +1,152 @@
+/*
+ * 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 com.alipay.sofa.jraft;
+
+import java.util.concurrent.Executor;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+
+import org.apache.commons.lang.StringUtils;
+
+import com.alipay.sofa.jraft.conf.Configuration;
+import com.alipay.sofa.jraft.core.NodeImpl;
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.option.BootstrapOptions;
+import com.alipay.sofa.jraft.util.Endpoint;
+import com.alipay.sofa.jraft.util.JRaftServiceLoader;
+import com.alipay.sofa.jraft.util.NamedThreadFactory;
+import com.alipay.sofa.jraft.util.ThreadPoolUtil;
+import com.alipay.sofa.jraft.util.timer.RaftTimerFactory;
+
+/**
+ * Some helper methods for jraft usage.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-23 3:48:45 PM
+ */
+public final class JRaftUtils {
+
+    private final static RaftTimerFactory TIMER_FACTORY = JRaftServiceLoader.load(RaftTimerFactory.class) //
+                                                            .first();
+
+    /**
+     * Get raft timer factory.
+     *
+     * @return {@link RaftTimerFactory}
+     */
+    public static RaftTimerFactory raftTimerFactory() {
+        return TIMER_FACTORY;
+    }
+
+    /**
+     * 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 Executor 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 SynchronousQueue<>()) //
+            .threadFactory(createThreadFactory(prefix)) //
+            .build();
+    }
+
+    /**
+     * Create a thread factory.
+     *
+     * @param prefixName the prefix name of thread
+     * @return a new {@link ThreadFactory} instance
+     *
+     * @since 0.0.3
+     */
+    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/main/java/com/alipay/sofa/jraft/Lifecycle.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/Lifecycle.java
new file mode 100644
index 0000000..85bff23
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/Lifecycle.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.alipay.sofa.jraft;
+
+/**
+ * Service life cycle mark interface.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Mar-12 3:47:04 PM
+ */
+public interface Lifecycle<T> {
+
+    /**
+     * Initialize the service.
+     *
+     * @return true when successes.
+     */
+    boolean init(final T opts);
+
+    /**
+     * Dispose the resources for service.
+     */
+    void shutdown();
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/Node.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/Node.java
new file mode 100644
index 0000000..2eab9a3
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/Node.java
@@ -0,0 +1,327 @@
+/*
+ * 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 com.alipay.sofa.jraft;
+
+import java.util.List;
+
+import com.alipay.sofa.jraft.closure.ReadIndexClosure;
+import com.alipay.sofa.jraft.conf.Configuration;
+import com.alipay.sofa.jraft.core.NodeMetrics;
+import com.alipay.sofa.jraft.core.Replicator;
+import com.alipay.sofa.jraft.entity.NodeId;
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.entity.Task;
+import com.alipay.sofa.jraft.entity.UserLog;
+import com.alipay.sofa.jraft.error.LogIndexOutOfBoundsException;
+import com.alipay.sofa.jraft.error.LogNotFoundException;
+import com.alipay.sofa.jraft.option.NodeOptions;
+import com.alipay.sofa.jraft.option.RaftOptions;
+import com.alipay.sofa.jraft.util.Describer;
+
+/**
+ * A raft replica node.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-03 4:06:55 PM
+ */
+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
+     *
+     * @since 0.0.3
+     */
+    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
+     * @since 1.2.6
+     */
+    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
+     * @since 1.3.0
+     */
+    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
+     * @since 1.3.0
+     */
+    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.un()
+     * 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
+     * @since 1.3.0
+     */
+    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
+     * @since 1.3.0
+     */
+    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
+     * @since 1.3.0
+     */
+    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);
+
+    /**
+     * SOFAJRaft 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> getReplicatorStatueListeners();
+
+    /**
+     * Get the node's target election priority value.
+     *
+     * @return node's target election priority value.
+     * @since 1.3.0
+     */
+    int getNodeTargetPriority();
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/NodeDescribeSignalHandler.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/NodeDescribeSignalHandler.java
new file mode 100644
index 0000000..42f6910
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/NodeDescribeSignalHandler.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.alipay.sofa.jraft;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.util.Describer;
+import com.alipay.sofa.jraft.util.FileOutputSignalHandler;
+import com.alipay.sofa.jraft.util.SystemPropertyUtil;
+
+/**
+ *
+ * @author jiachun.fjc
+ */
+public class NodeDescribeSignalHandler extends FileOutputSignalHandler {
+
+    private static Logger       LOG       = LoggerFactory.getLogger(NodeDescribeSignalHandler.class);
+
+    private static final String DIR       = SystemPropertyUtil.get("jraft.signal.node.describe.dir", "");
+    private static final String BASE_NAME = "node_describe.log";
+
+    @Override
+    public void handle(final String signalName) {
+        final List<Node> nodes = NodeManager.getInstance().getAllNodes();
+        if (nodes.isEmpty()) {
+            return;
+        }
+
+        try {
+            final File file = getOutputFile(DIR, BASE_NAME);
+
+            LOG.info("Describing raft nodes with signal: {} to file: {}.", signalName, file);
+
+            try (final PrintWriter out = new PrintWriter(new OutputStreamWriter(new FileOutputStream(file, true),
+                StandardCharsets.UTF_8))) {
+                final Describer.Printer printer = new Describer.DefaultPrinter(out);
+                for (final Node node : nodes) {
+                    node.describe(printer);
+                }
+            }
+        } catch (final IOException e) {
+            LOG.error("Fail to describe nodes: {}.", nodes, e);
+        }
+    }
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/NodeManager.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/NodeManager.java
new file mode 100644
index 0000000..236ebf7
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/NodeManager.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.alipay.sofa.jraft;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.stream.Collectors;
+
+import com.alipay.sofa.jraft.entity.NodeId;
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.util.Endpoint;
+import com.alipay.sofa.jraft.util.OnlyForTest;
+import com.alipay.sofa.jraft.util.Utils;
+import com.alipay.sofa.jraft.util.concurrent.ConcurrentHashSet;
+
+/**
+ * Raft nodes manager.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Mar-22 5:58:23 PM
+ */
+public class NodeManager {
+
+    private static final NodeManager                INSTANCE = new NodeManager();
+
+    private final ConcurrentMap<NodeId, Node>       nodeMap  = new ConcurrentHashMap<>();
+    private final ConcurrentMap<String, List<Node>> groupMap = new ConcurrentHashMap<>();
+    private final ConcurrentHashSet<Endpoint>       addrSet  = new ConcurrentHashSet<>();
+
+    public static NodeManager getInstance() {
+        return INSTANCE;
+    }
+
+    /**
+     * Return true when RPC service is registered.
+     */
+    public boolean serverExists(final Endpoint addr) {
+        if (addr.getIp().equals(Utils.IP_ANY)) {
+            return this.addrSet.contains(new Endpoint(Utils.IP_ANY, addr.getPort()));
+        }
+        return this.addrSet.contains(addr);
+    }
+
+    /**
+     * Remove a RPC service address.
+     */
+    public boolean removeAddress(final Endpoint addr) {
+        return this.addrSet.remove(addr);
+    }
+
+    /**
+     * Adds a RPC service address.
+     */
+    public void addAddress(final Endpoint addr) {
+        this.addrSet.add(addr);
+    }
+
+    /**
+     * Adds a node.
+     */
+    public boolean add(final Node node) {
+        // check address ok?
+        if (!serverExists(node.getNodeId().getPeerId().getEndpoint())) {
+            return false;
+        }
+        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 = Collections.synchronizedList(new ArrayList<>());
+                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();
+        this.addrSet.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());
+    }
+
+    private NodeManager() {
+    }
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/NodeMetricsSignalHandler.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/NodeMetricsSignalHandler.java
new file mode 100644
index 0000000..2d128cb
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/NodeMetricsSignalHandler.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 com.alipay.sofa.jraft;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.core.NodeMetrics;
+import com.alipay.sofa.jraft.util.FileOutputSignalHandler;
+import com.alipay.sofa.jraft.util.MetricReporter;
+import com.alipay.sofa.jraft.util.SystemPropertyUtil;
+import com.codahale.metrics.MetricRegistry;
+
+/**
+ *
+ * @author jiachun.fjc
+ */
+public class NodeMetricsSignalHandler extends FileOutputSignalHandler {
+
+    private static Logger       LOG       = LoggerFactory.getLogger(NodeMetricsSignalHandler.class);
+
+    private static final String DIR       = SystemPropertyUtil.get("jraft.signal.node.metrics.dir", "");
+    private static final String BASE_NAME = "node_metrics.log";
+
+    @Override
+    public void handle(final String signalName) {
+        final List<Node> nodes = NodeManager.getInstance().getAllNodes();
+        if (nodes.isEmpty()) {
+            return;
+        }
+
+        try {
+            final File file = getOutputFile(DIR, BASE_NAME);
+
+            LOG.info("Printing raft nodes metrics with signal: {} to file: {}.", signalName, file);
+
+            try (final PrintStream out = new PrintStream(new FileOutputStream(file, true))) {
+                for (final Node node : nodes) {
+                    final NodeMetrics nodeMetrics = node.getNodeMetrics();
+                    final MetricRegistry registry = nodeMetrics.getMetricRegistry();
+                    if (registry == null) {
+                        LOG.warn("Node: {} received a signal to print metric, but it does not have metric enabled.",
+                            node);
+                        continue;
+                    }
+                    final MetricReporter reporter = MetricReporter.forRegistry(registry) //
+                        .outputTo(out) //
+                        .prefixedWith("-- " + node.getNodeId()) //
+                        .build();
+                    reporter.report();
+                }
+            }
+        } catch (final IOException e) {
+            LOG.error("Fail to print nodes metrics: {}.", nodes, e);
+        }
+    }
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/RaftGroupService.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/RaftGroupService.java
new file mode 100644
index 0000000..df9f9d1
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/RaftGroupService.java
@@ -0,0 +1,257 @@
+/*
+ * 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 com.alipay.sofa.jraft;
+
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.option.NodeOptions;
+import com.alipay.sofa.jraft.option.RpcOptions;
+import com.alipay.sofa.jraft.rpc.ProtobufMsgFactory;
+import com.alipay.sofa.jraft.rpc.RaftRpcServerFactory;
+import com.alipay.sofa.jraft.rpc.RpcServer;
+import com.alipay.sofa.jraft.util.Endpoint;
+import com.alipay.sofa.jraft.util.Utils;
+
+/**
+ * A framework to implement a raft group service.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-08 7:53:03 PM
+ */
+public class RaftGroupService {
+
+    private static final Logger LOG     = LoggerFactory.getLogger(RaftGroupService.class);
+
+    static {
+        ProtobufMsgFactory.load();
+    }
+
+    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;
+
+    public RaftGroupService(final String groupId, final PeerId serverId, final NodeOptions nodeOptions) {
+        this(groupId, serverId, nodeOptions, RaftRpcServerFactory.createRaftRpcServer(serverId.getEndpoint(),
+            JRaftUtils.createExecutor("RAFT-RPC-executor-", nodeOptions.getRaftRpcThreadPoolSize()),
+            JRaftUtils.createExecutor("CLI-RPC-executor-", nodeOptions.getCliRpcThreadPoolSize())));
+    }
+
+    public RaftGroupService(final String groupId, final PeerId serverId, final NodeOptions nodeOptions,
+                            final RpcServer rpcServer) {
+        this(groupId, serverId, nodeOptions, rpcServer, false);
+    }
+
+    public RaftGroupService(final String groupId, final PeerId serverId, final NodeOptions nodeOptions,
+                            final RpcServer rpcServer, final boolean sharedRpcServer) {
+        super();
+        this.groupId = groupId;
+        this.serverId = serverId;
+        this.nodeOptions = nodeOptions;
+        this.rpcServer = rpcServer;
+        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);
+        }
+        //Adds RPC server to Server.
+        NodeManager.getInstance().addAddress(this.serverId.getEndpoint());
+
+        this.node = RaftServiceFactory.createAndInitRaftNode(this.groupId, this.serverId, this.nodeOptions);
+        if (startRpcServer) {
+            this.rpcServer.init(null);
+        } else {
+            LOG.warn("RPC server is not started in RaftGroupService.");
+        }
+        this.started = true;
+        LOG.info("Start the RaftGroupService successfully.");
+        return this.node;
+    }
+
+    /**
+     * Block thread to wait the server shutdown.
+     *
+     * @throws InterruptedException if the current thread is interrupted
+     *         while waiting
+     */
+    public synchronized void join() throws InterruptedException {
+        if (this.node != null) {
+            this.node.join();
+            this.node = null;
+        }
+    }
+
+    public synchronized void shutdown() {
+        if (!this.started) {
+            return;
+        }
+        if (this.rpcServer != null) {
+            try {
+                if (!this.sharedRpcServer) {
+                    this.rpcServer.shutdown();
+                }
+            } catch (final Exception ignored) {
+                // ignore
+            }
+            this.rpcServer = null;
+        }
+        this.node.shutdown();
+        NodeManager.getInstance().removeAddress(this.serverId.getEndpoint());
+        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/com/alipay/sofa/jraft/RaftServiceFactory.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/RaftServiceFactory.java
new file mode 100644
index 0000000..8aaea5c
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/RaftServiceFactory.java
@@ -0,0 +1,70 @@
+/*
+ * 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 com.alipay.sofa.jraft;
+
+import com.alipay.sofa.jraft.core.CliServiceImpl;
+import com.alipay.sofa.jraft.core.NodeImpl;
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.option.CliOptions;
+import com.alipay.sofa.jraft.option.NodeOptions;
+
+/**
+ * Service factory to create raft services, such as Node/CliService etc.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-May-03 11:06:02 AM
+ */
+public final class RaftServiceFactory {
+
+    /**
+     * Create a raft node with group id and it's serverId.
+     */
+    public static Node createRaftNode(final String groupId, final PeerId serverId) {
+        return new NodeImpl(groupId, serverId);
+    }
+
+    /**
+     * Create and initialize a raft node with node options.
+     * Throw {@link IllegalStateException} when fail to initialize.
+     */
+    public static Node createAndInitRaftNode(final String groupId, final PeerId serverId, final NodeOptions opts) {
+        final Node ret = createRaftNode(groupId, serverId);
+        if (!ret.init(opts)) {
+            throw new IllegalStateException("Fail to init node, please see the logs to find the reason.");
+        }
+        return ret;
+    }
+
+    /**
+     * Create a {@link CliService} instance.
+     */
+    public static CliService createCliService() {
+        return new CliServiceImpl();
+    }
+
+    /**
+     * Create and initialize a CliService instance.
+     */
+    public static CliService createAndInitCliService(final CliOptions cliOptions) {
+        final CliService ret = createCliService();
+        if (!ret.init(cliOptions)) {
+            throw new IllegalStateException("Fail to init CliService");
+        }
+        return ret;
+    }
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/ReadOnlyService.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/ReadOnlyService.java
new file mode 100644
index 0000000..cd14364
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/ReadOnlyService.java
@@ -0,0 +1,53 @@
+/*
+ * 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 com.alipay.sofa.jraft;
+
+import com.alipay.sofa.jraft.closure.ReadIndexClosure;
+import com.alipay.sofa.jraft.error.RaftException;
+import com.alipay.sofa.jraft.option.ReadOnlyServiceOptions;
+
+/**
+ * The read-only query service.
+ *
+ * @author dennis
+ *
+ */
+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/com/alipay/sofa/jraft/ReplicatorGroup.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/ReplicatorGroup.java
new file mode 100644
index 0000000..933bfbf
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/ReplicatorGroup.java
@@ -0,0 +1,229 @@
+/*
+ * 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 com.alipay.sofa.jraft;
+
+import java.util.List;
+
+import com.alipay.sofa.jraft.closure.CatchUpClosure;
+import com.alipay.sofa.jraft.conf.ConfigurationEntry;
+import com.alipay.sofa.jraft.core.ReplicatorType;
+import com.alipay.sofa.jraft.entity.NodeId;
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.option.ReplicatorGroupOptions;
+import com.alipay.sofa.jraft.rpc.RpcRequests.AppendEntriesResponse;
+import com.alipay.sofa.jraft.rpc.RpcResponseClosure;
+import com.alipay.sofa.jraft.util.Describer;
+import com.alipay.sofa.jraft.util.ThreadId;
+
+/**
+ * Replicators in a raft group.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-08 5:35:26 PM
+ */
+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}).
+     * @see #addReplicator(PeerId, ReplicatorType)
+     *
+     * @param peer target peer
+     * @return true on success
+     */
+    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/com/alipay/sofa/jraft/RouteTable.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/RouteTable.java
new file mode 100644
index 0000000..4d11be7
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/RouteTable.java
@@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.alipay.sofa.jraft;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.StampedLock;
+
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.conf.Configuration;
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.error.RaftError;
+import com.alipay.sofa.jraft.rpc.CliClientService;
+import com.alipay.sofa.jraft.rpc.CliRequests;
+import com.alipay.sofa.jraft.rpc.RpcRequests;
+import com.alipay.sofa.jraft.util.Describer;
+import com.alipay.sofa.jraft.util.Requires;
+import com.alipay.sofa.jraft.rpc.Message;
+
+/**
+ * Maintain routes to raft groups.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-09 10:41:21 AM
+ */
+public class RouteTable implements Describer {
+
+    private static final Logger                    LOG            = LoggerFactory.getLogger(RouteTable.class);
+
+    private static final RouteTable                INSTANCE       = new RouteTable();
+
+    // Map<groupId, groupConf>
+    private final ConcurrentMap<String, GroupConf> groupConfTable = new ConcurrentHashMap<>();
+
+    public static RouteTable getInstance() {
+        return INSTANCE;
+    }
+
+    /**
+     * Update configuration of group in route table.
+     *
+     * @param groupId raft group id
+     * @param conf    configuration to update
+     * @return true on success
+     */
+    public boolean updateConfiguration(final String groupId, final Configuration conf) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireNonNull(conf, "Null configuration");
+
+        final GroupConf gc = getOrCreateGroupConf(groupId);
+        final StampedLock stampedLock = gc.stampedLock;
+        final long stamp = stampedLock.writeLock();
+        try {
+            gc.conf = conf;
+            if (gc.leader != null && !gc.conf.contains(gc.leader)) {
+                gc.leader = null;
+            }
+        } finally {
+            stampedLock.unlockWrite(stamp);
+        }
+        return true;
+    }
+
+    private GroupConf getOrCreateGroupConf(final String groupId) {
+        GroupConf gc = this.groupConfTable.get(groupId);
+        if (gc == null) {
+            gc = new GroupConf();
+            final GroupConf old = this.groupConfTable.putIfAbsent(groupId, gc);
+            if (old != null) {
+                gc = old;
+            }
+        }
+        return gc;
+    }
+
+    /**
+     * Update configuration of group in route table.
+     *
+     * @param groupId raft group id
+     * @param confStr configuration string
+     * @return true on success
+     */
+    public boolean updateConfiguration(final String groupId, final String confStr) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireTrue(!StringUtils.isBlank(confStr), "Blank configuration");
+
+        final Configuration conf = new Configuration();
+        if (conf.parse(confStr)) {
+            return updateConfiguration(groupId, conf);
+        } else {
+            LOG.error("Fail to parse confStr: {}", confStr);
+            return false;
+        }
+    }
+
+    /**
+     * Get the cached leader of the group, return it when found, null otherwise.
+     * Make sure calls {@link #refreshLeader(CliClientService, String, int)} already
+     * before invoke this method.
+     *
+     * @param groupId raft group id
+     * @return peer of leader
+     */
+    public PeerId selectLeader(final String groupId) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+
+        final GroupConf gc = this.groupConfTable.get(groupId);
+        if (gc == null) {
+            return null;
+        }
+        final StampedLock stampedLock = gc.stampedLock;
+        long stamp = stampedLock.tryOptimisticRead();
+        PeerId leader = gc.leader;
+        if (!stampedLock.validate(stamp)) {
+            stamp = stampedLock.readLock();
+            try {
+                leader = gc.leader;
+            } finally {
+                stampedLock.unlockRead(stamp);
+            }
+        }
+        return leader;
+    }
+
+    /**
+     * Update leader info.
+     *
+     * @param groupId raft group id
+     * @param leader  peer of leader
+     * @return true on success
+     */
+    public boolean updateLeader(final String groupId, final PeerId leader) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+
+        if (leader != null) {
+            // If leader presents, it should not be empty.
+            Requires.requireTrue(!leader.isEmpty(), "Empty leader");
+        }
+
+        final GroupConf gc = getOrCreateGroupConf(groupId);
+        final StampedLock stampedLock = gc.stampedLock;
+        final long stamp = stampedLock.writeLock();
+        try {
+            gc.leader = leader;
+        } finally {
+            stampedLock.unlockWrite(stamp);
+        }
+        return true;
+    }
+
+    /**
+     * Update leader info.
+     *
+     * @param groupId   raft group id
+     * @param leaderStr peer string of leader
+     * @return true on success
+     */
+    public boolean updateLeader(final String groupId, final String leaderStr) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireTrue(!StringUtils.isBlank(leaderStr), "Blank leader");
+
+        final PeerId leader = new PeerId();
+        if (leader.parse(leaderStr)) {
+            return updateLeader(groupId, leader);
+        } else {
+            LOG.error("Fail to parse leaderStr: {}", leaderStr);
+            return false;
+        }
+    }
+
+    /**
+     * Get the configuration by groupId, returns null when not found.
+     *
+     * @param groupId raft group id
+     * @return configuration of the group id
+     */
+    public Configuration getConfiguration(final String groupId) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+
+        final GroupConf gc = this.groupConfTable.get(groupId);
+        if (gc == null) {
+            return null;
+        }
+        final StampedLock stampedLock = gc.stampedLock;
+        long stamp = stampedLock.tryOptimisticRead();
+        Configuration conf = gc.conf;
+        if (!stampedLock.validate(stamp)) {
+            stamp = stampedLock.readLock();
+            try {
+                conf = gc.conf;
+            } finally {
+                stampedLock.unlockRead(stamp);
+            }
+        }
+        return conf;
+    }
+
+    /**
+     * Blocking the thread until query_leader finishes.
+     *
+     * @param groupId   raft group id
+     * @param timeoutMs timeout millis
+     * @return operation status
+     */
+    public Status refreshLeader(final CliClientService cliClientService, final String groupId, final int timeoutMs)
+                                                                                                                   throws InterruptedException,
+                                                                                                                   TimeoutException {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireTrue(timeoutMs > 0, "Invalid timeout: " + timeoutMs);
+
+        final Configuration conf = getConfiguration(groupId);
+        if (conf == null) {
+            return new Status(RaftError.ENOENT,
+                "Group %s is not registered in RouteTable, forgot to call updateConfiguration?", groupId);
+        }
+        final Status st = Status.OK();
+        final CliRequests.GetLeaderRequest.Builder rb = CliRequests.GetLeaderRequest.newBuilder();
+        rb.setGroupId(groupId);
+        final CliRequests.GetLeaderRequest request = rb.build();
+        TimeoutException timeoutException = null;
+        for (final PeerId peer : conf) {
+            if (!cliClientService.connect(peer.getEndpoint())) {
+                if (st.isOk()) {
+                    st.setError(-1, "Fail to init channel to %s", peer);
+                } else {
+                    final String savedMsg = st.getErrorMsg();
+                    st.setError(-1, "%s, Fail to init channel to %s", savedMsg, peer);
+                }
+                continue;
+            }
+            final Future<Message> result = cliClientService.getLeader(peer.getEndpoint(), request, null);
+            try {
+                final Message msg = result.get(timeoutMs, TimeUnit.MILLISECONDS);
+                if (msg instanceof RpcRequests.ErrorResponse) {
+                    if (st.isOk()) {
+                        st.setError(-1, ((RpcRequests.ErrorResponse) msg).getErrorMsg());
+                    } else {
+                        final String savedMsg = st.getErrorMsg();
+                        st.setError(-1, "%s, %s", savedMsg, ((RpcRequests.ErrorResponse) msg).getErrorMsg());
+                    }
+                } else {
+                    final CliRequests.GetLeaderResponse response = (CliRequests.GetLeaderResponse) msg;
+                    updateLeader(groupId, response.getLeaderId());
+                    return Status.OK();
+                }
+            } catch (final TimeoutException e) {
+                timeoutException = e;
+            } catch (final ExecutionException e) {
+                if (st.isOk()) {
+                    st.setError(-1, e.getMessage());
+                } else {
+                    final String savedMsg = st.getErrorMsg();
+                    st.setError(-1, "%s, %s", savedMsg, e.getMessage());
+                }
+            }
+        }
+        if (timeoutException != null) {
+            throw timeoutException;
+        }
+
+        return st;
+    }
+
+    public Status refreshConfiguration(final CliClientService cliClientService, final String groupId,
+                                       final int timeoutMs) throws InterruptedException, TimeoutException {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireTrue(timeoutMs > 0, "Invalid timeout: " + timeoutMs);
+
+        final Configuration conf = getConfiguration(groupId);
+        if (conf == null) {
+            return new Status(RaftError.ENOENT,
+                "Group %s is not registered in RouteTable, forgot to call updateConfiguration?", groupId);
+        }
+        final Status st = Status.OK();
+        PeerId leaderId = selectLeader(groupId);
+        if (leaderId == null) {
+            refreshLeader(cliClientService, groupId, timeoutMs);
+            leaderId = selectLeader(groupId);
+        }
+        if (leaderId == null) {
+            st.setError(-1, "Fail to get leader of group %s", groupId);
+            return st;
+        }
+        if (!cliClientService.connect(leaderId.getEndpoint())) {
+            st.setError(-1, "Fail to init channel to %s", leaderId);
+            return st;
+        }
+        final CliRequests.GetPeersRequest.Builder rb = CliRequests.GetPeersRequest.newBuilder();
+        rb.setGroupId(groupId);
+        rb.setLeaderId(leaderId.toString());
+        try {
+            final Message result = cliClientService.getPeers(leaderId.getEndpoint(), rb.build(), null).get(timeoutMs,
+                TimeUnit.MILLISECONDS);
+            if (result instanceof CliRequests.GetPeersResponse) {
+                final CliRequests.GetPeersResponse resp = (CliRequests.GetPeersResponse) result;
+                final Configuration newConf = new Configuration();
+                for (final String peerIdStr : resp.getPeersList()) {
+                    final PeerId newPeer = new PeerId();
+                    newPeer.parse(peerIdStr);
+                    newConf.addPeer(newPeer);
+                }
+                if (!conf.equals(newConf)) {
+                    LOG.info("Configuration of replication group {} changed from {} to {}", groupId, conf, newConf);
+                }
+                updateConfiguration(groupId, newConf);
+            } else {
+                final RpcRequests.ErrorResponse resp = (RpcRequests.ErrorResponse) result;
+                st.setError(resp.getErrorCode(), resp.getErrorMsg());
+            }
+        } catch (final Exception e) {
+            st.setError(-1, e.getMessage());
+        }
+        return st;
+    }
+
+    /**
+     * Reset the states.
+     */
+    public void reset() {
+        this.groupConfTable.clear();
+    }
+
+    /**
+     * Remove the group from route table.
+     *
+     * @param groupId raft group id
+     * @return true on success
+     */
+    public boolean removeGroup(final String groupId) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+
+        return this.groupConfTable.remove(groupId) != null;
+    }
+
+    @Override
+    public String toString() {
+        return "RouteTable{" + "groupConfTable=" + groupConfTable + '}';
+    }
+
+    private RouteTable() {
+    }
+
+    @Override
+    public void describe(final Printer out) {
+        out.println("RouteTable:") //
+            .print("  ") //
+            .println(toString());
+    }
+
+    private static class GroupConf {
+
+        private final StampedLock stampedLock = new StampedLock();
+
+        private Configuration     conf;
+        private PeerId            leader;
+
+        @Override
+        public String toString() {
+            return "GroupConf{" + "conf=" + conf + ", leader=" + leader + '}';
+        }
+    }
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/StateMachine.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/StateMachine.java
new file mode 100644
index 0000000..ccf1aa2
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/StateMachine.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.alipay.sofa.jraft;
+
+import com.alipay.sofa.jraft.conf.Configuration;
+import com.alipay.sofa.jraft.entity.LeaderChangeContext;
+import com.alipay.sofa.jraft.error.RaftException;
+import com.alipay.sofa.jraft.storage.snapshot.SnapshotReader;
+import com.alipay.sofa.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.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-08 5:43:21 PM
+ */
+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/com/alipay/sofa/jraft/Status.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/Status.java
new file mode 100644
index 0000000..559a093
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/Status.java
@@ -0,0 +1,233 @@
+/*
+ * 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 com.alipay.sofa.jraft;
+
+import com.alipay.sofa.jraft.error.RaftError;
+import com.alipay.sofa.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.
+     *
+     * @author boyan (boyan@alibaba-inc.com)
+     *
+     * 2018-Apr-03 11:17:51 AM
+     */
+    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/com/alipay/sofa/jraft/ThreadPoolMetricsSignalHandler.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/ThreadPoolMetricsSignalHandler.java
new file mode 100644
index 0000000..6e89667
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/ThreadPoolMetricsSignalHandler.java
@@ -0,0 +1,60 @@
+/*
+ * 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 com.alipay.sofa.jraft;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.util.FileOutputSignalHandler;
+import com.alipay.sofa.jraft.util.MetricReporter;
+import com.alipay.sofa.jraft.util.SystemPropertyUtil;
+import com.alipay.sofa.jraft.util.ThreadPoolMetricRegistry;
+
+/**
+ *
+ * @author jiachun.fjc
+ */
+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/com/alipay/sofa/jraft/closure/CatchUpClosure.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/CatchUpClosure.java
new file mode 100644
index 0000000..54ea24c
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/CatchUpClosure.java
@@ -0,0 +1,72 @@
+/*
+ * 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 com.alipay.sofa.jraft.closure;
+
+import java.util.concurrent.ScheduledFuture;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.Status;
+
+/**
+ * A catchup closure for peer to catch up.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ * <p>
+ * 2018-Apr-04 2:15:05 PM
+ */
+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/com/alipay/sofa/jraft/closure/ClosureQueue.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/ClosureQueue.java
new file mode 100644
index 0000000..5aeded5
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/ClosureQueue.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.alipay.sofa.jraft.closure;
+
+import java.util.List;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import com.alipay.sofa.jraft.Closure;
+
+/**
+ * A thread-safe closure queue.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Mar-14 10:29:12 AM
+ */
+@ThreadSafe
+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/com/alipay/sofa/jraft/closure/ClosureQueueImpl.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/ClosureQueueImpl.java
new file mode 100644
index 0000000..10e9ed0
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/ClosureQueueImpl.java
@@ -0,0 +1,145 @@
+/*
+ * 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 com.alipay.sofa.jraft.closure;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.Status;
+import com.alipay.sofa.jraft.error.RaftError;
+import com.alipay.sofa.jraft.util.OnlyForTest;
+import com.alipay.sofa.jraft.util.Requires;
+import com.alipay.sofa.jraft.util.Utils;
+
+/**
+ * Closure queue implementation.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Mar-28 11:44:01 AM
+ */
+public class ClosureQueueImpl implements ClosureQueue {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ClosureQueueImpl.class);
+
+    private final Lock          lock;
+    private long                firstIndex;
+    private LinkedList<Closure> queue;
+
+    @OnlyForTest
+    public long getFirstIndex() {
+        return firstIndex;
+    }
+
+    @OnlyForTest
+    public LinkedList<Closure> getQueue() {
+        return queue;
+    }
+
+    public ClosureQueueImpl() {
+        super();
+        this.lock = new ReentrantLock();
+        this.firstIndex = 0;
+        this.queue = new LinkedList<>();
+    }
+
+    @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(() -> {
+            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/com/alipay/sofa/jraft/closure/JoinableClosure.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/JoinableClosure.java
new file mode 100644
index 0000000..5099b47
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/JoinableClosure.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 com.alipay.sofa.jraft.closure;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.Status;
+import com.alipay.sofa.jraft.util.Requires;
+
+/**
+ * @author jiachun.fjc
+ */
+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/main/java/com/alipay/sofa/jraft/closure/LoadSnapshotClosure.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/LoadSnapshotClosure.java
new file mode 100644
index 0000000..496d1df
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/LoadSnapshotClosure.java
@@ -0,0 +1,37 @@
+/*
+ * 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 com.alipay.sofa.jraft.closure;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.storage.snapshot.SnapshotReader;
+
+/**
+ * Load snapshot closure
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-04 2:20:09 PM
+ */
+public interface LoadSnapshotClosure extends Closure {
+
+    /**
+     * Start to load snapshot, returns a snapshot reader.
+     *
+     * @return a snapshot reader.
+     */
+    SnapshotReader start();
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/ReadIndexClosure.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/ReadIndexClosure.java
new file mode 100644
index 0000000..9bd23c6
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/ReadIndexClosure.java
@@ -0,0 +1,166 @@
+/*
+ * 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 com.alipay.sofa.jraft.closure;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.JRaftUtils;
+import com.alipay.sofa.jraft.Node;
+import com.alipay.sofa.jraft.Status;
+import com.alipay.sofa.jraft.error.RaftError;
+import com.alipay.sofa.jraft.util.SystemPropertyUtil;
+import com.alipay.sofa.jraft.util.timer.Timeout;
+import com.alipay.sofa.jraft.util.timer.Timer;
+import com.alipay.sofa.jraft.util.timer.TimerTask;
+
+/**
+ * Read index closure
+ *
+ * @author dennis
+ */
+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 long                                        DEFAULT_TIMEOUT   = SystemPropertyUtil.getInt(
+                                                                                           "jraft.read-index.timeout",
+                                                                                           2 * 1000);
+
+    private static final int                                         PENDING           = 0;
+    private static final int                                         COMPLETE          = 1;
+    private static final int                                         TIMEOUT           = 2;
+
+    /**
+     * 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;
+
+    public ReadIndexClosure() {
+        this(DEFAULT_TIMEOUT);
+    }
+
+    /**
+     * Create a read-index closure with a timeout parameter.
+     *
+     * @param timeoutMs timeout millis
+     */
+    public ReadIndexClosure(long timeoutMs) {
+        if (timeoutMs >= 0) {
+            // Lazy to init the timer
+            TimeoutScanner.TIMER.newTimeout(new TimeoutTask(this), timeoutMs, TimeUnit.MILLISECONDS);
+        }
+    }
+
+    /**
+     * 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);
+        }
+    }
+
+    static class TimeoutTask implements TimerTask {
+
+        private final ReadIndexClosure closure;
+
+        TimeoutTask(ReadIndexClosure closure) {
+            this.closure = closure;
+        }
+
+        @Override
+        public void run(final Timeout timeout) throws Exception {
+            if (!STATE_UPDATER.compareAndSet(this.closure, PENDING, TIMEOUT)) {
+                return;
+            }
+
+            final Status status = new Status(RaftError.ETIMEDOUT, "read-index request timeout");
+            try {
+                this.closure.run(status, INVALID_LOG_INDEX, null);
+            } catch (final Throwable t) {
+                LOG.error("[Timeout] fail to run ReadIndexClosure with status: {}.", status, t);
+            }
+        }
+    }
+
+    /**
+     * Lazy to create a timer
+     */
+    static class TimeoutScanner {
+        private static final Timer TIMER = JRaftUtils.raftTimerFactory().createTimer("read-index.timeout.scanner");
+    }
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/SaveSnapshotClosure.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/SaveSnapshotClosure.java
new file mode 100644
index 0000000..4b1b9b6
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/SaveSnapshotClosure.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.alipay.sofa.jraft.closure;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.entity.RaftOutter.SnapshotMeta;
+import com.alipay.sofa.jraft.storage.snapshot.SnapshotWriter;
+
+/**
+ * Save snapshot closure
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-04 2:21:30 PM
+ */
+public interface SaveSnapshotClosure extends Closure {
+
+    /**
+     * Starts to save snapshot, returns the writer.
+     *
+     * @param meta metadata of snapshot.
+     * @return returns snapshot writer.
+     */
+    SnapshotWriter start(final SnapshotMeta meta);
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/SynchronizedClosure.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/SynchronizedClosure.java
new file mode 100644
index 0000000..c1d2c2a
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/SynchronizedClosure.java
@@ -0,0 +1,83 @@
+/*
+ * 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 com.alipay.sofa.jraft.closure;
+
+import java.util.concurrent.CountDownLatch;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.Status;
+
+/**
+ * A special Closure which provides synchronization primitives.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Mar-16 2:45:34 PM
+ */
+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/main/java/com/alipay/sofa/jraft/closure/TaskClosure.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/TaskClosure.java
new file mode 100644
index 0000000..23c1415
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/closure/TaskClosure.java
@@ -0,0 +1,35 @@
+/*
+ * 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 com.alipay.sofa.jraft.closure;
+
+import com.alipay.sofa.jraft.Closure;
+
+/**
+ * Closure for task applying.
+ * @author dennis
+ */
+public interface TaskClosure extends Closure {
+
+    /**
+     * 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>
+     */
+    void onCommitted();
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/conf/Configuration.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/conf/Configuration.java
new file mode 100644
index 0000000..7175980
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/conf/Configuration.java
@@ -0,0 +1,324 @@
+/*
+ * 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 com.alipay.sofa.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.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.util.Copiable;
+import com.alipay.sofa.jraft.util.Requires;
+
+/**
+ * A configuration with a set of peers.
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Mar-15 11:00:26 AM
+ */
+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
+     * @since 1.3.0
+     */
+    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/com/alipay/sofa/jraft/conf/ConfigurationEntry.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/conf/ConfigurationEntry.java
new file mode 100644
index 0000000..d13fd17
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/conf/ConfigurationEntry.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.alipay.sofa.jraft.conf;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.entity.LogId;
+import com.alipay.sofa.jraft.entity.PeerId;
+
+/**
+ * A configuration entry with current peers and old peers.
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-04 2:25:06 PM
+ */
+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/com/alipay/sofa/jraft/conf/ConfigurationManager.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/conf/ConfigurationManager.java
new file mode 100644
index 0000000..3337699
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/conf/ConfigurationManager.java
@@ -0,0 +1,110 @@
+/*
+ * 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 com.alipay.sofa.jraft.conf;
+
+import java.util.LinkedList;
+import java.util.ListIterator;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.util.Requires;
+
+/**
+ * Configuration manager
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ * <p>
+ * 2018-Apr-04 2:24:54 PM
+ */
+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/com/alipay/sofa/jraft/core/BallotBox.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/BallotBox.java
new file mode 100644
index 0000000..64d2bf9
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/BallotBox.java
@@ -0,0 +1,283 @@
+/*
+ * 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 com.alipay.sofa.jraft.core;
+
+import java.util.concurrent.locks.StampedLock;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.FSMCaller;
+import com.alipay.sofa.jraft.Lifecycle;
+import com.alipay.sofa.jraft.closure.ClosureQueue;
+import com.alipay.sofa.jraft.conf.Configuration;
+import com.alipay.sofa.jraft.entity.Ballot;
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.option.BallotBoxOptions;
+import com.alipay.sofa.jraft.util.Describer;
+import com.alipay.sofa.jraft.util.OnlyForTest;
+import com.alipay.sofa.jraft.util.Requires;
+import com.alipay.sofa.jraft.util.SegmentList;
+
+/**
+ * Ballot box for voting.
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-04 2:32:10 PM
+ */
+@ThreadSafe
+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;
+    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?
+        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;
+            }
+            // 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();
+        if (!bl.init(conf, oldConf)) {
+            LOG.error("Fail to init ballot.");
+            return false;
+        }
+        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/com/alipay/sofa/jraft/core/CliServiceImpl.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java
new file mode 100644
index 0000000..ff14538
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java
@@ -0,0 +1,687 @@
+/*
+ * 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 com.alipay.sofa.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.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.CliService;
+import com.alipay.sofa.jraft.Status;
+import com.alipay.sofa.jraft.conf.Configuration;
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.error.JRaftException;
+import com.alipay.sofa.jraft.error.RaftError;
+import com.alipay.sofa.jraft.option.CliOptions;
+import com.alipay.sofa.jraft.rpc.CliClientService;
+import com.alipay.sofa.jraft.rpc.CliRequests.AddLearnersRequest;
+import com.alipay.sofa.jraft.rpc.CliRequests.AddPeerRequest;
+import com.alipay.sofa.jraft.rpc.CliRequests.AddPeerResponse;
+import com.alipay.sofa.jraft.rpc.CliRequests.ChangePeersRequest;
+import com.alipay.sofa.jraft.rpc.CliRequests.ChangePeersResponse;
+import com.alipay.sofa.jraft.rpc.CliRequests.GetLeaderRequest;
+import com.alipay.sofa.jraft.rpc.CliRequests.GetLeaderResponse;
+import com.alipay.sofa.jraft.rpc.CliRequests.GetPeersRequest;
+import com.alipay.sofa.jraft.rpc.CliRequests.GetPeersResponse;
+import com.alipay.sofa.jraft.rpc.CliRequests.LearnersOpResponse;
+import com.alipay.sofa.jraft.rpc.CliRequests.RemoveLearnersRequest;
+import com.alipay.sofa.jraft.rpc.CliRequests.RemovePeerRequest;
+import com.alipay.sofa.jraft.rpc.CliRequests.RemovePeerResponse;
+import com.alipay.sofa.jraft.rpc.CliRequests.ResetLearnersRequest;
+import com.alipay.sofa.jraft.rpc.CliRequests.ResetPeerRequest;
+import com.alipay.sofa.jraft.rpc.CliRequests.SnapshotRequest;
+import com.alipay.sofa.jraft.rpc.CliRequests.TransferLeaderRequest;
+import com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse;
+import com.alipay.sofa.jraft.rpc.impl.cli.CliClientServiceImpl;
+import com.alipay.sofa.jraft.util.Requires;
+import com.alipay.sofa.jraft.util.Utils;
+import com.alipay.sofa.jraft.rpc.Message;
+
+/**
+ * Cli service implementation.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ * @author jiachun.fjc
+ */
+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);
+                }
+                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());
+        }
+    }
+
+    // TODO refactor addPeer/removePeer/changePeers/transferLeader, remove duplicated code.
+    @Override
+    public Status changePeers(final String groupId, final Configuration conf, final Configuration newPeers) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireNonNull(conf, "Null configuration");
+        Requires.requireNonNull(newPeers, "Null new peers");
+
+        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 ChangePeersRequest.Builder rb = ChangePeersRequest.newBuilder() //
+            .setGroupId(groupId) //
+            .setLeaderId(leaderId.toString());
+        for (final PeerId peer : newPeers) {
+            rb.addNewPeers(peer.toString());
+        }
+
+        try {
+            final Message result = this.cliClientService.changePeers(leaderId.getEndpoint(), rb.build(), null).get();
+            if (result instanceof ChangePeersResponse) {
+                final ChangePeersResponse resp = (ChangePeersResponse) 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());
+        }
+    }
+
+    @Override
+    public Status resetPeer(final String groupId, final PeerId peerId, final Configuration newPeers) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireNonNull(peerId, "Null peerId");
+        Requires.requireNonNull(newPeers, "Null new peers");
+
+        if (!this.cliClientService.connect(peerId.getEndpoint())) {
+            return new Status(-1, "Fail to init channel to %s", peerId);
+        }
+
+        final ResetPeerRequest.Builder rb = ResetPeerRequest.newBuilder() //
+            .setGroupId(groupId) //
+            .setPeerId(peerId.toString());
+        for (final PeerId peer : newPeers) {
+            rb.addNewPeers(peer.toString());
+        }
+
+        try {
+            final Message result = this.cliClientService.resetPeer(peerId.getEndpoint(), rb.build(), null).get();
+            return statusFromResponse(result);
+        } catch (final Exception e) {
+            return new Status(-1, e.getMessage());
+        }
+    }
+
+    private void checkPeers(final Collection<PeerId> peers) {
+        for (final PeerId peer : peers) {
+            Requires.requireNonNull(peer, "Null peer in collection");
+        }
+    }
+
+    @Override
+    public Status addLearners(final String groupId, final Configuration conf, final List<PeerId> learners) {
+        checkLearnersOpParams(groupId, conf, learners);
+
+        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 AddLearnersRequest.Builder rb = AddLearnersRequest.newBuilder() //
+            .setGroupId(groupId) //
+            .setLeaderId(leaderId.toString());
+        for (final PeerId peer : learners) {
+            rb.addLearners(peer.toString());
+        }
+
+        try {
+            final Message result = this.cliClientService.addLearners(leaderId.getEndpoint(), rb.build(), null).get();
+            return processLearnersOpResponse(groupId, result, "adding learners: %s", learners);
+
+        } catch (final Exception e) {
+            return new Status(-1, e.getMessage());
+        }
+    }
+
+    private void checkLearnersOpParams(final String groupId, final Configuration conf, final List<PeerId> learners) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireNonNull(conf, "Null configuration");
+        Requires.requireTrue(learners != null && !learners.isEmpty(), "Empty peers");
+        checkPeers(learners);
+    }
+
+    private Status processLearnersOpResponse(final String groupId, final Message result, final String fmt,
+                                             final Object... formatArgs) {
+        if (result instanceof LearnersOpResponse) {
+            final LearnersOpResponse resp = (LearnersOpResponse) result;
+            final Configuration oldConf = new Configuration();
+            for (final String peerIdStr : resp.getOldLearnersList()) {
+                final PeerId oldPeer = new PeerId();
+                oldPeer.parse(peerIdStr);
+                oldConf.addLearner(oldPeer);
+            }
+            final Configuration newConf = new Configuration();
+            for (final String peerIdStr : resp.getNewLearnersList()) {
+                final PeerId newPeer = new PeerId();
+                newPeer.parse(peerIdStr);
+                newConf.addLearner(newPeer);
+            }
+
+            LOG.info("Learners of replication group {} changed from {} to {} after {}.", groupId, oldConf, newConf,
+                String.format(fmt, formatArgs));
+            return Status.OK();
+        } else {
+            return statusFromResponse(result);
+        }
+    }
+
+    @Override
+    public Status removeLearners(final String groupId, final Configuration conf, final List<PeerId> learners) {
+        checkLearnersOpParams(groupId, conf, learners);
+
+        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 RemoveLearnersRequest.Builder rb = RemoveLearnersRequest.newBuilder() //
+            .setGroupId(groupId) //
+            .setLeaderId(leaderId.toString());
+        for (final PeerId peer : learners) {
+            rb.addLearners(peer.toString());
+        }
+
+        try {
+            final Message result = this.cliClientService.removeLearners(leaderId.getEndpoint(), rb.build(), null).get();
+            return processLearnersOpResponse(groupId, result, "removing learners: %s", learners);
+
+        } catch (final Exception e) {
+            return new Status(-1, e.getMessage());
+        }
+    }
+
+    @Override
+    public Status resetLearners(final String groupId, final Configuration conf, final List<PeerId> learners) {
+        checkLearnersOpParams(groupId, conf, learners);
+
+        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 ResetLearnersRequest.Builder rb = ResetLearnersRequest.newBuilder() //
+            .setGroupId(groupId) //
+            .setLeaderId(leaderId.toString());
+        for (final PeerId peer : learners) {
+            rb.addLearners(peer.toString());
+        }
+
+        try {
+            final Message result = this.cliClientService.resetLearners(leaderId.getEndpoint(), rb.build(), null).get();
+            return processLearnersOpResponse(groupId, result, "resetting learners: %s", learners);
+
+        } catch (final Exception e) {
+            return new Status(-1, e.getMessage());
+        }
+    }
+
+    @Override
+    public Status transferLeader(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 TransferLeaderRequest.Builder rb = TransferLeaderRequest.newBuilder() //
+            .setGroupId(groupId) //
+            .setLeaderId(leaderId.toString());
+        if (!peer.isEmpty()) {
+            rb.setPeerId(peer.toString());
+        }
+
+        try {
+            final Message result = this.cliClientService.transferLeader(leaderId.getEndpoint(), rb.build(), null).get();
+            return statusFromResponse(result);
+        } catch (final Exception e) {
+            return new Status(-1, e.getMessage());
+        }
+    }
+
+    @Override
+    public Status snapshot(final String groupId, final PeerId peer) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireNonNull(peer, "Null peer");
+
+        if (!this.cliClientService.connect(peer.getEndpoint())) {
+            return new Status(-1, "Fail to init channel to %s", peer);
+        }
+
+        final SnapshotRequest.Builder rb = SnapshotRequest.newBuilder() //
+            .setGroupId(groupId) //
+            .setPeerId(peer.toString());
+
+        try {
+            final Message result = this.cliClientService.snapshot(peer.getEndpoint(), rb.build(), null).get();
+            return statusFromResponse(result);
+        } catch (final Exception e) {
+            return new Status(-1, e.getMessage());
+        }
+    }
+
+    @Override
+    public Status getLeader(final String groupId, final Configuration conf, final PeerId leaderId) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireNonNull(leaderId, "Null leader id");
+
+        if (conf == null || conf.isEmpty()) {
+            return new Status(RaftError.EINVAL, "Empty group configuration");
+        }
+
+        final Status st = new Status(-1, "Fail to get leader of group %s", groupId);
+        for (final PeerId peer : conf) {
+            if (!this.cliClientService.connect(peer.getEndpoint())) {
+                LOG.error("Fail to connect peer {} to get leader for group {}.", peer, groupId);
+                continue;
+            }
+
+            final GetLeaderRequest.Builder rb = GetLeaderRequest.newBuilder() //
+                .setGroupId(groupId) //
+                .setPeerId(peer.toString());
+
+            final Future<Message> result = this.cliClientService.getLeader(peer.getEndpoint(), rb.build(), null);
+            try {
+
+                final Message msg = result.get(
+                    this.cliOptions.getTimeoutMs() <= 0 ? this.cliOptions.getRpcDefaultTimeout() : this.cliOptions
+                        .getTimeoutMs(), TimeUnit.MILLISECONDS);
+                if (msg instanceof ErrorResponse) {
+                    if (st.isOk()) {
+                        st.setError(-1, ((ErrorResponse) msg).getErrorMsg());
+                    } else {
+                        final String savedMsg = st.getErrorMsg();
+                        st.setError(-1, "%s, %s", savedMsg, ((ErrorResponse) msg).getErrorMsg());
+                    }
+                } else {
+                    final GetLeaderResponse response = (GetLeaderResponse) msg;
+                    if (leaderId.parse(response.getLeaderId())) {
+                        break;
+                    }
+                }
+            } catch (final Exception e) {
+                if (st.isOk()) {
+                    st.setError(-1, e.getMessage());
+                } else {
+                    final String savedMsg = st.getErrorMsg();
+                    st.setError(-1, "%s, %s", savedMsg, e.getMessage());
+                }
+            }
+        }
+
+        if (leaderId.isEmpty()) {
+            return st;
+        }
+        return Status.OK();
+    }
+
+    @Override
+    public List<PeerId> getPeers(final String groupId, final Configuration conf) {
+        return getPeers(groupId, conf, false, false);
+    }
+
+    @Override
+    public List<PeerId> getAlivePeers(final String groupId, final Configuration conf) {
+        return getPeers(groupId, conf, false, true);
+    }
+
+    @Override
+    public List<PeerId> getLearners(final String groupId, final Configuration conf) {
+        return getPeers(groupId, conf, true, false);
+    }
+
+    @Override
+    public List<PeerId> getAliveLearners(final String groupId, final Configuration conf) {
+        return getPeers(groupId, conf, true, true);
+    }
+
+    @Override
+    public Status rebalance(final Set<String> balanceGroupIds, final Configuration conf,
+                            final Map<String, PeerId> rebalancedLeaderIds) {
+        Requires.requireNonNull(balanceGroupIds, "Null balance group ids");
+        Requires.requireTrue(!balanceGroupIds.isEmpty(), "Empty balance group ids");
+        Requires.requireNonNull(conf, "Null configuration");
+        Requires.requireTrue(!conf.isEmpty(), "No peers of configuration");
+
+        LOG.info("Rebalance start with raft groups={}.", balanceGroupIds);
+
+        final long start = Utils.monotonicMs();
+        int transfers = 0;
+        Status failedStatus = null;
+        final Queue<String> groupDeque = new ArrayDeque<>(balanceGroupIds);
+        final LeaderCounter leaderCounter = new LeaderCounter(balanceGroupIds.size(), conf.size());
+        for (;;) {
+            final String groupId = groupDeque.poll();
+            if (groupId == null) { // well done
+                break;
+            }
+
+            final PeerId leaderId = new PeerId();
+            final Status leaderStatus = getLeader(groupId, conf, leaderId);
+            if (!leaderStatus.isOk()) {
+                failedStatus = leaderStatus;
+                break;
+            }
+
+            if (rebalancedLeaderIds != null) {
+                rebalancedLeaderIds.put(groupId, leaderId);
+            }
+
+            if (leaderCounter.incrementAndGet(leaderId) <= leaderCounter.getExpectedAverage()) {
+                // The num of leaders is less than the expected average, we are going to deal with others
+                continue;
+            }
+
+            // Find the target peer and try to transfer the leader to this peer
+            final PeerId targetPeer = findTargetPeer(leaderId, groupId, conf, leaderCounter);
+            if (!targetPeer.isEmpty()) {
+                final Status transferStatus = transferLeader(groupId, conf, targetPeer);
+                transfers++;
+                if (!transferStatus.isOk()) {
+                    // The failure of `transfer leader` usually means the node is busy,
+                    // so we return failure status and should try `rebalance` again later.
+                    failedStatus = transferStatus;
+                    break;
+                }
+
+                LOG.info("Group {} transfer leader to {}.", groupId, targetPeer);
+                leaderCounter.decrementAndGet(leaderId);
+                groupDeque.add(groupId);
+                if (rebalancedLeaderIds != null) {
+                    rebalancedLeaderIds.put(groupId, targetPeer);
+                }
+            }
+        }
+
+        final Status status = failedStatus != null ? failedStatus : Status.OK();
+        if (LOG.isInfoEnabled()) {
+            LOG.info(
+                "Rebalanced raft groups={}, status={}, number of transfers={}, elapsed time={} ms, rebalanced result={}.",
+                balanceGroupIds, status, transfers, Utils.monotonicMs() - start, rebalancedLeaderIds);
+        }
+        return status;
+    }
+
+    private PeerId findTargetPeer(final PeerId self, final String groupId, final Configuration conf,
+                                  final LeaderCounter leaderCounter) {
+        for (final PeerId peerId : getAlivePeers(groupId, conf)) {
+            if (peerId.equals(self)) {
+                continue;
+            }
+            if (leaderCounter.get(peerId) >= leaderCounter.getExpectedAverage()) {
+                continue;
+            }
+            return peerId;
+        }
+        return PeerId.emptyPeer();
+    }
+
+    private List<PeerId> getPeers(final String groupId, final Configuration conf, final boolean returnLearners,
+                                  final boolean onlyGetAlive) {
+        Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id");
+        Requires.requireNonNull(conf, "Null conf");
+
+        final PeerId leaderId = new PeerId();
+        final Status st = getLeader(groupId, conf, leaderId);
+        if (!st.isOk()) {
+            throw new IllegalStateException(st.getErrorMsg());
+        }
+
+        if (!this.cliClientService.connect(leaderId.getEndpoint())) {
+            throw new IllegalStateException("Fail to init channel to leader " + leaderId);
+        }
+
+        final GetPeersRequest.Builder rb = GetPeersRequest.newBuilder() //
+            .setGroupId(groupId) //
+            .setLeaderId(leaderId.toString()) //
+            .setOnlyAlive(onlyGetAlive);
+
+        try {
+            final Message result = this.cliClientService.getPeers(leaderId.getEndpoint(), rb.build(), null).get(
+                this.cliOptions.getTimeoutMs() <= 0 ? this.cliOptions.getRpcDefaultTimeout()
+                    : this.cliOptions.getTimeoutMs(), TimeUnit.MILLISECONDS);
+            if (result instanceof GetPeersResponse) {
+                final GetPeersResponse resp = (GetPeersResponse) result;
+                final List<PeerId> peerIdList = new ArrayList<>();
+                final List<java.lang.String> responsePeers = returnLearners ? resp.getLearnersList() : resp.getPeersList();
+                for (final String peerIdStr : responsePeers) {
+                    final PeerId newPeer = new PeerId();
+                    newPeer.parse(peerIdStr);
+                    peerIdList.add(newPeer);
+                }
+                return peerIdList;
+            } else {
+                final ErrorResponse resp = (ErrorResponse) result;
+                throw new JRaftException(resp.getErrorMsg());
+            }
+        } catch (final JRaftException e) {
+            throw e;
+        } catch (final Exception e) {
+            throw new JRaftException(e);
+        }
+    }
+
+    public CliClientService getCliClientService() {
+        return this.cliClientService;
+    }
+
+    private static class LeaderCounter {
+
+        private final Map<PeerId, Integer> counter = new HashMap<>();
+        // The expected average leader number on every peerId
+        private final int                  expectedAverage;
+
+        public LeaderCounter(final int groupCount, final int peerCount) {
+            this.expectedAverage = (int) Math.ceil((double) groupCount / peerCount);
+        }
+
+        public int getExpectedAverage() {
+            return this.expectedAverage;
+        }
+
+        public int incrementAndGet(final PeerId peerId) {
+            return this.counter.compute(peerId, (ignored, num) -> num == null ? 1 : num + 1);
+        }
+
+        public int decrementAndGet(final PeerId peerId) {
+            return this.counter.compute(peerId, (ignored, num) -> num == null ? 0 : num - 1);
+        }
+
+        public int get(final PeerId peerId) {
+            return this.counter.getOrDefault(peerId, 0);
+        }
+    }
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/core/DefaultJRaftServiceFactory.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/DefaultJRaftServiceFactory.java
new file mode 100644
index 0000000..5707ba5
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/DefaultJRaftServiceFactory.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.alipay.sofa.jraft.core;
+
+import com.alipay.sofa.jraft.entity.codec.v1.LogEntryV1CodecFactory;
+import org.apache.commons.lang.StringUtils;
+
+import com.alipay.sofa.jraft.JRaftServiceFactory;
+import com.alipay.sofa.jraft.entity.codec.LogEntryCodecFactory;
+import com.alipay.sofa.jraft.option.RaftOptions;
+import com.alipay.sofa.jraft.storage.LogStorage;
+import com.alipay.sofa.jraft.storage.RaftMetaStorage;
+import com.alipay.sofa.jraft.storage.SnapshotStorage;
+import com.alipay.sofa.jraft.storage.impl.LocalRaftMetaStorage;
+import com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage;
+import com.alipay.sofa.jraft.storage.snapshot.local.LocalSnapshotStorage;
+import com.alipay.sofa.jraft.util.Requires;
+import com.alipay.sofa.jraft.util.SPI;
+
+/**
+ * The default factory for JRaft services.
+ * @author boyan(boyan@antfin.com)
+ * @since 1.2.6
+ *
+ */
+@SPI
+public class DefaultJRaftServiceFactory implements JRaftServiceFactory {
+
+    public static DefaultJRaftServiceFactory newInstance() {
+        return new DefaultJRaftServiceFactory();
+    }
+
+    @Override
+    public LogStorage createLogStorage(final String uri, final RaftOptions raftOptions) {
+        Requires.requireTrue(StringUtils.isNotBlank(uri), "Blank log storage uri.");
+        return new RocksDBLogStorage(uri, raftOptions);
+    }
+
+    @Override
+    public SnapshotStorage createSnapshotStorage(final String uri, final RaftOptions raftOptions) {
+        Requires.requireTrue(!StringUtils.isBlank(uri), "Blank snapshot storage uri.");
+        return new LocalSnapshotStorage(uri, raftOptions);
+    }
+
+    @Override
+    public RaftMetaStorage createRaftMetaStorage(final String uri, final RaftOptions raftOptions) {
+        Requires.requireTrue(!StringUtils.isBlank(uri), "Blank raft meta storage uri.");
+        return new LocalRaftMetaStorage(uri, raftOptions);
+    }
+
+    @Override
+    public LogEntryCodecFactory createLogEntryCodecFactory() {
+        return LogEntryV1CodecFactory.getInstance();
+    }
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/core/ElectionPriority.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/ElectionPriority.java
new file mode 100644
index 0000000..16cae25
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/ElectionPriority.java
@@ -0,0 +1,40 @@
+/*
+ * 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 com.alipay.sofa.jraft.core;
+
+/**
+ * ElectionPriority Type
+ *
+ * @author zongtanghu
+ */
+public class ElectionPriority {
+
+    /**
+     * Priority -1 represents this node disabled the priority election function.
+     */
+    public static final int Disabled   = -1;
+
+    /**
+     * Priority 0 is a special value so that a node will never participate in election.
+     */
+    public static final int NotElected = 0;
+
+    /**
+     * Priority 1 is a minimum value for priority election.
+     */
+    public static final int MinValue   = 1;
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/core/FSMCallerImpl.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/FSMCallerImpl.java
new file mode 100644
index 0000000..7aa2b28
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/FSMCallerImpl.java
@@ -0,0 +1,729 @@
+/*
+ * 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 com.alipay.sofa.jraft.core;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.FSMCaller;
+import com.alipay.sofa.jraft.StateMachine;
+import com.alipay.sofa.jraft.Status;
+import com.alipay.sofa.jraft.closure.ClosureQueue;
+import com.alipay.sofa.jraft.closure.LoadSnapshotClosure;
+import com.alipay.sofa.jraft.closure.SaveSnapshotClosure;
+import com.alipay.sofa.jraft.closure.TaskClosure;
+import com.alipay.sofa.jraft.conf.Configuration;
+import com.alipay.sofa.jraft.conf.ConfigurationEntry;
+import com.alipay.sofa.jraft.entity.EnumOutter;
+import com.alipay.sofa.jraft.entity.EnumOutter.ErrorType;
+import com.alipay.sofa.jraft.entity.LeaderChangeContext;
+import com.alipay.sofa.jraft.entity.LogEntry;
+import com.alipay.sofa.jraft.entity.LogId;
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.entity.RaftOutter;
+import com.alipay.sofa.jraft.error.RaftError;
+import com.alipay.sofa.jraft.error.RaftException;
+import com.alipay.sofa.jraft.option.FSMCallerOptions;
+import com.alipay.sofa.jraft.storage.LogManager;
+import com.alipay.sofa.jraft.storage.snapshot.SnapshotReader;
+import com.alipay.sofa.jraft.storage.snapshot.SnapshotWriter;
+import com.alipay.sofa.jraft.util.DisruptorBuilder;
+import com.alipay.sofa.jraft.util.DisruptorMetricSet;
+import com.alipay.sofa.jraft.util.LogExceptionHandler;
+import com.alipay.sofa.jraft.util.NamedThreadFactory;
+import com.alipay.sofa.jraft.util.OnlyForTest;
+import com.alipay.sofa.jraft.util.Requires;
+import com.alipay.sofa.jraft.util.Utils;
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.EventFactory;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.EventTranslator;
+import com.lmax.disruptor.RingBuffer;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
+
+/**
+ * The finite state machine caller implementation.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-03 11:12:14 AM
+ */
+public class FSMCallerImpl implements FSMCaller {
+
+    private static final Logger LOG = LoggerFactory.getLogger(FSMCallerImpl.class);
+
+    /**
+     * Task type
+     * @author boyan (boyan@alibaba-inc.com)
+     *
+     * 2018-Apr-03 11:12:25 AM
+     */
+    private enum TaskType {
+        IDLE, //
+        COMMITTED, //
+        SNAPSHOT_SAVE, //
+        SNAPSHOT_LOAD, //
+        LEADER_STOP, //
+        LEADER_START, //
+        START_FOLLOWING, //
+        STOP_FOLLOWING, //
+        SHUTDOWN, //
+        FLUSH, //
+        ERROR;
+
+        private String metricName;
+
+        public String metricName() {
+            if (this.metricName == null) {
+                this.metricName = "fsm-" + name().toLowerCase().replaceAll("_", "-");
+            }
+            return this.metricName;
+        }
+    }
+
+    /**
+     * Apply task for disruptor.
+     *
+     * @author boyan (boyan@alibaba-inc.com)
+     *
+     * 2018-Apr-03 11:12:35 AM
+     */
+    private static class ApplyTask {
+        TaskType            type;
+        // union fields
+        long                committedIndex;
+        long                term;
+        Status              status;
+        LeaderChangeContext leaderChangeCtx;
+        Closure             done;
+        CountDownLatch      shutdownLatch;
+
+        public void reset() {
+            this.type = null;
+            this.committedIndex = 0;
+            this.term = 0;
+            this.status = null;
+            this.leaderChangeCtx = null;
+            this.done = null;
+            this.shutdownLatch = null;
+        }
+    }
+
+    private static class ApplyTaskFactory implements EventFactory<ApplyTask> {
+
+        @Override
+        public ApplyTask newInstance() {
+            return new ApplyTask();
+        }
+    }
+
+    private class ApplyTaskHandler implements EventHandler<ApplyTask> {
+        // max committed index in current batch, reset to -1 every batch
+        private long maxCommittedIndex = -1;
+
+        @Override
+        public void onEvent(final ApplyTask event, final long sequence, final boolean endOfBatch) throws Exception {
+            this.maxCommittedIndex = runApplyTask(event, this.maxCommittedIndex, endOfBatch);
+        }
+    }
+
+    private LogManager                                              logManager;
+    private StateMachine                                            fsm;
+    private ClosureQueue                                            closureQueue;
+    private final AtomicLong                                        lastAppliedIndex;
+    private long                                                    lastAppliedTerm;
+    private Closure                                                 afterShutdown;
+    private NodeImpl                                                node;
+    private volatile TaskType                                       currTask;
+    private final AtomicLong                                        applyingIndex;
+    private volatile RaftException                                  error;
+    private Disruptor<ApplyTask>                                    disruptor;
+    private RingBuffer<ApplyTask>                                   taskQueue;
+    private volatile CountDownLatch                                 shutdownLatch;
+    private NodeMetrics                                             nodeMetrics;
+    private final CopyOnWriteArrayList<LastAppliedLogIndexListener> lastAppliedLogIndexListeners = new CopyOnWriteArrayList<>();
+
+    public FSMCallerImpl() {
+        super();
+        this.currTask = TaskType.IDLE;
+        this.lastAppliedIndex = new AtomicLong(0);
+        this.applyingIndex = new AtomicLong(0);
+    }
+
+    @Override
+    public boolean init(final FSMCallerOptions opts) {
+        this.logManager = opts.getLogManager();
+        this.fsm = opts.getFsm();
+        this.closureQueue = opts.getClosureQueue();
+        this.afterShutdown = opts.getAfterShutdown();
+        this.node = opts.getNode();
+        this.nodeMetrics = this.node.getNodeMetrics();
+        this.lastAppliedIndex.set(opts.getBootstrapId().getIndex());
+        notifyLastAppliedIndexUpdated(this.lastAppliedIndex.get());
+        this.lastAppliedTerm = opts.getBootstrapId().getTerm();
+        this.disruptor = DisruptorBuilder.<ApplyTask> newInstance() //
+            .setEventFactory(new ApplyTaskFactory()) //
+            .setRingBufferSize(opts.getDisruptorBufferSize()) //
+            .setThreadFactory(new NamedThreadFactory("JRaft-FSMCaller-Disruptor-", true)) //
+            .setProducerType(ProducerType.MULTI) //
+            .setWaitStrategy(new BlockingWaitStrategy()) //
+            .build();
+        this.disruptor.handleEventsWith(new ApplyTaskHandler());
+        this.disruptor.setDefaultExceptionHandler(new LogExceptionHandler<Object>(getClass().getSimpleName()));
+        this.taskQueue = this.disruptor.start();
+        if (this.nodeMetrics.getMetricRegistry() != null) {
+            this.nodeMetrics.getMetricRegistry().register("jraft-fsm-caller-disruptor",
+                new DisruptorMetricSet(this.taskQueue));
+        }
+        this.error = new RaftException(EnumOutter.ErrorType.ERROR_TYPE_NONE);
+        LOG.info("Starts FSMCaller successfully.");
+        return true;
+    }
+
+    @Override
+    public synchronized void shutdown() {
+        if (this.shutdownLatch != null) {
+            return;
+        }
+        LOG.info("Shutting down FSMCaller...");
+
+        if (this.taskQueue != null) {
+            final CountDownLatch latch = new CountDownLatch(1);
+            this.shutdownLatch = latch;
+            Utils.runInThread(() -> this.taskQueue.publishEvent((task, sequence) -> {
+                task.reset();
+                task.type = TaskType.SHUTDOWN;
+                task.shutdownLatch = latch;
+            }));
+        }
+        doShutdown();
+    }
+
+    @Override
+    public void addLastAppliedLogIndexListener(final LastAppliedLogIndexListener listener) {
+        this.lastAppliedLogIndexListeners.add(listener);
+    }
+
+    private boolean enqueueTask(final EventTranslator<ApplyTask> tpl) {
+        if (this.shutdownLatch != null) {
+            // Shutting down
+            LOG.warn("FSMCaller is stopped, can not apply new task.");
+            return false;
+        }
+        if (!this.taskQueue.tryPublishEvent(tpl)) {
+            setError(new RaftException(ErrorType.ERROR_TYPE_STATE_MACHINE, new Status(RaftError.EBUSY,
+                "FSMCaller is overload.")));
+            return false;
+        }
+        return true;
+    }
+
+    @Override
+    public boolean onCommitted(final long committedIndex) {
+        return enqueueTask((task, sequence) -> {
+            task.type = TaskType.COMMITTED;
+            task.committedIndex = committedIndex;
+        });
+    }
+
+    /**
+     * Flush all events in disruptor.
+     */
+    @OnlyForTest
+    void flush() throws InterruptedException {
+        final CountDownLatch latch = new CountDownLatch(1);
+        enqueueTask((task, sequence) -> {
+            task.type = TaskType.FLUSH;
+            task.shutdownLatch = latch;
+        });
+        latch.await();
+    }
+
+    @Override
+    public boolean onSnapshotLoad(final LoadSnapshotClosure done) {
+        return enqueueTask((task, sequence) -> {
+            task.type = TaskType.SNAPSHOT_LOAD;
+            task.done = done;
+        });
+    }
+
+    @Override
+    public boolean onSnapshotSave(final SaveSnapshotClosure done) {
+        return enqueueTask((task, sequence) -> {
+            task.type = TaskType.SNAPSHOT_SAVE;
+            task.done = done;
+        });
+    }
+
+    @Override
+    public boolean onLeaderStop(final Status status) {
+        return enqueueTask((task, sequence) -> {
+            task.type = TaskType.LEADER_STOP;
+            task.status = new Status(status);
+        });
+    }
+
+    @Override
+    public boolean onLeaderStart(final long term) {
+        return enqueueTask((task, sequence) -> {
+            task.type = TaskType.LEADER_START;
+            task.term = term;
+        });
+    }
+
+    @Override
+    public boolean onStartFollowing(final LeaderChangeContext ctx) {
+        return enqueueTask((task, sequence) -> {
+            task.type = TaskType.START_FOLLOWING;
+            task.leaderChangeCtx = new LeaderChangeContext(ctx.getLeaderId(), ctx.getTerm(), ctx.getStatus());
+        });
+    }
+
+    @Override
+    public boolean onStopFollowing(final LeaderChangeContext ctx) {
+        return enqueueTask((task, sequence) -> {
+            task.type = TaskType.STOP_FOLLOWING;
+            task.leaderChangeCtx = new LeaderChangeContext(ctx.getLeaderId(), ctx.getTerm(), ctx.getStatus());
+        });
+    }
+
+    /**
+     * Closure runs with an error.
+     * @author boyan (boyan@alibaba-inc.com)
+     *
+     * 2018-Apr-04 2:20:31 PM
+     */
+    public class OnErrorClosure implements Closure {
+        private RaftException error;
+
+        public OnErrorClosure(final RaftException error) {
+            super();
+            this.error = error;
+        }
+
+        public RaftException getError() {
+            return this.error;
+        }
+
+        public void setError(final RaftException error) {
+            this.error = error;
+        }
+
+        @Override
+        public void run(final Status st) {
+        }
+    }
+
+    @Override
+    public boolean onError(final RaftException error) {
+        if (!this.error.getStatus().isOk()) {
+            LOG.warn("FSMCaller already in error status, ignore new error: {}", error);
+            return false;
+        }
+        final OnErrorClosure c = new OnErrorClosure(error);
+        return enqueueTask((task, sequence) -> {
+            task.type = TaskType.ERROR;
+            task.done = c;
+        });
+    }
+
+    @Override
+    public long getLastAppliedIndex() {
+        return this.lastAppliedIndex.get();
+    }
+
+    @Override
+    public synchronized void join() throws InterruptedException {
+        if (this.shutdownLatch != null) {
+            this.shutdownLatch.await();
+            this.disruptor.shutdown();
+            if (this.afterShutdown != null) {
+                this.afterShutdown.run(Status.OK());
+                this.afterShutdown = null;
+            }
+            this.shutdownLatch = null;
+        }
+    }
+
+    @SuppressWarnings("ConstantConditions")
+    private long runApplyTask(final ApplyTask task, long maxCommittedIndex, final boolean endOfBatch) {
+        CountDownLatch shutdown = null;
+        if (task.type == TaskType.COMMITTED) {
+            if (task.committedIndex > maxCommittedIndex) {
+                maxCommittedIndex = task.committedIndex;
+            }
+        } else {
+            if (maxCommittedIndex >= 0) {
+                this.currTask = TaskType.COMMITTED;
+                doCommitted(maxCommittedIndex);
+                maxCommittedIndex = -1L; // reset maxCommittedIndex
+            }
+            final long startMs = Utils.monotonicMs();
+            try {
+                switch (task.type) {
+                    case COMMITTED:
+                        Requires.requireTrue(false, "Impossible");
+                        break;
+                    case SNAPSHOT_SAVE:
+                        this.currTask = TaskType.SNAPSHOT_SAVE;
+                        if (passByStatus(task.done)) {
+                            doSnapshotSave((SaveSnapshotClosure) task.done);
+                        }
+                        break;
+                    case SNAPSHOT_LOAD:
+                        this.currTask = TaskType.SNAPSHOT_LOAD;
+                        if (passByStatus(task.done)) {
+                            doSnapshotLoad((LoadSnapshotClosure) task.done);
+                        }
+                        break;
+                    case LEADER_STOP:
+                        this.currTask = TaskType.LEADER_STOP;
+                        doLeaderStop(task.status);
+                        break;
+                    case LEADER_START:
+                        this.currTask = TaskType.LEADER_START;
+                        doLeaderStart(task.term);
+                        break;
+                    case START_FOLLOWING:
+                        this.currTask = TaskType.START_FOLLOWING;
+                        doStartFollowing(task.leaderChangeCtx);
+                        break;
+                    case STOP_FOLLOWING:
+                        this.currTask = TaskType.STOP_FOLLOWING;
+                        doStopFollowing(task.leaderChangeCtx);
+                        break;
+                    case ERROR:
+                        this.currTask = TaskType.ERROR;
+                        doOnError((OnErrorClosure) task.done);
+                        break;
+                    case IDLE:
+                        Requires.requireTrue(false, "Can't reach here");
+                        break;
+                    case SHUTDOWN:
+                        this.currTask = TaskType.SHUTDOWN;
+                        shutdown = task.shutdownLatch;
+                        break;
+                    case FLUSH:
+                        this.currTask = TaskType.FLUSH;
+                        shutdown = task.shutdownLatch;
+                        break;
+                }
+            } finally {
+                this.nodeMetrics.recordLatency(task.type.metricName(), Utils.monotonicMs() - startMs);
+            }
+        }
+        try {
+            if (endOfBatch && maxCommittedIndex >= 0) {
+                this.currTask = TaskType.COMMITTED;
+                doCommitted(maxCommittedIndex);
+                maxCommittedIndex = -1L; // reset maxCommittedIndex
+            }
+            this.currTask = TaskType.IDLE;
+            return maxCommittedIndex;
+        } finally {
+            if (shutdown != null) {
+                shutdown.countDown();
+            }
+        }
+    }
+
+    private void doShutdown() {
+        if (this.node != null) {
+            this.node = null;
+        }
+        if (this.fsm != null) {
+            this.fsm.onShutdown();
+        }
+    }
+
+    private void notifyLastAppliedIndexUpdated(final long lastAppliedIndex) {
+        for (final LastAppliedLogIndexListener listener : this.lastAppliedLogIndexListeners) {
+            listener.onApplied(lastAppliedIndex);
+        }
+    }
+
+    private void doCommitted(final long committedIndex) {
+        if (!this.error.getStatus().isOk()) {
+            return;
+        }
+        final long lastAppliedIndex = this.lastAppliedIndex.get();
+        // We can tolerate the disorder of committed_index
+        if (lastAppliedIndex >= committedIndex) {
+            return;
+        }
+        final long startMs = Utils.monotonicMs();
+        try {
+            final List<Closure> closures = new ArrayList<>();
+            final List<TaskClosure> taskClosures = new ArrayList<>();
+            final long firstClosureIndex = this.closureQueue.popClosureUntil(committedIndex, closures, taskClosures);
+
+            // Calls TaskClosure#onCommitted if necessary
+            onTaskCommitted(taskClosures);
+
+            Requires.requireTrue(firstClosureIndex >= 0, "Invalid firstClosureIndex");
+            final IteratorImpl iterImpl = new IteratorImpl(this.fsm, this.logManager, closures, firstClosureIndex,
+                lastAppliedIndex, committedIndex, this.applyingIndex);
+            while (iterImpl.isGood()) {
+                final LogEntry logEntry = iterImpl.entry();
+                if (logEntry.getType() != EnumOutter.EntryType.ENTRY_TYPE_DATA) {
+                    if (logEntry.getType() == EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION) {
+                        if (logEntry.getOldPeers() != null && !logEntry.getOldPeers().isEmpty()) {
+                            // Joint stage is not supposed to be noticeable by end users.
+                            this.fsm.onConfigurationCommitted(new Configuration(iterImpl.entry().getPeers()));
+                        }
+                    }
+                    if (iterImpl.done() != null) {
+                        // For other entries, we have nothing to do besides flush the
+                        // pending tasks and run this closure to notify the caller that the
+                        // entries before this one were successfully committed and applied.
+                        iterImpl.done().run(Status.OK());
+                    }
+                    iterImpl.next();
+                    continue;
+                }
+
+                // Apply data task to user state machine
+                doApplyTasks(iterImpl);
+            }
+
+            if (iterImpl.hasError()) {
+                setError(iterImpl.getError());
+                iterImpl.runTheRestClosureWithError();
+            }
+            final long lastIndex = iterImpl.getIndex() - 1;
+            final long lastTerm = this.logManager.getTerm(lastIndex);
+            final LogId lastAppliedId = new LogId(lastIndex, lastTerm);
+            this.lastAppliedIndex.set(lastIndex);
+            this.lastAppliedTerm = lastTerm;
+            this.logManager.setAppliedId(lastAppliedId);
+            notifyLastAppliedIndexUpdated(lastIndex);
+        } finally {
+            this.nodeMetrics.recordLatency("fsm-commit", Utils.monotonicMs() - startMs);
+        }
+    }
+
+    private void onTaskCommitted(final List<TaskClosure> closures) {
+        for (int i = 0, size = closures.size(); i < size; i++) {
+            final TaskClosure done = closures.get(i);
+            done.onCommitted();
+        }
+    }
+
+    private void doApplyTasks(final IteratorImpl iterImpl) {
+        final IteratorWrapper iter = new IteratorWrapper(iterImpl);
+        final long startApplyMs = Utils.monotonicMs();
+        final long startIndex = iter.getIndex();
+        try {
+            this.fsm.onApply(iter);
+        } finally {
+            this.nodeMetrics.recordLatency("fsm-apply-tasks", Utils.monotonicMs() - startApplyMs);
+            this.nodeMetrics.recordSize("fsm-apply-tasks-count", iter.getIndex() - startIndex);
+        }
+        if (iter.hasNext()) {
+            LOG.error("Iterator is still valid, did you return before iterator reached the end?");
+        }
+        // Try move to next in case that we pass the same log twice.
+        iter.next();
+    }
+
+    private void doSnapshotSave(final SaveSnapshotClosure done) {
+        Requires.requireNonNull(done, "SaveSnapshotClosure is null");
+        final long lastAppliedIndex = this.lastAppliedIndex.get();
+        final RaftOutter.SnapshotMeta.Builder metaBuilder = RaftOutter.SnapshotMeta.newBuilder() //
+            .setLastIncludedIndex(lastAppliedIndex) //
+            .setLastIncludedTerm(this.lastAppliedTerm);
+        final ConfigurationEntry confEntry = this.logManager.getConfiguration(lastAppliedIndex);
+        if (confEntry == null || confEntry.isEmpty()) {
+            LOG.error("Empty conf entry for lastAppliedIndex={}", lastAppliedIndex);
+            Utils.runClosureInThread(done, new Status(RaftError.EINVAL, "Empty conf entry for lastAppliedIndex=%s",
+                lastAppliedIndex));
+            return;
+        }
+        for (final PeerId peer : confEntry.getConf()) {
+            metaBuilder.addPeers(peer.toString());
+        }
+        for (final PeerId peer : confEntry.getConf().getLearners()) {
+            metaBuilder.addLearners(peer.toString());
+        }
+        if (confEntry.getOldConf() != null) {
+            for (final PeerId peer : confEntry.getOldConf()) {
+                metaBuilder.addOldPeers(peer.toString());
+            }
+            for (final PeerId peer : confEntry.getOldConf().getLearners()) {
+                metaBuilder.addOldLearners(peer.toString());
+            }
+        }
+        final SnapshotWriter writer = done.start(metaBuilder.build());
+        if (writer == null) {
+            done.run(new Status(RaftError.EINVAL, "snapshot_storage create SnapshotWriter failed"));
+            return;
+        }
+        this.fsm.onSnapshotSave(writer, done);
+    }
+
+    @Override
+    public String toString() {
+        final StringBuilder sb = new StringBuilder("StateMachine [");
+        switch (this.currTask) {
+            case IDLE:
+                sb.append("Idle");
+                break;
+            case COMMITTED:
+                sb.append("Applying logIndex=").append(this.applyingIndex);
+                break;
+            case SNAPSHOT_SAVE:
+                sb.append("Saving snapshot");
+                break;
+            case SNAPSHOT_LOAD:
+                sb.append("Loading snapshot");
+                break;
+            case ERROR:
+                sb.append("Notifying error");
+                break;
+            case LEADER_STOP:
+                sb.append("Notifying leader stop");
+                break;
+            case LEADER_START:
+                sb.append("Notifying leader start");
+                break;
+            case START_FOLLOWING:
+                sb.append("Notifying start following");
+                break;
+            case STOP_FOLLOWING:
+                sb.append("Notifying stop following");
+                break;
+            case SHUTDOWN:
+                sb.append("Shutting down");
+                break;
+            default:
+                break;
+        }
+        return sb.append(']').toString();
+    }
+
+    private void doSnapshotLoad(final LoadSnapshotClosure done) {
+        Requires.requireNonNull(done, "LoadSnapshotClosure is null");
+        final SnapshotReader reader = done.start();
+        if (reader == null) {
+            done.run(new Status(RaftError.EINVAL, "open SnapshotReader failed"));
+            return;
+        }
+        final RaftOutter.SnapshotMeta meta = reader.load();
+        if (meta == null) {
+            done.run(new Status(RaftError.EINVAL, "SnapshotReader load meta failed"));
+            if (reader.getRaftError() == RaftError.EIO) {
+                final RaftException err = new RaftException(EnumOutter.ErrorType.ERROR_TYPE_SNAPSHOT, RaftError.EIO,
+                    "Fail to load snapshot meta");
+                setError(err);
+            }
+            return;
+        }
+        final LogId lastAppliedId = new LogId(this.lastAppliedIndex.get(), this.lastAppliedTerm);
+        final LogId snapshotId = new LogId(meta.getLastIncludedIndex(), meta.getLastIncludedTerm());
+        if (lastAppliedId.compareTo(snapshotId) > 0) {
+            done.run(new Status(
+                RaftError.ESTALE,
+                "Loading a stale snapshot last_applied_index=%d last_applied_term=%d snapshot_index=%d snapshot_term=%d",
+                lastAppliedId.getIndex(), lastAppliedId.getTerm(), snapshotId.getIndex(), snapshotId.getTerm()));
+            return;
+        }
+        if (!this.fsm.onSnapshotLoad(reader)) {
+            done.run(new Status(-1, "StateMachine onSnapshotLoad failed"));
+            final RaftException e = new RaftException(EnumOutter.ErrorType.ERROR_TYPE_STATE_MACHINE,
+                RaftError.ESTATEMACHINE, "StateMachine onSnapshotLoad failed");
+            setError(e);
+            return;
+        }
+        if (meta.getOldPeersCount() == 0) {
+            // Joint stage is not supposed to be noticeable by end users.
+            final Configuration conf = new Configuration();
+            for (int i = 0, size = meta.getPeersCount(); i < size; i++) {
+                final PeerId peer = new PeerId();
+                Requires.requireTrue(peer.parse(meta.getPeers(i)), "Parse peer failed");
+                conf.addPeer(peer);
+            }
+            this.fsm.onConfigurationCommitted(conf);
+        }
+        this.lastAppliedIndex.set(meta.getLastIncludedIndex());
+        this.lastAppliedTerm = meta.getLastIncludedTerm();
+        done.run(Status.OK());
+    }
+
+    private void doOnError(final OnErrorClosure done) {
+        setError(done.getError());
+    }
+
+    private void doLeaderStop(final Status status) {
+        this.fsm.onLeaderStop(status);
+    }
+
+    private void doLeaderStart(final long term) {
+        this.fsm.onLeaderStart(term);
+    }
+
+    private void doStartFollowing(final LeaderChangeContext ctx) {
+        this.fsm.onStartFollowing(ctx);
+    }
+
+    private void doStopFollowing(final LeaderChangeContext ctx) {
+        this.fsm.onStopFollowing(ctx);
+    }
+
+    private void setError(final RaftException e) {
+        if (this.error.getType() != EnumOutter.ErrorType.ERROR_TYPE_NONE) {
+            // already report
+            return;
+        }
+        this.error = e;
+        if (this.fsm != null) {
+            this.fsm.onError(e);
+        }
+        if (this.node != null) {
+            this.node.onError(e);
+        }
+    }
+
+    @OnlyForTest
+    RaftException getError() {
+        return this.error;
+    }
+
+    private boolean passByStatus(final Closure done) {
+        final Status status = this.error.getStatus();
+        if (!status.isOk()) {
+            if (done != null) {
+                done.run(new Status(RaftError.EINVAL, "FSMCaller is in bad status=`%s`", status));
+                return false;
+            }
+        }
+        return true;
+    }
+
+    @Override
+    public void describe(final Printer out) {
+        out.print("  ") //
+            .println(toString());
+    }
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/core/IteratorImpl.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/IteratorImpl.java
new file mode 100644
index 0000000..a272b2f
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/IteratorImpl.java
@@ -0,0 +1,161 @@
+/*
+ * 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 com.alipay.sofa.jraft.core;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.StateMachine;
+import com.alipay.sofa.jraft.Status;
+import com.alipay.sofa.jraft.entity.EnumOutter;
+import com.alipay.sofa.jraft.entity.LogEntry;
+import com.alipay.sofa.jraft.error.LogEntryCorruptedException;
+import com.alipay.sofa.jraft.error.RaftError;
+import com.alipay.sofa.jraft.error.RaftException;
+import com.alipay.sofa.jraft.storage.LogManager;
+import com.alipay.sofa.jraft.util.Requires;
+import com.alipay.sofa.jraft.util.Utils;
+
+/**
+ * The iterator implementation.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-03 3:28:37 PM
+ */
+public class IteratorImpl {
+
+    private final StateMachine  fsm;
+    private final LogManager    logManager;
+    private final List<Closure> closures;
+    private final long          firstClosureIndex;
+    private long                currentIndex;
+    private final long          committedIndex;
+    private LogEntry            currEntry = new LogEntry(); // blank entry
+    private final AtomicLong    applyingIndex;
+    private RaftException       error;
+
+    public IteratorImpl(final StateMachine fsm, final LogManager logManager, final List<Closure> closures,
+                        final long firstClosureIndex, final long lastAppliedIndex, final long committedIndex,
+                        final AtomicLong applyingIndex) {
+        super();
+        this.fsm = fsm;
+        this.logManager = logManager;
+        this.closures = closures;
+        this.firstClosureIndex = firstClosureIndex;
+        this.currentIndex = lastAppliedIndex;
+        this.committedIndex = committedIndex;
+        this.applyingIndex = applyingIndex;
+        next();
+    }
+
+    @Override
+    public String toString() {
+        return "IteratorImpl [fsm=" + this.fsm + ", logManager=" + this.logManager + ", closures=" + this.closures
+               + ", firstClosureIndex=" + this.firstClosureIndex + ", currentIndex=" + this.currentIndex
+               + ", committedIndex=" + this.committedIndex + ", currEntry=" + this.currEntry + ", applyingIndex="
+               + this.applyingIndex + ", error=" + this.error + "]";
+    }
+
+    public LogEntry entry() {
+        return this.currEntry;
+    }
+
+    public RaftException getError() {
+        return this.error;
+    }
+
+    public boolean isGood() {
+        return this.currentIndex <= this.committedIndex && !hasError();
+    }
+
+    public boolean hasError() {
+        return this.error != null;
+    }
+
+    /**
+     * Move to next
+     */
+    public void next() {
+        this.currEntry = null; //release current entry
+        //get next entry
+        if (this.currentIndex <= this.committedIndex) {
+            ++this.currentIndex;
+            if (this.currentIndex <= this.committedIndex) {
+                try {
+                    this.currEntry = this.logManager.getEntry(this.currentIndex);
+                    if (this.currEntry == null) {
+                        getOrCreateError().setType(EnumOutter.ErrorType.ERROR_TYPE_LOG);
+                        getOrCreateError().getStatus().setError(-1,
+                            "Fail to get entry at index=%d while committed_index=%d", this.currentIndex,
+                            this.committedIndex);
+                    }
+                } catch (final LogEntryCorruptedException e) {
+                    getOrCreateError().setType(EnumOutter.ErrorType.ERROR_TYPE_LOG);
+                    getOrCreateError().getStatus().setError(RaftError.EINVAL, e.getMessage());
+                }
+                this.applyingIndex.set(this.currentIndex);
+            }
+        }
+    }
+
+    public long getIndex() {
+        return this.currentIndex;
+    }
+
+    public Closure done() {
+        if (this.currentIndex < this.firstClosureIndex) {
+            return null;
+        }
+        return this.closures.get((int) (this.currentIndex - this.firstClosureIndex));
+    }
+
+    protected void runTheRestClosureWithError() {
+        for (long i = Math.max(this.currentIndex, this.firstClosureIndex); i <= this.committedIndex; i++) {
+            final Closure done = this.closures.get((int) (i - this.firstClosureIndex));
+            if (done != null) {
+                Requires.requireNonNull(this.error, "error");
+                Requires.requireNonNull(this.error.getStatus(), "error.status");
+                final Status status = this.error.getStatus();
+                Utils.runClosureInThread(done, status);
+            }
+        }
+    }
+
+    public void setErrorAndRollback(final long ntail, final Status st) {
+        Requires.requireTrue(ntail > 0, "Invalid ntail=" + ntail);
+        if (this.currEntry == null || this.currEntry.getType() != EnumOutter.EntryType.ENTRY_TYPE_DATA) {
+            this.currentIndex -= ntail;
+        } else {
+            this.currentIndex -= ntail - 1;
+        }
+        this.currEntry = null;
+        getOrCreateError().setType(EnumOutter.ErrorType.ERROR_TYPE_STATE_MACHINE);
+        getOrCreateError().getStatus().setError(RaftError.ESTATEMACHINE,
+            "StateMachine meet critical error when applying one or more tasks since index=%d, %s", this.currentIndex,
+            st != null ? st.toString() : "none");
+
+    }
+
+    private RaftException getOrCreateError() {
+        if (this.error == null) {
+            this.error = new RaftException();
+        }
+        return this.error;
+    }
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/core/IteratorWrapper.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/IteratorWrapper.java
new file mode 100644
index 0000000..ce504d0
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/IteratorWrapper.java
@@ -0,0 +1,75 @@
+/*
+ * 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 com.alipay.sofa.jraft.core;
+
+import java.nio.ByteBuffer;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.Iterator;
+import com.alipay.sofa.jraft.Status;
+import com.alipay.sofa.jraft.entity.EnumOutter;
+import com.alipay.sofa.jraft.entity.LogEntry;
+
+public class IteratorWrapper implements Iterator {
+
+    private final IteratorImpl impl;
+
+    public IteratorWrapper(IteratorImpl iterImpl) {
+        super();
+        this.impl = iterImpl;
+    }
+
+    @Override
+    public boolean hasNext() {
+        return this.impl.isGood() && this.impl.entry().getType() == EnumOutter.EntryType.ENTRY_TYPE_DATA;
+    }
+
+    @Override
+    public ByteBuffer next() {
+        final ByteBuffer data = getData();
+        if (hasNext()) {
+            this.impl.next();
+        }
+        return data;
+    }
+
+    @Override
+    public ByteBuffer getData() {
+        final LogEntry entry = this.impl.entry();
+        return entry != null ? entry.getData() : null;
+    }
+
+    @Override
+    public long getIndex() {
+        return this.impl.getIndex();
+    }
+
+    @Override
+    public long getTerm() {
+        return this.impl.entry().getId().getTerm();
+    }
+
+    @Override
+    public Closure done() {
+        return this.impl.done();
+    }
+
+    @Override
+    public void setErrorAndRollback(final long ntail, final Status st) {
+        this.impl.setErrorAndRollback(ntail, st);
+    }
+}
diff --git a/modules/raft/src/main/java/com/alipay/sofa/jraft/core/NodeImpl.java b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/NodeImpl.java
new file mode 100644
index 0000000..09139d5
--- /dev/null
+++ b/modules/raft/src/main/java/com/alipay/sofa/jraft/core/NodeImpl.java
@@ -0,0 +1,3464 @@
+/*
+ * 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 com.alipay.sofa.jraft.core;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.FSMCaller;
+import com.alipay.sofa.jraft.JRaftServiceFactory;
+import com.alipay.sofa.jraft.JRaftUtils;
+import com.alipay.sofa.jraft.Node;
+import com.alipay.sofa.jraft.NodeManager;
+import com.alipay.sofa.jraft.ReadOnlyService;
+import com.alipay.sofa.jraft.ReplicatorGroup;
+import com.alipay.sofa.jraft.Status;
+import com.alipay.sofa.jraft.closure.CatchUpClosure;
+import com.alipay.sofa.jraft.closure.ClosureQueue;
+import com.alipay.sofa.jraft.closure.ClosureQueueImpl;
+import com.alipay.sofa.jraft.closure.ReadIndexClosure;
+import com.alipay.sofa.jraft.closure.SynchronizedClosure;
+import com.alipay.sofa.jraft.conf.Configuration;
+import com.alipay.sofa.jraft.conf.ConfigurationEntry;
+import com.alipay.sofa.jraft.conf.ConfigurationManager;
+import com.alipay.sofa.jraft.entity.Ballot;
+import com.alipay.sofa.jraft.entity.EnumOutter;
+import com.alipay.sofa.jraft.entity.LeaderChangeContext;
+import com.alipay.sofa.jraft.entity.LogEntry;
+import com.alipay.sofa.jraft.entity.LogId;
+import com.alipay.sofa.jraft.entity.NodeId;
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.entity.RaftOutter;
+import com.alipay.sofa.jraft.entity.Task;
+import com.alipay.sofa.jraft.entity.UserLog;
+import com.alipay.sofa.jraft.error.LogIndexOutOfBoundsException;
+import com.alipay.sofa.jraft.error.LogNotFoundException;
+import com.alipay.sofa.jraft.error.RaftError;
+import com.alipay.sofa.jraft.error.RaftException;
+import com.alipay.sofa.jraft.option.BallotBoxOptions;
+import com.alipay.sofa.jraft.option.BootstrapOptions;
+import com.alipay.sofa.jraft.option.FSMCallerOptions;
+import com.alipay.sofa.jraft.option.LogManagerOptions;
+import com.alipay.sofa.jraft.option.NodeOptions;
+import com.alipay.sofa.jraft.option.RaftMetaStorageOptions;
+import com.alipay.sofa.jraft.option.RaftOptions;
+import com.alipay.sofa.jraft.option.ReadOnlyOption;
+import com.alipay.sofa.jraft.option.ReadOnlyServiceOptions;
+import com.alipay.sofa.jraft.option.ReplicatorGroupOptions;
+import com.alipay.sofa.jraft.option.SnapshotExecutorOptions;
+import com.alipay.sofa.jraft.rpc.RaftClientService;
+import com.alipay.sofa.jraft.rpc.RaftServerService;
+import com.alipay.sofa.jraft.rpc.RpcRequestClosure;
+import com.alipay.sofa.jraft.rpc.RpcRequests.AppendEntriesRequest;
+import com.alipay.sofa.jraft.rpc.RpcRequests.AppendEntriesResponse;
+import com.alipay.sofa.jraft.rpc.RpcRequests.InstallSnapshotRequest;
+import com.alipay.sofa.jraft.rpc.RpcRequests.InstallSnapshotResponse;
+import com.alipay.sofa.jraft.rpc.RpcRequests.ReadIndexRequest;
+import com.alipay.sofa.jraft.rpc.RpcRequests.ReadIndexResponse;
+import com.alipay.sofa.jraft.rpc.RpcRequests.RequestVoteRequest;
+import com.alipay.sofa.jraft.rpc.RpcRequests.RequestVoteResponse;
+import com.alipay.sofa.jraft.rpc.RpcRequests.TimeoutNowRequest;
+import com.alipay.sofa.jraft.rpc.RpcRequests.TimeoutNowResponse;
+import com.alipay.sofa.jraft.rpc.RpcResponseClosure;
+import com.alipay.sofa.jraft.rpc.RpcResponseClosureAdapter;
+import com.alipay.sofa.jraft.rpc.impl.core.DefaultRaftClientService;
+import com.alipay.sofa.jraft.storage.LogManager;
+import com.alipay.sofa.jraft.storage.LogStorage;
+import com.alipay.sofa.jraft.storage.RaftMetaStorage;
+import com.alipay.sofa.jraft.storage.SnapshotExecutor;
+import com.alipay.sofa.jraft.storage.impl.LogManagerImpl;
+import com.alipay.sofa.jraft.storage.snapshot.SnapshotExecutorImpl;
+import com.alipay.sofa.jraft.util.Describer;
+import com.alipay.sofa.jraft.util.DisruptorBuilder;
+import com.alipay.sofa.jraft.util.DisruptorMetricSet;
+import com.alipay.sofa.jraft.util.JRaftServiceLoader;
+import com.alipay.sofa.jraft.util.JRaftSignalHandler;
+import com.alipay.sofa.jraft.util.LogExceptionHandler;
+import com.alipay.sofa.jraft.util.NamedThreadFactory;
+import com.alipay.sofa.jraft.util.OnlyForTest;
+import com.alipay.sofa.jraft.util.Platform;
+import com.alipay.sofa.jraft.util.RepeatedTimer;
+import com.alipay.sofa.jraft.util.Requires;
+import com.alipay.sofa.jraft.util.RpcFactoryHelper;
+import com.alipay.sofa.jraft.util.SignalHelper;
+import com.alipay.sofa.jraft.util.SystemPropertyUtil;
+import com.alipay.sofa.jraft.util.ThreadHelper;
+import com.alipay.sofa.jraft.util.ThreadId;
+import com.alipay.sofa.jraft.util.Utils;
+import com.alipay.sofa.jraft.util.concurrent.LongHeldDetectingReadWriteLock;
+import com.alipay.sofa.jraft.util.timer.RaftTimerFactory;
+import com.alipay.sofa.jraft.rpc.Message;
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.EventFactory;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.EventTranslator;
+import com.lmax.disruptor.RingBuffer;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
+
+/**
+ * The raft replica node implementation.
+ *
+ * @author boyan (boyan@alibaba-inc.com)
+ *
+ * 2018-Apr-03 4:26:51 PM
+ */
+public class NodeImpl implements Node, RaftServerService {
+
+    private static final Logger                                            LOG                      = LoggerFactory
+                                                                                                        .getLogger(NodeImpl.class);
+
+    static {
+        try {
+            if (SignalHelper.supportSignal()) {
+                // TODO support windows signal
+                if (!Platform.isWindows()) {
+                    final List<JRaftSignalHandler> handlers = JRaftServiceLoader.load(JRaftSignalHandler.class) //
+                        .sort();
+                    SignalHelper.addSignal(SignalHelper.SIG_USR2, handlers);
+                }
+            }
+        } catch (final Throwable t) {
+            LOG.error("Fail to add signal.", t);
+        }
+    }
+
+    public final static RaftTimerFactory                                   TIMER_FACTORY            = JRaftUtils
+                                                                                                        .raftTimerFactory();
+
+    // Max retry times when applying tasks.
+    private static final int                                               MAX_APPLY_RETRY_TIMES    = 3;
+
+    public static final AtomicInteger                                      GLOBAL_NUM_NODES         = new AtomicInteger(
+                                                                                                        0);
+
+    /** Internal states */
+    private final ReadWriteLock                                            readWriteLock            = new NodeReadWriteLock(
+                                                                                                        this);
+    protected final Lock                                                   writeLock                = this.readWriteLock
+                                                                                                        .writeLock();
+    protected final Lock                                                   readLock                 = this.readWriteLock
+                                                                                                        .readLock();
+    private volatile State                                                 state;
+    private volatile CountDownLatch                                        shutdownLatch;
+    private long                                                           currTerm;
+    private volatile long                                                  lastLeaderTimestamp;
+    private PeerId                                                         leaderId                 = new PeerId();
+    private PeerId                                                         votedId;
+    private final Ballot                                                   voteCtx                  = new Ballot();
+    private final Ballot                                                   prevVoteCtx              = new Ballot();
+    private ConfigurationEntry                                             conf;
+    private StopTransferArg                                                stopTransferArg;
+    /** Raft group and node options and identifier */
+    private final String                                                   groupId;
+    private NodeOptions                                                    options;
+    private RaftOptions                                                    raftOptions;
+    private final PeerId                                                   serverId;
+    /** Other services */
+    private final ConfigurationCtx                                         confCtx;
+    private LogStorage                                                     logStorage;
+    private RaftMetaStorage                                                metaStorage;
+    private ClosureQueue                                                   closureQueue;
+    private ConfigurationManager                                           configManager;
+    private LogManager                                                     logManager;
+    private FSMCaller                                                      fsmCaller;
+    private BallotBox                                                      ballotBox;
+    private SnapshotExecutor                                               snapshotExecutor;
+    private ReplicatorGroup                                                replicatorGroup;
+    private final List<Closure>                                            shutdownContinuations    = new ArrayList<>();
+    private RaftClientService                                              rpcService;
+    private ReadOnlyService                                                readOnlyService;
+    /** Timers */
+    private Scheduler                                                      timerManager;
+    private RepeatedTimer                                                  electionTimer;
+    private RepeatedTimer                                                  voteTimer;
+    private RepeatedTimer                                                  stepDownTimer;
+    private RepeatedTimer                                                  snapshotTimer;
+    private ScheduledFuture<?>                                             transferTimer;
+    private ThreadId                                                       wakingCandidate;
+    /** Disruptor to run node service */
+    private Disruptor<LogEntryAndClosure>                                  applyDisruptor;
+    private RingBuffer<LogEntryAndClosure>                                 applyQueue;
+
+    /** Metrics */
+    private NodeMetrics                                                    metrics;
+
+    private NodeId                                                         nodeId;
+    private JRaftServiceFactory                                            serviceFactory;
+
+    /** ReplicatorStateListeners */
+    private final CopyOnWriteArrayList<Replicator.ReplicatorStateListener> replicatorStateListeners = new CopyOnWriteArrayList<>();
+    /** Node's target leader election priority value */
+    private volatile int                                                   targetPriority;
+    /** The number of elections time out for current node */
+    private volatile int                                                   electionTimeoutCounter;
+
+    private static class NodeReadWriteLock extends LongHeldDetectingReadWriteLock {
+
+        static final long  MAX_BLOCKING_MS_TO_REPORT = SystemPropertyUtil.getLong(
+                                                         "jraft.node.detecting.lock.max_blocking_ms_to_report", -1);
+
+        private final Node node;
+
+        public NodeReadWriteLock(final Node node) {
+            super(MAX_BLOCKING_MS_TO_REPORT, TimeUnit.MILLISECONDS);
+            this.node = node;
+        }
+
+        @Override
+        public void report(final AcquireMode acquireMode, final Thread heldThread,
+                           final Collection<Thread> queuedThreads, final long blockedNanos) {
+            final long blockedMs = TimeUnit.NANOSECONDS.toMillis(blockedNanos);
+            LOG.warn(
+                "Raft-Node-Lock report: currentThread={}, acquireMode={}, heldThread={}, queuedThreads={}, blockedMs={}.",
+                Thread.currentThread(), acquireMode, heldThread, queuedThreads, blockedMs);
+
+            final NodeMetrics metrics = this.node.getNodeMetrics();
+            if (metrics != null) {
+                metrics.recordLatency("node-lock-blocked", blockedMs);
+            }
+        }
+    }
+
+    /**
+     * Node service event.
+     *
+     * @author boyan (boyan@alibaba-inc.com)
+     *
+     * 2018-Apr-03 4:29:55 PM
+     */
+    private static class LogEntryAndClosure {
+        LogEntry       entry;
+        Closure        done;
+        long           expectedTerm;
+        CountDownLatch shutdownLatch;
+
+        public void reset() {
+            this.entry = null;
+            this.done = null;
+            this.expectedTerm = 0;
+            this.shutdownLatch = null;
+        }
+    }
+
+    private static class LogEntryAndClosureFactory implements EventFactory<LogEntryAndClosure> {
+
+        @Override
+        public LogEntryAndClosure newInstance() {
+            return new LogEntryAndClosure();
+        }
+    }
+
+    /**
+     * Event handler.
+     *
+     * @author boyan (boyan@alibaba-inc.com)
+     *
+     * 2018-Apr-03 4:30:07 PM
+     */
+    private class LogEntryAndClosureHandler implements EventHandler<LogEntryAndClosure> {
+        // task list for batch
+        private final List<LogEntryAndClosure> tasks = new ArrayList<>(NodeImpl.this.raftOptions.getApplyBatch());
+
+        @Override
+        public void onEvent(final LogEntryAndClosure event, final long sequence, final boolean endOfBatch)
+                                                                                                          throws Exception {
+            if (event.shutdownLatch != null) {
+                if (!this.tasks.isEmpty()) {
+                    executeApplyingTasks(this.tasks);
+                    this.tasks.clear();
+                }
+                final int num = GLOBAL_NUM_NODES.decrementAndGet();
+                LOG.info("The number of active nodes decrement to {}.", num);
+                event.shutdownLatch.countDown();
+                return;
+            }
+
+            this.tasks.add(event);
+            if (this.tasks.size() >= NodeImpl.this.raftOptions.getApplyBatch() || endOfBatch) {
+                executeApplyingTasks(this.tasks);
+                this.tasks.clear();
+            }
+        }
+    }
+
+    /**
+     * Configuration commit context.
+     *
+     * @author boyan (boyan@alibaba-inc.com)
+     *
+     * 2018-Apr-03 4:29:38 PM
+     */
+    private static class ConfigurationCtx {
+        enum Stage {
+            STAGE_NONE, // none stage
+            STAGE_CATCHING_UP, // the node is catching-up
+            STAGE_JOINT, // joint stage
+            STAGE_STABLE // stable stage
+        }
+
+        final NodeImpl node;
+        Stage          stage;
+        // Peers change times
+        int            nchanges;
+        long           version;
+        // peers
+        List<PeerId>   newPeers    = new ArrayList<>();
+        List<PeerId>   oldPeers    = new ArrayList<>();
+        List<PeerId>   addingPeers = new ArrayList<>();
+        // learners
+        List<PeerId>   newLearners = new ArrayList<>();
+        List<PeerId>   oldLearners = new ArrayList<>();
+        Closure        done;
+
+        public ConfigurationCtx(final NodeImpl node) {
+            super();
+            this.node = node;
+            this.stage = Stage.STAGE_NONE;
+            this.version = 0;
+            this.done = null;
+        }
+
+        /**
+         * Start change configuration.
+         */
+        void start(final Configuration oldConf, final Configuration newConf, final Closure done) {
+            if (isBusy()) {
+                if (done != null) {
+                    Utils.runClosureInThread(done, new Status(RaftError.EBUSY, "Already in busy stage."));
+                }
+                throw new IllegalStateException("Busy stage");
+            }
+            if (this.done != null) {
+                if (done != null) {
+                    Utils.runClosureInThread(done, new Status(RaftError.EINVAL, "Already have done closure."));
+                }
+                throw new IllegalArgumentException("Already have done closure");
+            }
+            this.done = done;
+            this.stage = Stage.STAGE_CATCHING_UP;
+            this.oldPeers = oldConf.listPeers();
+            this.newPeers = newConf.listPeers();
+            this.oldLearners = oldConf.listLearners();
+            this.newLearners = newConf.listLearners();
+            final Configuration adding = new Configuration();
+            final Configuration removing = new Configuration();
+            newConf.diff(oldConf, adding, removing);
+            this.nchanges = adding.size() + removing.size();
+
+            addNewLearners();
+            if (adding.isEmpty()) {
+                nextStage();
+                return;
+            }
+            addNewPeers(adding);
+        }
+
+        private void addNewPeers(final Configuration adding) {
+            this.addingPeers = adding.listPeers();
+            LOG.info("Adding peers: {}.", this.addingPeers);
+            for (final PeerId newPeer : this.addingPeers) {
+                if (!this.node.replicatorGroup.addReplicator(newPeer)) {
+                    LOG.error("Node {} start the replicator failed, peer={}.", this.node.getNodeId(), newPeer);
+                    onCaughtUp(this.version, newPeer, false);
+                    return;
+                }
+                final OnCaughtUp caughtUp = new OnCaughtUp(this.node, this.node.currTerm, newPeer, this.version);
+                final long dueTime = Utils.nowMs() + this.node.options.getElectionTimeoutMs();
+                if (!this.node.replicatorGroup.waitCaughtUp(newPeer, this.node.options.getCatchupMargin(), dueTime,
+                    caughtUp)) {
+                    LOG.error("Node {} waitCaughtUp, peer={}.", this.node.getNodeId(), newPeer);
+                    onCaughtUp(this.version, newPeer, false);
+                    return;
+                }
+            }
+        }
+
+        private void addNewLearners() {
+            final Set<PeerId> addingLearners = new HashSet<>(this.newLearners);
+            addingLearners.removeAll(this.oldLearners);
+            LOG.info("Adding learners: {}.", this.addingPeers);
+            for (final PeerId newLearner : addingLearners) {
+                if (!this.node.replicatorGroup.addReplicator(newLearner, ReplicatorType.Learner)) {
+                    LOG.error("Node {} start the learner replicator failed, peer={}.", this.node.getNodeId(),
+                        newLearner);
+                }
+            }
+        }
+
+        void onCaughtUp(final long version, final PeerId peer, final boolean success) {
+            if (version != this.version) {
+                LOG.warn("Ignore onCaughtUp message, mismatch configuration context version, expect {}, but is {}.",
+                    this.version, version);
+                return;
+            }
+            Requires.requireTrue(this.stage == Stage.STAGE_CATCHING_UP, "Stage is not in STAGE_CATCHING_UP");
+            if (success) {
+                this.addingPeers.remove(peer);
+                if (this.addingPeers.isEmpty()) {
+                    nextStage();
+                    return;
+                }
+                return;
+            }
+            LOG.warn("Node {} fail to catch up peer {} when trying to change peers from {} to {}.",
+                this.node.getNodeId(), peer, this.oldPeers, this.newPeers);
+            reset(new Status(RaftError.ECATCHUP, "Peer %s failed to catch up.", peer));
+        }
+
+        void reset() {
+            reset(null);
+        }
+
+        void reset(final Status st) {
+            if (st != null && st.isOk()) {
+                this.node.stopReplicator(this.newPeers, this.oldPeers);
+                this.node.stopReplicator(this.newLearners, this.oldLearners);
+            } else {
+                this.node.stopReplicator(this.oldPeers, this.newPeers);
+                this.node.stopReplicator(this.oldLearners, this.newLearners);
+            }
+            clearPeers();
+            clearLearners();
+
+            this.version++;
+            this.stage = Stage.STAGE_NONE;
+            this.nchanges = 0;
+            if (this.done != null) {
+                Utils.runClosureInThread(this.done, st != null ? st : new Status(RaftError.EPERM,
+                    "Leader stepped down."));
+                this.done = null;
+            }
+        }
+
+        private void clearLearners() {
+            this.newLearners.clear();
+            this.oldLearners.clear();
+        }
+
+        private void clearPeers() {
+            this.newPeers.clear();
+            this.oldPeers.clear();
+            this.addingPeers.clear();
+        }
+
+        /**
+         * Invoked when this node becomes the leader, write a configuration change log as the first log.
+         */
+        void flush(final Configuration conf, final Configuration oldConf) {
+            Requires.requireTrue(!isBusy(), "Flush when busy");
+            this.newPeers = conf.listPeers();
+            this.newLearners = conf.listLearners();
+            if (oldConf == null || oldConf.isEmpty()) {
+                this.stage = Stage.STAGE_STABLE;
+                this.oldPeers = this.newPeers;
+                this.oldLearners = this.newLearners;
+            } else {
+                this.stage = Stage.STAGE_JOINT;
+                this.oldPeers = oldConf.listPeers();
+                this.oldLearners = oldConf.listLearners();
+            }
+            this.node.unsafeApplyConfiguration(conf, oldConf == null || oldConf.isEmpty() ? null : oldConf, true);
+        }
+
+        void nextStage() {
+            Requires.requireTrue(isBusy(), "Not in busy stage");
+            switch (this.stage) {
+                case STAGE_CATCHING_UP:
+                    if (this.nchanges > 0) {
+                        this.stage = Stage.STAGE_JOINT;
+                        this.node.unsafeApplyConfiguration(new Configuration(this.newPeers, this.newLearners),
+                            new Configuration(this.oldPeers), false);
+                        return;
+                    }
+                case STAGE_JOINT:
+                    this.stage = Stage.STAGE_STABLE;
+                    this.node.unsafeApplyConfiguration(new Configuration(this.newPeers, this.newLearners), null, false);
+                    break;
+                case STAGE_STABLE:
+                    final boolean shouldStepDown = !this.newPeers.contains(this.node.serverId);
+                    reset(new Status());
+                    if (shouldStepDown) {
+                        this.node.stepDown(this.node.currTerm, true, new Status(RaftError.ELEADERREMOVED,
+                            "This node was removed."));
+                    }
+                    break;
+                case STAGE_NONE:
+                    // noinspection ConstantConditions
+                    Requires.requireTrue(false, "Can't reach here");
+                    break;
+            }
+        }
+
+        boolean isBusy() {
+            return this.stage != Stage.STAGE_NONE;
+        }
+    }
+
+    public NodeImpl() {
+        this(null, null);
+    }
+
+    public NodeImpl(final String groupId, final PeerId serverId) {
+        super();
+        if (groupId != null) {
+            Utils.verifyGroupId(groupId);
+        }
+        this.groupId = groupId;
+        this.serverId = serverId != null ? serverId.copy() : null;
+        this.state = State.STATE_UNINITIALIZED;
+        this.currTerm = 0;
+        updateLastLeaderTimestamp(Utils.monotonicMs());
+        this.confCtx = new ConfigurationCtx(this);
+        this.wakingCandidate = null;
+        final int num = GLOBAL_NUM_NODES.incrementAndGet();
+        LOG.info("The number of active nodes increment to {}.", num);
+    }
+
+    private boolean initSnapshotStorage() {
+        if (StringUtils.isEmpty(this.options.getSnapshotUri())) {
+            LOG.warn("Do not set snapshot uri, ignore initSnapshotStorage.");
+            return true;
+        }
+        this.snapshotExecutor = new SnapshotExecutorImpl();
+        final SnapshotExecutorOptions opts = new SnapshotExecutorOptions();
+        opts.setUri(this.options.getSnapshotUri());
+        opts.setFsmCaller(this.fsmCaller);
+        opts.setNode(this);
+        opts.setLogManager(this.logManager);
+        opts.setAddr(this.serverId != null ? this.serverId.getEndpoint() : null);
+        opts.setInitTerm(this.currTerm);
+        opts.setFilterBeforeCopyRemote(this.options.isFilterBeforeCopyRemote());
+        // get snapshot throttle
+        opts.setSnapshotThrottle(this.options.getSnapshotThrottle());
+        return this.snapshotExecutor.init(opts);
+    }
+
+    private boolean initLogStorage() {
+        Requires.requireNonNull(this.fsmCaller, "Null fsm caller");
+        this.logStorage = this.serviceFactory.createLogStorage(this.options.getLogUri(), this.raftOptions);
+        this.logManager = new LogManagerImpl();
+        final LogManagerOptions opts = new LogManagerOptions();
+        opts.setLogEntryCodecFactory(this.serviceFactory.createLogEntryCodecFactory());
+        opts.setLogStorage(this.logStorage);
+        opts.setConfigurationManager(this.configManager);
+        opts.setFsmCaller(this.fsmCaller);
+        opts.setNodeMetrics(this.metrics);
+        opts.setDisruptorBufferSize(this.raftOptions.getDisruptorBufferSize());
+        opts.setRaftOptions(this.raftOptions);
+        return this.logManager.init(opts);
+    }
+
+    private boolean initMetaStorage() {
+        this.metaStorage = this.serviceFactory.createRaftMetaStorage(this.options.getRaftMetaUri(), this.raftOptions);
+        RaftMetaStorageOptions opts = new RaftMetaStorageOptions();
+        opts.setNode(this);
+        if (!this.metaStorage.init(opts)) {
+            LOG.error("Node {} init meta storage failed, uri={}.", this.serverId, this.options.getRaftMetaUri());
+            return false;
+        }
+        this.currTerm = this.metaStorage.getTerm();
+        this.votedId = this.metaStorage.getVotedFor().copy();
+        return true;
+    }
+
+    private void handleSnapshotTimeout() {
+        this.writeLock.lock();
+        try {
+            if (!this.state.isActive()) {
+                return;
+            }
+        } finally {
+            this.writeLock.unlock();
+        }
+        // do_snapshot in another thread to avoid blocking the timer thread.
+        Utils.runInThread(() -> doSnapshot(null));
+    }
+
+    private void handleElectionTimeout() {
+        boolean doUnlock = true;
+        this.writeLock.lock();
+        try {
+            if (this.state != State.STATE_FOLLOWER) {
+                return;
+            }
+            if (isCurrentLeaderValid()) {
+                return;
+            }
+            resetLeaderId(PeerId.emptyPeer(), new Status(RaftError.ERAFTTIMEDOUT, "Lost connection from leader %s.",
+                this.leaderId));
+
+            // Judge whether to launch a election.
+            if (!allowLaunchElection()) {
+                return;
+            }
+
+            doUnlock = false;
+            preVote();
+
+        } finally {
+            if (doUnlock) {
+                this.writeLock.unlock();
+            }
+        }
+    }
+
+    /**
+     * Whether to allow for launching election or not by comparing node's priority with target
+     * priority. And at the same time, if next leader is not elected until next election
+     * timeout, it decays its local target priority exponentially.
+     *
+     * @return Whether current node will launch election or not.
+     */
+    @SuppressWarnings("NonAtomicOperationOnVolatileField")
+    private boolean allowLaunchElection() {
+
+        // Priority 0 is a special value so that a node will never participate in election.
+        if (this.serverId.isPriorityNotElected()) {
+            LOG.warn("Node {} will never participate in election, because it's priority={}.", getNodeId(),
+                this.serverId.getPriority());
+            return false;
+        }
+
+        // If this nodes disable priority election, then it can make a election.
+        if (this.serverId.isPriorityDisabled()) {
+            return true;
+        }
+
+        // If current node's priority < target_priority, it does not initiate leader,
+        // election and waits for the next election timeout.
+        if (this.serverId.getPriority() < this.targetPriority) {
+            this.electionTimeoutCounter++;
+
+            // If next leader is not elected until next election timeout, it
+            // decays its local target priority exponentially.
+            if (this.electionTimeoutCounter > 1) {
+                decayTargetPriority();
+                this.electionTimeoutCounter = 0;
+            }
+
+            if (this.electionTimeoutCounter == 1) {
+                LOG.debug("Node {} does not initiate leader election and waits for the next election timeout.",
+                    getNodeId());
+                return false;
+            }
+        }
+
+        return this.serverId.getPriority() >= this.targetPriority;
+    }
+
+    /**
+     * Decay targetPriority value based on gap value.
+     */
+    @SuppressWarnings("NonAtomicOperationOnVolatileField")
+    private void decayTargetPriority() {
+        // Default Gap value should be bigger than 10.
+        final int decayPriorityGap = Math.max(this.options.getDecayPriorityGap(), 10);
+        final int gap = Math.max(decayPriorityGap, (this.targetPriority / 5));
+
+        final int prevTargetPriority = this.targetPriority;
+        this.targetPriority = Math.max(ElectionPriority.MinValue, (this.targetPriority - gap));
+        LOG.info("Node {} priority decay, from: {}, to: {}.", getNodeId(), prevTargetPriority, this.targetPriority);
+    }
+
+    /**
+     * Check and set configuration for node.At the same time, if configuration is changed,
+     * then compute and update the target priority value.
+     *
+     * @param inLock whether the writeLock has already been locked in other place.
+     *
+     */
+    private void checkAndSetConfiguration(final boolean inLock) {
+        if (!inLock) {
+            this.writeLock.lock();
+        }
+        try {
+            final ConfigurationEntry prevConf = this.conf;
+            this.conf = this.logManager.checkAndSetConfiguration(prevConf);
+
+            if (this.conf != prevConf) {
+                // Update target priority value
+                final int prevTargetPriority = this.targetPriority;
+                this.targetPriority = getMaxPriorityOfNodes(this.conf.getConf().getPeers());
+                if (prevTargetPriority != this.targetPriority) {
+                    LOG.info("Node {} target priority value has changed from: {}, to: {}.", getNodeId(),
+                        prevTargetPriority, this.targetPriority);
+                }
+                this.electionTimeoutCounter = 0;
+            }
+        } finally {
+            if (!inLock) {
+                this.writeLock.unlock();
+            }
+        }
+    }
+
+    /**
+     * Get max priority value for all nodes in the same Raft group, and update current node's target priority value.
+     *
+     * @param peerIds peer nodes in the same Raft group
+     *
+     */
+    private int getMaxPriorityOfNodes(final List<PeerId> peerIds) {
+        Requires.requireNonNull(peerIds, "Null peer list");
+
+        int maxPriority = Integer.MIN_VALUE;
+        for (final PeerId peerId : peerIds) {
+            final int priorityVal = peerId.getPriority();
+            maxPriority = Math.max(priorityVal, maxPriority);
+        }
+
+        return maxPriority;
+    }
+
+    private boolean initFSMCaller(final LogId bootstrapId) {
+        if (this.fsmCaller == null) {
+            LOG.error("Fail to init fsm caller, null instance, bootstrapId={}.", bootstrapId);
+            return false;
+        }
+        this.closureQueue = new ClosureQueueImpl();
+        final FSMCallerOptions opts = new FSMCallerOptions();
+        opts.setAfterShutdown(status -> afterShutdown());
+        opts.setLogManager(this.logManager);
+        opts.setFsm(this.options.getFsm());
+        opts.setClosureQueue(this.closureQueue);
+        opts.setNode(this);
+        opts.setBootstrapId(bootstrapId);
+        opts.setDisruptorBufferSize(this.raftOptions.getDisruptorBufferSize());
+        return this.fsmCaller.init(opts);
+    }
+
+    private static class BootstrapStableClosure extends LogManager.StableClosure {
+
+        private final SynchronizedClosure done = new SynchronizedClosure();
+
+        public BootstrapStableClosure() {
+            super(null);
+        }
+
+        public Status await() throws InterruptedException {
+            return this.done.await();
+        }
+
+        @Override
+        public void run(final Status status) {
+            this.done.run(status);
+        }
+    }
+
+    public boolean bootstrap(final BootstrapOptions opts) throws InterruptedException {
+        if (opts.getLastLogIndex() > 0 && (opts.getGroupConf().isEmpty() || opts.getFsm() == null)) {
+            LOG.error("Invalid arguments for bootstrap, groupConf={}, fsm={}, lastLogIndex={}.", opts.getGroupConf(),
+                opts.getFsm(), opts.getLastLogIndex());
+            return false;
+        }
+        if (opts.getGroupConf().isEmpty()) {
+            LOG.error("Bootstrapping an empty node makes no sense.");
+            return false;
+        }
+        Requires.requireNonNull(opts.getServiceFactory(), "Null jraft service factory");
+        this.serviceFactory = opts.getServiceFactory();
+        // Term is not an option since changing it is very dangerous
+        final long bootstrapLogTerm = opts.getLastLogIndex() > 0 ? 1 : 0;
+        final LogId bootstrapId = new LogId(opts.getLastLogIndex(), bootstrapLogTerm);
+        this.options = new NodeOptions();
+        this.raftOptions = this.options.getRaftOptions();
+        this.metrics = new NodeMetrics(opts.isEnableMetrics());
+        this.options.setFsm(opts.getFsm());
+        this.options.setLogUri(opts.getLogUri());
+        this.options.setRaftMetaUri(opts.getRaftMetaUri());
+        this.options.setSnapshotUri(opts.getSnapshotUri());
+
+        this.configManager = new ConfigurationManager();
+        // Create fsmCaller at first as logManager needs it to report error
+        this.fsmCaller = new FSMCallerImpl();
+
+        if (!initLogStorage()) {
+            LOG.error("Fail to init log storage.");
+            return false;
+        }
+        if (!initMetaStorage()) {
+            LOG.error("Fail to init meta storage.");
+            return false;
+        }
+        if (this.currTerm == 0) {
+            this.currTerm = 1;
+            if (!this.metaStorage.setTermAndVotedFor(1, new PeerId())) {
+                LOG.error("Fail to set term.");
+                return false;
+            }
+        }
+
+        if (opts.getFsm() != null && !initFSMCaller(bootstrapId)) {
+            LOG.error("Fail to init fsm caller.");
+            return false;
+        }
+
+        final LogEntry entry = new LogEntry(EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION);
+        entry.getId().setTerm(this.currTerm);
+        entry.setPeers(opts.getGroupConf().listPeers());
+        entry.setLearners(opts.getGroupConf().listLearners());
+
+        final List<LogEntry> entries = new ArrayList<>();
+        entries.add(entry);
+
+        final BootstrapStableClosure bootstrapDone = new BootstrapStableClosure();
+        this.logManager.appendEntries(entries, bootstrapDone);
+        if (!bootstrapDone.await().isOk()) {
+            LOG.error("Fail to append configuration.");
+            return false;
+        }
+
+        if (opts.getLastLogIndex() > 0) {
+            if (!initSnapshotStorage()) {
+                LOG.error("Fail to init snapshot storage.");
+                return false;
+            }
+            final SynchronizedClosure snapshotDone = new SynchronizedClosure();
+            this.snapshotExecutor.doSnapshot(snapshotDone);
+            if (!snapshotDone.await().isOk()) {
+                LOG.error("Fail to save snapshot, status={}.", snapshotDone.getStatus());
+                return false;
+            }
+        }
+
+        if (this.logManager.getFirstLogIndex() != opts.getLastLogIndex() + 1) {
+            throw new IllegalStateException("First and last log index mismatch");
+        }
+        if (opts.getLastLogIndex() > 0) {
+            if (this.logManager.getLastLogIndex() != opts.getLastLogIndex()) {
+                throw new IllegalStateException("Last log index mismatch");
+            }
+        } else {
+            if (this.logManager.getLastLogIndex() != opts.getLastLogIndex() + 1) {
+                throw new IllegalStateException("Last log index mismatch");
+            }
+        }
+
+        return true;
+    }
+
+    private int heartbeatTimeout(final int electionTimeout) {
+        return Math.max(electionTimeout / this.raftOptions.getElectionHeartbeatFactor(), 10);
+    }
+
+    private int randomTimeout(final int timeoutMs) {
+        return ThreadLocalRandom.current().nextInt(timeoutMs, timeoutMs + this.raftOptions.getMaxElectionDelayMs());
+    }
+
+    @Override
+    public boolean init(final NodeOptions opts) {
+        Requires.requireNonNull(opts, "Null node options");
+        Requires.requireNonNull(opts.getRaftOptions(), "Null raft options");
+        Requires.requireNonNull(opts.getServiceFactory(), "Null jraft service factory");
+        this.serviceFactory = opts.getServiceFactory();
+        this.options = opts;
+        this.raftOptions = opts.getRaftOptions();
+        this.metrics = new NodeMetrics(opts.isEnableMetrics());
+        this.serverId.setPriority(opts.getElectionPriority());
+        this.electionTimeoutCounter = 0;
+
+        if (this.serverId.getIp().equals(Utils.IP_ANY)) {
+            LOG.error("Node can't started from IP_ANY.");
+            return false;
+        }
+
+        if (!NodeManager.getInstance().serverExists(this.serverId.getEndpoint())) {
+            LOG.error("No RPC server attached to, did you forget to call addService?");
+            return false;
+        }
+
+        this.timerManager = TIMER_FACTORY.getRaftScheduler(this.options.isSharedTimerPool(),
+            this.options.getTimerPoolSize(), "JRaft-Node-ScheduleThreadPool");
+
+        // Init timers
+        final String suffix = getNodeId().toString();
+        String name = "JRaft-VoteTimer-" + suffix;
+        this.voteTimer = new RepeatedTimer(name, this.options.getElectionTimeoutMs(), TIMER_FACTORY.getVoteTimer(
+            this.options.isSharedVoteTimer(), name)) {
+
+            @Override
+            protected void onTrigger() {
+                handleVoteTimeout();
+            }
+
+            @Override
+            protected int adjustTimeout(final int timeoutMs) {
+                return randomTimeout(timeoutMs);
+            }
+        };
+        name = "JRaft-ElectionTimer-" + suffix;
+        this.electionTimer = new RepeatedTimer(name, this.options.getElectionTimeoutMs(),
+            TIMER_FACTORY.getElectionTimer(this.options.isSharedElectionTimer(), name)) {
+
+            @Override
+            protected void onTrigger() {
+                handleElectionTimeout();
+            }
+
+            @Override
+            protected int adjustTimeout(final int timeoutMs) {
+                return randomTimeout(timeoutMs);
+            }
+        };
+        name = "JRaft-StepDownTimer-" + suffix;
+        this.stepDownTimer = new RepeatedTimer(name, this.options.getElectionTimeoutMs() >> 1,
+            TIMER_FACTORY.getStepDownTimer(this.options.isSharedStepDownTimer(), name)) {
+
+            @Override
+            protected void onTrigger() {
+                handleStepDownTimeout();
+            }
+        };
+        name = "JRaft-SnapshotTimer-" + suffix;
+        this.snapshotTimer = new RepeatedTimer(name, this.options.getSnapshotIntervalSecs() * 1000,
+            TIMER_FACTORY.getSnapshotTimer(this.options.isSharedSnapshotTimer(), name)) {
+
+            private volatile boolean firstSchedule = true;
+
+            @Override
+            protected void onTrigger() {
+                handleSnapshotTimeout();
+            }
+
+            @Override
+            protected int adjustTimeout(final int timeoutMs) {
+                if (!this.firstSchedule) {
+                    return timeoutMs;
+                }
+
+                // Randomize the first snapshot trigger timeout
+                this.firstSchedule = false;
+                if (timeoutMs > 0) {
+                    int half = timeoutMs / 2;
+                    return half + ThreadLocalRandom.current().nextInt(half);
+                } else {
+                    return timeoutMs;
+                }
+            }
+        };
+
+        this.configManager = new ConfigurationManager();
+
+        this.applyDisruptor = DisruptorBuilder.<LogEntryAndClosure> newInstance() //
+            .setRingBufferSize(this.raftOptions.getDisruptorBufferSize()) //
+            .setEventFactory(new LogEntryAndClosureFactory()) //
+            .setThreadFactory(new NamedThreadFactory("JRaft-NodeImpl-Disruptor-", true)) //
+            .setProducerType(ProducerType.MULTI) //
+            .setWaitStrategy(new BlockingWaitStrategy()) //
+            .build();
+        this.applyDisruptor.handleEventsWith(new LogEntryAndClosureHandler());
+        this.applyDisruptor.setDefaultExceptionHandler(new LogExceptionHandler<Object>(getClass().getSimpleName()));
+        this.applyQueue = this.applyDisruptor.start();
+        if (this.metrics.getMetricRegistry() != null) {
+            this.metrics.getMetricRegistry().register("jraft-node-impl-disruptor",
+                new DisruptorMetricSet(this.applyQueue));
+        }
+
+        this.fsmCaller = new FSMCallerImpl();
+        if (!initLogStorage()) {
+            LOG.error("Node {} initLogStorage failed.", getNodeId());
+            return false;
+        }
+        if (!initMetaStorage()) {
+            LOG.error("Node {} initMetaStorage failed.", getNodeId());
+            return false;
+        }
+        if (!initFSMCaller(new LogId(0, 0))) {
+            LOG.error("Node {} initFSMCaller failed.", getNodeId());
+            return false;
+        }
+        this.ballotBox = new BallotBox();
+        final BallotBoxOptions ballotBoxOpts = new BallotBoxOptions();
+        ballotBoxOpts.setWaiter(this.fsmCaller);
+        ballotBoxOpts.setClosureQueue(this.closureQueue);
+        if (!this.ballotBox.init(ballotBoxOpts)) {
+            LOG.error("Node {} init ballotBox failed.", getNodeId());
+            return false;
+        }
+
+        if (!initSnapshotStorage()) {
+            LOG.error("Node {} initSnapshotStorage failed.", getNodeId());
+            return false;
+        }
+
+        final Status st = this.logManager.checkConsistency();
+        if (!st.isOk()) {
+            LOG.error("Node {} is initialized with inconsistent log, status={}.", getNodeId(), st);
+            return false;
+        }
+        this.conf = new ConfigurationEntry();
+        this.conf.setId(new LogId());
+        // if have log using conf in log, else using conf in options
+        if (this.logManager.getLastLogIndex() > 0) {
+            checkAndSetConfiguration(false);
+        } else {
+            this.conf.setConf(this.options.getInitialConf());
+            // initially set to max(priority of all nodes)
+            this.targetPriority = getMaxPriorityOfNodes(this.conf.getConf().getPeers());
+        }
+
+        if (!this.conf.isEmpty()) {
+            Requires.requireTrue(this.conf.isValid(), "Invalid conf: %s", this.conf);
+        } else {
+            LOG.info("Init node {} with empty conf.", this.serverId);
+        }
+
+        // TODO RPC service and ReplicatorGroup is in cycle dependent, refactor it
+        this.replicatorGroup = new ReplicatorGroupImpl();
+        this.rpcService = new DefaultRaftClientService(this.replicatorGroup);
+        final ReplicatorGroupOptions rgOpts = new ReplicatorGroupOptions();
+        rgOpts.setHeartbeatTimeoutMs(heartbeatTimeout(this.options.getElectionTimeoutMs()));
+        rgOpts.setElectionTimeoutMs(this.options.getElectionTimeoutMs());
+        rgOpts.setLogManager(this.logManager);
+        rgOpts.setBallotBox(this.ballotBox);
+        rgOpts.setNode(this);
+        rgOpts.setRaftRpcClientService(this.rpcService);
+        rgOpts.setSnapshotStorage(this.snapshotExecutor != null ? this.snapshotExecutor.getSnapshotStorage() : null);
+        rgOpts.setRaftOptions(this.raftOptions);
+        rgOpts.setTimerManager(this.timerManager);
+
+        // Adds metric registry to RPC service.
+        this.options.setMetricRegistry(this.metrics.getMetricRegistry());
+
+        if (!this.rpcService.init(this.options)) {
+            LOG.error("Fail to init rpc service.");
+            return false;
+        }
+        this.replicatorGroup.init(new NodeId(this.groupId, this.serverId), rgOpts);
+
+        this.readOnlyService = new ReadOnlyServiceImpl();
+        final ReadOnlyServiceOptions rosOpts = new ReadOnlyServiceOptions();
+        rosOpts.setFsmCaller(this.fsmCaller);
+        rosOpts.setNode(this);
+        rosOpts.setRaftOptions(this.raftOptions);
+
+        if (!this.readOnlyService.init(rosOpts)) {
+            LOG.error("Fail to init readOnlyService.");
+            return false;
+        }
+
+        // set state to follower
+        this.state = State.STATE_FOLLOWER;
+
+        if (LOG.isInfoEnabled()) {
+            LOG.info("Node {} init, term={}, lastLogId={}, conf={}, oldConf={}.", getNodeId(), this.currTerm,
+                this.logManager.getLastLogId(false), this.conf.getConf(), this.conf.getOldConf());
+        }
+
+        if (this.snapshotExecutor != null && this.options.getSnapshotIntervalSecs() > 0) {
+            LOG.debug("Node {} start snapshot timer, term={}.", getNodeId(), this.currTerm);
+            this.snapshotTimer.start();
+        }
+
+        if (!this.conf.isEmpty()) {
+            stepDown(this.currTerm, false, new Status());
+        }
+
+        if (!NodeManager.getInstance().add(this)) {
+            LOG.error("NodeManager add {} failed.", getNodeId());
+            return false;
+        }
+
+        // Now the raft node is started , have to acquire the writeLock to avoid race
+        // conditions
+        this.writeLock.lock();
+        if (this.conf.isStable() && this.conf.getConf().size() == 1 && this.conf.getConf().contains(this.serverId)) {
+            // The group contains only this server which must be the LEADER, trigger
+            // the timer immediately.
+            electSelf();
+        } else {
+            this.writeLock.unlock();
+        }
+
+        return true;
+    }
+
+    @OnlyForTest
+    void tryElectSelf() {
+        this.writeLock.lock();
+        // unlock in electSelf
+        electSelf();
+    }
+
+    // should be in writeLock
+    private void electSelf() {
+        long oldTerm;
+        try {
+            LOG.info("Node {} start vote and grant vote self, term={}.", getNodeId(), this.currTerm);
+            if (!this.conf.contains(this.serverId)) {
+                LOG.warn("Node {} can't do electSelf as it is not in {}.", getNodeId(), this.conf);
+                return;
+            }
+            if (this.state == State.STATE_FOLLOWER) {
+                LOG.debug("Node {} stop election timer, term={}.", getNodeId(), this.currTerm);
+                this.electionTimer.stop();
+            }
+            resetLeaderId(PeerId.emptyPeer(), new Status(RaftError.ERAFTTIMEDOUT,
+                "A follower's leader_id is reset to NULL as it begins to request_vote."));
+            this.state = State.STATE_CANDIDATE;
+            this.currTerm++;
+            this.votedId = this.serverId.copy();
+            LOG.debug("Node {} start vote timer, term={} .", getNodeId(), this.currTerm);
+            this.voteTimer.start();
+            this.voteCtx.init(this.conf.getConf(), this.conf.isStable() ? null : this.conf.getOldConf());
+            oldTerm = this.currTerm;
+        } finally {
+            this.writeLock.unlock();
+        }
+
+        final LogId lastLogId = this.logManager.getLastLogId(true);
+
+        this.writeLock.lock();
+        try {
+            // vote need defense ABA after unlock&writeLock
+            if (oldTerm != this.currTerm) {
+                LOG.warn("Node {} raise term {} when getLastLogId.", getNodeId(), this.currTerm);
+                return;
+            }
+            for (final PeerId peer : this.conf.listPeers()) {
+                if (peer.equals(this.serverId)) {
+                    continue;
+                }
+                if (!this.rpcService.connect(peer.getEndpoint())) {
+                    LOG.warn("Node {} channel init failed, address={}.", getNodeId(), peer.getEndpoint());
+                    continue;
+                }
+                final OnRequestVoteRpcDone done = new OnRequestVoteRpcDone(peer, this.currTerm, this);
+                done.request = RequestVoteRequest.newBuilder() //
+                    .setPreVote(false) // It's not a pre-vote request.
+                    .setGroupId(this.groupId) //
+                    .setServerId(this.serverId.toString()) //
+                    .setPeerId(peer.toString()) //
+                    .setTerm(this.currTerm) //
+                    .setLastLogIndex(lastLogId.getIndex()) //
+                    .setLastLogTerm(lastLogId.getTerm()) //
+                    .build();
+                this.rpcService.requestVote(peer.getEndpoint(), done.request, done);
+            }
+
+            this.metaStorage.setTermAndVotedFor(this.currTerm, this.serverId);
+            this.voteCtx.grant(this.serverId);
+            if (this.voteCtx.isGranted()) {
+                becomeLeader();
+            }
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    private void resetLeaderId(final PeerId newLeaderId, final Status status) {
+        if (newLeaderId.isEmpty()) {
+            if (!this.leaderId.isEmpty() && this.state.compareTo(State.STATE_TRANSFERRING) > 0) {
+                this.fsmCaller.onStopFollowing(new LeaderChangeContext(this.leaderId.copy(), this.currTerm, status));
+            }
+            this.leaderId = PeerId.emptyPeer();
+        } else {
+            if (this.leaderId == null || this.leaderId.isEmpty()) {
+                this.fsmCaller.onStartFollowing(new LeaderChangeContext(newLeaderId, this.currTerm, status));
+            }
+            this.leaderId = newLeaderId.copy();
+        }
+    }
+
+    // in writeLock
+    private void checkStepDown(final long requestTerm, final PeerId serverId) {
+        final Status status = new Status();
+        if (requestTerm > this.currTerm) {
+            status.setError(RaftError.ENEWLEADER, "Raft node receives message from new leader with higher term.");
+            stepDown(requestTerm, false, status);
+        } else if (this.state != State.STATE_FOLLOWER) {
+            status.setError(RaftError.ENEWLEADER, "Candidate receives message from new leader with the same term.");
+            stepDown(requestTerm, false, status);
+        } else if (this.leaderId.isEmpty()) {
+            status.setError(RaftError.ENEWLEADER, "Follower receives message from new leader with the same term.");
+            stepDown(requestTerm, false, status);
+        }
+        // save current leader
+        if (this.leaderId == null || this.leaderId.isEmpty()) {
+            resetLeaderId(serverId, status);
+        }
+    }
+
+    private void becomeLeader() {
+        Requires.requireTrue(this.state == State.STATE_CANDIDATE, "Illegal state: " + this.state);
+        LOG.info("Node {} become leader of group, term={}, conf={}, oldConf={}.", getNodeId(), this.currTerm,
+            this.conf.getConf(), this.conf.getOldConf());
+        // cancel candidate vote timer
+        stopVoteTimer();
+        this.state = State.STATE_LEADER;
+        this.leaderId = this.serverId.copy();
+        this.replicatorGroup.resetTerm(this.currTerm);
+        // Start follower's replicators
+        for (final PeerId peer : this.conf.listPeers()) {
+            if (peer.equals(this.serverId)) {
+                continue;
+            }
+            LOG.debug("Node {} add a replicator, term={}, peer={}.", getNodeId(), this.currTerm, peer);
+            if (!this.replicatorGroup.addReplicator(peer)) {
+                LOG.error("Fail to add a replicator, peer={}.", peer);
+            }
+        }
+
+        // Start learner's replicators
+        for (final PeerId peer : this.conf.listLearners()) {
+            LOG.debug("Node {} add a learner replicator, term={}, peer={}.", getNodeId(), this.currTerm, peer);
+            if (!this.replicatorGroup.addReplicator(peer, ReplicatorType.Learner)) {
+                LOG.error("Fail to add a learner replicator, peer={}.", peer);
+            }
+        }
+
+        // init commit manager
+        this.ballotBox.resetPendingIndex(this.logManager.getLastLogIndex() + 1);
+        // Register _conf_ctx to reject configuration changing before the first log
+        // is committed.
+        if (this.confCtx.isBusy()) {
+            throw new IllegalStateException();
+        }
+        this.confCtx.flush(this.conf.getConf(), this.conf.getOldConf());
+        this.stepDownTimer.start();
+    }
+
+    // should be in writeLock
+    private void stepDown(final long term, final boolean wakeupCandidate, final Status status) {
+        LOG.debug("Node {} stepDown, term={}, newTerm={}, wakeupCandidate={}.", getNodeId(), this.currTerm, term,
+            wakeupCandidate);
+        if (!this.state.isActive()) {
+            return;
+        }
+        if (this.state == State.STATE_CANDIDATE) {
+            stopVoteTimer();
+        } else if (this.state.compareTo(State.STATE_TRANSFERRING) <= 0) {
+            stopStepDownTimer();
+            this.ballotBox.clearPendingTasks();
+            // signal fsm leader stop immediately
+            if (this.state == State.STATE_LEADER) {
+                onLeaderStop(status);
+            }
+        }
+        // reset leader_id
+        resetLeaderId(PeerId.emptyPeer(), status);
+
+        // soft state in memory
+        this.state = State.STATE_FOLLOWER;
+        this.confCtx.reset();
+        updateLastLeaderTimestamp(Utils.monotonicMs());
+        if (this.snapshotExecutor != null) {
+            this.snapshotExecutor.interruptDownloadingSnapshots(term);
+        }
+
+        // meta state
+        if (term > this.currTerm) {
+            this.currTerm = term;
+            this.votedId = PeerId.emptyPeer();
+            this.metaStorage.setTermAndVotedFor(term, this.votedId);
+        }
+
+        if (wakeupCandidate) {
+            this.wakingCandidate = this.replicatorGroup.stopAllAndFindTheNextCandidate(this.conf);
+            if (this.wakingCandidate != null) {
+                Replicator.sendTimeoutNowAndStop(this.wakingCandidate, this.options.getElectionTimeoutMs());
+            }
+        } else {
+            this.replicatorGroup.stopAll();
+        }
+        if (this.stopTransferArg != null) {
+            if (this.transferTimer != null) {
+                this.transferTimer.cancel(true);
+            }
+            // There is at most one StopTransferTimer at the same term, it's safe to
+            // mark stopTransferArg to NULL
+            this.stopTransferArg = null;
+        }
+        // Learner node will not trigger the election timer.
+        if (!isLearner()) {
+            this.electionTimer.restart();
+        } else {
+            LOG.info("Node {} is a learner, election timer is not started.", this.nodeId);
+        }
+    }
+
+    // Should be in readLock
+    private boolean isLearner() {
+        return this.conf.listLearners().contains(this.serverId);
+    }
+
+    private void stopStepDownTimer() {
+        if (this.stepDownTimer != null) {
+            this.stepDownTimer.stop();
+        }
+    }
+
+    private void stopVoteTimer() {
+        if (this.voteTimer != null) {
+            this.voteTimer.stop();
+        }
+    }
+
+    class LeaderStableClosure extends LogManager.StableClosure {
+
+        public LeaderStableClosure(final List<LogEntry> entries) {
+            super(entries);
+        }
+
+        @Override
+        public void run(final Status status) {
+            if (status.isOk()) {
+                NodeImpl.this.ballotBox.commitAt(this.firstLogIndex, this.firstLogIndex + this.nEntries - 1,
+                    NodeImpl.this.serverId);
+            } else {
+                LOG.error("Node {} append [{}, {}] failed, status={}.", getNodeId(), this.firstLogIndex,
+                    this.firstLogIndex + this.nEntries - 1, status);
+            }
+        }
+    }
+
+    private void executeApplyingTasks(final List<LogEntryAndClosure> tasks) {
+        this.writeLock.lock();
+        try {
+            final int size = tasks.size();
+            if (this.state != State.STATE_LEADER) {
+                final Status st = new Status();
+                if (this.state != State.STATE_TRANSFERRING) {
+                    st.setError(RaftError.EPERM, "Is not leader.");
+                } else {
+                    st.setError(RaftError.EBUSY, "Is transferring leadership.");
+                }
+                LOG.debug("Node {} can't apply, status={}.", getNodeId(), st);
+                final List<LogEntryAndClosure> savedTasks = new ArrayList<>(tasks);
+                Utils.runInThread(() -> {
+                    for (int i = 0; i < size; i++) {
+                        savedTasks.get(i).done.run(st);
+                    }
+                });
+                return;
+            }
+            final List<LogEntry> entries = new ArrayList<>(size);
+            for (int i = 0; i < size; i++) {
+                final LogEntryAndClosure task = tasks.get(i);
+                if (task.expectedTerm != -1 && task.expectedTerm != this.currTerm) {
+                    LOG.debug("Node {} can't apply task whose expectedTerm={} doesn't match currTerm={}.", getNodeId(),
+                        task.expectedTerm, this.currTerm);
+                    if (task.done != null) {
+                        final Status st = new Status(RaftError.EPERM, "expected_term=%d doesn't match current_term=%d",
+                            task.expectedTerm, this.currTerm);
+                        Utils.runClosureInThread(task.done, st);
+                    }
+                    continue;
+                }
+                if (!this.ballotBox.appendPendingTask(this.conf.getConf(),
+                    this.conf.isStable() ? null : this.conf.getOldConf(), task.done)) {
+                    Utils.runClosureInThread(task.done, new Status(RaftError.EINTERNAL, "Fail to append task."));
+                    continue;
+                }
+                // set task entry info before adding to list.
+                task.entry.getId().setTerm(this.currTerm);
+                task.entry.setType(EnumOutter.EntryType.ENTRY_TYPE_DATA);
+                entries.add(task.entry);
+            }
+            this.logManager.appendEntries(entries, new LeaderStableClosure(entries));
+            // update conf.first
+            checkAndSetConfiguration(true);
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    /**
+     * Returns the node metrics.
+     *
+     * @return returns metrics of current node.
+     */
+    @Override
+    public NodeMetrics getNodeMetrics() {
+        return this.metrics;
+    }
+
+    /**
+     * Returns the JRaft service factory for current node.
+     * @since 1.2.6
+     * @return the service factory
+     */
+    public JRaftServiceFactory getServiceFactory() {
+        return this.serviceFactory;
+    }
+
+    @Override
+    public void readIndex(final byte[] requestContext, final ReadIndexClosure done) {
+        if (this.shutdownLatch != null) {
+            Utils.runClosureInThread(done, new Status(RaftError.ENODESHUTDOWN, "Node is shutting down."));
+            throw new IllegalStateException("Node is shutting down");
+        }
+        Requires.requireNonNull(done, "Null closure");
+        this.readOnlyService.addRequest(requestContext, done);
+    }
+
+    /**
+     * ReadIndex response closure
+     * @author dennis
+     */
+    private class ReadIndexHeartbeatResponseClosure extends RpcResponseClosureAdapter<AppendEntriesResponse> {
+        final ReadIndexResponse.Builder             respBuilder;
+        final RpcResponseClosure<ReadIndexResponse> closure;
+        final int                                   quorum;
+        final int                                   failPeersThreshold;
+        int                                         ackSuccess;
+        int                                         ackFailures;
+        boolean                                     isDone;
+
+        public ReadIndexHeartbeatResponseClosure(final RpcResponseClosure<ReadIndexResponse> closure,
+                                                 final ReadIndexResponse.Builder rb, final int quorum,
+                                                 final int peersCount) {
+            super();
+            this.closure = closure;
+            this.respBuilder = rb;
+            this.quorum = quorum;
+            this.failPeersThreshold = peersCount % 2 == 0 ? (quorum - 1) : quorum;
+            this.ackSuccess = 0;
+            this.ackFailures = 0;
+            this.isDone = false;
+        }
+
+        @Override
+        public synchronized void run(final Status status) {
+            if (this.isDone) {
+                return;
+            }
+            if (status.isOk() && getResponse().getSuccess()) {
+                this.ackSuccess++;
+            } else {
+                this.ackFailures++;
+            }
+            // Include leader self vote yes.
+            if (this.ackSuccess + 1 >= this.quorum) {
+                this.respBuilder.setSuccess(true);
+                this.closure.setResponse(this.respBuilder.build());
+                this.closure.run(Status.OK());
+                this.isDone = true;
+            } else if (this.ackFailures >= this.failPeersThreshold) {
+                this.respBuilder.setSuccess(false);
+                this.closure.setResponse(this.respBuilder.build());
+                this.closure.run(Status.OK());
+                this.isDone = true;
+            }
+        }
+    }
+
+    /**
+     * Handle read index request.
+     */
+    @Override
+    public void handleReadIndexRequest(final ReadIndexRequest request, final RpcResponseClosure<ReadIndexResponse> done) {
+        final long startMs = Utils.monotonicMs();
+        this.readLock.lock();
+        try {
+            switch (this.state) {
+                case STATE_LEADER:
+                    readLeader(request, ReadIndexResponse.newBuilder(), done);
+                    break;
+                case STATE_FOLLOWER:
+                    readFollower(request, done);
+                    break;
+                case STATE_TRANSFERRING:
+                    done.run(new Status(RaftError.EBUSY, "Is transferring leadership."));
+                    break;
+                default:
+                    done.run(new Status(RaftError.EPERM, "Invalid state for readIndex: %s.", this.state));
+                    break;
+            }
+        } finally {
+            this.readLock.unlock();
+            this.metrics.recordLatency("handle-read-index", Utils.monotonicMs() - startMs);
+            this.metrics.recordSize("handle-read-index-entries", request.getEntriesCount());
+        }
+    }
+
+    private int getQuorum() {
+        final Configuration c = this.conf.getConf();
+        if (c.isEmpty()) {
+            return 0;
+        }
+        return c.getPeers().size() / 2 + 1;
+    }
+
+    private void readFollower(final ReadIndexRequest request, final RpcResponseClosure<ReadIndexResponse> closure) {
+        if (this.leaderId == null || this.leaderId.isEmpty()) {
+            closure.run(new Status(RaftError.EPERM, "No leader at term %d.", this.currTerm));
+            return;
+        }
+        // send request to leader.
+        final ReadIndexRequest newRequest = ReadIndexRequest.newBuilder() //
+            .mergeFrom(request) //
+            .setPeerId(this.leaderId.toString()) //
+            .build();
+        this.rpcService.readIndex(this.leaderId.getEndpoint(), newRequest, -1, closure);
+    }
+
+    private void readLeader(final ReadIndexRequest request, final ReadIndexResponse.Builder respBuilder,
+                            final RpcResponseClosure<ReadIndexResponse> closure) {
+        final int quorum = getQuorum();
+        if (quorum <= 1) {
+            // Only one peer, fast path.
+            respBuilder.setSuccess(true) //
+                .setIndex(this.ballotBox.getLastCommittedIndex());
+            closure.setResponse(respBuilder.build());
+            closure.run(Status.OK());
+            return;
+        }
+
+        final long lastCommittedIndex = this.ballotBox.getLastCommittedIndex();
+        if (this.logManager.getTerm(lastCommittedIndex) != this.currTerm) {
+            // Reject read only request when this leader has not committed any log entry at its term
+            closure
+                .run(new Status(
+                    RaftError.EAGAIN,
+                    "ReadIndex request rejected because leader has not committed any log entry at its term, logIndex=%d, currTerm=%d.",
+                    lastCommittedIndex, this.currTerm));
+            return;
+        }
+        respBuilder.setIndex(lastCommittedIndex);
+
+        if (request.getPeerId() != null) {
+            // request from follower or learner, check if the follower/learner is in current conf.
+            final PeerId peer = new PeerId();
+            peer.parse(request.getServerId());
+            if (!this.conf.contains(peer) && !this.conf.containsLearner(peer)) {
+                closure
+                    .run(new Status(RaftError.EPERM, "Peer %s is not in current configuration: %s.", peer, this.conf));
+                return;
+            }
+        }
+
+        ReadOnlyOption readOnlyOpt = this.raftOptions.getReadOnlyOptions();
+        if (readOnlyOpt == ReadOnlyOption.ReadOnlyLeaseBased && !isLeaderLeaseValid()) {
+            // If leader lease timeout, we must change option to ReadOnlySafe
+            readOnlyOpt = ReadOnlyOption.ReadOnlySafe;
+        }
+
+        switch (readOnlyOpt) {
+            case ReadOnlySafe:
+                final List<PeerId> peers = this.conf.getConf().getPeers();
+                Requires.requireTrue(peers != null && !peers.isEmpty(), "Empty peers");
+                final ReadIndexHeartbeatResponseClosure heartbeatDone = new ReadIndexHeartbeatResponseClosure(closure,
+                    respBuilder, quorum, peers.size());
+                // Send heartbeat requests to followers
+                for (final PeerId peer : peers) {
+                    if (peer.equals(this.serverId)) {
+                        continue;
+                    }
+                    this.replicatorGroup.sendHeartbeat(peer, heartbeatDone);
+                }
+                break;
+            case ReadOnlyLeaseBased:
+                // Responses to followers and local node.
+                respBuilder.setSuccess(true);
+                closure.setResponse(respBuilder.build());
+                closure.run(Status.OK());
+                break;
+        }
+    }
+
+    @Override
+    public void apply(final Task task) {
+        if (this.shutdownLatch != null) {
+            Utils.runClosureInThread(task.getDone(), new Status(RaftError.ENODESHUTDOWN, "Node is shutting down."));
+            throw new IllegalStateException("Node is shutting down");
+        }
+        Requires.requireNonNull(task, "Null task");
+
+        final LogEntry entry = new LogEntry();
+        entry.setData(task.getData());
+        int retryTimes = 0;
+        try {
+            final EventTranslator<LogEntryAndClosure> translator = (event, sequence) -> {
+                event.reset();
+                event.done = task.getDone();
+                event.entry = entry;
+                event.expectedTerm = task.getExpectedTerm();
+            };
+            while (true) {
+                if (this.applyQueue.tryPublishEvent(translator)) {
+                    break;
+                } else {
+                    retryTimes++;
+                    if (retryTimes > MAX_APPLY_RETRY_TIMES) {
+                        Utils.runClosureInThread(task.getDone(),
+                            new Status(RaftError.EBUSY, "Node is busy, has too many tasks."));
+                        LOG.warn("Node {} applyQueue is overload.", getNodeId());
+                        this.metrics.recordTimes("apply-task-overload-times", 1);
+                        return;
+                    }
+                    ThreadHelper.onSpinWait();
+                }
+            }
+
+        } catch (final Exception e) {
+            LOG.error("Fail to apply task.", e);
+            Utils.runClosureInThread(task.getDone(), new Status(RaftError.EPERM, "Node is down."));
+        }
+    }
+
+    @Override
+    public Message handlePreVoteRequest(final RequestVoteRequest request) {
+        boolean doUnlock = true;
+        this.writeLock.lock();
+        try {
+            if (!this.state.isActive()) {
+                LOG.warn("Node {} is not in active state, currTerm={}.", getNodeId(), this.currTerm);
+                return RpcFactoryHelper //
+                    .responseFactory() //
+                    .newResponse(RequestVoteResponse.getDefaultInstance(), RaftError.EINVAL,
+                        "Node %s is not in active state, state %s.", getNodeId(), this.state.name());
+            }
+            final PeerId candidateId = new PeerId();
+            if (!candidateId.parse(request.getServerId())) {
+                LOG.warn("Node {} received PreVoteRequest from {} serverId bad format.", getNodeId(),
+                    request.getServerId());
+                return RpcFactoryHelper //
+                    .responseFactory() //
+                    .newResponse(RequestVoteResponse.getDefaultInstance(), RaftError.EINVAL,
+                        "Parse candidateId failed: %s.", request.getServerId());
+            }
+            boolean granted = false;
+            // noinspection ConstantConditions
+            do {
+                if (!this.conf.contains(candidateId)) {
+                    LOG.warn("Node {} ignore PreVoteRequest from {} as it is not in conf <{}>.", getNodeId(),
+                        request.getServerId(), this.conf);
+                    break;
+                }
+                if (this.leaderId != null && !this.leaderId.isEmpty() && isCurrentLeaderValid()) {
+                    LOG.info(
+                        "Node {} ignore PreVoteRequest from {}, term={}, currTerm={}, because the leader {}'s lease is still valid.",
+                        getNodeId(), request.getServerId(), request.getTerm(), this.currTerm, this.leaderId);
+                    break;
+                }
+                if (request.getTerm() < this.currTerm) {
+                    LOG.info("Node {} ignore PreVoteRequest from {}, term={}, currTerm={}.", getNodeId(),
+                        request.getServerId(), request.getTerm(), this.currTerm);
+                    // A follower replicator may not be started when this node become leader, so we must check it.
+                    checkReplicator(candidateId);
+                    break;
+                }
+                // A follower replicator may not be started when this node become leader, so we must check it.
+                // check replicator state
+                checkReplicator(candidateId);
+
+                doUnlock = false;
+                this.writeLock.unlock();
+
+                final LogId lastLogId = this.logManager.getLastLogId(true);
+
+                doUnlock = true;
+                this.writeLock.lock();
+                final LogId requestLastLogId = new LogId(request.getLastLogIndex(), request.getLastLogTerm());
+                granted = requestLastLogId.compareTo(lastLogId) >= 0;
+
+                LOG.info(
+                    "Node {} received PreVoteRequest from {}, term={}, currTerm={}, granted={}, requestLastLogId={}, lastLogId={}.",
+                    getNodeId(), request.getServerId(), request.getTerm(), this.currTerm, granted, requestLastLogId,
+                    lastLogId);
+            } while (false);
+
+            return RequestVoteResponse.newBuilder() //
+                .setTerm(this.currTerm) //
+                .setGranted(granted) //
+                .build();
+        } finally {
+            if (doUnlock) {
+                this.writeLock.unlock();
+            }
+        }
+    }
+
+    // in read_lock
+    private boolean isLeaderLeaseValid() {
+        final long monotonicNowMs = Utils.monotonicMs();
+        if (checkLeaderLease(monotonicNowMs)) {
+            return true;
+        }
+        checkDeadNodes0(this.conf.getConf().getPeers(), monotonicNowMs, false, null);
+        return checkLeaderLease(monotonicNowMs);
+    }
+
+    private boolean checkLeaderLease(final long monotonicNowMs) {
+        return monotonicNowMs - this.lastLeaderTimestamp < this.options.getLeaderLeaseTimeoutMs();
+    }
+
+    private boolean isCurrentLeaderValid() {
+        return Utils.monotonicMs() - this.lastLeaderTimestamp < this.options.getElectionTimeoutMs();
+    }
+
+    private void updateLastLeaderTimestamp(final long lastLeaderTimestamp) {
+        this.lastLeaderTimestamp = lastLeaderTimestamp;
+    }
+
+    private void checkReplicator(final PeerId candidateId) {
+        if (this.state == State.STATE_LEADER) {
+            this.replicatorGroup.checkReplicator(candidateId, false);
+        }
+    }
+
+    @Override
+    public Message handleRequestVoteRequest(final RequestVoteRequest request) {
+        boolean doUnlock = true;
+        this.writeLock.lock();
+        try {
+            if (!this.state.isActive()) {
+                LOG.warn("Node {} is not in active state, currTerm={}.", getNodeId(), this.currTerm);
+                return RpcFactoryHelper //
+                    .responseFactory() //
+                    .newResponse(RequestVoteResponse.getDefaultInstance(), RaftError.EINVAL,
+                        "Node %s is not in active state, state %s.", getNodeId(), this.state.name());
+            }
+            final PeerId candidateId = new PeerId();
+            if (!candidateId.parse(request.getServerId())) {
+                LOG.warn("Node {} received RequestVoteRequest from {} serverId bad format.", getNodeId(),
+                    request.getServerId());
+                return RpcFactoryHelper //
+                    .responseFactory() //
+                    .newResponse(RequestVoteResponse.getDefaultInstance(), RaftError.EINVAL,
+                        "Parse candidateId failed: %s.", request.getServerId());
+            }
+
+            // noinspection ConstantConditions
+            do {
+                // check term
+                if (request.getTerm() >= this.currTerm) {
+                    LOG.info("Node {} received RequestVoteRequest from {}, term={}, currTerm={}.", getNodeId(),
+                        request.getServerId(), request.getTerm(), this.currTerm);
+                    // increase current term, change state to follower
+                    if (request.getTerm() > this.currTerm) {
+                        stepDown(request.getTerm(), false, new Status(RaftError.EHIGHERTERMRESPONSE,
+                            "Raft node receives higher term RequestVoteRequest."));
+                    }
+                } else {
+                    // ignore older term
+                    LOG.info("Node {} ignore RequestVoteRequest from {}, term={}, currTerm={}.", getNodeId(),
+                        request.getServerId(), request.getTerm(), this.currTerm);
+                    break;
+                }
+                doUnlock = false;
+                this.writeLock.unlock();
+
+                final LogId lastLogId = this.logManager.getLastLogId(true);
+
+                doUnlock = true;
+                this.writeLock.lock();
+                // vote need ABA check after unlock&writeLock
+                if (request.getTerm() != this.currTerm) {
+                    LOG.warn("Node {} raise term {} when get lastLogId.", getNodeId(), this.currTerm);
+                    break;
+                }
+
+                final boolean logIsOk = new LogId(request.getLastLogIndex(), request.getLastLogTerm())
+                    .compareTo(lastLogId) >= 0;
+
+                if (logIsOk && (this.votedId == null || this.votedId.isEmpty())) {
+                    stepDown(request.getTerm(), false, new Status(RaftError.EVOTEFORCANDIDATE,
+                        "Raft node votes for some candidate, step down to restart election_timer."));
+                    this.votedId = candidateId.copy();
+                    this.metaStorage.setVotedFor(candidateId);
+                }
+            } while (false);
+
+            return RequestVoteResponse.newBuilder() //
+                .setTerm(this.currTerm) //
+                .setGranted(request.getTerm() == this.currTerm && candidateId.equals(this.votedId)) //
+                .build();
+        } finally {
+            if (doUnlock) {
+                this.writeLock.unlock();
+            }
+        }
+    }
+
+    private static class FollowerStableClosure extends LogManager.StableClosure {
+
+        final long                          committedIndex;
+        final AppendEntriesResponse.Builder responseBuilder;
+        final NodeImpl                      node;
+        final RpcRequestClosure             done;
+        final long                          term;
+
+        public FollowerStableClosure(final AppendEntriesRequest request,
+                                     final AppendEntriesResponse.Builder responseBuilder, final NodeImpl node,
+                                     final RpcRequestClosure done, final long term) {
+            super(null);
+            this.committedIndex = Math.min(
+            // committed index is likely less than the lastLogIndex
+                request.getCommittedIndex(),
+                // The logs after the appended entries can not be trust, so we can't commit them even if their indexes are less than request's committed index.
+                request.getPrevLogIndex() + request.getEntriesCount());
+            this.responseBuilder = responseBuilder;
+            this.node = node;
+            this.done = done;
+            this.term = term;
+        }
+
+        @Override
+        public void run(final Status status) {
+
+            if (!status.isOk()) {
+                this.done.run(status);
+                return;
+            }
+
+            this.node.readLock.lock();
+            try {
+                if (this.term != this.node.currTerm) {
+                    // The change of term indicates that leader has been changed during
+                    // appending entries, so we can't respond ok to the old leader
+                    // because we are not sure if the appended logs would be truncated
+                    // by the new leader:
+                    //  - If they won't be truncated and we respond failure to the old
+                    //    leader, the new leader would know that they are stored in this
+                    //    peer and they will be eventually committed when the new leader
+                    //    found that quorum of the cluster have stored.
+                    //  - If they will be truncated and we responded success to the old
+                    //    leader, the old leader would possibly regard those entries as
+                    //    committed (very likely in a 3-nodes cluster) and respond
+                    //    success to the clients, which would break the rule that
+                    //    committed entries would never be truncated.
+                    // So we have to respond failure to the old leader and set the new
+                    // term to make it stepped down if it didn't.
+                    this.responseBuilder.setSuccess(false).setTerm(this.node.currTerm);
+                    this.done.sendResponse(this.responseBuilder.build());
+                    return;
+                }
+            } finally {
+                // It's safe to release lock as we know everything is ok at this point.
+                this.node.readLock.unlock();
+            }
+
+            // Don't touch node any more.
+            this.responseBuilder.setSuccess(true).setTerm(this.term);
+
+            // Ballot box is thread safe and tolerates disorder.
+            this.node.ballotBox.setLastCommittedIndex(this.committedIndex);
+
+            this.done.sendResponse(this.responseBuilder.build());
+        }
+    }
+
+    @Override
+    public Message handleAppendEntriesRequest(final AppendEntriesRequest request, final RpcRequestClosure done) {
+        boolean doUnlock = true;
+        final long startMs = Utils.monotonicMs();
+        this.writeLock.lock();
+        final int entriesCount = request.getEntriesCount();
+        try {
+            if (!this.state.isActive()) {
+                LOG.warn("Node {} is not in active state, currTerm={}.", getNodeId(), this.currTerm);
+                return RpcFactoryHelper //
+                    .responseFactory() //
+                    .newResponse(AppendEntriesResponse.getDefaultInstance(), RaftError.EINVAL,
+                        "Node %s is not in active state, state %s.", getNodeId(), this.state.name());
+            }
+
+            final PeerId serverId = new PeerId();
+            if (!serverId.parse(request.getServerId())) {
+                LOG.warn("Node {} received AppendEntriesRequest from {} serverId bad format.", getNodeId(),
+                    request.getServerId());
+                return RpcFactoryHelper //
+                    .responseFactory() //
+                    .newResponse(AppendEntriesResponse.getDefaultInstance(), RaftError.EINVAL,
+                        "Parse serverId failed: %s.", request.getServerId());
+            }
+
+            // Check stale term
+            if (request.getTerm() < this.currTerm) {
+                LOG.warn("Node {} ignore stale AppendEntriesRequest from {}, term={}, currTerm={}.", getNodeId(),
+                    request.getServerId(), request.getTerm(), this.currTerm);
+                return AppendEntriesResponse.newBuilder() //
+                    .setSuccess(false) //
+                    .setTerm(this.currTerm) //
+                    .build();
+            }
+
+            // Check term and state to step down
+            checkStepDown(request.getTerm(), serverId);
+            if (!serverId.equals(this.leaderId)) {
+                LOG.error("Another peer {} declares that it is the leader at term {} which was occupied by leader {}.",
+                    serverId, this.currTerm, this.leaderId);
+                // Increase the term by 1 and make both leaders step down to minimize the
+                // loss of split brain
+                stepDown(request.getTerm() + 1, false, new Status(RaftError.ELEADERCONFLICT,
+                    "More than one leader in the same term."));
+                return AppendEntriesResponse.newBuilder() //
+                    .setSuccess(false) //
+                    .setTerm(request.getTerm() + 1) //
+                    .build();
+            }
+
+            updateLastLeaderTimestamp(Utils.monotonicMs());
+
+            if (entriesCount > 0 && this.snapshotExecutor != null && this.snapshotExecutor.isInstallingSnapshot()) {
+                LOG.warn("Node {} received AppendEntriesRequest while installing snapshot.", getNodeId());
+                return RpcFactoryHelper //
+                    .responseFactory() //
+                    .newResponse(AppendEntriesResponse.getDefaultInstance(), RaftError.EBUSY,
+                        "Node %s:%s is installing snapshot.", this.groupId, this.serverId);
+            }
+
+            final long prevLogIndex = request.getPrevLogIndex();
+            final long prevLogTerm = request.getPrevLogTerm();
+            final long localPrevLogTerm = this.logManager.getTerm(prevLogIndex);
+            if (localPrevLogTerm != prevLogTerm) {
+                final long lastLogIndex = this.logManager.getLastLogIndex();
+
+                LOG.warn(
+                    "Node {} reject term_unmatched AppendEntriesRequest from {}, term={}, prevLogIndex={}, prevLogTerm={}, localPrevLogTerm={}, lastLogIndex={}, entriesSize={}.",
+                    getNodeId(), request.getServerId(), request.getTerm(), prevLogIndex, prevLogTerm, localPrevLogTerm,
+                    lastLogIndex, entriesCount);
+
+                return AppendEntriesResponse.newBuilder() //
+                    .setSuccess(false) //
+                    .setTerm(this.currTerm) //
+                    .setLastLogIndex(lastLogIndex) //
+                    .build();
+            }
+
+            if (entriesCount == 0) {
+                // heartbeat or probe request
+                final AppendEntriesResponse.Builder respBuilder = AppendEntriesResponse.newBuilder() //
+                    .setSuccess(true) //
+                    .setTerm(this.currTerm) //
+                    .setLastLogIndex(this.logManager.getLastLogIndex());
+                doUnlock = false;
+                this.writeLock.unlock();
+                // see the comments at FollowerStableClosure#run()
+                this.ballotBox.setLastCommittedIndex(Math.min(request.getCommittedIndex(), prevLogIndex));
+                return respBuilder.build();
+            }
+
+            // Parse request
+            long index = prevLogIndex;
+            final List<LogEntry> entries = new ArrayList<>(entriesCount);
+            ByteBuffer allData = request.getData().asReadOnlyByteBuffer();
+
+            final List<RaftOutter.EntryMeta> entriesList = request.getEntriesList();
+            for (int i = 0; i < entriesCount; i++) {
+                index++;
+                final RaftOutter.EntryMeta entry = entriesList.get(i);
+
+                final LogEntry logEntry = logEntryFromMeta(index, allData, entry);
+
+                if (logEntry != null) {
+                    // Validate checksum
+                    if (this.raftOptions.isEnableLogEntryChecksum() && logEntry.isCorrupted()) {
+                        long realChecksum = logEntry.checksum();
+                        LOG.error(
+                            "Corrupted log entry received from leader, index={}, term={}, expectedChecksum={}, realChecksum={}",
+                            logEntry.getId().getIndex(), logEntry.getId().getTerm(), logEntry.getChecksum(),
+                            realChecksum);
+                        return RpcFactoryHelper //
+                            .responseFactory() //
+                            .newResponse(AppendEntriesResponse.getDefaultInstance(), RaftError.EINVAL,
+                                "The log entry is corrupted, index=%d, term=%d, expectedChecksum=%d, realChecksum=%d",
+                                logEntry.getId().getIndex(), logEntry.getId().getTerm(), logEntry.getChecksum(),
+                                realChecksum);
+                    }
+                    entries.add(logEntry);
+                }
+            }
+
+            final FollowerStableClosure closure = new FollowerStableClosure(request, AppendEntriesResponse.newBuilder()
+                .setTerm(this.currTerm), this, done, this.currTerm);
+            this.logManager.appendEntries(entries, closure);
+            // update configuration after _log_manager updated its memory status
+            checkAndSetConfiguration(true);
+            return null;
+        } finally {
+            if (doUnlock) {
+                this.writeLock.unlock();
+            }
+            this.metrics.recordLatency("handle-append-entries", Utils.monotonicMs() - startMs);
+            this.metrics.recordSize("handle-append-entries-count", entriesCount);
+        }
+    }
+
+    private LogEntry logEntryFromMeta(final long index, final ByteBuffer allData, final RaftOutter.EntryMeta entry) {
+        if (entry.getType() != EnumOutter.EntryType.ENTRY_TYPE_UNKNOWN) {
+            final LogEntry logEntry = new LogEntry();
+            logEntry.setId(new LogId(index, entry.getTerm()));
+            logEntry.setType(entry.getType());
+            logEntry.setChecksum(entry.getChecksum()); // since 1.2.6
+
+            final long dataLen = entry.getDataLen();
+            if (dataLen > 0) {
+                final byte[] bs = new byte[(int) dataLen];
+                assert allData != null;
+                allData.get(bs, 0, bs.length);
+                logEntry.setData(ByteBuffer.wrap(bs));
+            }
+
+            if (entry.getPeersCount() > 0) {
+                if (entry.getType() != EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION) {
+                    throw new IllegalStateException(
+                        "Invalid log entry that contains peers but is not ENTRY_TYPE_CONFIGURATION type: "
+                                + entry.getType());
+                }
+
+                fillLogEntryPeers(entry, logEntry);
+            } else if (entry.getType() == EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION) {
+                throw new IllegalStateException(
+                    "Invalid log entry that contains zero peers but is ENTRY_TYPE_CONFIGURATION type");
+            }
+            return logEntry;
+        }
+        return null;
+    }
+
+    private void fillLogEntryPeers(final RaftOutter.EntryMeta entry, final LogEntry logEntry) {
+        // TODO refactor
+        if (entry.getPeersCount() > 0) {
+            final List<PeerId> peers = new ArrayList<>(entry.getPeersCount());
+            for (final String peerStr : entry.getPeersList()) {
+                final PeerId peer = new PeerId();
+                peer.parse(peerStr);
+                peers.add(peer);
+            }
+            logEntry.setPeers(peers);
+        }
+
+        if (entry.getOldPeersCount() > 0) {
+            final List<PeerId> oldPeers = new ArrayList<>(entry.getOldPeersCount());
+            for (final String peerStr : entry.getOldPeersList()) {
+                final PeerId peer = new PeerId();
+                peer.parse(peerStr);
+                oldPeers.add(peer);
+            }
+            logEntry.setOldPeers(oldPeers);
+        }
+
+        if (entry.getLearnersCount() > 0) {
+            final List<PeerId> peers = new ArrayList<>(entry.getLearnersCount());
+            for (final String peerStr : entry.getLearnersList()) {
+                final PeerId peer = new PeerId();
+                peer.parse(peerStr);
+                peers.add(peer);
+            }
+            logEntry.setLearners(peers);
+        }
+
+        if (entry.getOldLearnersCount() > 0) {
+            final List<PeerId> peers = new ArrayList<>(entry.getOldLearnersCount());
+            for (final String peerStr : entry.getOldLearnersList()) {
+                final PeerId peer = new PeerId();
+                peer.parse(peerStr);
+                peers.add(peer);
+            }
+            logEntry.setOldLearners(peers);
+        }
+    }
+
+    // called when leader receive greater term in AppendEntriesResponse
+    void increaseTermTo(final long newTerm, final Status status) {
+        this.writeLock.lock();
+        try {
+            if (newTerm < this.currTerm) {
+                return;
+            }
+            stepDown(newTerm, false, status);
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    /**
+     * Peer catch up callback
+     * @author boyan (boyan@alibaba-inc.com)
+     *
+     * 2018-Apr-11 2:10:02 PM
+     */
+    private static class OnCaughtUp extends CatchUpClosure {
+        private final NodeImpl node;
+        private final long     term;
+        private final PeerId   peer;
+        private final long     version;
+
+        public OnCaughtUp(final NodeImpl node, final long term, final PeerId peer, final long version) {
+            super();
+            this.node = node;
+            this.term = term;
+            this.peer = peer;
+            this.version = version;
+        }
+
+        @Override
+        public void run(final Status status) {
+            this.node.onCaughtUp(this.peer, this.term, this.version, status);
+        }
+    }
+
+    private void onCaughtUp(final PeerId peer, final long term, final long version, final Status st) {
+        this.writeLock.lock();
+        try {
+            // check current_term and state to avoid ABA problem
+            if (term != this.currTerm && this.state != State.STATE_LEADER) {
+                // term has changed and nothing should be done, otherwise there will be
+                // an ABA problem.
+                return;
+            }
+            if (st.isOk()) {
+                // Caught up successfully
+                this.confCtx.onCaughtUp(version, peer, true);
+                return;
+            }
+            // Retry if this peer is still alive
+            if (st.getCode() == RaftError.ETIMEDOUT.getNumber()
+                && Utils.monotonicMs() - this.replicatorGroup.getLastRpcSendTimestamp(peer) <= this.options
+                    .getElectionTimeoutMs()) {
+                LOG.debug("Node {} waits peer {} to catch up.", getNodeId(), peer);
+                final OnCaughtUp caughtUp = new OnCaughtUp(this, term, peer, version);
+                final long dueTime = Utils.nowMs() + this.options.getElectionTimeoutMs();
+                if (this.replicatorGroup.waitCaughtUp(peer, this.options.getCatchupMargin(), dueTime, caughtUp)) {
+                    return;
+                }
+                LOG.warn("Node {} waitCaughtUp failed, peer={}.", getNodeId(), peer);
+            }
+            LOG.warn("Node {} caughtUp failed, status={}, peer={}.", getNodeId(), st, peer);
+            this.confCtx.onCaughtUp(version, peer, false);
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    private boolean checkDeadNodes(final Configuration conf, final long monotonicNowMs,
+                                   final boolean stepDownOnCheckFail) {
+        // Check learner replicators at first.
+        for (final PeerId peer : conf.getLearners()) {
+            checkReplicator(peer);
+        }
+        // Ensure quorum nodes alive.
+        final List<PeerId> peers = conf.listPeers();
+        final Configuration deadNodes = new Configuration();
+        if (checkDeadNodes0(peers, monotonicNowMs, true, deadNodes)) {
+            return true;
+        }
+        if (stepDownOnCheckFail) {
+            LOG.warn("Node {} steps down when alive nodes don't satisfy quorum, term={}, deadNodes={}, conf={}.",
+                getNodeId(), this.currTerm, deadNodes, conf);
+            final Status status = new Status();
+            status.setError(RaftError.ERAFTTIMEDOUT, "Majority of the group dies: %d/%d", deadNodes.size(),
+                peers.size());
+            stepDown(this.currTerm, false, status);
+        }
+        return false;
+    }
+
+    private boolean checkDeadNodes0(final List<PeerId> peers, final long monotonicNowMs, final boolean checkReplicator,
+                                    final Configuration deadNodes) {
+        final int leaderLeaseTimeoutMs = this.options.getLeaderLeaseTimeoutMs();
+        int aliveCount = 0;
+        long startLease = Long.MAX_VALUE;
+        for (final PeerId peer : peers) {
+            if (peer.equals(this.serverId)) {
+                aliveCount++;
+                continue;
+            }
+            if (checkReplicator) {
+                checkReplicator(peer);
+            }
+            final long lastRpcSendTimestamp = this.replicatorGroup.getLastRpcSendTimestamp(peer);
+            if (monotonicNowMs - lastRpcSendTimestamp <= leaderLeaseTimeoutMs) {
+                aliveCount++;
+                if (startLease > lastRpcSendTimestamp) {
+                    startLease = lastRpcSendTimestamp;
+                }
+                continue;
+            }
+            if (deadNodes != null) {
+                deadNodes.addPeer(peer);
+            }
+        }
+        if (aliveCount >= peers.size() / 2 + 1) {
+            updateLastLeaderTimestamp(startLease);
+            return true;
+        }
+        return false;
+    }
+
+    // in read_lock
+    private List<PeerId> getAliveNodes(final Collection<PeerId> peers, final long monotonicNowMs) {
+        final int leaderLeaseTimeoutMs = this.options.getLeaderLeaseTimeoutMs();
+        final List<PeerId> alivePeers = new ArrayList<>();
+        for (final PeerId peer : peers) {
+            if (peer.equals(this.serverId)) {
+                alivePeers.add(peer.copy());
+                continue;
+            }
+            if (monotonicNowMs - this.replicatorGroup.getLastRpcSendTimestamp(peer) <= leaderLeaseTimeoutMs) {
+                alivePeers.add(peer.copy());
+            }
+        }
+        return alivePeers;
+    }
+
+    @SuppressWarnings({ "LoopStatementThatDoesntLoop", "ConstantConditions" })
+    private void handleStepDownTimeout() {
+        do {
+            this.readLock.lock();
+            try {
+                if (this.state.compareTo(State.STATE_TRANSFERRING) > 0) {
+                    LOG.debug("Node {} stop step-down timer, term={}, state={}.", getNodeId(), this.currTerm,
+                        this.state);
+                    return;
+                }
+                final long monotonicNowMs = Utils.monotonicMs();
+                if (!checkDeadNodes(this.conf.getConf(), monotonicNowMs, false)) {
+                    break;
+                }
+                if (!this.conf.getOldConf().isEmpty()) {
+                    if (!checkDeadNodes(this.conf.getOldConf(), monotonicNowMs, false)) {
+                        break;
+                    }
+                }
+                return;
+            } finally {
+                this.readLock.unlock();
+            }
+        } while (false);
+
+        this.writeLock.lock();
+        try {
+            if (this.state.compareTo(State.STATE_TRANSFERRING) > 0) {
+                LOG.debug("Node {} stop step-down timer, term={}, state={}.", getNodeId(), this.currTerm, this.state);
+                return;
+            }
+            final long monotonicNowMs = Utils.monotonicMs();
+            checkDeadNodes(this.conf.getConf(), monotonicNowMs, true);
+            if (!this.conf.getOldConf().isEmpty()) {
+                checkDeadNodes(this.conf.getOldConf(), monotonicNowMs, true);
+            }
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    /**
+     * Configuration changed callback.
+     *
+     * @author boyan (boyan@alibaba-inc.com)
+     *
+     * 2018-Apr-11 2:53:43 PM
+     */
+    private class ConfigurationChangeDone implements Closure {
+        private final long    term;
+        private final boolean leaderStart;
+
+        public ConfigurationChangeDone(final long term, final boolean leaderStart) {
+            super();
+            this.term = term;
+            this.leaderStart = leaderStart;
+        }
+
+        @Override
+        public void run(final Status status) {
+            if (status.isOk()) {
+                onConfigurationChangeDone(this.term);
+                if (this.leaderStart) {
+                    getOptions().getFsm().onLeaderStart(this.term);
+                }
+            } else {
+                LOG.error("Fail to run ConfigurationChangeDone, status: {}.", status);
+            }
+        }
+    }
+
+    private void unsafeApplyConfiguration(final Configuration newConf, final Configuration oldConf,
+                                          final boolean leaderStart) {
+        Requires.requireTrue(this.confCtx.isBusy(), "ConfigurationContext is not busy");
+        final LogEntry entry = new LogEntry(EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION);
+        entry.setId(new LogId(0, this.currTerm));
+        entry.setPeers(newConf.listPeers());
+        entry.setLearners(newConf.listLearners());
+        if (oldConf != null) {
+            entry.setOldPeers(oldConf.listPeers());
+            entry.setOldLearners(oldConf.listLearners());
+        }
+        final ConfigurationChangeDone configurationChangeDone = new ConfigurationChangeDone(this.currTerm, leaderStart);
+        // Use the new_conf to deal the quorum of this very log
+        if (!this.ballotBox.appendPendingTask(newConf, oldConf, configurationChangeDone)) {
+            Utils.runClosureInThread(configurationChangeDone, new Status(RaftError.EINTERNAL, "Fail to append task."));
+            return;
+        }
+        final List<LogEntry> entries = new ArrayList<>();
+        entries.add(entry);
+        this.logManager.appendEntries(entries, new LeaderStableClosure(entries));
+        checkAndSetConfiguration(false);
+    }
+
+    private void unsafeRegisterConfChange(final Configuration oldConf, final Configuration newConf, final Closure done) {
+
+        Requires.requireTrue(newConf.isValid(), "Invalid new conf: %s", newConf);
+        // The new conf entry(will be stored in log manager) should be valid
+        Requires.requireTrue(new ConfigurationEntry(null, newConf, oldConf).isValid(), "Invalid conf entry: %s",
+            newConf);
+
+        if (this.state != State.STATE_LEADER) {
+            LOG.warn("Node {} refused configuration changing as the state={}.", getNodeId(), this.state);
+            if (done != null) {
+                final Status status = new Status();
+                if (this.state == State.STATE_TRANSFERRING) {
+                    status.setError(RaftError.EBUSY, "Is transferring leadership.");
+                } else {
+                    status.setError(RaftError.EPERM, "Not leader");
+                }
+                Utils.runClosureInThread(done, status);
+            }
+            return;
+        }
+        // check concurrent conf change
+        if (this.confCtx.isBusy()) {
+            LOG.warn("Node {} refused configuration concurrent changing.", getNodeId());
+            if (done != null) {
+                Utils.runClosureInThread(done, new Status(RaftError.EBUSY, "Doing another configuration change."));
+            }
+            return;
+        }
+        // Return immediately when the new peers equals to current configuration
+        if (this.conf.getConf().equals(newConf)) {
+            Utils.runClosureInThread(done);
+            return;
+        }
+        this.confCtx.start(oldConf, newConf, done);
+    }
+
+    private void afterShutdown() {
+        List<Closure> savedDoneList = null;
+        this.writeLock.lock();
+        try {
+            if (!this.shutdownContinuations.isEmpty()) {
+                savedDoneList = new ArrayList<>(this.shutdownContinuations);
+            }
+            if (this.logStorage != null) {
+                this.logStorage.shutdown();
+            }
+            this.state = State.STATE_SHUTDOWN;
+        } finally {
+            this.writeLock.unlock();
+        }
+        if (savedDoneList != null) {
+            for (final Closure closure : savedDoneList) {
+                Utils.runClosureInThread(closure);
+            }
+        }
+    }
+
+    @Override
+    public NodeOptions getOptions() {
+        return this.options;
+    }
+
+    public Scheduler getTimerManager() {
+        return this.timerManager;
+    }
+
+    @Override
+    public RaftOptions getRaftOptions() {
+        return this.raftOptions;
+    }
+
+    @OnlyForTest
+    long getCurrentTerm() {
+        this.readLock.lock();
+        try {
+            return this.currTerm;
+        } finally {
+            this.readLock.unlock();
+        }
+    }
+
+    @OnlyForTest
+    ConfigurationEntry getConf() {
+        this.readLock.lock();
+        try {
+            return this.conf;
+        } finally {
+            this.readLock.unlock();
+        }
+    }
+
+    @Override
+    public void shutdown() {
+        shutdown(null);
+    }
+
+    public void onConfigurationChangeDone(final long term) {
+        this.writeLock.lock();
+        try {
+            if (term != this.currTerm || this.state.compareTo(State.STATE_TRANSFERRING) > 0) {
+                LOG.warn("Node {} process onConfigurationChangeDone at term {} while state={}, currTerm={}.",
+                    getNodeId(), term, this.state, this.currTerm);
+                return;
+            }
+            this.confCtx.nextStage();
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    @Override
+    public PeerId getLeaderId() {
+        this.readLock.lock();
+        try {
+            return this.leaderId.isEmpty() ? null : this.leaderId;
+        } finally {
+            this.readLock.unlock();
+        }
+    }
+
+    @Override
+    public String getGroupId() {
+        return this.groupId;
+    }
+
+    public PeerId getServerId() {
+        return this.serverId;
+    }
+
+    @Override
+    public NodeId getNodeId() {
+        if (this.nodeId == null) {
+            this.nodeId = new NodeId(this.groupId, this.serverId);
+        }
+        return this.nodeId;
+    }
+
+    public RaftClientService getRpcService() {
+        return this.rpcService;
+    }
+
+    public void onError(final RaftException error) {
+        LOG.warn("Node {} got error: {}.", getNodeId(), error);
+        if (this.fsmCaller != null) {
+            // onError of fsmCaller is guaranteed to be executed once.
+            this.fsmCaller.onError(error);
+        }
+        if (this.readOnlyService != null) {
+            this.readOnlyService.setError(error);
+        }
+        this.writeLock.lock();
+        try {
+            // If it is leader, need to wake up a new one;
+            // If it is follower, also step down to call on_stop_following.
+            if (this.state.compareTo(State.STATE_FOLLOWER) <= 0) {
+                stepDown(this.currTerm, this.state == State.STATE_LEADER, new Status(RaftError.EBADNODE,
+                    "Raft node(leader or candidate) is in error."));
+            }
+            if (this.state.compareTo(State.STATE_ERROR) < 0) {
+                this.state = State.STATE_ERROR;
+            }
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    public void handleRequestVoteResponse(final PeerId peerId, final long term, final RequestVoteResponse response) {
+        this.writeLock.lock();
+        try {
+            if (this.state != State.STATE_CANDIDATE) {
+                LOG.warn("Node {} received invalid RequestVoteResponse from {}, state not in STATE_CANDIDATE but {}.",
+                    getNodeId(), peerId, this.state);
+                return;
+            }
+            // check stale term
+            if (term != this.currTerm) {
+                LOG.warn("Node {} received stale RequestVoteResponse from {}, term={}, currTerm={}.", getNodeId(),
+                    peerId, term, this.currTerm);
+                return;
+            }
+            // check response term
+            if (response.getTerm() > this.currTerm) {
+                LOG.warn("Node {} received invalid RequestVoteResponse from {}, term={}, expect={}.", getNodeId(),
+                    peerId, response.getTerm(), this.currTerm);
+                stepDown(response.getTerm(), false, new Status(RaftError.EHIGHERTERMRESPONSE,
+                    "Raft node receives higher term request_vote_response."));
+                return;
+            }
+            // check granted quorum?
+            if (response.getGranted()) {
+                this.voteCtx.grant(peerId);
+                if (this.voteCtx.isGranted()) {
+                    becomeLeader();
+                }
+            }
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    private class OnRequestVoteRpcDone extends RpcResponseClosureAdapter<RequestVoteResponse> {
+
+        final long         startMs;
+        final PeerId       peer;
+        final long         term;
+        final NodeImpl     node;
+        RequestVoteRequest request;
+
+        public OnRequestVoteRpcDone(final PeerId peer, final long term, final NodeImpl node) {
+            super();
+            this.startMs = Utils.monotonicMs();
+            this.peer = peer;
+            this.term = term;
+            this.node = node;
+        }
+
+        @Override
+        public void run(final Status status) {
+            NodeImpl.this.metrics.recordLatency("request-vote", Utils.monotonicMs() - this.startMs);
+            if (!status.isOk()) {
+                LOG.warn("Node {} RequestVote to {} error: {}.", this.node.getNodeId(), this.peer, status);
+            } else {
+                this.node.handleRequestVoteResponse(this.peer, this.term, getResponse());
+            }
+        }
+    }
+
+    public void handlePreVoteResponse(final PeerId peerId, final long term, final RequestVoteResponse response) {
+        boolean doUnlock = true;
+        this.writeLock.lock();
+        try {
+            if (this.state != State.STATE_FOLLOWER) {
+                LOG.warn("Node {} received invalid PreVoteResponse from {}, state not in STATE_FOLLOWER but {}.",
+                    getNodeId(), peerId, this.state);
+                return;
+            }
+            if (term != this.currTerm) {
+                LOG.warn("Node {} received invalid PreVoteResponse from {}, term={}, currTerm={}.", getNodeId(),
+                    peerId, term, this.currTerm);
+                return;
+            }
+            if (response.getTerm() > this.currTerm) {
+                LOG.warn("Node {} received invalid PreVoteResponse from {}, term {}, expect={}.", getNodeId(), peerId,
+                    response.getTerm(), this.currTerm);
+                stepDown(response.getTerm(), false, new Status(RaftError.EHIGHERTERMRESPONSE,
+                    "Raft node receives higher term pre_vote_response."));
+                return;
+            }
+            LOG.info("Node {} received PreVoteResponse from {}, term={}, granted={}.", getNodeId(), peerId,
+                response.getTerm(), response.getGranted());
+            // check granted quorum?
+            if (response.getGranted()) {
+                this.prevVoteCtx.grant(peerId);
+                if (this.prevVoteCtx.isGranted()) {
+                    doUnlock = false;
+                    electSelf();
+                }
+            }
+        } finally {
+            if (doUnlock) {
+                this.writeLock.unlock();
+            }
+        }
+    }
+
+    private class OnPreVoteRpcDone extends RpcResponseClosureAdapter<RequestVoteResponse> {
+
+        final long         startMs;
+        final PeerId       peer;
+        final long         term;
+        RequestVoteRequest request;
+
+        public OnPreVoteRpcDone(final PeerId peer, final long term) {
+            super();
+            this.startMs = Utils.monotonicMs();
+            this.peer = peer;
+            this.term = term;
+        }
+
+        @Override
+        public void run(final Status status) {
+            NodeImpl.this.metrics.recordLatency("pre-vote", Utils.monotonicMs() - this.startMs);
+            if (!status.isOk()) {
+                LOG.warn("Node {} PreVote to {} error: {}.", getNodeId(), this.peer, status);
+            } else {
+                handlePreVoteResponse(this.peer, this.term, getResponse());
+            }
+        }
+    }
+
+    // in writeLock
+    private void preVote() {
+        long oldTerm;
+        try {
+            LOG.info("Node {} term {} start preVote.", getNodeId(), this.currTerm);
+            if (this.snapshotExecutor != null && this.snapshotExecutor.isInstallingSnapshot()) {
+                LOG.warn(
+                    "Node {} term {} doesn't do preVote when installing snapshot as the configuration may be out of date.",
+                    getNodeId(), this.currTerm);
+                return;
+            }
+            if (!this.conf.contains(this.serverId)) {
+                LOG.warn("Node {} can't do preVote as it is not in conf <{}>.", getNodeId(), this.conf);
+                return;
+            }
+            oldTerm = this.currTerm;
+        } finally {
+            this.writeLock.unlock();
+        }
+
+        final LogId lastLogId = this.logManager.getLastLogId(true);
+
+        boolean doUnlock = true;
+        this.writeLock.lock();
+        try {
+            // pre_vote need defense ABA after unlock&writeLock
+            if (oldTerm != this.currTerm) {
+                LOG.warn("Node {} raise term {} when get lastLogId.", getNodeId(), this.currTerm);
+                return;
+            }
+            this.prevVoteCtx.init(this.conf.getConf(), this.conf.isStable() ? null : this.conf.getOldConf());
+            for (final PeerId peer : this.conf.listPeers()) {
+                if (peer.equals(this.serverId)) {
+                    continue;
+                }
+                if (!this.rpcService.connect(peer.getEndpoint())) {
+                    LOG.warn("Node {} channel init failed, address={}.", getNodeId(), peer.getEndpoint());
+                    continue;
+                }
+                final OnPreVoteRpcDone done = new OnPreVoteRpcDone(peer, this.currTerm);
+                done.request = RequestVoteRequest.newBuilder() //
+                    .setPreVote(true) // it's a pre-vote request.
+                    .setGroupId(this.groupId) //
+                    .setServerId(this.serverId.toString()) //
+                    .setPeerId(peer.toString()) //
+                    .setTerm(this.currTerm + 1) // next term
+                    .setLastLogIndex(lastLogId.getIndex()) //
+                    .setLastLogTerm(lastLogId.getTerm()) //
+                    .build();
+                this.rpcService.preVote(peer.getEndpoint(), done.request, done);
+            }
+            this.prevVoteCtx.grant(this.serverId);
+            if (this.prevVoteCtx.isGranted()) {
+                doUnlock = false;
+                electSelf();
+            }
+        } finally {
+            if (doUnlock) {
+                this.writeLock.unlock();
+            }
+        }
+    }
+
+    private void handleVoteTimeout() {
+        this.writeLock.lock();
+        if (this.state != State.STATE_CANDIDATE) {
+            this.writeLock.unlock();
+            return;
+        }
+
+        if (this.raftOptions.isStepDownWhenVoteTimedout()) {
+            LOG.warn(
+                "Candidate node {} term {} steps down when election reaching vote timeout: fail to get quorum vote-granted.",
+                this.nodeId, this.currTerm);
+            stepDown(this.currTerm, false, new Status(RaftError.ETIMEDOUT,
+                "Vote timeout: fail to get quorum vote-granted."));
+            // unlock in preVote
+            preVote();
+        } else {
+            LOG.debug("Node {} term {} retry to vote self.", getNodeId(), this.currTerm);
+            // unlock in electSelf
+            electSelf();
+        }
+    }
+
+    @Override
+    public boolean isLeader() {
+        return isLeader(true);
+    }
+
+    @Override
+    public boolean isLeader(final boolean blocking) {
+        if (!blocking) {
+            return this.state == State.STATE_LEADER;
+        }
+        this.readLock.lock();
+        try {
+            return this.state == State.STATE_LEADER;
+        } finally {
+            this.readLock.unlock();
+        }
+    }
+
+    @Override
+    public void shutdown(final Closure done) {
+        List<RepeatedTimer> timers = null;
+        this.writeLock.lock();
+        try {
+            LOG.info("Node {} shutdown, currTerm={} state={}.", getNodeId(), this.currTerm, this.state);
+            if (this.state.compareTo(State.STATE_SHUTTING) < 0) {
+                NodeManager.getInstance().remove(this);
+                // If it is leader, set the wakeup_a_candidate with true;
+                // If it is follower, call on_stop_following in step_down
+                if (this.state.compareTo(State.STATE_FOLLOWER) <= 0) {
+                    stepDown(this.currTerm, this.state == State.STATE_LEADER,
+                            new Status(RaftError.ESHUTDOWN, "Raft node is going to quit."));
+                }
+                this.state = State.STATE_SHUTTING;
+                // Stop all timers
+                timers = stopAllTimers();
+                if (this.readOnlyService != null) {
+                    this.readOnlyService.shutdown();
+                }
+                if (this.logManager != null) {
+                    this.logManager.shutdown();
+                }
+                if (this.metaStorage != null) {
+                    this.metaStorage.shutdown();
+                }
+                if (this.snapshotExecutor != null) {
+                    this.snapshotExecutor.shutdown();
+                }
+                if (this.wakingCandidate != null) {
+                    Replicator.stop(this.wakingCandidate);
+                }
+                if (this.fsmCaller != null) {
+                    this.fsmCaller.shutdown();
+                }
+                if (this.rpcService != null) {
+                    this.rpcService.shutdown();
+                }
+                if (this.applyQueue != null) {
+                    final CountDownLatch latch = new CountDownLatch(1);
+                    this.shutdownLatch = latch;
+                    Utils.runInThread(
+                        () -> this.applyQueue.publishEvent((event, sequence) -> event.shutdownLatch = latch));
+                } else {
+                    final int num = GLOBAL_NUM_NODES.decrementAndGet();
+                    LOG.info("The number of active nodes decrement to {}.", num);
+                }
+                if (this.timerManager != null) {
+                    this.timerManager.shutdown();
+                }
+            }
+
+            if (this.state != State.STATE_SHUTDOWN) {
+                if (done != null) {
+                    this.shutdownContinuations.add(done);
+                }
+                return;
+            }
+
+            // This node is down, it's ok to invoke done right now. Don't invoke this
+            // in place to avoid the dead writeLock issue when done.Run() is going to acquire
+            // a writeLock which is already held by the caller
+            if (done != null) {
+                Utils.runClosureInThread(done);
+            }
+        } finally {
+            this.writeLock.unlock();
+
+            // Destroy all timers out of lock
+            if (timers != null) {
+                destroyAllTimers(timers);
+            }
+        }
+    }
+
+    // Should in lock
+    private List<RepeatedTimer> stopAllTimers() {
+        final List<RepeatedTimer> timers = new ArrayList<>();
+        if (this.electionTimer != null) {
+            this.electionTimer.stop();
+            timers.add(this.electionTimer);
+        }
+        if (this.voteTimer != null) {
+            this.voteTimer.stop();
+            timers.add(this.voteTimer);
+        }
+        if (this.stepDownTimer != null) {
+            this.stepDownTimer.stop();
+            timers.add(this.stepDownTimer);
+        }
+        if (this.snapshotTimer != null) {
+            this.snapshotTimer.stop();
+            timers.add(this.snapshotTimer);
+        }
+        return timers;
+    }
+
+    private void destroyAllTimers(final List<RepeatedTimer> timers) {
+        for (final RepeatedTimer timer : timers) {
+            timer.destroy();
+        }
+    }
+
+    @Override
+    public synchronized void join() throws InterruptedException {
+        if (this.shutdownLatch != null) {
+            if (this.readOnlyService != null) {
+                this.readOnlyService.join();
+            }
+            if (this.logManager != null) {
+                this.logManager.join();
+            }
+            if (this.snapshotExecutor != null) {
+                this.snapshotExecutor.join();
+            }
+            if (this.wakingCandidate != null) {
+                Replicator.join(this.wakingCandidate);
+            }
+            this.shutdownLatch.await();
+            this.applyDisruptor.shutdown();
+            this.shutdownLatch = null;
+        }
+        if (this.fsmCaller != null) {
+            this.fsmCaller.join();
+        }
+    }
+
+    private static class StopTransferArg {
+        final NodeImpl node;
+        final long     term;
+        final PeerId   peer;
+
+        public StopTransferArg(final NodeImpl node, final long term, final PeerId peer) {
+            super();
+            this.node = node;
+            this.term = term;
+            this.peer = peer;
+        }
+    }
+
+    private void handleTransferTimeout(final long term, final PeerId peer) {
+        LOG.info("Node {} failed to transfer leadership to peer {}, reached timeout.", getNodeId(), peer);
+        this.writeLock.lock();
+        try {
+            if (term == this.currTerm) {
+                this.replicatorGroup.stopTransferLeadership(peer);
+                if (this.state == State.STATE_TRANSFERRING) {
+                    this.fsmCaller.onLeaderStart(term);
+                    this.state = State.STATE_LEADER;
+                    this.stopTransferArg = null;
+                }
+            }
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    private void onTransferTimeout(final StopTransferArg arg) {
+        arg.node.handleTransferTimeout(arg.term, arg.peer);
+    }
+
+    /**
+     * Retrieve current configuration this node seen so far. It's not a reliable way to
+     * retrieve cluster peers info, you should use {@link #listPeers()} instead.
+     *
+     * @return current configuration.
+     *
+     * @since 1.0.3
+     */
+    public Configuration getCurrentConf() {
+        this.readLock.lock();
+        try {
+            if (this.conf != null && this.conf.getConf() != null) {
+                return this.conf.getConf().copy();
+            }
+            return null;
+        } finally {
+            this.readLock.unlock();
+        }
+    }
+
+    @Override
+    public List<PeerId> listPeers() {
+        this.readLock.lock();
+        try {
+            if (this.state != State.STATE_LEADER) {
+                throw new IllegalStateException("Not leader");
+            }
+            return this.conf.getConf().listPeers();
+        } finally {
+            this.readLock.unlock();
+        }
+    }
+
+    @Override
+    public List<PeerId> listAlivePeers() {
+        this.readLock.lock();
+        try {
+            if (this.state != State.STATE_LEADER) {
+                throw new IllegalStateException("Not leader");
+            }
+            return getAliveNodes(this.conf.getConf().getPeers(), Utils.monotonicMs());
+        } finally {
+            this.readLock.unlock();
+        }
+    }
+
+    @Override
+    public List<PeerId> listLearners() {
+        this.readLock.lock();
+        try {
+            if (this.state != State.STATE_LEADER) {
+                throw new IllegalStateException("Not leader");
+            }
+            return this.conf.getConf().listLearners();
+        } finally {
+            this.readLock.unlock();
+        }
+    }
+
+    @Override
+    public List<PeerId> listAliveLearners() {
+        this.readLock.lock();
+        try {
+            if (this.state != State.STATE_LEADER) {
+                throw new IllegalStateException("Not leader");
+            }
+            return getAliveNodes(this.conf.getConf().getLearners(), Utils.monotonicMs());
+        } finally {
+            this.readLock.unlock();
+        }
+    }
+
+    @Override
+    public void addPeer(final PeerId peer, final Closure done) {
+        Requires.requireNonNull(peer, "Null peer");
+        this.writeLock.lock();
+        try {
+            Requires.requireTrue(!this.conf.getConf().contains(peer), "Peer already exists in current configuration");
+
+            final Configuration newConf = new Configuration(this.conf.getConf());
+            newConf.addPeer(peer);
+            unsafeRegisterConfChange(this.conf.getConf(), newConf, done);
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    @Override
+    public void removePeer(final PeerId peer, final Closure done) {
+        Requires.requireNonNull(peer, "Null peer");
+        this.writeLock.lock();
+        try {
+            Requires.requireTrue(this.conf.getConf().contains(peer), "Peer not found in current configuration");
+
+            final Configuration newConf = new Configuration(this.conf.getConf());
+            newConf.removePeer(peer);
+            unsafeRegisterConfChange(this.conf.getConf(), newConf, done);
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    @Override
+    public void changePeers(final Configuration newPeers, final Closure done) {
+        Requires.requireNonNull(newPeers, "Null new peers");
+        Requires.requireTrue(!newPeers.isEmpty(), "Empty new peers");
+        this.writeLock.lock();
+        try {
+            LOG.info("Node {} change peers from {} to {}.", getNodeId(), this.conf.getConf(), newPeers);
+            unsafeRegisterConfChange(this.conf.getConf(), newPeers, done);
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    @Override
+    public Status resetPeers(final Configuration newPeers) {
+        Requires.requireNonNull(newPeers, "Null new peers");
+        Requires.requireTrue(!newPeers.isEmpty(), "Empty new peers");
+        Requires.requireTrue(newPeers.isValid(), "Invalid new peers: %s", newPeers);
+        this.writeLock.lock();
+        try {
+            if (newPeers.isEmpty()) {
+                LOG.warn("Node {} set empty peers.", getNodeId());
+                return new Status(RaftError.EINVAL, "newPeers is empty");
+            }
+            if (!this.state.isActive()) {
+                LOG.warn("Node {} is in state {}, can't set peers.", getNodeId(), this.state);
+                return new Status(RaftError.EPERM, "Bad state: %s", this.state);
+            }
+            // bootstrap?
+            if (this.conf.getConf().isEmpty()) {
+                LOG.info("Node {} set peers to {} from empty.", getNodeId(), newPeers);
+                this.conf.setConf(newPeers);
+                stepDown(this.currTerm + 1, false, new Status(RaftError.ESETPEER, "Set peer from empty configuration"));
+                return Status.OK();
+            }
+            if (this.state == State.STATE_LEADER && this.confCtx.isBusy()) {
+                LOG.warn("Node {} set peers need wait current conf changing.", getNodeId());
+                return new Status(RaftError.EBUSY, "Changing to another configuration");
+            }
+            // check equal, maybe retry direct return
+            if (this.conf.getConf().equals(newPeers)) {
+                return Status.OK();
+            }
+            final Configuration newConf = new Configuration(newPeers);
+            LOG.info("Node {} set peers from {} to {}.", getNodeId(), this.conf.getConf(), newPeers);
+            this.conf.setConf(newConf);
+            this.conf.getOldConf().reset();
+            stepDown(this.currTerm + 1, false, new Status(RaftError.ESETPEER, "Raft node set peer normally"));
+            return Status.OK();
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    @Override
+    public void addLearners(final List<PeerId> learners, final Closure done) {
+        checkPeers(learners);
+        this.writeLock.lock();
+        try {
+            final Configuration newConf = new Configuration(this.conf.getConf());
+            for (final PeerId peer : learners) {
+                newConf.addLearner(peer);
+            }
+            unsafeRegisterConfChange(this.conf.getConf(), newConf, done);
+        } finally {
+            this.writeLock.unlock();
+        }
+
+    }
+
+    private void checkPeers(final List<PeerId> peers) {
+        Requires.requireNonNull(peers, "Null peers");
+        Requires.requireTrue(!peers.isEmpty(), "Empty peers");
+        for (final PeerId peer : peers) {
+            Requires.requireNonNull(peer, "Null peer");
+        }
+    }
+
+    @Override
+    public void removeLearners(final List<PeerId> learners, final Closure done) {
+        checkPeers(learners);
+        this.writeLock.lock();
+        try {
+            final Configuration newConf = new Configuration(this.conf.getConf());
+            for (final PeerId peer : learners) {
+                newConf.removeLearner(peer);
+            }
+            unsafeRegisterConfChange(this.conf.getConf(), newConf, done);
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    @Override
+    public void resetLearners(final List<PeerId> learners, final Closure done) {
+        checkPeers(learners);
+        this.writeLock.lock();
+        try {
+            final Configuration newConf = new Configuration(this.conf.getConf());
+            newConf.setLearners(new LinkedHashSet<>(learners));
+            unsafeRegisterConfChange(this.conf.getConf(), newConf, done);
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    @Override
+    public void snapshot(final Closure done) {
+        doSnapshot(done);
+    }
+
+    private void doSnapshot(final Closure done) {
+        if (this.snapshotExecutor != null) {
+            this.snapshotExecutor.doSnapshot(done);
+        } else {
+            if (done != null) {
+                final Status status = new Status(RaftError.EINVAL, "Snapshot is not supported");
+                Utils.runClosureInThread(done, status);
+            }
+        }
+    }
+
+    @Override
+    public void resetElectionTimeoutMs(final int electionTimeoutMs) {
+        Requires.requireTrue(electionTimeoutMs > 0, "Invalid electionTimeoutMs");
+        this.writeLock.lock();
+        try {
+            this.options.setElectionTimeoutMs(electionTimeoutMs);
+            this.replicatorGroup.resetHeartbeatInterval(heartbeatTimeout(this.options.getElectionTimeoutMs()));
+            this.replicatorGroup.resetElectionTimeoutInterval(electionTimeoutMs);
+            LOG.info("Node {} reset electionTimeout, currTimer {} state {} new electionTimeout {}.", getNodeId(),
+                this.currTerm, this.state, electionTimeoutMs);
+            this.electionTimer.reset(electionTimeoutMs);
+        } finally {
+            this.writeLock.unlock();
+        }
+    }
+
+    @Override
+    public Status transferLeadershipTo(final PeerId peer) {
+        Requires.requireNonNull(peer, "Null peer");
+        this.writeLock.lock();
+        try {
+            if (this.state != State.STATE_LEADER) {
+                LOG.warn("Node {} can't transfer leadership to peer {} as it is in state {}.", getNodeId(), peer,
+                    this.state);
+                return new Status(this.state == State.STATE_TRANSFERRING ? RaftError.EBUSY : RaftError.EPERM,
+                        "Not a leader");
+            }
+            if (this.confCtx.isBusy()) {
+                // It's very messy to deal with the case when the |peer| received
+                // TimeoutNowRequest and increase the term while somehow another leader
+                // which was not replicated with the newest configuration has been
+                // elected. If no add_peer with this very |peer| is to be invoked ever
+                // after nor this peer is to be killed, this peer will spin in the voting
+                // procedure and make the each new leader stepped down when the peer
+                // reached vote timeout and it starts to vote (because it will increase
+                // the term of the group)
+                // To make things simple, refuse the operation and force users to
+                // invoke transfer_leadership_to after configuration changing is
+                // completed so that the peer's configuration is up-to-date when it
+                // receives the TimeOutNowRequest.
+                LOG.warn(
+                    "Node {} refused to transfer leadership to peer {} when the leader is changing the configuration.",
+                    getNodeId(), peer);
+                return new Status(RaftError.EBUSY, "Changing the configuration");
+            }
+
+            PeerId peerId = peer.copy();
+            // if peer_id is ANY_PEER(0.0.0.0:0:0), the peer with the largest
+            // last_log_id will be selected.
+            if (peerId.equals(PeerId.ANY_PEER)) {
+                LOG.info("Node {} starts to transfer leadership to any peer.", getNodeId());
+                if ((peerId = this.replicatorGroup.findTheNextCandidate(this.conf)) == null) {
+                    return new Status(-1, "Candidate not found for any peer");
+                }
+            }
+            if (peerId.equals(this.serverId)) {
+                LOG.info("Node {} transferred leadership to self.", this.serverId);
+                return Status.OK();
+            }
+            if (!this.conf.contains(peerId)) {
+                LOG.info("Node {} refused to transfer leadership to peer {} as it is not in {}.", getNodeId(), peer,
+                    this.conf);
+                return new Status(RaftError.EINVAL, "Not in current configuration");
+            }
+
+            final long lastLogIndex = this.logManager.getLastLogIndex();
+            if (!this.replicatorGroup.transferLeadershipTo(peerId, lastLogIndex)) {
+                LOG.warn("No such peer {}.", peer);
+                return new Status(RaftError.EINVAL, "No such peer %s", peer);
+            }
+            this.state = State.STATE_TRANSFERRING;
+            final Status status = new Status(RaftError.ETRANSFERLEADERSHIP,
+                "Raft leader is transferring leadership to %s", peerId);
+            onLeaderStop(status);
+            LOG.info("Node {} starts to transfer leadership to peer {}.", getNodeId(), peer);
+            final StopTransferArg stopArg = new StopTransferArg(this, this.currTerm, peerId);
+            this.stopTransferArg = stopArg;
+            this.transferTimer = this.timerManager.schedule(() -> onTransferTimeout(stopArg),
+                this.options.getElectionTimeoutMs(), TimeUnit.MILLISECONDS);
+
+        } finally {
+            this.writeLock.unlock();
+        }
+        return Status.OK();
+    }
+
+    private void onLeaderStop(final Status status) {
+        this.replicatorGroup.clearFailureReplicators();
+        this.fsmCaller.onLeaderStop(status);
+    }
+
+    @Override
+    public Message handleTimeoutNowRequest(final TimeoutNowRequest request, final RpcRequestClosure done) {
+        boolean doUnlock = true;
+        this.writeLock.lock();
+        try {
+            if (request.getTerm() != this.currTerm) {
+                final long savedCurrTerm = this.currTerm;
+                if (request.getTerm() > this.currTerm) {
+                    stepDown(request.getTerm(), false, new Status(RaftError.EHIGHERTERMREQUEST,
+                        "Raft node receives higher term request"));
+                }
+                LOG.info("Node {} received TimeoutNowRequest from {} while currTerm={} didn't match requestTerm={}.",
+                    getNodeId(), request.getPeerId(), savedCurrTerm, request.getTerm());
+                return TimeoutNowResponse.newBuilder() //
+                    .setTerm(this.currTerm) //
+                    .setSuccess(false) //
+                    .build();
+            }
+            if (this.state != State.STATE_FOLLOWER) {
+                LOG.info("Node {} received TimeoutNowRequest from {}, while state={}, term={}.", getNodeId(),
+                    request.getServerId(), this.state, this.currTerm);
+                return TimeoutNowResponse.newBuilder() //
+                    .setTerm(this.currTerm) //
+                    .setSuccess(false) //
+                    .build();
+            }
+
+            final long savedTerm = this.currTerm;
+            final TimeoutNowResponse resp = TimeoutNowResponse.newBuilder() //
+                .setTerm(this.currTerm + 1) //
+                .setSuccess(true) //
+                .build();
+            // Parallelize response and election
+            done.sendResponse(resp);
+            doUnlock = false;
+            electSelf();
+            LOG.info("Node {} received TimeoutNowRequest from {}, term={}.", getNodeId(), request.getServerId(),
+                savedTerm);
+        } finally {
+            if (doUnlock) {
+                this.writeLock.unlock();
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Message handleInstallSnapshot(final InstallSnapshotRequest request, final RpcRequestClosure done) {
+        if (this.snapshotExecutor == null) {
+            return RpcFactoryHelper //
+                .responseFactory() //
+                .newResponse(InstallSnapshotResponse.getDefaultInstance(), RaftError.EINVAL, "Not supported snapshot");
+        }
+        final PeerId serverId = new PeerId();
+        if (!serverId.parse(request.getServerId())) {
+            LOG.warn("Node {} ignore InstallSnapshotRequest from {} bad server id.", getNodeId(), request.getServerId());
+            return RpcFactoryHelper //
+                .responseFactory() //
+                .newResponse(InstallSnapshotResponse.getDefaultInstance(), RaftError.EINVAL,
+                    "Parse serverId failed: %s", request.getServerId());
+        }
+
+        this.writeLock.lock();
+        try {
+            if (!this.state.isActive()) {
+                LOG.warn("Node {} ignore InstallSnapshotRequest as it is not in active state {}.", getNodeId(),
+                    this.state);
+                return RpcFactoryHelper //
+                    .responseFactory() //
+                    .newResponse(InstallSnapshotResponse.getDefaultInstance(), RaftError.EINVAL,
+                        "Node %s:%s is not in active state, state %s.", this.groupId, this.serverId, this.state.name());
+            }
+
+            if (request.getTerm() < this.currTerm) {
+                LOG.warn("Node {} ignore stale InstallSnapshotRequest from {}, term={}, currTerm={}.", getNodeId(),
+                    request.getPeerId(), request.getTerm(), this.currTerm);
+                return InstallSnapshotResponse.newBuilder() //
+                    .setTerm(this.currTerm) //
+                    .setSuccess(false) //
+                    .build();
+            }
+
+            checkStepDown(request.getTerm(), serverId);
+
+            if (!serverId.equals(this.leaderId)) {
+                LOG.error("Another peer {} declares that it is the leader at term {} which was occupied by leader {}.",
+                    serverId, this.currTerm, this.leaderId);
+                // Increase the term by 1 and make both leaders step down to minimize the
+                // loss of split brain
+                stepDown(request.getTerm() + 1, false, new Status(RaftError.ELEADERCONFLICT,
+                    "More than one leader in the same term."));
+                return InstallSnapshotResponse.newBuilder() //
+                    .setTerm(request.getTerm() + 1) //
+                    .setSuccess(false) //
+                    .build();
+            }
+
+        } finally {
+            this.writeLock.unlock();
+        }
+        final long startMs = Utils.monotonicMs();
+        try {
+            if (LOG.isInfoEnabled()) {
+                LOG.info(
+                    "Node {} received InstallSnapshotRequest from {}, lastIncludedLogIndex={}, lastIncludedLogTerm={}, lastLogId={}.",
+                    getNodeId(), request.getServerId(), request.getMeta().getLastIncludedIndex(), request.getMeta()
+                        .getLastIncludedTerm(), this.logManager.getLastLogId(false));
+            }
+            this.snapshotExecutor.installSnapshot(request, InstallSnapshotResponse.newBuilder(), done);
+            return null;
+        } finally {
+            this.metrics.recordLatency("install-snapshot", Utils.monotonicMs() - startMs);
+        }
+    }
+
+    public void updateConfigurationAfterInstallingSnapshot() {
+        checkAndSetConfiguration(false);
... 35455 lines suppressed ...