diff --git a/jraft-core/pom.xml b/jraft-core/pom.xml
new file mode 100644
index 0000000..04ee660
--- /dev/null
+++ b/jraft-core/pom.xml
@@ -0,0 +1,131 @@
+
+
+
+ 4.0.0
+
+ jraft-parent
+ com.alipay.sofa
+ 1.3.10.bugfix
+
+ jraft-core
+ jar
+ jraft-core ${project.version}
+
+
+
+
+ junit
+ junit
+ test
+
+
+ junit
+ junit-dep
+ test
+
+
+ org.ow2.asm
+ asm
+
+
+ com.google.code.findbugs
+ jsr305
+
+
+
+ org.rocksdb
+ rocksdbjni
+
+
+
+ net.java.dev.jna
+ jna
+
+
+
+ org.jctools
+ jctools-core
+
+
+
+ org.mockito
+ mockito-all
+ test
+
+
+ org.powermock
+ powermock-api-mockito
+ test
+
+
+ org.powermock
+ powermock-module-junit4
+ test
+
+
+
+ org.slf4j
+ slf4j-api
+
+
+ org.apache.logging.log4j
+ log4j-api
+ test
+
+
+ org.apache.logging.log4j
+ log4j-core
+ test
+
+
+ org.apache.logging.log4j
+ log4j-slf4j-impl
+ test
+
+
+ org.apache.logging.log4j
+ log4j-jcl
+ test
+
+
+ com.lmax
+ disruptor
+
+
+ com.google.protobuf
+ protobuf-java
+
+
+
+ commons-io
+ commons-io
+
+
+ commons-lang
+ commons-lang
+
+
+
+ com.alipay.sofa
+ bolt
+
+
+ com.alipay.sofa
+ hessian
+
+
+
+ io.dropwizard.metrics
+ metrics-core
+
+
+
+ org.openjdk.jmh
+ jmh-core
+
+
+ org.openjdk.jmh
+ jmh-generator-annprocess
+
+
+
diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/CliService.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/CliService.java
new file mode 100644
index 0000000..e364b30
--- /dev/null
+++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/CliService.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package 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 {
+
+ /**
+ * 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 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 learners);
+
+ /**
+ * Converts the specified learner to follower of |conf|.
+ * return OK status when success.
+ *
+ * @param groupId the raft group id
+ * @param conf current configuration
+ * @param learner learner peer
+ * @return operation status
+ * @since 1.3.8
+ *
+ */
+ Status learner2Follower(final String groupId, final Configuration conf, final PeerId learner);
+
+ /**
+ * 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 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 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 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 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 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 balanceGroupIds, final Configuration conf,
+ final Map balancedLeaderIds);
+}
diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/Closure.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/Closure.java
new file mode 100644
index 0000000..7d51264
--- /dev/null
+++ b/jraft-core/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/jraft-core/src/main/java/com/alipay/sofa/jraft/FSMCaller.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/FSMCaller.java
new file mode 100644
index 0000000..71b003e
--- /dev/null
+++ b/jraft-core/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, 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/jraft-core/src/main/java/com/alipay/sofa/jraft/Iterator.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/Iterator.java
new file mode 100644
index 0000000..c80de1f
--- /dev/null
+++ b/jraft-core/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 {
+
+ /**
+ * 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/jraft-core/src/main/java/com/alipay/sofa/jraft/JRaftServiceFactory.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/JRaftServiceFactory.java
new file mode 100644
index 0000000..84a78fc
--- /dev/null
+++ b/jraft-core/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 entries.
+ */
+ 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/jraft-core/src/main/java/com/alipay/sofa/jraft/JRaftUtils.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/JRaftUtils.java
new file mode 100644
index 0000000..020137f
--- /dev/null
+++ b/jraft-core/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/jraft-core/src/main/java/com/alipay/sofa/jraft/Lifecycle.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/Lifecycle.java
new file mode 100644
index 0000000..85bff23
--- /dev/null
+++ b/jraft-core/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 {
+
+ /**
+ * Initialize the service.
+ *
+ * @return true when successes.
+ */
+ boolean init(final T opts);
+
+ /**
+ * Dispose the resources for service.
+ */
+ void shutdown();
+}
diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/Node.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/Node.java
new file mode 100644
index 0000000..990d131
--- /dev/null
+++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/Node.java
@@ -0,0 +1,336 @@
+/*
+ * 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.core.State;
+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, 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] 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
+ *
+ * @return the peer list
+ */
+ List listPeers();
+
+ /**
+ * List all alive peers of this raft group, only leader returns.
+ *
+ * [NOTE] 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.
+ *
+ * @return the alive peer list
+ * @since 1.2.6
+ */
+ List listAlivePeers();
+
+ /**
+ * List all learners of this raft group, only leader returns.
+ *
+ * [NOTE] 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
+ *
+ * @return the learners set
+ * @since 1.3.0
+ */
+ List listLearners();
+
+ /**
+ * List all alive learners of this raft group, only leader returns.
+ *
+ * [NOTE] 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
+ *
+ * @return the alive learners set
+ * @since 1.3.0
+ */
+ List listAliveLearners();
+
+ /**
+ * Add a new peer to the raft group. done.run() would be invoked after this
+ * operation finishes, describing the detailed result.
+ *
+ * @param peer peer to add
+ * @param done callback
+ */
+ void addPeer(final PeerId peer, final Closure done);
+
+ /**
+ * Remove the peer from the raft group. done.run() would be invoked after
+ * operation finishes, describing the detailed result.
+ *
+ * @param peer peer to remove
+ * @param done callback
+ */
+ void removePeer(final PeerId peer, final Closure done);
+
+ /**
+ * Change the configuration of the raft group to |newPeers| , done.run()
+ * would be invoked after this operation finishes, describing the detailed result.
+ *
+ * @param newPeers new peers to change
+ * @param done callback
+ */
+ void changePeers(final Configuration newPeers, final Closure done);
+
+ /**
+ * Reset the configuration of this node individually, without any replication
+ * to other peers before this node becomes the leader. This function is
+ * supposed to be invoked when the majority of the replication group are
+ * dead and you'd like to revive the service in the consideration of
+ * availability.
+ * Notice that neither consistency nor consensus are guaranteed in this
+ * case, BE CAREFUL 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 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 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 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 getReplicatorStatueListeners();
+
+ /**
+ * Get the node's target election priority value.
+ *
+ * @return node's target election priority value.
+ * @since 1.3.0
+ */
+ int getNodeTargetPriority();
+
+ /**
+ * Get the node's state.
+ *
+ * @return node's state.
+ * @since 1.3.8
+ */
+ State getNodeState();
+}
diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/NodeDescribeSignalHandler.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/NodeDescribeSignalHandler.java
new file mode 100644
index 0000000..42f6910
--- /dev/null
+++ b/jraft-core/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 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/jraft-core/src/main/java/com/alipay/sofa/jraft/NodeManager.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/NodeManager.java
new file mode 100644
index 0000000..236ebf7
--- /dev/null
+++ b/jraft-core/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 nodeMap = new ConcurrentHashMap<>();
+ private final ConcurrentMap> groupMap = new ConcurrentHashMap<>();
+ private final ConcurrentHashSet 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 nodes = this.groupMap.get(groupId);
+ if (nodes == null) {
+ nodes = Collections.synchronizedList(new ArrayList<>());
+ List 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 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 getNodesByGroupId(final String groupId) {
+ return this.groupMap.get(groupId);
+ }
+
+ /**
+ * Get all nodes
+ */
+ public List getAllNodes() {
+ return this.groupMap.values().stream().flatMap(Collection::stream).collect(Collectors.toList());
+ }
+
+ private NodeManager() {
+ }
+}
diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/NodeMetricsSignalHandler.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/NodeMetricsSignalHandler.java
new file mode 100644
index 0000000..2d128cb
--- /dev/null
+++ b/jraft-core/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 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/jraft-core/src/main/java/com/alipay/sofa/jraft/RaftGroupService.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/RaftGroupService.java
new file mode 100644
index 0000000..d206c2c
--- /dev/null
+++ b/jraft-core/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/jraft-core/src/main/java/com/alipay/sofa/jraft/RaftServiceFactory.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/RaftServiceFactory.java
new file mode 100644
index 0000000..8aaea5c
--- /dev/null
+++ b/jraft-core/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/jraft-core/src/main/java/com/alipay/sofa/jraft/ReadOnlyService.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/ReadOnlyService.java
new file mode 100644
index 0000000..cd14364
--- /dev/null
+++ b/jraft-core/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 {
+
+ /**
+ * 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/jraft-core/src/main/java/com/alipay/sofa/jraft/ReplicatorGroup.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/ReplicatorGroup.java
new file mode 100644
index 0000000..7054194
--- /dev/null
+++ b/jraft-core/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 group
+ * @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 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 listReplicators();
+}
diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/RouteTable.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/RouteTable.java
new file mode 100644
index 0000000..94578a2
--- /dev/null
+++ b/jraft-core/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.google.protobuf.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
+ private final ConcurrentMap 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 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/jraft-core/src/main/java/com/alipay/sofa/jraft/StateMachine.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/StateMachine.java
new file mode 100644
index 0000000..ccf1aa2
--- /dev/null
+++ b/jraft-core/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/jraft-core/src/main/java/com/alipay/sofa/jraft/Status.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/Status.java
new file mode 100644
index 0000000..559a093
--- /dev/null
+++ b/jraft-core/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 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/jraft-core/src/main/java/com/alipay/sofa/jraft/ThreadPoolMetricsSignalHandler.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/ThreadPoolMetricsSignalHandler.java
new file mode 100644
index 0000000..6e89667
--- /dev/null
+++ b/jraft-core/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/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/CatchUpClosure.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/CatchUpClosure.java
new file mode 100644
index 0000000..5e8558f
--- /dev/null
+++ b/jraft-core/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)
+ *
+ * 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/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/ClosureQueue.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/ClosureQueue.java
new file mode 100644
index 0000000..5aeded5
--- /dev/null
+++ b/jraft-core/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 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 closures, final List taskClosures);
+}
diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/ClosureQueueImpl.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/ClosureQueueImpl.java
new file mode 100644
index 0000000..10e9ed0
--- /dev/null
+++ b/jraft-core/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 queue;
+
+ @OnlyForTest
+ public long getFirstIndex() {
+ return firstIndex;
+ }
+
+ @OnlyForTest
+ public LinkedList getQueue() {
+ return queue;
+ }
+
+ public ClosureQueueImpl() {
+ super();
+ this.lock = new ReentrantLock();
+ this.firstIndex = 0;
+ this.queue = new LinkedList<>();
+ }
+
+ @Override
+ public void clear() {
+ List 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 closures) {
+ return popClosureUntil(endIndex, closures, null);
+ }
+
+ @Override
+ public long popClosureUntil(final long endIndex, final List closures, final List 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/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/JoinableClosure.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/JoinableClosure.java
new file mode 100644
index 0000000..2f18a4e
--- /dev/null
+++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/JoinableClosure.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package 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) {
+ try {
+ this.closure.run(status);
+ } finally {
+ 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/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/LoadSnapshotClosure.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/LoadSnapshotClosure.java
new file mode 100644
index 0000000..496d1df
--- /dev/null
+++ b/jraft-core/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/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/ReadIndexClosure.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/ReadIndexClosure.java
new file mode 100644
index 0000000..9bd23c6
--- /dev/null
+++ b/jraft-core/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 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/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/SaveSnapshotClosure.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/SaveSnapshotClosure.java
new file mode 100644
index 0000000..4b1b9b6
--- /dev/null
+++ b/jraft-core/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/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/SynchronizedClosure.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/SynchronizedClosure.java
new file mode 100644
index 0000000..c1d2c2a
--- /dev/null
+++ b/jraft-core/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/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/TaskClosure.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/closure/TaskClosure.java
new file mode 100644
index 0000000..23c1415
--- /dev/null
+++ b/jraft-core/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.
+ *
+ * Note: user implementation should not block
+ * this method and throw any exceptions.
+ */
+ void onCommitted();
+}
diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/conf/Configuration.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/conf/Configuration.java
new file mode 100644
index 0000000..7175980
--- /dev/null
+++ b/jraft-core/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, Copiable {
+
+ private static final Logger LOG = LoggerFactory.getLogger(Configuration.class);
+
+ private static final String LEARNER_POSTFIX = "/learner";
+
+ private List peers = new ArrayList<>();
+
+ // use LinkedHashSet to keep insertion order.
+ private LinkedHashSet learners = new LinkedHashSet<>();
+
+ public Configuration() {
+ super();
+ }
+
+ /**
+ * Construct a configuration instance with peers.
+ *
+ * @param conf configuration
+ */
+ public Configuration(final Iterable 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 conf, final Iterable learners) {
+ Requires.requireNonNull(conf, "conf");
+ for (final PeerId peer : conf) {
+ this.peers.add(peer.copy());
+ }
+ addLearners(learners);
+ }
+
+ public void setLearners(final LinkedHashSet 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 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 getLearners() {
+ return this.learners;
+ }
+
+ /**
+ * Retrieve the learners set copy.
+ *
+ * @return learners
+ */
+ public List 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 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 iterator() {
+ return this.peers.iterator();
+ }
+
+ public Set getPeerSet() {
+ return new HashSet<>(this.peers);
+ }
+
+ public List listPeers() {
+ return new ArrayList<>(this.peers);
+ }
+
+ public List getPeers() {
+ return this.peers;
+ }
+
+ public void setPeers(final List peers) {
+ this.peers.clear();
+ for (final PeerId peer : peers) {
+ this.peers.add(peer.copy());
+ }
+ }
+
+ public void appendPeers(final Collection 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 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/jraft-core/src/main/java/com/alipay/sofa/jraft/conf/ConfigurationEntry.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/conf/ConfigurationEntry.java
new file mode 100644
index 0000000..d13fd17
--- /dev/null
+++ b/jraft-core/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 listPeers() {
+ final Set 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 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 listLearners() {
+ final Set 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/jraft-core/src/main/java/com/alipay/sofa/jraft/conf/ConfigurationManager.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/conf/ConfigurationManager.java
new file mode 100644
index 0000000..3337699
--- /dev/null
+++ b/jraft-core/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)
+ *
+ * 2018-Apr-04 2:24:54 PM
+ */
+public class ConfigurationManager {
+
+ private static final Logger LOG = LoggerFactory.getLogger(ConfigurationManager.class);
+
+ private final LinkedList 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 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/jraft-core/src/main/java/com/alipay/sofa/jraft/core/BallotBox.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/BallotBox.java
new file mode 100644
index 0000000..64d2bf9
--- /dev/null
+++ b/jraft-core/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, 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 pendingMetaQueue = new SegmentList<>(false);
+
+ @OnlyForTest
+ long getPendingIndex() {
+ return this.pendingIndex;
+ }
+
+ @OnlyForTest
+ SegmentList 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/jraft-core/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java
new file mode 100644
index 0000000..3bc56b9
--- /dev/null
+++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java
@@ -0,0 +1,672 @@
+/*
+ * 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.Arrays;
+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.google.protobuf.Message;
+import com.google.protobuf.ProtocolStringList;
+
+/**
+ * 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;
+ }
+
+ private void recordConfigurationChange(final String groupId, final List oldPeersList,
+ final List newPeersList) {
+ final Configuration oldConf = new Configuration();
+ for (final String peerIdStr : oldPeersList) {
+ final PeerId oldPeer = new PeerId();
+ oldPeer.parse(peerIdStr);
+ oldConf.addPeer(oldPeer);
+ }
+ final Configuration newConf = new Configuration();
+ for (final String peerIdStr : newPeersList) {
+ final PeerId newPeer = new PeerId();
+ newPeer.parse(peerIdStr);
+ newConf.addPeer(newPeer);
+ }
+ LOG.info("Configuration of replication group {} changed from {} to {}.", groupId, oldConf, newConf);
+ }
+
+ private Status checkLeaderAndConnect(final String groupId, final Configuration conf, final PeerId leaderId) {
+ 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);
+ }
+
+ return Status.OK();
+ }
+
+ @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 = checkLeaderAndConnect(groupId, conf, leaderId);
+ if (!st.isOk()) {
+ return st;
+ }
+
+ 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;
+ recordConfigurationChange(groupId, resp.getOldPeersList(), resp.getNewPeersList());
+ 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 = checkLeaderAndConnect(groupId, conf, leaderId);
+ if (!st.isOk()) {
+ return st;
+ }
+ 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;
+ recordConfigurationChange(groupId, resp.getOldPeersList(), resp.getNewPeersList());
+ return Status.OK();
+ } else {
+ return statusFromResponse(result);
+
+ }
+ } catch (final Exception e) {
+ return new Status(-1, e.getMessage());
+ }
+ }
+
+ @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 = checkLeaderAndConnect(groupId, conf, leaderId);
+ if (!st.isOk()) {
+ return st;
+ }
+
+ 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;
+ recordConfigurationChange(groupId, resp.getOldPeersList(), resp.getNewPeersList());
+ 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 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 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 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 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 learner2Follower(final String groupId, final Configuration conf, final PeerId learner) {
+ Status status = removeLearners(groupId, conf, Arrays.asList(learner));
+ if (status.isOk()) {
+ status = addPeer(groupId, conf, new PeerId(learner.getIp(), learner.getPort()));
+ }
+ return status;
+ }
+
+ @Override
+ public Status resetLearners(final String groupId, final Configuration conf, final List 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 = checkLeaderAndConnect(groupId, conf, leaderId);
+ if (!st.isOk()) {
+ return st;
+ }
+
+ 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 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 getPeers(final String groupId, final Configuration conf) {
+ return getPeers(groupId, conf, false, false);
+ }
+
+ @Override
+ public List getAlivePeers(final String groupId, final Configuration conf) {
+ return getPeers(groupId, conf, false, true);
+ }
+
+ @Override
+ public List getLearners(final String groupId, final Configuration conf) {
+ return getPeers(groupId, conf, true, false);
+ }
+
+ @Override
+ public List getAliveLearners(final String groupId, final Configuration conf) {
+ return getPeers(groupId, conf, true, true);
+ }
+
+ @Override
+ public Status rebalance(final Set balanceGroupIds, final Configuration conf,
+ final Map 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 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 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 peerIdList = new ArrayList<>();
+ final ProtocolStringList 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 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/jraft-core/src/main/java/com/alipay/sofa/jraft/core/DefaultJRaftServiceFactory.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/DefaultJRaftServiceFactory.java
new file mode 100644
index 0000000..26321f4
--- /dev/null
+++ b/jraft-core/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 org.apache.commons.lang.StringUtils;
+
+import com.alipay.sofa.jraft.JRaftServiceFactory;
+import com.alipay.sofa.jraft.entity.codec.LogEntryCodecFactory;
+import com.alipay.sofa.jraft.entity.codec.v2.LogEntryV2CodecFactory;
+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 LogEntryV2CodecFactory.getInstance();
+ }
+}
diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/ElectionPriority.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/ElectionPriority.java
new file mode 100644
index 0000000..16cae25
--- /dev/null
+++ b/jraft-core/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/jraft-core/src/main/java/com/alipay/sofa/jraft/core/FSMCallerImpl.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/FSMCallerImpl.java
new file mode 100644
index 0000000..2f05340
--- /dev/null
+++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/FSMCallerImpl.java
@@ -0,0 +1,726 @@
+/*
+ * 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.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 {
+
+ @Override
+ public ApplyTask newInstance() {
+ return new ApplyTask();
+ }
+ }
+
+ private class ApplyTaskHandler implements EventHandler {
+ // 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 disruptor;
+ private RingBuffer taskQueue;
+ private volatile CountDownLatch shutdownLatch;
+ private NodeMetrics nodeMetrics;
+ private final CopyOnWriteArrayList 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. 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