×
Community Blog Brief Introduction to Distributed Consensus: Raft and SOFAJRaft

Brief Introduction to Distributed Consensus: Raft and SOFAJRaft

This article gives a brief overview of Distributed Consensus.

By Jiachun

1

1. Distributed Consensus Algorithm

1.1 How Do I Understand Distributed Consensus?

Multiple participants reached a complete agreement on one thing: one conclusion for one thing.

The conclusion on which agreement has been reached cannot be overturned.

1.2 Which Distributed Consensus Algorithms Are Available?

  • Paxos: It is considered the most original distributed consensus algorithm, and others are its variants. However, the paper about Paxos only describes the process of a single proposal instead of a detailed description of the Multi-Paxos required in state machine replication. The implementation of Paxos is complex in solving engineering issues, such as multi-node write and log discontinuity.
  • Zab: It is widely used in ZooKeeper but is not abstracted into a general-purpose library.
  • Raft: It is famous for its ease of understanding. Many Raft implementations have also emerged in the industry, such as etcd, Braft, and TiKV.

2. An Introduction to Raft

2.1 Feature: Strong Leader

  • The system must have a leader in place, and only one leader exists at a time. Only the leader can accept the request sent from clients.
  • The leader is responsible for actively communicating with all followers, sending "proposals" to all followers, and collecting responses from the majority of followers.
  • The leader also needs to actively send a heartbeat to all followers to maintain its leadership status (maintain presence.)

A leader must keep sending heartbeats all the time.

2

2.2 State Machine Replication

For an infinitely growing sequence a[1, 2, 3...], if for any integer i, the value of a[i] is aligned with distributed consensus, the system meets the requirement of the consensus state machine.

All real systems run continuously, and it is not enough to agree on a specific value. A real system usually converts the operation to a write-ahead-log (WAL) to ensure the consistency of all replicas. Then, all replicas in the system are kept consistent with the WAL, so each replica performs operations in the WAL in sequence. This ensures that the final status is consistent.

3

  • The client sends a write request to the leader.
  • The leader converts the "operation" into a WAL, writes the local log to the local log, and copies the log to all followers.
  • After receiving the responses from the majority of followers, the leader applies the "operation" corresponding to the log to the state machine.
  • The leader returns the processing result to the client.

2.3 Basic Concepts in Raft

Three Roles or States of Raft-Node

4

  • Follower: It is completely passive and cannot send any requests. It only receives and responds to messages from the leader and the candidate. The initial state of a node after it is started must be follower.
  • Leader: It processes all requests from the client and copies the log to all followers.
  • Candidate: It is used to select a new leader. If a follower times out, it becomes a candidate.

Three Types of Messages

  • RequestVote RPC: Sent by the candidate
  • AppendEntries (heartbeat) RPC: Sent by the leader
  • InstallSnapshot RPC: Sent by the leader

Term Logical Clock

  • Time is divided into terms, and term IDs increase monotonically on a timeline.
  • The leader is elected at the beginning of each term. After the election, the leader manages the entire cluster during the term, which is "election + routine operations."
  • A maximum of one leader can be specified in each term. The absence of the leader (caused by split-vote) is allowed.

5

2.4 Raft Function Description

Leader Election

Timeout Driver: Heartbeat / Election Timeout

Random Timeout Value: Reduces the probability that votes are divided due to election conflict

Election Process: Follower :arrow_right: Candidate (triggered by election timeout)

  • Win the Election: Candidate :arrow_right: Leader
  • Another Node Wins the Election: Candidate :arrow_right: Follower
  • No Node Wins the Election for a Period of Time: Candidate :arrow_right: Candidate

Election Action:

  • Current Term++
  • Send RequestVote RPC

Selection Principle of New Leader (Maximum Commitment Principle)

  • Candidates include log info in RequestVote RPCs (index & term of last log entry.)
  • During elections, choose the candidate with a log most likely to contain all committed entries.
  • Voting server V denies vote if its log is "more complete": (lastTermV > lastTermC) ||((lastTermV == lastTermC) && (lastIndexV > lastIndexC)).
  • Leader will have "most complete" log among electing majority.

Security: In one term, up to one leader can be elected. If no leader is elected, start the election in the next term.

6

Several time parameters affecting the election success rate in Raft:

  • Round Trip Time (RTT): Network latency
  • Heartbeat Timeout: The heartbeat interval is usually ten times smaller than election timeout. The aim is to enable the leader to send heartbeats continuously to prevent followers from triggering the election.
  • Election Timeout: The communication timeout between the leader and followers that trigger the election
  • Meantime Between Failure (MTBF): The interval of continuous routine failure of a server
    RTT << Heartbeat Timeout < Election Timeout (ET) << MTBF

Time for Triggering a Random Leader Selection: Random(ET, 2ET)

Log Replication

7

Raft Log Format

  • (TermId, LogIndex, LogValue)
  • (TermId and LogIndex) can determine a unique log.

Key Points of Log Replication

  • Continuity: Discontinuity is not allowed in logs.
  • Validity:

    1. Values of logs with the same term and logIndex must be the same across different nodes.
    2. Logs on the leader must be valid.
    3. The validity of logs in the follower is checked by comparing with logs in the leader. (How?)

Log Validity Check in Followers

  • AppendEntries RPC also carries the unique identifier (prevTermId or prevLogIndex) of the previous log.
  • Recursive Deduction

Log Recovery in Followers

  • The leader decrements nextIndex and resends AppendEntries until they are consistent with those of the leader log.

8

Commit Index Promotion

CommitIndex (TermId and LogIndex)

  • The commitIndex is the latest log location that has reached a consensus among the majority and can be applied to the state machine.
  • Logs are copied to followers and then persisted. However, the logs cannot be applied to the state machine immediately.
  • Only the leader knows whether the log is consistent among the majority and whether it can be applied to the state machine.
  • Followers record the current commitIndex received from the leader. All the logs smaller than or equal to the commitIndex can be applied to the state machine.

CommitIndex Promotion

  • In the next AppendEntries RPC (including Heartbeat), the leader carries the current commitIndex.
  • After followers check the validity of the logs, they accept AppendEntries and update the local commitIndex. Finally, all the logs smaller than or equal to the commitIndex are applied to the state machine.

AppendEntries RPC

  • Complete Information: (currentTerm, logEntries[], prevTerm, prevLogIndex, commitTerm, commitLogIndex)
  • currentTerm and logEntries[]: This is the log information. Considering the efficiency, there are usually multiple logs.
  • prevTerm and prevLogIndex: Log validity check
  • commitTerm and commitLogIndex: The latest log commitment point (commitIndex)

Summary: What Can We Do Now with Raft?

  • Identify multiple proposals in a row and ensure that the states of each system node in the cluster are the same
  • Select a leader automatically to ensure service availability in the case of downtime of a few nodes
  • Conduct strong synchronization of logs to ensure zero data loss after downtime

3. SOFAJRaft

It is a Raft implementation library only based on Java. All Raft functions are rewritten using Java with some improvements and optimizations.

3.1 Overall Functions of SOFAJRaft

9

Functions

  • Leader Election: The election of the leader
  • Log Replication and Recovery: Log recovery ensures that the committed data is not lost. Log recovery includes two aspects:

    1. Log Recovery for the Current Term: It involves the log recovery when follower nodes are restarted and added to the cluster, or a new follower node is added.
    2. Log Recovery for the Previous Term: It targets the consistency of logs before and after leader switchover.
  • Snapshot and Log Compaction: A snapshot is generated at a scheduled time to implement fast startup and recovery of log compaction and data copy to followers with InstallSnapshot.

10

  • Membership Change: The online change of cluster configurations, such as adding, deleting, and replacing nodes
  • Transfer Leader: Active transfer of leader for restarting, maintenance, and load balancing
  • Symmetric Network Partition Tolerance: The tolerance for symmetric network partitions

11

  • Pre-Vote: As is shown in the preceding figure, S1 is the current leader, and the network partition causes S2 to increase the local term continuously. After the network is recovered, the step-down of S1 that is running conscientiously may occur due to leader election initiated by S2. Thus, the entire cluster initiates a new election. To avoid this, pre-vote(currentTerm + 1, lastLogIndex, lastLogTerm) will be performed before the request-vote. After the majority agrees on a consensus, S2 changes the status to the candidate and initiates the real request-vote. Therefore, the pre-vote of the nodes after partitioning will not succeed, and the cluster will be unable to provide services normally for a period of time.
  • Asymmetric Network Partition Tolerance: The tolerance for asymmetric network partitions

12

As is shown in the preceding figure, S1 indicates the current leader, and S2 triggers leader selection through a continuous timeout. S3 interrupts the current lease by improving the term to reject the leader update. At this time, a trick check can be added, and each follower maintains a timestamp to record the time when the leader receives data updates (including the heartbeat.) A request-vote request is only allowed when the election timeout is exceeded.

  • Fault Tolerance: The faults of the minority do not affect the overall system availability:

    1. Machine Power Failure
    2. Force Shutdown of Applications
    3. Slow nodes, such as GC and OOM nodes
    4. Network Exception
    5. Raft node exception due to various other strange reasons
  • Workaround When Quorum Peers Are Dead: When failure occurs in the majority, the entire group is no longer available. At this moment, a safe choice is to wait for the majority of nodes to recover. This is the only way to ensure data security. However, if the service lays more emphasis on availability and gives up data consistency, the reset_peers command can recreate the cluster quickly to ensure the availability of the cluster.
  • Metrics: SOFAJRaft provides a variety of built-in performance metrics based on metrics class libraries.
  • Jepsen: In addition to unit tests, SOFAJRaft uses Jepsen, a distributed verification and fault injection testing framework, to simulate many situations, all of which have been verified:

    1. Random Partition: One primary network partition and one secondary network partition
    2. Add and remove nodes randomly
    3. Stop and start nodes randomly
    4. Run kill -9 and start nodes randomly
    5. Divide into two groups, connect through an intermediate node, and simulate the partition randomly
    6. Divide into different majority groups randomly

Performance Optimization

  • Batch: The entire procedure in SOFAJRaft is in batch mode and consumes in batches by relying on the MPSC model in disruptor, including but not limited to:

    1. Batch Submission of Tasks
    2. Batch Network Sending
    3. Batch Local I/O Write – Fsync is generally required for each log entry to ensure that logs are not lost, which is time-consuming. Therefore, SOFAJRaft has implemented merged write.
    4. Batch Application to State Machines
  • Replication Pipeline: Replication in a pipeline manner. The log synchronization between the leader and the follower nodes is in serial batch mode. After each batch is sent, the next batch (ping-pong) is sent after the current batch is synchronized, which causes a long latency. Pipeline replication between the leader and followers nodes can reduce the update latency and improve the throughput.
  • Append Log in Parallel: The leader persists log entries and sends log entries to followers in parallel.
  • Fully Concurrent Replication: The leader sends logs to all followers concurrently.
  • Asynchronous: The entire procedure in Jraft is almost free of blocking. It is completely asynchronous and is a callback programming model.
  • ReadIndex: The performance of reading Raft logs is optimized. Only the commitIndex is recorded when reading logs each time. Then, all peer heartbeats are sent to confirm the leader's identity. If the leader identity is confirmed, when applied index is greater than or equal to commitIndex, the result of client read can be returned. Linearly consistent reading can be performed based on ReadIndex. However, commitIndex should be obtained from the leader, which means one more round of RPC.
  • Lease Read: A lease is used to ensure the identity of the leader, eliminating the need for readIndex to confirm the leader's identity each time through the heartbeat, which provides better performance. However, it is not secure to maintain leases using the clock. The default configuration in Jraft is readIndex because readIndex performance is good enough.

3.2 SOFAJRaft Design

SOFAJRaft – Raft Node

13

  • Node: This is a node in the Raft group that connects and encapsulates all services in the underlying layer. The main service interfaces seen by users, especially the apply(task), submits new tasks to the state machine replication cluster composed by the Raft group. Then, tasks are applied to service state machines.

Storage

  • Log Storage: It records the logs of tasks submitted by users of Raft and copies logs from the leader to other nodes. LogStorage is the implementation of storage. LogManager is responsible for calling the underlying storage and caching, submitting in batches, checking, and optimizing the calls.
  • Metadata Storage: It stores the metadata and records the internal states of the Raft implementation, such as the current term and the vote result.
  • Snapshot Storage: It is used for storing snapshots and metadata of the user's state machine. This one is optional. SnapshotStorage is the implementation of snapshot storage. SnapshotExecutor is used to manage the storage, remote installation, and replication of snapshots.

State Machines

  • StateMachine: It implements the core logic of the user. The core is the onApply(Iterator) method. Applications submit logs to service state machines using Node#apply(task).
  • FSMCaller: It encapsulates calls for state transition of service StateMachine and writes of logs. It is used to implement a finite state machine and perform necessary checks, merged request submission, and concurrent processing.

Replication

  • Replicator: It replicates logs from the leader to followers, namely, the AppendEntries calls in Raft, including the state check through the heartbeat.
  • ReplicatorGroup: It is used to manage all replicators in a single Raft group. It checks and grants the necessary permissions.

RPC module for network communication between nodes

  • RPC Server: It is an RPC server built in a node. It can receive requests from other nodes or clients and transfer them to the corresponding service for processing.
  • RPC Client: It is used to initiate requests to other nodes, such as voting, log replication, and heartbeat requests.
  • KV Store: SOFAJRaft is only a lib. KV Store is a typical application scenario of SOFAJRaft. Put it in the figure to understand SOFAJRaft more.

SOFAJRaft – Raft Group

14

SOFAJRaft – Multi Raft Group

15

3.3 SOFAJRaft Implementation Details

Efficient Linearly Consistent Reading

What Is Linearly Consistent Reading?

Here is a simple example of linearly consistent reading. When we write a value at t1, we can read this value after t1, but it is impossible to read the old value before t1. Think about the volatile keyword in Java. Simply speaking, linearly consistent reading is implementing volatile semantics in a distributed system.

16

The client A, B, C, and D in the preceding figure all conform to linearly consistent reading. Among them, D seems to be a stale read, but it is not. The request of D spans three stages, but reading may occur at any time, so reading 1 or 2 is OK.

Note: The following discussion is based on the major premise that the implementation of the service state machine must meet linearly consistent reading. In other words, it must also have Java volatile semantics.

1) To put it simply, can we read data directly from the current leader?

How can we determine whether the current leader is the real leader (network partition)?

2) The simplest implementation method is to apply the Raft protocol to the read request.

17

What's the Problem?

  • Besides the overhead of writing logs to disks, the RPC overhead of log replication also exists. It is unacceptable in a system where read takes a large proportion.
  • A large number of Raft "log read" operations appear

3) ReadIndex Read

This is an optimization solution mentioned in the Raft paper. Specifically:

  • Record the commit index of the current log to a local variable ReadIndex
  • Initiate a heartbeat request to other nodes. If the corresponding heartbeat response is returned by most nodes, the leader can be sure that it is still the leader. The leader has proved itself.
  • The leader waits for the execution of its own state machine until the apply index exceeds the ReadIndex so that Linearizable Read can be provided safely. We don't have to consider if the leader has gone at the time of reading. Thinking: why can't I perform the read request before the value of apply index exceeds ReadIndex?
  • The leader executes the read request and returns the result to the client.

With ReadIndex, it is also easy to provide linearly consistent reading on the follower nodes:

  • The follower node asks for the latest ReadIndex from the leader.
  • The leader performs the preceding first three steps to determine whether it is a leader and then returns ReadIndex to followers.
  • The follower waits for its apply index to exceed the ReadIndex. What is the problem? Is it a slow node?
  • The follower executes the read request and returns the result to the client.

ReadIndex Summary:

  • Compared with the Raft log solution, ReadIndex saves the disk overhead in reading data and can improve throughput significantly. The read throughput of the leader is close to the upper limit of RPC after combining with the batch + pipeline ack + full asynchronization of SOFAJRaft in three-replica mode.
  • The latency depends on the slowest heartbeat response in the majority. Theoretically, it has little effect on reducing the latency.

4) Lease Read

Lease read is similar to ReadIndex. It saves the logs and the network interaction. It can improve the read throughput and reduce the latency significantly.

The leader selects a lease period smaller than election timeout (preferably ten times smaller), and no election will take place during the lease period. This ensures that the leader does not change, so Step 2 of ReadIndex can be skipped, which reduces the latency. As we can see, the correctness of lease read is closely related to time, so the implementation of time is very important. If the drift is serious, this mechanism will go wrong.

Implementation:

  • Timed heartbeats receive responses from the majority to confirm the leader's validity. In SOFAJRaft, the default heartbeat interval is one-tenth of the election timeout.
  • Before the lease expires, the current leader can be considered the only valid leader in the Raft group. Step 2 in ReadIndex (confirming leader identity by heartbeat) can be ignored.
  • The leader waits for the execution of its own state machine until the apply index exceeds the ReadIndex, which can provide Linearizable Read safely.

5) Go Further: Wait Free

So far, lease has dropped Step 2 of ReadIndex (heartbeat) and can go further, omitting Step 3.

Let's think about the essence of the previous implementation solution? The state machine of the current node is in the same or newer state at the same time point as the "read" moment.

Then, a stricter constraint is that the state machine of the current node is up-to-date at the current time.

Here is the question: can we guarantee that the leader's state machine is up-to-date?

  • First, the log of the leader node must be up-to-date. Even the newly elected leader must contain all the commit logs, but its state machine may lag behind the old leader.
  • However, after the leader applies the first log in the current term, its state machine must be up-to-date.
  • Therefore, it can be concluded that after the leader has applied the first log in its own term successfully, it doesn't have to obtain the commit index or wait for the state machine. It can read directly, which is linearly consistent.

Summary

The Wait Free mechanism will minimize the read latency. SOFAJRaft has not implemented the wait free optimization yet, but it is already in the plans.

Initiate a linearly consistent read request in SOFAJRaft:

// KV storage realizes linearly consistent read.
public void readFromQuorum(String key, AsyncContext asyncContext) {
    // Request ID is passed in as request context.
    byte[] reqContext = new byte[4];
    Bits.putInt(reqContext, 0, requestId.incrementAndGet());
    // Call the readIndex method and wait for the callback to execute.
    this.node.readIndex(reqContext, new ReadIndexClosure() {

        @Override
        public void run(Status status, long index, byte[] reqCtx) {
            if (status.isOk()) {
                try {
                    // The ReadIndexClosure callback is successful. Read the latest data from the state machine and return it.
                    // If the status implementation involves version, read data based on the index number of input log.
                    asyncContext.sendResponse(new ValueCommand(fsm.getValue(key)));
                } catch (KeyNotFoundException e) {
                    asyncContext.sendResponse(GetCommandProcessor.createKeyNotFoundResponse());
                }
            } else {
                // In specific cases, for example, election, the read request will fail.
                asyncContext.sendResponse(new BooleanCommand(false, status.getErrorMsg()));
            }
        }
    });
}

4. SOFAJRaft Application Scenarios

4.1 What Can SOFAJRaft Do?

  • Election
  • Distributed lock services, such as ZooKeeper
  • Highly reliable metadata management
  • Distributed storage systems, such as distributed message queues, distributed file systems, and distributed block systems

4.2 User Case

  • AntQ Streams QCoordinator: It uses SOFAJRaft to conduct elections and store metadata in the coordinator cluster.
  • Schema Registry: It provides a highly reliable schema management service, similar to the schema registry of Kafka.
  • The Meta Information Management Module of the SOFA Service Registry: IP data information registration requires consistent data writes in each node. The normal storage of data cannot be affected when the minority nodes fail.
  • RheaKV: It is an embedded, distributed, highly available, and highly consistent KV storage class library based on SOFAJRaft and RocksDB.

4.3 Simple Practice: Design a Simple KV Store Based on SOFAJRaft

18

So far, it seems that we have not seen anything special about SOFAJRaft as a lib because zk and etcd can also do what SOFAJRaft can. As such, is using SOFAJRaft necessary?

Next, I will introduce a more complex SOFAJRaft-based practice to show that SOFAJRaft leaves much scope for imagination and scalability.

4.4 More Complex Practice: Design of Rhea KV Based on SOFAJRaft

19

Concepts

  • PD: It refers to the global central control node that schedules the entire cluster. PD is not required for clusters that don't require self-management. A PD can manage multiple isolated clusters based on clusterId.
  • Store: It is a physical storage node in a cluster. A store contains one or more regions.
  • Region: It is the smallest KV data unit. Each region has a left-closed and right-open interval [startKey, endKey) that supports automatic splitting and automatic copy migration based on metrics, such as request traffic, load, and data volume.

Features

  • Embedment
  • Strong Consistency
  • Self-Driving: It supports self-diagnosis, self-optimization, automatic decision-making, and self-recovery. The preceding points (especially features 2 and 3) are achieved based on the function of SOFAJRaft.

References

0 1 0
Share on

block

2 posts | 0 followers

You may also like

Comments

block

2 posts | 0 followers

Related Products