├── .github └── workflows │ └── rust.yml ├── .gitignore ├── BENCHMARKING.md ├── Cargo.toml ├── LICENSE ├── README.md └── core ├── Cargo.toml ├── build.rs ├── examples ├── README.md ├── gouge.rs └── gouged.rs ├── proto └── proto.proto └── src ├── database.rs ├── errors.rs ├── lib.rs ├── rpc.rs └── server.rs /.github/workflows/rust.yml: -------------------------------------------------------------------------------- 1 | name: Rust 2 | 3 | on: 4 | push: 5 | branches: [ main ] 6 | pull_request: 7 | branches: [ main ] 8 | 9 | env: 10 | CARGO_TERM_COLOR: always 11 | 12 | jobs: 13 | build: 14 | 15 | runs-on: ubuntu-latest 16 | 17 | steps: 18 | - uses: actions/checkout@v2 19 | - name: Build 20 | run: cargo build --verbose 21 | - name: Run tests 22 | run: cargo test --verbose 23 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | /target 2 | Cargo.lock 3 | -------------------------------------------------------------------------------- /BENCHMARKING.md: -------------------------------------------------------------------------------- 1 | ## Benchmarking ChiselStore 2 | 3 | First download the source code and build the benchmark: 4 | 5 | ``` 6 | mkdir benchmark && cd benchmark 7 | git clone git@github.com:chiselstrike/chiselstore.git 8 | git clone git@github.com:chiselstrike/ycsb-rs.git 9 | cd ycsb-rs 10 | cargo build --release 11 | ``` 12 | 13 | Now we're ready to benchmark! 14 | 15 | Let's start with SQLite. First load some data: 16 | 17 | ``` 18 | ./target/release/ycsb --database sqlite --workload workloads/workloadc.toml load 19 | [OVERALL], ThreadCount, 1 20 | [OVERALL], RunTime(ms), 3825 21 | [OVERALL], Throughput(ops/sec), 261.437908496732 22 | ``` 23 | 24 | Then edit `workloads/workloadc.toml` to perform 1 M operations to make the benchmark run for a longer period of time: 25 | 26 | ``` 27 | operationcount = 1000000 28 | ``` 29 | 30 | and then run the benchmark: 31 | 32 | ``` 33 | ./target/release/ycsb --database sqlite --workload workloads/workloadc.toml --threads 20 run 34 | [OVERALL], ThreadCount, 20 35 | [OVERALL], RunTime(ms), 1288 36 | [OVERALL], Throughput(ops/sec), 776397.5155279503 37 | ``` 38 | 39 | Next, let's benchmark ChiselStore. 40 | 41 | First, start two ChiselStore replicas in the `chiselstore` repository: 42 | 43 | ``` 44 | $ cargo run --release --example gouged -- --id 2 --peers 1 3 45 | $ cargo run --release --example gouged -- --id 3 --peers 1 2 46 | ``` 47 | 48 | Then load data to the ChiselStore cluster (remember to change `operationscount` to 1000 in `workloadc.toml`): 49 | 50 | ``` 51 | ./target/release/ycsb --database chiselstore --workload workloads/workloadc.toml load 52 | [OVERALL], ThreadCount, 1 53 | [OVERALL], RunTime(ms), 7048 54 | [OVERALL], Throughput(ops/sec), 141.88422247446084 55 | ``` 56 | 57 | Finally, bump `operationcount` to `1000000`, and run the benchmark: 58 | 59 | ``` 60 | ./target/release/ycsb --database chiselstore --workload workloads/workloadc.toml run --threads 20 61 | [OVERALL], ThreadCount, 20 62 | [OVERALL], RunTime(ms), 1369 63 | [OVERALL], Throughput(ops/sec), 730460.1899196494 64 | ``` 65 | -------------------------------------------------------------------------------- /Cargo.toml: -------------------------------------------------------------------------------- 1 | [workspace] 2 | resolver = "2" 3 | 4 | members = [ 5 | "core", 6 | ] 7 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright (c) 2021 ChiselStrike Inc. 2 | 3 | Permission is hereby granted, free of charge, to any person obtaining a copy 4 | of this software and associated documentation files (the "Software"), to deal 5 | in the Software without restriction, including without limitation the rights 6 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | copies of the Software, and to permit persons to whom the Software is 8 | furnished to do so, subject to the following conditions: 9 | 10 | The above copyright notice and this permission notice shall be included in 11 | all copies or substantial portions of the Software. 12 | 13 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | THE SOFTWARE. 20 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # ChiselStore 2 | 3 | [![Rust](https://github.com/chiselstrike/chiselstore/actions/workflows/rust.yml/badge.svg)](https://github.com/chiselstrike/chiselstore/actions/workflows/rust.yml) 4 | [![MIT licensed](https://img.shields.io/badge/license-MIT-blue.svg)](./LICENSE) 5 | 6 | ChiselStore is an embeddable, distributed [SQLite](https://www.sqlite.org/index.html) for Rust, powered by [Little Raft](https://github.com/andreev-io/little-raft). 7 | 8 | SQLite is a fast and compact relational database management system, but it is limited to single-node configurations. 9 | ChiselStore extends SQLite to run on a cluster of machines with the [Raft consensus algorithm](https://raft.github.io). 10 | With ChiselStore, you get the benefits of easy-to-use, embeddable SQLite but with Raft's high availability and fault tolerance. 11 | 12 | For more information, check out the following [blog post](https://glaubercosta-11125.medium.com/winds-of-change-in-web-data-728187331f53). 13 | 14 | ## Features 15 | 16 | * SQLite with Raft's high availability and fault tolerance 17 | * Strong consistency with optional relaxed reads 18 | * Embeddable Rust library 19 | 20 | ### Roadmap 21 | 22 | * Efficient node restarts (with Raft snapshots) 23 | * Dynamic cluster membership (with Raft joint consensus) 24 | * Support executing non-deterministic SQL functions 25 | 26 | ## Getting Started 27 | 28 | See the [example server](core/examples) of how to use the ChiselStore library. 29 | 30 | ## License 31 | 32 | This project is licensed under the [MIT license](LICENSE). 33 | 34 | ### Contribution 35 | 36 | Unless you explicitly state otherwise, any contribution intentionally submitted 37 | for inclusion in ChiselStore by you, shall be licensed as MIT, without any additional 38 | terms or conditions. 39 | -------------------------------------------------------------------------------- /core/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "chiselstore" 3 | version = "0.1.0" 4 | edition = "2021" 5 | 6 | [dependencies] 7 | anyhow = { version = "1.0.63", features = ["backtrace"] } 8 | async-mutex = "1.4.0" 9 | async-notify = "0.2.0" 10 | async-trait = "0.1.52" 11 | bytes = "1.1.0" 12 | crossbeam = "0.8.1" 13 | crossbeam-channel = "0.5.1" 14 | derivative = "2.2.0" 15 | little_raft = { git = "https://github.com/chiselstrike/little-raft.git", rev = "58e00c105c25b4aae54a10147ee06449b10fbd88" } 16 | log = "0.4.17" 17 | pretty_env_logger = "0.4.0" 18 | prost = "0.8.0" 19 | sqlite = "0.26.0" 20 | thiserror = "1.0.30" 21 | tokio = { version = "1.11.0", features = ["full"] } 22 | tonic = "0.5.2" 23 | 24 | [build-dependencies] 25 | tonic-build = "0.5.2" 26 | 27 | [dev-dependencies] 28 | structopt = "0.3.25" 29 | -------------------------------------------------------------------------------- /core/build.rs: -------------------------------------------------------------------------------- 1 | fn main() -> std::io::Result<()> { 2 | let proto = "proto/proto.proto"; 3 | tonic_build::compile_protos(proto)?; 4 | println!("cargo:rerun-if-changed={}", proto); 5 | Ok(()) 6 | } 7 | -------------------------------------------------------------------------------- /core/examples/README.md: -------------------------------------------------------------------------------- 1 | # Gouge 2 | 3 | Gouge is a distributed SQL server built on ChiselStore and gRPC. 4 | 5 | ## Getting Started 6 | 7 | Start a cluster of three nodes: 8 | 9 | ``` 10 | cargo run --example gouged -- --id 1 --peers 2 3 11 | cargo run --example gouged -- --id 2 --peers 1 3 12 | cargo run --example gouged -- --id 3 --peers 1 2 13 | ``` 14 | 15 | Then run some SQL commands: 16 | 17 | ``` 18 | cargo run --example gouge 19 | ``` 20 | -------------------------------------------------------------------------------- /core/examples/gouge.rs: -------------------------------------------------------------------------------- 1 | use std::io::Write; 2 | use tokio::io::{AsyncBufReadExt, BufReader}; 3 | 4 | pub mod proto { 5 | tonic::include_proto!("proto"); 6 | } 7 | 8 | use proto::rpc_client::RpcClient; 9 | use proto::{Consistency, Query}; 10 | 11 | #[tokio::main] 12 | async fn main() -> Result<(), Box> { 13 | let stdin = tokio::io::stdin(); 14 | let rdr = BufReader::new(stdin); 15 | let mut lines = rdr.lines(); 16 | print!("gouge=# "); 17 | std::io::stdout().flush().unwrap(); 18 | while let Some(line) = lines.next_line().await? { 19 | let addr = "http://127.0.0.1:50001"; 20 | let mut client = RpcClient::connect(addr).await?; 21 | let query = tonic::Request::new(Query { 22 | sql: line.to_string(), 23 | consistency: Consistency::RelaxedReads as i32, 24 | }); 25 | let response = client.execute(query).await?; 26 | let response = response.into_inner(); 27 | for row in response.rows { 28 | println!("{:?}", row.values); 29 | } 30 | print!("gouge=# "); 31 | std::io::stdout().flush().unwrap(); 32 | } 33 | Ok(()) 34 | } 35 | -------------------------------------------------------------------------------- /core/examples/gouged.rs: -------------------------------------------------------------------------------- 1 | use anyhow::Result; 2 | use chiselstore::{Config, Database}; 3 | use std::sync::Arc; 4 | use structopt::StructOpt; 5 | 6 | #[derive(StructOpt, Debug)] 7 | #[structopt(name = "gouged")] 8 | struct Opt { 9 | /// The ID of this server. 10 | #[structopt(short, long)] 11 | id: usize, 12 | /// The IDs of peers. 13 | #[structopt(short, long, required = false)] 14 | peers: Vec, 15 | } 16 | 17 | /// Node authority (host and port) in the cluster. 18 | fn node_authority(id: usize) -> (&'static str, u16) { 19 | let host = "127.0.0.1"; 20 | let port = 50000 + (id as u16); 21 | (host, port) 22 | } 23 | 24 | #[tokio::main] 25 | async fn main() -> Result<()> { 26 | pretty_env_logger::init(); 27 | let opt = Opt::from_args(); 28 | let config = Config { 29 | id: opt.id, 30 | peers: opt.peers, 31 | node_addr: Arc::new(node_authority), 32 | }; 33 | let db = Database::new(config); 34 | db.run().await?; 35 | Ok(()) 36 | } 37 | -------------------------------------------------------------------------------- /core/proto/proto.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | package proto; 4 | 5 | message Void { 6 | } 7 | 8 | enum Consistency { 9 | STRONG = 0; 10 | RELAXED_READS = 1; 11 | } 12 | 13 | message Query { 14 | string sql = 1; 15 | Consistency consistency = 2; 16 | } 17 | 18 | message QueryResults { 19 | repeated QueryRow rows = 1; 20 | } 21 | 22 | message QueryRow { 23 | repeated string values = 1; 24 | } 25 | 26 | message VoteRequest { 27 | uint64 from_id = 1; 28 | uint64 term = 2; 29 | uint64 last_log_index = 3; 30 | uint64 last_log_term = 4; 31 | } 32 | 33 | message VoteResponse { 34 | uint64 from_id = 1; 35 | uint64 term = 2; 36 | bool vote_granted = 3; 37 | } 38 | 39 | message AppendEntriesRequest { 40 | uint64 from_id = 1; 41 | uint64 term = 2; 42 | uint64 prev_log_index = 3; 43 | uint64 prev_log_term = 4; 44 | repeated LogEntry entries = 5; 45 | uint64 commit_index = 6; 46 | } 47 | 48 | message LogEntry { 49 | uint64 id = 1; 50 | string sql = 2; 51 | uint64 index = 3; 52 | uint64 term = 4; 53 | } 54 | 55 | message AppendEntriesResponse { 56 | uint64 from_id = 1; 57 | uint64 term = 2; 58 | bool success = 3; 59 | uint64 last_index = 4; 60 | optional uint64 mismatch_index = 5; 61 | } 62 | 63 | service RPC { 64 | rpc Execute(Query) returns (QueryResults); 65 | rpc Vote(VoteRequest) returns (Void); 66 | rpc RespondToVote(VoteResponse) returns (Void); 67 | rpc AppendEntries(AppendEntriesRequest) returns (Void); 68 | rpc RespondToAppendEntries(AppendEntriesResponse) returns (Void); 69 | } 70 | -------------------------------------------------------------------------------- /core/src/database.rs: -------------------------------------------------------------------------------- 1 | //! ChiselStore database API. 2 | 3 | use crate::rpc::proto::rpc_server::RpcServer; 4 | use crate::{ 5 | rpc::{NodeAddrFn, RpcService, RpcTransport}, 6 | StoreServer, 7 | }; 8 | use anyhow::Result; 9 | use derivative::Derivative; 10 | use std::sync::Arc; 11 | use tonic::transport::Server; 12 | 13 | /// ChiselStore database configuration. 14 | #[derive(Derivative)] 15 | #[derivative(Debug)] 16 | pub struct Config { 17 | /// The ID of this ChiselStore node. 18 | pub id: usize, 19 | /// The peers of in the ChiselStore cluster. 20 | pub peers: Vec, 21 | /// Node address lookup function. 22 | #[derivative(Debug = "ignore")] 23 | pub node_addr: Arc, 24 | } 25 | 26 | /// ChiselStore database API. 27 | #[derive(Debug)] 28 | pub struct Database { 29 | /// Configuration of this ChiselStore database. 30 | config: Config, 31 | } 32 | 33 | impl Database { 34 | /// Creates a new `Database` object. 35 | pub fn new(config: Config) -> Self { 36 | Self { config } 37 | } 38 | 39 | /// Runs the database main loop. 40 | pub async fn run(&self) -> Result<()> { 41 | let (host, port) = (*self.config.node_addr)(self.config.id); 42 | let rpc_listen_addr = format!("{}:{}", host, port).parse().unwrap(); 43 | let transport = RpcTransport::new(self.config.node_addr.clone()); 44 | let server = StoreServer::start(self.config.id, self.config.peers.clone(), transport)?; 45 | let server = Arc::new(server); 46 | let f = { 47 | let server = server.clone(); 48 | tokio::task::spawn_blocking(move || { 49 | server.run(); 50 | }) 51 | }; 52 | let rpc = RpcService::new(server); 53 | let g = tokio::task::spawn(async move { 54 | println!("RPC listening to {} ...", rpc_listen_addr); 55 | let ret = Server::builder() 56 | .add_service(RpcServer::new(rpc)) 57 | .serve(rpc_listen_addr) 58 | .await; 59 | ret 60 | }); 61 | let results = tokio::try_join!(f, g)?; 62 | results.1?; 63 | Ok(()) 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /core/src/errors.rs: -------------------------------------------------------------------------------- 1 | //! ChiselStore errors. 2 | 3 | use thiserror::Error; 4 | 5 | /// Errors encountered in the store layer. 6 | #[derive(Error, Debug)] 7 | pub enum StoreError { 8 | /// SQLite error. 9 | #[error("SQLite error: {0}")] 10 | SQLiteError(#[from] sqlite::Error), 11 | /// This node is not a leader and cannot therefore execute the command. 12 | #[error("Node is not a leader")] 13 | NotLeader, 14 | } 15 | -------------------------------------------------------------------------------- /core/src/lib.rs: -------------------------------------------------------------------------------- 1 | //! ChiselStore is an embeddable, distributed [SQLite][1] for Rust, powered 2 | //! by [Little Raft][2]. 3 | //! 4 | //! ## Getting Started 5 | //! 6 | //! ChiselStore is a distributed SQLite that you can embed in your 7 | //! application. With ChiselStore, clients (external applications) connect to 8 | //! one of the cluster's nodes to execute SQL statements, such as `CREATE TABLE`, 9 | //! `INSERT` or `SELECT` statements. 10 | //! 11 | //! Under the hood, ChiselStore uses the Raft consensus protocol to replicate 12 | //! the SQL statements to all nodes in the cluster, which apply the statements 13 | //! to their local in-memory SQLite instance. Raft guarantees that all of the 14 | //! SQLite instances in the cluster have identical contents, which allows the 15 | //! cluster to keep operating even if some of the nodes become unavailable. 16 | //! 17 | //! As ChiselStore uses the Raft consensus algorithm, it provides strong 18 | //! consistency (linearizability) by default. SQL statements on a cluster of 19 | //! ChiselStore appear to execute as if there is only one copy of the data 20 | //! because SQL statements execute on the Raft cluster leader node. As strong 21 | //! consistency limits performance, ChiselStore provides an optional 22 | //! consistency [`Consistency::RelaxedReads`] mode, allowing clients to 23 | //! perform read operations on the local node. The relaxed read mode can, 24 | //! however, result in reading stale data so use it with caution. 25 | //! 26 | //! ChiselStore is currently not suitable for production use because it lacks 27 | //! support for Raft snapshots and joint consensus. That is, the replicated 28 | //! log of SQL statements is never truncated, and it is not possible for 29 | //! nodes to join and leave a cluster dynamically. There is, however, a plan 30 | //! to implement support for the missing features to make ChiselStore suitable 31 | //! for production use cases. 32 | //! 33 | //! ChiselStore comes with batteries included and embedding it to your 34 | //! application as simple as: 35 | //! 36 | //! ```no_run 37 | //! use anyhow::Result; 38 | //! use chiselstore::{Config, Database}; 39 | //! use std::sync::Arc; 40 | //! 41 | //! /// Node authority (host and port) in the cluster. 42 | //! fn node_authority(id: usize) -> (&'static str, u16) { 43 | //! let host = "127.0.0.1"; 44 | //! let port = 50000 + (id as u16); 45 | //! (host, port) 46 | //! } 47 | //! 48 | //! #[tokio::main] 49 | //! async fn main() -> Result<()> { 50 | //! // The ID of this node: 51 | //! let id = 1; 52 | //! // A list of IDs of peer nodes: 53 | //! let peers = vec![2, 3]; 54 | //! let config = Config { 55 | //! id: id, 56 | //! peers: peers, 57 | //! node_addr: Arc::new(node_authority), 58 | //! }; 59 | //! let db = Database::new(config); 60 | //! db.run().await?; 61 | //! Ok(()) 62 | //! } 63 | //! ``` 64 | //! 65 | //! [1]: https://www.sqlite.org/index.html 66 | //! [2]: https://github.com/andreev-io/little-raft 67 | 68 | #![warn(missing_docs, missing_debug_implementations, rust_2018_idioms)] 69 | 70 | pub mod database; 71 | pub mod errors; 72 | pub mod rpc; 73 | pub mod server; 74 | 75 | pub use errors::StoreError; 76 | pub use server::Consistency; 77 | pub use server::StoreCommand; 78 | pub use server::StoreServer; 79 | pub use server::StoreTransport; 80 | 81 | pub use database::{Config, Database}; 82 | -------------------------------------------------------------------------------- /core/src/rpc.rs: -------------------------------------------------------------------------------- 1 | //! ChiselStore RPC module. 2 | 3 | use crate::rpc::proto::rpc_server::Rpc; 4 | use crate::{Consistency, StoreCommand, StoreServer, StoreTransport}; 5 | use async_mutex::Mutex; 6 | use async_trait::async_trait; 7 | use bytes::Bytes; 8 | use crossbeam::queue::ArrayQueue; 9 | use derivative::Derivative; 10 | use little_raft::message::Message; 11 | use std::collections::HashMap; 12 | use std::sync::Arc; 13 | use tonic::{Request, Response, Status}; 14 | 15 | #[allow(missing_docs)] 16 | pub mod proto { 17 | tonic::include_proto!("proto"); 18 | } 19 | 20 | use proto::rpc_client::RpcClient; 21 | use proto::{ 22 | AppendEntriesRequest, AppendEntriesResponse, LogEntry, Query, QueryResults, QueryRow, Void, 23 | VoteRequest, VoteResponse, 24 | }; 25 | 26 | /// Node address lookup function. 27 | pub type NodeAddrFn = dyn Fn(usize) -> (&'static str, u16) + Send + Sync; 28 | 29 | #[derive(Debug)] 30 | struct ConnectionPool { 31 | connections: ArrayQueue>, 32 | } 33 | 34 | struct Connection { 35 | conn: RpcClient, 36 | pool: Arc, 37 | } 38 | 39 | impl Drop for Connection { 40 | fn drop(&mut self) { 41 | self.pool.replenish(self.conn.clone()) 42 | } 43 | } 44 | 45 | impl ConnectionPool { 46 | fn new() -> Arc { 47 | Arc::new(Self { 48 | connections: ArrayQueue::new(16), 49 | }) 50 | } 51 | 52 | async fn connection( 53 | &self, 54 | addr: S, 55 | ) -> Result, tonic::transport::Error> { 56 | let addr = addr.to_string(); 57 | match self.connections.pop() { 58 | Some(x) => Ok(x), 59 | None => RpcClient::connect(addr).await, 60 | } 61 | } 62 | 63 | fn replenish(&self, conn: RpcClient) { 64 | let _ = self.connections.push(conn); 65 | } 66 | } 67 | 68 | #[derive(Debug, Clone)] 69 | struct Connections(Arc>>>); 70 | 71 | impl Connections { 72 | fn new() -> Self { 73 | Self(Arc::new(Mutex::new(HashMap::new()))) 74 | } 75 | 76 | async fn connection( 77 | &self, 78 | addr: S, 79 | ) -> Result { 80 | let mut conns = self.0.lock().await; 81 | let addr = addr.to_string(); 82 | let pool = conns 83 | .entry(addr.clone()) 84 | .or_insert_with(ConnectionPool::new); 85 | Ok(Connection { 86 | conn: pool.connection(addr).await?, 87 | pool: pool.clone(), 88 | }) 89 | } 90 | 91 | async fn invalidate(&self, addr: S) { 92 | let mut conns = self.0.lock().await; 93 | let addr = addr.to_string(); 94 | conns.remove(&addr); 95 | } 96 | } 97 | 98 | /// RPC transport. 99 | #[derive(Derivative)] 100 | #[derivative(Debug)] 101 | pub struct RpcTransport { 102 | /// Node address mapping function. 103 | #[derivative(Debug = "ignore")] 104 | node_addr: Arc, 105 | connections: Connections, 106 | } 107 | 108 | impl RpcTransport { 109 | /// Creates a new RPC transport. 110 | pub fn new(node_addr: Arc) -> Self { 111 | RpcTransport { 112 | node_addr, 113 | connections: Connections::new(), 114 | } 115 | } 116 | 117 | /// Node RPC address in cluster. 118 | fn node_rpc_addr(&self, id: usize) -> String { 119 | let (host, port) = (*self.node_addr)(id); 120 | format!("http://{}:{}", host, port) 121 | } 122 | } 123 | 124 | #[async_trait] 125 | impl StoreTransport for RpcTransport { 126 | fn send(&self, to_id: usize, msg: Message) { 127 | match msg { 128 | Message::AppendEntryRequest { 129 | from_id, 130 | term, 131 | prev_log_index, 132 | prev_log_term, 133 | entries, 134 | commit_index, 135 | } => { 136 | let from_id = from_id as u64; 137 | let term = term as u64; 138 | let prev_log_index = prev_log_index as u64; 139 | let prev_log_term = prev_log_term as u64; 140 | let entries = entries 141 | .iter() 142 | .map(|entry| { 143 | let id = entry.transition.id as u64; 144 | let index = entry.index as u64; 145 | let sql = entry.transition.sql.clone(); 146 | let term = entry.term as u64; 147 | LogEntry { 148 | id, 149 | sql, 150 | index, 151 | term, 152 | } 153 | }) 154 | .collect(); 155 | let commit_index = commit_index as u64; 156 | let request = AppendEntriesRequest { 157 | from_id, 158 | term, 159 | prev_log_index, 160 | prev_log_term, 161 | entries, 162 | commit_index, 163 | }; 164 | let peer = self.node_rpc_addr(to_id); 165 | let pool = self.connections.clone(); 166 | tokio::task::spawn(async move { 167 | if let Ok(mut client) = pool.connection(&peer).await { 168 | let request = tonic::Request::new(request.clone()); 169 | if client.conn.append_entries(request).await.is_err() { 170 | pool.invalidate(peer).await 171 | } 172 | } 173 | }); 174 | } 175 | Message::AppendEntryResponse { 176 | from_id, 177 | term, 178 | success, 179 | last_index, 180 | mismatch_index, 181 | } => { 182 | let from_id = from_id as u64; 183 | let term = term as u64; 184 | let last_index = last_index as u64; 185 | let mismatch_index = mismatch_index.map(|idx| idx as u64); 186 | let request = AppendEntriesResponse { 187 | from_id, 188 | term, 189 | success, 190 | last_index, 191 | mismatch_index, 192 | }; 193 | let peer = self.node_rpc_addr(to_id); 194 | let pool = self.connections.clone(); 195 | tokio::task::spawn(async move { 196 | if let Ok(mut client) = pool.connection(&peer).await { 197 | let request = tonic::Request::new(request.clone()); 198 | if client 199 | .conn 200 | .respond_to_append_entries(request) 201 | .await 202 | .is_err() 203 | { 204 | pool.invalidate(peer).await 205 | } 206 | } 207 | }); 208 | } 209 | Message::VoteRequest { 210 | from_id, 211 | term, 212 | last_log_index, 213 | last_log_term, 214 | } => { 215 | let from_id = from_id as u64; 216 | let term = term as u64; 217 | let last_log_index = last_log_index as u64; 218 | let last_log_term = last_log_term as u64; 219 | let request = VoteRequest { 220 | from_id, 221 | term, 222 | last_log_index, 223 | last_log_term, 224 | }; 225 | let peer = self.node_rpc_addr(to_id); 226 | let pool = self.connections.clone(); 227 | tokio::task::spawn(async move { 228 | if let Ok(mut client) = pool.connection(&peer).await { 229 | let vote = tonic::Request::new(request.clone()); 230 | if client.conn.vote(vote).await.is_err() { 231 | pool.invalidate(peer).await 232 | } 233 | } 234 | }); 235 | } 236 | Message::VoteResponse { 237 | from_id, 238 | term, 239 | vote_granted, 240 | } => { 241 | let peer = self.node_rpc_addr(to_id); 242 | tokio::task::spawn(async move { 243 | let from_id = from_id as u64; 244 | let term = term as u64; 245 | let response = VoteResponse { 246 | from_id, 247 | term, 248 | vote_granted, 249 | }; 250 | if let Ok(mut client) = RpcClient::connect(peer.to_string()).await { 251 | let response = tonic::Request::new(response.clone()); 252 | client.respond_to_vote(response).await.unwrap(); 253 | } 254 | }); 255 | } 256 | Message::InstallSnapshotRequest { .. } => { 257 | todo!("Snapshotting is not implemented."); 258 | } 259 | Message::InstallSnapshotResponse { .. } => { 260 | todo!("Snapshotting is not implemented."); 261 | } 262 | } 263 | } 264 | 265 | async fn delegate( 266 | &self, 267 | to_id: usize, 268 | sql: String, 269 | consistency: Consistency, 270 | ) -> Result { 271 | let addr = self.node_rpc_addr(to_id); 272 | let mut client = self.connections.connection(addr.clone()).await.unwrap(); 273 | let query = tonic::Request::new(Query { 274 | sql, 275 | consistency: consistency as i32, 276 | }); 277 | let response = client.conn.execute(query).await.unwrap(); 278 | let response = response.into_inner(); 279 | let mut rows = vec![]; 280 | for row in response.rows { 281 | rows.push(crate::server::QueryRow { values: row.values }); 282 | } 283 | Ok(crate::server::QueryResults { rows }) 284 | } 285 | } 286 | 287 | /// RPC service. 288 | #[derive(Debug)] 289 | pub struct RpcService { 290 | /// The ChiselStore server access via this RPC service. 291 | pub server: Arc>, 292 | } 293 | 294 | impl RpcService { 295 | /// Creates a new RPC service. 296 | pub fn new(server: Arc>) -> Self { 297 | Self { server } 298 | } 299 | } 300 | 301 | #[tonic::async_trait] 302 | impl Rpc for RpcService { 303 | async fn execute( 304 | &self, 305 | request: Request, 306 | ) -> Result, tonic::Status> { 307 | let query = request.into_inner(); 308 | let consistency = 309 | proto::Consistency::from_i32(query.consistency).unwrap_or(proto::Consistency::Strong); 310 | let consistency = match consistency { 311 | proto::Consistency::Strong => Consistency::Strong, 312 | proto::Consistency::RelaxedReads => Consistency::RelaxedReads, 313 | }; 314 | let server = self.server.clone(); 315 | let results = match server.query(query.sql, consistency).await { 316 | Ok(results) => results, 317 | Err(e) => return Err(Status::internal(format!("{}", e))), 318 | }; 319 | let mut rows = vec![]; 320 | for row in results.rows { 321 | rows.push(QueryRow { 322 | values: row.values.clone(), 323 | }) 324 | } 325 | Ok(Response::new(QueryResults { rows })) 326 | } 327 | 328 | async fn vote(&self, request: Request) -> Result, tonic::Status> { 329 | let msg = request.into_inner(); 330 | let from_id = msg.from_id as usize; 331 | let term = msg.term as usize; 332 | let last_log_index = msg.last_log_index as usize; 333 | let last_log_term = msg.last_log_term as usize; 334 | let msg = little_raft::message::Message::VoteRequest { 335 | from_id, 336 | term, 337 | last_log_index, 338 | last_log_term, 339 | }; 340 | let server = self.server.clone(); 341 | server.recv_msg(msg); 342 | Ok(Response::new(Void {})) 343 | } 344 | 345 | async fn respond_to_vote( 346 | &self, 347 | request: Request, 348 | ) -> Result, tonic::Status> { 349 | let msg = request.into_inner(); 350 | let from_id = msg.from_id as usize; 351 | let term = msg.term as usize; 352 | let vote_granted = msg.vote_granted; 353 | let msg = little_raft::message::Message::VoteResponse { 354 | from_id, 355 | term, 356 | vote_granted, 357 | }; 358 | let server = self.server.clone(); 359 | server.recv_msg(msg); 360 | Ok(Response::new(Void {})) 361 | } 362 | 363 | async fn append_entries( 364 | &self, 365 | request: Request, 366 | ) -> Result, tonic::Status> { 367 | let msg = request.into_inner(); 368 | let from_id = msg.from_id as usize; 369 | let term = msg.term as usize; 370 | let prev_log_index = msg.prev_log_index as usize; 371 | let prev_log_term = msg.prev_log_term as usize; 372 | let entries: Vec> = msg 373 | .entries 374 | .iter() 375 | .map(|entry| { 376 | let id = entry.id as usize; 377 | let sql = entry.sql.to_string(); 378 | let transition = StoreCommand { id, sql }; 379 | let index = entry.index as usize; 380 | let term = entry.term as usize; 381 | little_raft::message::LogEntry { 382 | transition, 383 | index, 384 | term, 385 | } 386 | }) 387 | .collect(); 388 | let commit_index = msg.commit_index as usize; 389 | let msg = little_raft::message::Message::AppendEntryRequest { 390 | from_id, 391 | term, 392 | prev_log_index, 393 | prev_log_term, 394 | entries, 395 | commit_index, 396 | }; 397 | let server = self.server.clone(); 398 | server.recv_msg(msg); 399 | Ok(Response::new(Void {})) 400 | } 401 | 402 | async fn respond_to_append_entries( 403 | &self, 404 | request: tonic::Request, 405 | ) -> Result, tonic::Status> { 406 | let msg = request.into_inner(); 407 | let from_id = msg.from_id as usize; 408 | let term = msg.term as usize; 409 | let success = msg.success; 410 | let last_index = msg.last_index as usize; 411 | let mismatch_index = msg.mismatch_index.map(|idx| idx as usize); 412 | let msg = little_raft::message::Message::AppendEntryResponse { 413 | from_id, 414 | term, 415 | success, 416 | last_index, 417 | mismatch_index, 418 | }; 419 | let server = self.server.clone(); 420 | server.recv_msg(msg); 421 | Ok(Response::new(Void {})) 422 | } 423 | } 424 | -------------------------------------------------------------------------------- /core/src/server.rs: -------------------------------------------------------------------------------- 1 | //! ChiselStore server module. 2 | 3 | use crate::errors::StoreError; 4 | use async_notify::Notify; 5 | use async_trait::async_trait; 6 | use bytes::Bytes; 7 | use crossbeam_channel as channel; 8 | use crossbeam_channel::{Receiver, Sender}; 9 | use derivative::Derivative; 10 | use little_raft::{ 11 | cluster::Cluster, 12 | message::Message, 13 | replica::{Replica, ReplicaID}, 14 | state_machine::{Snapshot, StateMachine, StateMachineTransition, TransitionState}, 15 | }; 16 | use log::debug; 17 | use sqlite::{Connection, OpenFlags}; 18 | use std::collections::HashMap; 19 | use std::sync::atomic::{AtomicBool, AtomicU64, Ordering}; 20 | use std::sync::{Arc, Mutex}; 21 | use std::time::Duration; 22 | 23 | /// ChiselStore transport layer. 24 | /// 25 | /// Your application should implement this trait to provide network access 26 | /// to the ChiselStore server. 27 | #[async_trait] 28 | pub trait StoreTransport { 29 | /// Send a store command message `msg` to `to_id` node. 30 | fn send(&self, to_id: usize, msg: Message); 31 | 32 | /// Delegate command to another node. 33 | async fn delegate( 34 | &self, 35 | to_id: usize, 36 | sql: String, 37 | consistency: Consistency, 38 | ) -> Result; 39 | } 40 | 41 | /// Consistency mode. 42 | #[derive(Debug)] 43 | pub enum Consistency { 44 | /// Strong consistency. Both reads and writes go through the Raft leader, 45 | /// which makes them linearizable. 46 | Strong, 47 | /// Relaxed reads. Reads are performed on the local node, which relaxes 48 | /// read consistency and allows stale reads. 49 | RelaxedReads, 50 | } 51 | 52 | /// Store command. 53 | /// 54 | /// A store command is a SQL statement that is replicated in the Raft cluster. 55 | #[derive(Clone, Debug)] 56 | pub struct StoreCommand { 57 | /// Unique ID of this command. 58 | pub id: usize, 59 | /// The SQL statement of this command. 60 | pub sql: String, 61 | } 62 | 63 | impl StateMachineTransition for StoreCommand { 64 | type TransitionID = usize; 65 | 66 | fn get_id(&self) -> Self::TransitionID { 67 | self.id 68 | } 69 | } 70 | 71 | /// Store configuration. 72 | #[derive(Debug)] 73 | struct StoreConfig { 74 | /// Connection pool size. 75 | conn_pool_size: usize, 76 | } 77 | 78 | #[derive(Derivative)] 79 | #[derivative(Debug)] 80 | struct Store { 81 | /// ID of the node this Cluster objecti s on. 82 | this_id: usize, 83 | /// Is this node the leader? 84 | leader: Option, 85 | leader_exists: AtomicBool, 86 | waiters: Vec>, 87 | /// Pending messages 88 | pending_messages: Vec>, 89 | /// Transport layer. 90 | transport: Arc, 91 | #[derivative(Debug = "ignore")] 92 | conn_pool: Vec>>, 93 | conn_idx: usize, 94 | pending_transitions: Vec, 95 | command_completions: HashMap>, 96 | results: HashMap>, 97 | } 98 | 99 | impl Store { 100 | pub fn new(this_id: usize, transport: T, config: StoreConfig) -> Self { 101 | let mut conn_pool = vec![]; 102 | let conn_pool_size = config.conn_pool_size; 103 | for _ in 0..conn_pool_size { 104 | // FIXME: Let's use the 'memdb' VFS of SQLite, which allows concurrent threads 105 | // accessing the same in-memory database. 106 | let flags = OpenFlags::new() 107 | .set_read_write() 108 | .set_create() 109 | .set_no_mutex(); 110 | let mut conn = 111 | Connection::open_with_flags(format!("node{}.db", this_id), flags).unwrap(); 112 | conn.set_busy_timeout(5000).unwrap(); 113 | conn_pool.push(Arc::new(Mutex::new(conn))); 114 | } 115 | let conn_idx = 0; 116 | Store { 117 | this_id, 118 | leader: None, 119 | leader_exists: AtomicBool::new(false), 120 | waiters: Vec::new(), 121 | pending_messages: Vec::new(), 122 | transport: Arc::new(transport), 123 | conn_pool, 124 | conn_idx, 125 | pending_transitions: Vec::new(), 126 | command_completions: HashMap::new(), 127 | results: HashMap::new(), 128 | } 129 | } 130 | 131 | pub fn is_leader(&self) -> bool { 132 | match self.leader { 133 | Some(id) => id == self.this_id, 134 | _ => false, 135 | } 136 | } 137 | 138 | pub fn get_connection(&mut self) -> Arc> { 139 | let idx = self.conn_idx % self.conn_pool.len(); 140 | let conn = &self.conn_pool[idx]; 141 | self.conn_idx += 1; 142 | conn.clone() 143 | } 144 | } 145 | 146 | fn query(conn: Arc>, sql: String) -> Result { 147 | let conn = conn.lock().unwrap(); 148 | let mut rows = vec![]; 149 | conn.iterate(sql, |pairs| { 150 | let mut row = QueryRow::new(); 151 | for &(_, value) in pairs.iter() { 152 | row.values.push(value.unwrap().to_string()); 153 | } 154 | rows.push(row); 155 | true 156 | })?; 157 | Ok(QueryResults { rows }) 158 | } 159 | 160 | impl StateMachine for Store { 161 | fn register_transition_state(&mut self, transition_id: usize, state: TransitionState) { 162 | debug!( 163 | "Registering transition state: {} -> {:?}", 164 | transition_id, state 165 | ); 166 | match state { 167 | TransitionState::Applied | TransitionState::Abandoned(_) => { 168 | if let Some(completion) = self.command_completions.remove(&(transition_id as u64)) { 169 | completion.notify(); 170 | } 171 | } 172 | _ => (), 173 | } 174 | } 175 | 176 | fn apply_transition(&mut self, transition: StoreCommand) { 177 | if transition.id == NOP_TRANSITION_ID { 178 | return; 179 | } 180 | let conn = self.get_connection(); 181 | let results = query(conn, transition.sql); 182 | if self.is_leader() { 183 | self.results.insert(transition.id as u64, results); 184 | } 185 | } 186 | 187 | fn get_pending_transitions(&mut self) -> Vec { 188 | let cur = self.pending_transitions.clone(); 189 | self.pending_transitions = Vec::new(); 190 | cur 191 | } 192 | 193 | fn get_snapshot(&mut self) -> Option> { 194 | None 195 | } 196 | 197 | fn create_snapshot(&mut self, _index: usize, _term: usize) -> Snapshot { 198 | todo!("Snapshotting is not implemented."); 199 | } 200 | 201 | fn set_snapshot(&mut self, _snapshot: Snapshot) { 202 | todo!("Snapshotting is not implemented."); 203 | } 204 | } 205 | 206 | impl Cluster for Store { 207 | fn register_leader(&mut self, leader_id: Option) { 208 | if let Some(id) = leader_id { 209 | debug!("Registering {} as leader at node {}", id, self.this_id); 210 | self.leader = Some(id); 211 | self.leader_exists.store(true, Ordering::SeqCst); 212 | } else { 213 | debug!("No leader leader at node {}", self.this_id); 214 | self.leader = None; 215 | self.leader_exists.store(false, Ordering::SeqCst); 216 | } 217 | let waiters = self.waiters.clone(); 218 | self.waiters = Vec::new(); 219 | for waiter in waiters { 220 | waiter.notify(); 221 | } 222 | } 223 | 224 | fn send_message(&mut self, to_id: usize, message: Message) { 225 | debug!( 226 | "Node {} sending message to {} -> {:?}", 227 | self.this_id, to_id, message 228 | ); 229 | self.transport.send(to_id, message); 230 | } 231 | 232 | fn receive_messages(&mut self) -> Vec> { 233 | let cur = self.pending_messages.clone(); 234 | self.pending_messages = Vec::new(); 235 | debug!("Node {} received message(s) -> {:?}", self.this_id, cur); 236 | cur 237 | } 238 | 239 | fn halt(&self) -> bool { 240 | false 241 | } 242 | } 243 | 244 | type StoreReplica = Replica, Store, StoreCommand, Bytes>; 245 | 246 | /// ChiselStore server. 247 | #[derive(Derivative)] 248 | #[derivative(Debug)] 249 | pub struct StoreServer { 250 | next_cmd_id: AtomicU64, 251 | store: Arc>>, 252 | #[derivative(Debug = "ignore")] 253 | replica: Arc>>, 254 | message_notifier_rx: Receiver<()>, 255 | message_notifier_tx: Sender<()>, 256 | transition_notifier_rx: Receiver<()>, 257 | transition_notifier_tx: Sender<()>, 258 | } 259 | 260 | /// Query row. 261 | #[derive(Debug)] 262 | pub struct QueryRow { 263 | /// Column values of the row. 264 | pub values: Vec, 265 | } 266 | 267 | impl QueryRow { 268 | fn new() -> Self { 269 | QueryRow { values: Vec::new() } 270 | } 271 | } 272 | 273 | /// Query results. 274 | #[derive(Debug)] 275 | pub struct QueryResults { 276 | /// Query result rows. 277 | pub rows: Vec, 278 | } 279 | 280 | const NOP_TRANSITION_ID: usize = 0; 281 | const HEARTBEAT_TIMEOUT: Duration = Duration::from_millis(500); 282 | const MIN_ELECTION_TIMEOUT: Duration = Duration::from_millis(750); 283 | const MAX_ELECTION_TIMEOUT: Duration = Duration::from_millis(950); 284 | 285 | impl StoreServer { 286 | /// Start a new server as part of a ChiselStore cluster. 287 | pub fn start(this_id: usize, peers: Vec, transport: T) -> Result { 288 | let config = StoreConfig { conn_pool_size: 20 }; 289 | let store = Arc::new(Mutex::new(Store::new(this_id, transport, config))); 290 | let noop = StoreCommand { 291 | id: NOP_TRANSITION_ID, 292 | sql: "".to_string(), 293 | }; 294 | let (message_notifier_tx, message_notifier_rx) = channel::unbounded(); 295 | let (transition_notifier_tx, transition_notifier_rx) = channel::unbounded(); 296 | let replica = Replica::new( 297 | this_id, 298 | peers, 299 | store.clone(), 300 | store.clone(), 301 | 0, // snapshotting is disabled 302 | noop, 303 | HEARTBEAT_TIMEOUT, 304 | (MIN_ELECTION_TIMEOUT, MAX_ELECTION_TIMEOUT), 305 | ); 306 | let replica = Arc::new(Mutex::new(replica)); 307 | Ok(StoreServer { 308 | next_cmd_id: AtomicU64::new(1), // zero is reserved for no-op. 309 | store, 310 | replica, 311 | message_notifier_rx, 312 | message_notifier_tx, 313 | transition_notifier_rx, 314 | transition_notifier_tx, 315 | }) 316 | } 317 | 318 | /// Run the blocking event loop. 319 | pub fn run(&self) { 320 | self.replica.lock().unwrap().start( 321 | self.message_notifier_rx.clone(), 322 | self.transition_notifier_rx.clone(), 323 | ); 324 | } 325 | 326 | /// Execute a SQL statement on the ChiselStore cluster. 327 | pub async fn query>( 328 | &self, 329 | stmt: S, 330 | consistency: Consistency, 331 | ) -> Result { 332 | // If the statement is a read statement, let's use whatever 333 | // consistency the user provided; otherwise fall back to strong 334 | // consistency. 335 | let consistency = if is_read_statement(stmt.as_ref()) { 336 | consistency 337 | } else { 338 | Consistency::Strong 339 | }; 340 | let results = match consistency { 341 | Consistency::Strong => { 342 | self.wait_for_leader().await; 343 | let (delegate, leader, transport) = { 344 | let store = self.store.lock().unwrap(); 345 | (!store.is_leader(), store.leader, store.transport.clone()) 346 | }; 347 | if delegate { 348 | if let Some(leader_id) = leader { 349 | return transport 350 | .delegate(leader_id, stmt.as_ref().to_string(), consistency) 351 | .await; 352 | } 353 | return Err(StoreError::NotLeader); 354 | } 355 | let (notify, id) = { 356 | let mut store = self.store.lock().unwrap(); 357 | let id = self.next_cmd_id.fetch_add(1, Ordering::SeqCst); 358 | let cmd = StoreCommand { 359 | id: id as usize, 360 | sql: stmt.as_ref().to_string(), 361 | }; 362 | let notify = Arc::new(Notify::new()); 363 | store.command_completions.insert(id, notify.clone()); 364 | store.pending_transitions.push(cmd); 365 | (notify, id) 366 | }; 367 | self.transition_notifier_tx.send(()).unwrap(); 368 | notify.notified().await; 369 | if let Some(results) = self.store.lock().unwrap().results.remove(&id) { 370 | results? 371 | } else { 372 | return Err(StoreError::NotLeader); 373 | } 374 | } 375 | Consistency::RelaxedReads => { 376 | let conn = { 377 | let mut store = self.store.lock().unwrap(); 378 | store.get_connection() 379 | }; 380 | query(conn, stmt.as_ref().to_string())? 381 | } 382 | }; 383 | Ok(results) 384 | } 385 | 386 | /// Wait for a leader to be elected. 387 | pub async fn wait_for_leader(&self) { 388 | loop { 389 | let notify = { 390 | let mut store = self.store.lock().unwrap(); 391 | if store.leader_exists.load(Ordering::SeqCst) { 392 | break; 393 | } 394 | let notify = Arc::new(Notify::new()); 395 | store.waiters.push(notify.clone()); 396 | notify 397 | }; 398 | if self 399 | .store 400 | .lock() 401 | .unwrap() 402 | .leader_exists 403 | .load(Ordering::SeqCst) 404 | { 405 | break; 406 | } 407 | // TODO: add a timeout and fail if necessary 408 | notify.notified().await; 409 | } 410 | } 411 | 412 | /// Receive a message from the ChiselStore cluster. 413 | pub fn recv_msg(&self, msg: little_raft::message::Message) { 414 | let mut cluster = self.store.lock().unwrap(); 415 | cluster.pending_messages.push(msg); 416 | self.message_notifier_tx.send(()).unwrap(); 417 | } 418 | } 419 | 420 | fn is_read_statement(stmt: &str) -> bool { 421 | stmt.to_lowercase().starts_with("select") 422 | } 423 | --------------------------------------------------------------------------------