Paxakos is a pure Rust implementation of a distributed consensus algorithm

Overview

paxakos

Paxakos is a pure Rust implementation of a distributed consensus algorithm based on Leslie Lamport's Paxos. It enables distributed systems to consistently modify shared state across their network, even in the presence of failures.

crates.io docs.rs GPLv3 licensed

Usage

In order to use Paxakos, the traits [LogEntry], [State], [NodeInfo] and Communicator need to be implemented. The first two describe what state will be replicated across the network and which operations on it are available. The latter describe the nodes in your network and how to communicate between them.

Below are two partial implementations of LogEntry and State. The other two traits are more abstract and won't be illustrated here. Please refer to the examples to get a fuller picture.

use std::collections::HashSet;

use paxakos::{LogEntry, State};
use uuid::Uuid;

#[derive(Clone, Copy, Debug, serde::Deserialize, serde::Serialize)]
pub enum CalcOp {
    Add(f64, Uuid),
    Div(f64, Uuid),
    Mul(f64, Uuid),
    Sub(f64, Uuid),
}

impl LogEntry for CalcOp {
    type Id = Uuid;

    fn id(&self) -> Self::Id {
        match self {
            CalcOp::Add(_, id)
            | CalcOp::Div(_, id)
            | CalcOp::Mul(_, id)
            | CalcOp::Sub(_, id) => {
                *id
            }
        }
    }
}

#[derive(Clone, Debug)]
pub struct CalcState {
    applied: HashSet<Uuid>,
    value: f64,
}

impl State for CalcState {
    type Context = ();

    type LogEntry = CalcOp;
    type Outcome = f64;
    type Event = f64;

#
#
    fn apply(
        &mut self,
        log_entry: &Self::LogEntry,
        _context: &mut (),
    ) -> (Self::Outcome, Self::Event) {
        if self.applied.insert(log_entry.id()) {
            match log_entry {
                CalcOp::Add(v, _) => {
                    self.value += v;
                }
                CalcOp::Div(v, _) => {
                    self.value /= v;
                }
                CalcOp::Mul(v, _) => {
                    self.value *= v;
                }
                CalcOp::Sub(v, _) => {
                    self.value -= v;
                }
            }
        }

        (self.value, self.value)
    }
}

Motivation

Rust is a great language with which to implement efficient and truly reliable, fault-tolerant services. And while there already are several Rust implementations of consensus algorithms, they are either rudimentary or incomplete or their API was not approachable enough for this author.

Priorities

The project's priorities are as follows.

  1. Correctness

    The highest priority is correctness, which, in the context of consensus, requires stability, consistency and liveness.

    • Stability implies that once a node learns that a log entry a has been appended to the distributed log, it will never learn that a different entry b belongs in its place.
    • Consistency means that all nodes in the Paxakos network agree on the contents of their shared log. While nodes may temporarily fall behind, i.e. their log may be shorter than other nodes', their logs must be otherwise equivalent.
    • Liveness means that the system won't get stuck, i.e. progress is always eventually made (assuming a there is no contention/some degree of cooperation).
  2. Simplicity

    Paxakos aims to be simple by providing few orthogonal primitives. To be clear, the goal is not to arbitrarily limit complexity. The goal is to have unentangled primitves; providing the same features with the least amount of complexity possible.

  3. Ergonomics

    Using Paxakos should be as pleasant as possible without sacrificing correctness or simplicity. The biggest challenge in this area are, at present, build times. If you have other concerns, please open an issue.

Features

Paxakos is a Multi-Paxos implementation. It supports membership changes, concurrency, as well as taking and installing snapshots.

Membership Changes

The State trait exposes the cluster_at method. By implementing it, arbitrary membership changes may be made. No restrictions are imposed and it is up to users and implementors to make sure that any transition is safe.

Concurrency

Multi-Paxos allows for any number of rounds to be settled concurrently. This can be exploited by implementing State's [concurrency][State::concurrency] method.

Please note that when concurrency is enabled, gaps may appear in the log. These must be closed before entries after them can be applied to the state. This is typically done by appending no-op entries. A utility for doing so automatically is provided, but its API is far from stable.

Snapshots

A node may take a snapshot of its current state. The result is a combination of the application specific State as well as pakakos specifc information. These snapshots may be used for graceful shutdown and restart, to bootstrap nodes which have joined the cluster or to catch up nodes that have fallen behind.

Decorations

Implementations of the [Decoration][crate::decoration::Decoration] trait can provide reusable functionality than can but need not be used. Paxakos comes with several decorations (see below).

Heartbeats (heartbeats flag)

The heartbeats decoration sends a heartbeat message at regular intervals.

Autofill (autofill flag)

From time to time gaps will appear in the distributed log. For example due to concurrency or dropped messages. When that happens, the autofill decoration will fill the gap, implicitly catching the node up or making sure that queued log entries may be applied.

Track Leadership (track-leadership flag)

The leadership tracking decoration infers which nodes are leading the cluster.

Ensure Leadership (experimental, ensure-leadership flag)

Similar to heartbeats, the ensure leadership decoration will append a log entry after none has been for a certain amount of time. However the goal of this decoration is to ensure there is a leader.

Leases

A cluster will often have shared resources which must be locked before they may be accessed. To account for node failures, locks time out unless they are refreshed. Such locks are commonly referred to as "leases".

Leaser (experimental, leaser flag)

Leaser makes taking, refreshing and releasing a lease as convenient as calling [take_lease][crate::leases::leaser::Leaser::take_lease]. The lease is refreshed as long as the returned value is held onto.

Releaser (experimental, releaser flag)

The releaser decoration makes sure that leases that have timed out are cleared away, releasing the underlying resource.

Master Leases (experimental, master-leases flag)

Master leases are a mechanism to allow passive local reads. A comprehensive description may be found in Paxos Made Live.

Protocol

This section describes the Paxakos protocol. It is, for the most part, a typical Multi-Paxos protocol. Multi-Paxos generalizes Paxos to be run for multiple rounds, where each round represents a slot in the distributed log. Nodes may propose log entries to place in those slots. The liveness property guarantees that the cluster will, for each round, eventually converge on one of the proposed entries.

A central component of the protocol are coordination numbers. These are usually called "proposal numbers". However, because they are used throughout the protocol, Paxakos uses the more abstract term.

Appending an entry to the distributed log takes the following five steps.

  1. Prepare (RoundNum, CoordNum)

    In order for a node to append an entry to the distributed log, it must first become leader for the given round. If it already believes itself leader for the round, it will skip to step 3.

    To become leader for a round the node will send a prepare message containing the round number and a coordination number. The coordination number is chosen so that it is

    • higher than any previously encountered coordination number and
    • guaranteed not to be used by another node.

    The former is important for liveness. The latter is required for stability and consistency and is achieved by exploiting the deterministic order of nodes returned by cluster_at.

  2. Vote

    When a node receives a prepare request, it checks that it hasn't accepted a previous such request with a coordination number that's higher than the given one. If it has, it will reply with a conflict. If it hasn't, the node will either abstain, i.e., choose not to give its vote, or it sends back a promise not to accept any more proposals with a coordination number that's less the given one.

    1. Promise (Vec<(RoundNum, CoordNum, LogEntry)>)

      The promise is a set of triples, each consisting of a round number, a coordination number and a log entry. It can be thought to mean "I acknowledge your bid to become leader and give you my vote. However, in return you must respect these previous commitments I've made."

    2. Conflict (CoordNum, Option<(CoordNum, LogEntry)>)

      A rejection is sent with the greatest observed coordination number so far. For the special case that the round has already converged and the node still has it available, it will send it along as well.

    3. Abstention A

      The node chose to abstain. By default nodes will never abstain. This can be changed by providing a custom Voter implementation. The argument type A is defined by Communicator::Abstain and Voter::Abstain.

  3. Propose (RoundNum, CoordNum, LogEntry)

    When a node sent a prepare(r, c) request and received a quorum or more of promises in return (counting its own), it will believe itself to be leader for all rounds r... It may now start proposing log entries for any of these rounds, using c as the coordination number.

    The only restriction is that it must respect the promises it has received. If multiple promises contain a triple with the same round number, the one with the greatest coordination number wins. (Triples with the same round and coordination number will have the same log entry as well.)

  4. Vote

    When a node receives a proposal, it will check whether it has made any conflicting promises. If it has, it responds with a conflict. If it hasn't, it may choose to accept or reject the proposal and reply accordingly.

    1. Acceptance Y / Rejection N

      By default nodes accept all proposals with Y = (). This can be changed by providing a custom Voter implementation. The argument types Y and N are defined by Communicator::Yea, Communicator::Nay, Voter::Yea and Voter::Nay.

    2. Conflict (CoordNum, Option<(CoordNum, LogEntry)>)

      See 2.2.

  5. Commit (RoundNum, CoordNum, LogEntry::Id) / CommitById (RoundNum, CoordNum, LogEntry)

    After having received a quorum of acceptances, the round has converged on the proposed entry. The leader node will commit the entry locally and inform other nodes as well. Nodes who sent an acceptance will only be sent the log entry's id, others will receive the full entry.

Project Status

This section examines different aspects of paxakos, how far along they are and what's missing or in need of improvement.

☀️ Consensus Implementation ☀️

The core algorithm of paxakos is well-tested and has been exercised a lot. There is reason to be confident in its correctness.

Passive Mode

[Passive mode][crate::node::Participation] is implemented and superficially tested. Thorough testing is still needed.

Serialization

Snapshot serialization is serde based and still maturing.

API Stability

The API has been fairly stable and there is no reason to expect big changes. The decorations may be reworked so that their configuration can be moved into the State.

Efficiency

Paxakos supports concurrency and has a (for now rudimentary) implementation of master leases. Assuming a scheme to delegate to the current master, this combination is highly efficient.

Future Direction

This is a side project. I will work on the following as my time allows (in no particular order).

  • Tests
  • Adding comments and documentation
  • Rounding off existing decorations
  • Additional decorations, e.g.,
    • for consistency checks
    • for delegation to the current leader
  • Serialization

License: GPL-3.0-only

You might also like...
Easy-to-use beanstalkd client for Rust (IronMQ compatible)

rust-beanstalkd Easy-to-use beanstalkd client for Rust (IronMQ compatible) Install Add this dependency to your Cargo.toml beanstalkd = "*" Documentati

libhdfs binding and wrapper APIs for Rust

hdfs-rs libhdfs binding library and rust APIs which safely wraps libhdfs binding APIs Current Status Alpha Status (Rust wrapping APIs can be changed)

Raft distributed consensus algorithm implemented in Rust.
Raft distributed consensus algorithm implemented in Rust.

Raft Problem and Importance When building a distributed system one principal goal is often to build in fault-tolerance. That is, if one particular nod

🚣‍♀️ 1kloc, well-documented Raft consensus algorithm implementation

miniraft A 1kloc, well-documented Raft consensus algorithm implementation This crate is a minimal implementation of the Raft consensus protocol with

Raft distributed consensus for WebAssembly in Rust

WRaft: Raft in WebAssembly What is this? A toy implementation of the Raft Consensus Algorithm for WebAssembly, written in Rust. Basically, it synchron

The lightest distributed consensus library. Run your own replicated state machine! ❤️

Little Raft The lightest distributed consensus library. Run your own replicated state machine! ❤️ Installing Simply import the crate. In your Cargo.to

The labs of Raft consensus algorithm based on MadSim.

MadRaft The labs of Raft consensus algorithm based on MadSim. Some codes are derived from MIT 6.824 and PingCAP Talent Plan: Raft Lab. Thanks for thei

Damavand is a quantum circuit simulator. It can  run on laptops or High Performance Computing architectures, such CPU distributed architectures or multi GPU distributed architectures.
Damavand is a quantum circuit simulator. It can run on laptops or High Performance Computing architectures, such CPU distributed architectures or multi GPU distributed architectures.

Damavand is a quantum circuit simulator. It can run on laptops or High Performance Computing architectures, such CPU distributed architectures or multi GPU distributed architectures.

open source training courses about distributed database and distributed systemes
open source training courses about distributed database and distributed systemes

Welcome to learn Talent Plan Courses! Talent Plan is an open source training program initiated by PingCAP. It aims to create or combine some open sour

Damavand is a quantum circuit simulator. It can  run on laptops or High Performance Computing architectures, such CPU distributed architectures or multi GPU distributed architectures.
Damavand is a quantum circuit simulator. It can run on laptops or High Performance Computing architectures, such CPU distributed architectures or multi GPU distributed architectures.

Damavand is a code that simulates quantum circuits. In order to learn more about damavand, refer to the documentation. Development status Core feature

An implementation of the FP-Growth algorithm in pure Rust.

fp-growth-rs An implementation of the FP-Growth algorithm in pure Rust, which is inspired by enaeseth/python-fp-growth. Usage Add this to your Cargo.t

Pure Rust implementation of the Double Ratchet algorithm
Pure Rust implementation of the Double Ratchet algorithm

Double Ratchet A pure Rust implementation of the Double Ratchet, as specified by Trevor Perrin and Moxie Marlinspike. The Double Ratchet allows two us

Implementation of Proof of Existence consensus using Substrate Framework, Frame, Pallets, RUST

Substrate Node Template A fresh FRAME-based Substrate node, ready for hacking 🚀 Getting Started Follow the steps below to get started with the Node T

 An implementation of the paper
An implementation of the paper "Honey Badger of BFT Protocols" in Rust. This is a modular library of consensus.

Honey Badger Byzantine Fault Tolerant (BFT) consensus algorithm Welcome to a Rust library of the Honey Badger Byzantine Fault Tolerant (BFT) consensus

Suite for automatically testing algorithm questions from the Polish Algorithm Olympiad.

oisuite Your number #1 tool to managing your algo questions! This software only works on UNIX-based operating systems (macOS, Linux, BSD, etc.) Projec

EVM compatible chain with NPoS/PoC consensus

Reef Chain Reef chain is written in Rust. A basic familiarity with Rust tooling is required. To learn more about Reef chain, please refer to Documenta

Narwhal and Tusk A DAG-based Mempool and Efficient BFT Consensus.

This repo contains a prototype of Narwhal and Tusk. It supplements the paper Narwhal and Tusk: A DAG-based Mempool and Efficient BFT Consensus.

Ed25519 suitable for use in consensus-critical contexts.

Ed25519 for consensus-critical contexts This library provides an Ed25519 implementation with validation rules intended for consensus-critical contexts

Nimbus is a framework for building parachain consensus systems on cumulus-based parachains.

Cumulo -- Nimbus ⛈️ Nimbus is a framework for building parachain consensus systems on cumulus-based parachains. Given the regular six-second pulse-lik

Owner
Pavan Ananth Sharma
Ethereum 2.0
Pavan Ananth Sharma
The lightest distributed consensus library. Run your own replicated state machine! ❤️

Little Raft The lightest distributed consensus library. Run your own replicated state machine! ❤️ Installing Simply import the crate. In your Cargo.to

Ilya Andreev 359 Dec 26, 2022
Fluvio is a high-performance distributed streaming platform that's written in Rust

Fluvio is a high-performance distributed streaming platform that's written in Rust, built to make it easy to develop real-time applications.

InfinyOn 1.6k Dec 30, 2022
Magical Automatic Deterministic Simulator for distributed systems in Rust.

MadSim Magical Automatic Deterministic Simulator for distributed systems. Deterministic simulation MadSim is a Rust async runtime similar to tokio, bu

MadSys Research Group 249 Dec 28, 2022
A model checker for implementing distributed systems.

A model checker for implementing distributed systems.

Stateright Actor Framework 1.3k Dec 15, 2022
Sorock is an experimental "so rocking" scale-out distributed object storage

Sorock is an experimental "so rocking" scale-out distributed object storage

Akira Hayakawa 6 Jun 13, 2022
A universal, distributed package manager

Cask A universal, distributed package manager. Installation | Usage | How to publish package? | Design | Contributing | Cask.toml If you are tired of:

null 39 Dec 30, 2022
The Raft algorithm implement by Rust.

Raft The Raft algorithm implement by Rust. This project refers to Eli Bendersky's website, the link as follows: https://eli.thegreenplace.net/2020/imp

Qiang Zhao 1 Oct 23, 2021
Easy Hadoop Streaming and MapReduce interfaces in Rust

Efflux Efflux is a set of Rust interfaces for MapReduce and Hadoop Streaming. It enables Rust developers to run batch jobs on Hadoop infrastructure wh

Isaac Whitfield 31 Nov 22, 2022
A fully asynchronous, futures-based Kafka client library for Rust based on librdkafka

rust-rdkafka A fully asynchronous, futures-enabled Apache Kafka client library for Rust based on librdkafka. The library rust-rdkafka provides a safe

Federico Giraud 1.1k Jan 8, 2023
Rust client for Apache Kafka

Kafka Rust Client Project Status This project is starting to be maintained by John Ward, the current status is that I am bringing the project up to da

Yousuf Fauzan 902 Jan 2, 2023