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