Showing preview only (653K chars total). Download the full file or copy to clipboard to get everything.
Repository: ShisoftResearch/bifrost
Branch: develop
Commit: b826201b0482
Files: 48
Total size: 628.3 KB
Directory structure:
gitextract_g24w9kwe/
├── .gitignore
├── .travis.yml
├── Cargo.toml
├── LICENSE
├── MEMBERSHIP_GUIDE.md
├── README.md
├── RECOVERY_IMPROVEMENTS.md
├── SNAPSHOT_GUIDE.md
├── TYPE2_LAZY_LOADING_CONTRACT.md
├── examples/
│ └── graceful_shutdown.rs
├── src/
│ ├── conshash/
│ │ ├── mod.rs
│ │ └── weights.rs
│ ├── hasher/
│ │ ├── Cargo.toml
│ │ └── src/
│ │ └── lib.rs
│ ├── lib.rs
│ ├── membership/
│ │ ├── client.rs
│ │ ├── member.rs
│ │ ├── mod.rs
│ │ └── server.rs
│ ├── plugins/
│ │ ├── Cargo.toml
│ │ └── src/
│ │ └── lib.rs
│ ├── proc_macro/
│ │ ├── Cargo.toml
│ │ └── src/
│ │ └── lib.rs
│ ├── raft/
│ │ ├── client.rs
│ │ ├── disk.rs
│ │ ├── mod.rs
│ │ └── state_machine/
│ │ ├── callback/
│ │ │ ├── client.rs
│ │ │ ├── mod.rs
│ │ │ └── server.rs
│ │ ├── configs.rs
│ │ ├── macros.rs
│ │ ├── master.rs
│ │ └── mod.rs
│ ├── rpc/
│ │ ├── cluster.rs
│ │ ├── mod.rs
│ │ └── proto.rs
│ ├── tcp/
│ │ ├── client.rs
│ │ ├── mod.rs
│ │ ├── server.rs
│ │ └── shortcut.rs
│ ├── utils/
│ │ ├── bindings.rs
│ │ ├── math.rs
│ │ ├── mod.rs
│ │ ├── serde.rs
│ │ └── time.rs
│ └── vector_clock/
│ └── mod.rs
└── tests/
├── graceful_shutdown_tests.rs
└── single_node_recovery_test.rs
================================================
FILE CONTENTS
================================================
================================================
FILE: .gitignore
================================================
target
Cargo.lock
.idea/
bifrost.iml
================================================
FILE: .travis.yml
================================================
language: rust
rust:
- nightly
================================================
FILE: Cargo.toml
================================================
[package]
name = "bifrost"
version = "0.1.0"
authors = ["Hao Shi <shisoftgenius@gmail.com>"]
edition = "2018"
[lib]
name = "bifrost"
[dependencies]
serde_cbor = "0.11.1"
serde_json = "1.0.51"
byteorder = "1"
log = "*"
serde = { version = "1.0", features = ["derive"] }
bifrost_plugins = { path = "src/plugins" }
bifrost_hasher = { path = "src/hasher" }
bifrost_proc_macro = { path = "src/proc_macro" }
rand = "*"
lazy_static = "*"
threadpool = "1"
num_cpus = "1"
parking_lot = {version = "*", features = ["nightly"]}
thread-id = "5"
tokio = { version = "1", features = ["full"] }
tokio-util = {version = "0.7", features = ["full"]}
tokio-stream = "0.1"
bytes = "1"
crc32fast = "*"
futures = {version = "0.3", features = ["executor", "thread-pool"] }
futures-timer = "3"
async-std = "1"
lightning-containers = { git = "ssh://git@192.168.10.134/shisoft-x/Lightning.git", branch = "develop" }
[dev-dependencies]
env_logger = "*"
================================================
FILE: LICENSE
================================================
MIT License
Copyright (c) 2020 Hao Shi
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all
copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
SOFTWARE.
================================================
FILE: MEMBERSHIP_GUIDE.md
================================================
# Membership Guide
This document explains how membership works in Bifrost and the difference between **Raft Cluster Membership** and the **Membership Service**.
## Two Types of Membership
Bifrost has two distinct membership systems that serve different purposes:
### 1. Raft Cluster Membership (PERSISTED ✅)
**Location**: `src/raft/state_machine/configs.rs`
**Purpose**: Tracks which servers are part of the Raft consensus cluster
**Persistence**: **YES** - Fully persisted to disk via:
- Write-Ahead Log (WAL)
- Snapshots
**Members**: Raft servers that participate in consensus (leader election, log replication)
**Operations**:
- `new_member_(address)` - Add a Raft server to the cluster
- `del_member_(address)` - Remove a Raft server from the cluster
- `member_address()` - Query all Raft cluster members
**Recovery**: On restart, Raft cluster membership is recovered from:
1. Latest snapshot on disk
2. WAL log replay
**Why Persisted?**: Critical for Raft consensus. The cluster must know its membership to:
- Calculate quorum (majority)
- Elect leaders
- Replicate logs correctly
**Code Example**:
```rust
// These members are persisted and recovered on restart
service.join(&vec!["node1:5000".to_string()]).await;
```
### 2. Membership Service (NOT PERSISTED ❌)
**Location**: `src/membership/server.rs`
**Purpose**: Tracks member groups, heartbeat status, and online/offline state
**Persistence**: **NO** - Intentionally ephemeral
**Members**: Applications or clients using the membership service for:
- Group membership
- Leader election within groups
- Liveness tracking
- Membership change notifications
**Operations**:
- `join(address)` - Join as a member
- `leave(id)` - Leave the membership service
- `join_group(group_name, id)` - Join a group
- `leave_group(group, id)` - Leave a group
- `ping(id)` - Send heartbeat
**Recovery**: On restart, starts with **empty state** and rebuilds through:
1. Members calling `join()` again
2. Heartbeat `ping()` messages
3. Group operations
**Why NOT Persisted?**:
- Membership should reflect **current network reality**
- Stale disk state would be misleading after crashes
- Members must actively rejoin to prove they're alive
- Groups are transient application-level constructs
**Code Example**:
```rust
// After restart, this state is gone - members must rejoin
let client = MemberClient::new(...).await;
client.join().await; // Must be called again after restart
client.join_group("workers".to_string()).await;
```
## Comparison Table
| Feature | Raft Cluster Membership | Membership Service |
|---------|------------------------|-------------------|
| **Persisted** | ✅ Yes (WAL + Snapshot) | ❌ No (Always fresh) |
| **Purpose** | Raft consensus | Application groups/heartbeats |
| **Scope** | Cluster-wide | Per-service |
| **Recovery** | From disk | From network rediscovery |
| **State Machine ID** | `CONFIG_SM_ID` (1) | `DEFAULT_SERVICE_ID` |
| **Critical for Raft** | ✅ Yes | ❌ No |
| **Survives Restart** | ✅ Yes | ❌ No |
## How They Work Together
```
┌─────────────────────────────────────────────────────────┐
│ Bifrost Cluster │
├─────────────────────────────────────────────────────────┤
│ │
│ Raft Cluster Membership (Persisted) │
│ ┌─────────┐ ┌─────────┐ ┌─────────┐ │
│ │ Server1 │ │ Server2 │ │ Server3 │ │
│ │ :5000 │ │ :5001 │ │ :5002 │ │
│ └────┬────┘ └────┬────┘ └────┬────┘ │
│ │ │ │ │
│ ├────────────┴────────────┤ │
│ │ Raft Consensus │ │
│ │ (Leader Election, │ │
│ │ Log Replication) │ │
│ └─────────────────────────┘ │
│ │
│ Membership Service (NOT Persisted - Fresh on restart) │
│ ┌─────────────────────────────────────────────┐ │
│ │ Members: { │ │
│ │ App1 -> online, groups: [workers] │ │
│ │ App2 -> online, groups: [workers] │ │
│ │ App3 -> offline, groups: [storage] │ │
│ │ } │ │
│ │ Groups: { │ │
│ │ workers -> leader: App1, members: [1,2] │ │
│ │ storage -> leader: None, members: [3] │ │
│ │ } │ │
│ └─────────────────────────────────────────────┘ │
│ ↑ This is cleared on restart │
└─────────────────────────────────────────────────────────┘
```
## Startup Sequence After Restart
### Raft Cluster Membership (Automatic)
```rust
// Server restarts
let raft_service = RaftService::new(options);
RaftService::start(&raft_service).await;
// ✅ Cluster membership automatically recovered from disk
// ✅ Knows about Server1, Server2, Server3
// ✅ Can participate in consensus immediately
```
### Membership Service (Manual Rejoin Required)
```rust
// Server restarts
let membership_client = MemberClient::new(...).await;
// ❌ Membership service starts EMPTY
// ❌ Previous groups/members are forgotten
// ✅ Applications must rejoin
membership_client.join().await; // Rejoin as member
membership_client.join_group("workers").await; // Rejoin group
membership_client.start_heartbeat(); // Start sending pings
// ✅ Membership service rebuilds state from these actions
```
## Why This Design?
### Raft Cluster Membership: Persisted
- **Safety**: Raft consensus requires consistent membership for quorum
- **Correctness**: Must survive crashes to maintain cluster integrity
- **Availability**: Cluster can restart without manual intervention
### Membership Service: NOT Persisted
- **Freshness**: Ensures membership reflects current reality
- **Simplicity**: No stale data to reconcile
- **Self-Healing**: Dead members naturally disappear (no heartbeat = offline)
- **Flexibility**: Applications control their own membership lifecycle
## Code Examples
### Example 1: Raft Member Survives Restart
```rust
// Initial setup
let raft = RaftService::new(Options {
storage: Storage::DISK(disk_opts),
address: "node1:5000".to_string(),
service_id: DEFAULT_SERVICE_ID,
});
raft.join(&vec!["node2:5000".to_string()]).await;
// --- CRASH AND RESTART ---
// After restart
let raft = RaftService::new(Options {
storage: Storage::DISK(disk_opts), // Same disk path
address: "node1:5000".to_string(),
service_id: DEFAULT_SERVICE_ID,
});
RaftService::start(&raft).await;
// ✅ Still knows about node2:5000 (recovered from disk)
// ✅ Can participate in cluster immediately
```
### Example 2: Membership Service Starts Fresh
```rust
// Initial setup
let member = MemberClient::new(...).await;
member.join().await;
member.join_group("workers").await;
// --- CRASH AND RESTART ---
// After restart
let member = MemberClient::new(...).await;
// ❌ Not in any groups
// ❌ Not registered as a member
// Must rejoin explicitly
member.join().await; // Required
member.join_group("workers").await; // Required
member.start_heartbeat(); // Required
```
## Best Practices
### For Raft Cluster Members
1. Use `Storage::DISK` for production deployments
2. Membership changes are committed via Raft consensus
3. No need to rejoin after restart
### For Membership Service Users
1. **Always rejoin after restart**:
```rust
async fn on_startup() {
member_client.join().await;
for group in my_groups {
member_client.join_group(group).await;
}
member_client.start_heartbeat();
}
```
2. **Handle disconnections gracefully** - May need to rejoin
3. **Monitor membership changes** via subscriptions:
```rust
client.on_any_member_joined(|member, version| {
println!("New member joined: {:?}", member);
}).await;
```
## Summary
- **Raft Cluster Membership**: Persisted for consensus correctness ✅
- **Membership Service**: NOT persisted, learns from network ❌
- Both serve different purposes and have different persistence requirements
- This design ensures both safety (for Raft) and freshness (for membership)
================================================
FILE: README.md
================================================
# bifrost
[](https://travis-ci.org/ShisoftResearch/bifrost)
Pure rust building block for distributed systems
### Objective
The objective of bifrost is to build a solid foundation for distributed systems in rust.
It is similar to one of my Clojure project [cluster-connecter](https://github.com/shisoft/cluster-connector), but no longer require any third-party software like Zookeeper or etcd.
Bifrost will ship with it's own reliable data store based on [raft consensus algorithm](https://raft.github.io/) state machines. Users are also able to build their own reliable data structures by implementing state machine commands.
**Bifrost is still in very early stage of development and it is not suggested to be used in any kinds of projects until it is stabilized and fully tested**
### Progress Check List
- [ ] RPC
- [x] TCP Server
- [x] Protocol
- [x] Event driven server
- [x] Sync client
- [x] Async client
- [X] Multiplexing pluggable services
- [X] Shortcut (for both TCP and RPC APIs)
- [ ] Raft (data replication)
- [x] Leader election
- [x] Log replication
- [x] Master/subs state machine framework
- [ ] State machine client
- [x] Sync
- [x] PubSub
- [ ] Master state machine snapshot
- [x] Generate
- [x] Install
- [ ] Generate in chunks
- [ ] Install in chunks
- [ ] Automation
- [ ] Persistent to disk
- [ ] Recover from disk
- [ ] Incremental snapshot
- [ ] Membership changes
- [x] State machine
- [x] New Member
- [x] Delete Member
- [x] Snapshot
- [x] Recover
- [X] Interfaces
- [X] Update procedures
- [x] Cluster bootstrap
- [x] Client
- [x] Command
- [x] Query
- [x] Concurrency
- [x] Failover
- [x] Membership changes
- [x] Subscription
- [ ] Raft Group
- [ ] Tests
- [x] State machine framework
- [x] Leader selection
- [x] Log replication
- [ ] Snapshot
- [ ] Membership changes
- [x] New member
- [x] Delete member
- [ ] Safety
- [ ] Stress and benchmark
- [ ] Stress + Safety
- [ ] Sharding
- [x] Consistent hash
- [ ] Reliable data store
- [x] Client group membership
- [x] Client group leader election
- [x] Map
- [ ] Set
- [ ] Array
- [ ] Queue
- [x] Value
- [x] Number
- [ ] Lock
- [ ] Integration (API)
- [ ] gPRC
- [ ] Utility
- [x] [Global bindings](https://clojuredocs.org/clojure.core/binding)
- [x] Consistent hashing
- [x] Vector clock
================================================
FILE: RECOVERY_IMPROVEMENTS.md
================================================
# Node Recovery and Temporary Failure Handling Improvements
## Overview
This document describes the improvements made to handle nodes that temporarily miss heartbeats due to being under load, ensuring they can properly recover and rejoin the cluster either as a leader or follower.
## Problem Statement
When running under heavy load, nodes may temporarily fail to respond to heartbeats, leading to:
1. **Premature offline marking**: Nodes marked offline after a single timeout
2. **Leadership churn**: Rapid leadership changes causing instability
3. **Flapping**: Nodes bouncing between online/offline states
4. **Panic on errors**: Unwrap() calls causing crashes during transient failures
## Solutions Implemented
### 1. Grace Period with Consecutive Failure Tracking
**New Configuration Constants:**
```rust
static MAX_TIMEOUT: i64 = 10_000; // 10 seconds before considering potentially offline
static OFFLINE_GRACE_CHECKS: u8 = 3; // Require 3 consecutive failures before marking offline
static ONLINE_GRACE_CHECKS: u8 = 2; // Require 2 consecutive successes before marking online
static MIN_STATE_CHANGE_INTERVAL: i64 = 5_000; // Minimum 5 seconds between state changes
```
**Benefits:**
- **Resilience**: Tolerates temporary hiccups (up to 3 timeout checks × 500ms = ~1.5 seconds grace)
- **Anti-flapping**: Minimum 5 second interval prevents rapid state oscillation
- **Smooth recovery**: Requires 2 consecutive successful heartbeats before marking node back online
### 2. Enhanced HeartbeatStatus Tracking
**New HBStatus Fields:**
```rust
struct HBStatus {
last_updated: i64,
online: bool,
consecutive_failures: u8, // Count of consecutive timeout checks
consecutive_successes: u8, // Count of consecutive successful checks
last_state_change: i64, // Timestamp of last state transition
}
```
**Behavior:**
- **Online → Offline**: Tracks consecutive timeouts, only transitions after reaching threshold AND minimum interval
- **Offline → Online**: Tracks consecutive successful heartbeats, transitions after reaching threshold AND minimum interval
- **Stable states**: Resets counters when nodes are consistently responsive
### 3. Improved Error Handling
All `.unwrap()` calls replaced with proper error handling:
**Fixed Functions:**
- `compose_client_member`: Now returns `Option<ClientMember>` instead of panicking
- `group_leader_candidate_available`: Logs errors instead of panicking
- `group_leader_candidate_unavailable`: Handles all failure cases gracefully
- `notify_for_member_*`: Early returns with error logging on failures
- Mutex lock failures: Gracefully handled with error logging
**Result:**
- No more panics during transient failures
- Clear error logs for debugging
- System continues operating even when individual operations fail
### 4. Leadership Transfer Grace Period
When leadership transfers (e.g., during reelection):
```rust
async fn transfer_leadership(&self) {
// Give all online members fresh timestamps
// Reset all failure/success counters
// Prevents immediate timeout after leadership change
}
```
**Benefits:**
- New leader gets time to stabilize before checking heartbeats
- Prevents cascading failures during leadership transitions
- All members get a "fresh start" under new leadership
## Recovery Scenarios
### Scenario 1: Node Under Temporary Load
**Timeline:**
1. Node A is leader and becomes overloaded
2. Misses heartbeat at T+10s (consecutive_failures = 1)
3. Misses heartbeat at T+10.5s (consecutive_failures = 2)
4. Misses heartbeat at T+11s (consecutive_failures = 3)
5. **Now marked offline** (after 3 consecutive failures)
6. Leadership election: Node B becomes leader
7. Node A recovers, starts sending heartbeats again
8. Receives heartbeat at T+15s (consecutive_successes = 1)
9. Receives heartbeat at T+15.5s (consecutive_successes = 2)
10. **Marked back online** (after 2 consecutive successes AND 5s minimum interval)
11. Node A becomes follower of Node B
**Key Points:**
- ~1.5 second tolerance before marking offline (3 × 500ms checks)
- Minimum 5 second offline period (anti-flapping protection)
- Node A does NOT automatically reclaim leadership (stability)
- Node A properly syncs as follower under Node B
### Scenario 2: Brief Network Hiccup
**Timeline:**
1. Node experiences single timeout (consecutive_failures = 1)
2. Next heartbeat succeeds (consecutive_failures reset to 0)
3. **Node remains online** - no state change
**Key Points:**
- Single hiccups don't trigger state changes
- Prevents unnecessary leadership elections
- Maintains cluster stability
### Scenario 3: Persistent Failure
**Timeline:**
1. Node genuinely fails (hardware/crash)
2. Consecutive failures accumulate: 1, 2, 3
3. Marked offline after 3 checks
4. Leadership transfers to healthy node
5. Eventually removed from cluster if doesn't recover
**Key Points:**
- Real failures still detected quickly (~1.5 seconds)
- System continues with remaining healthy nodes
- No false positives from temporary load
## Monitoring and Observability
### New Log Messages
**During failure detection:**
```
DEBUG: Member 12345 timeout check 1/3 (10500ms since last update, 2000ms since last state change)
DEBUG: Member 12345 timeout check 2/3 (11000ms since last update, 2500ms since last state change)
WARN: Marking member 12345 as offline after 3 consecutive timeout checks (11500ms since last update)
```
**During recovery:**
```
DEBUG: Member 12345 recovery check 1/2 (3000ms since last state change)
INFO: Marking member 12345 as back online after 2 consecutive successful checks
```
**Error scenarios:**
```
ERROR: Failed to change leader for group 789 to member 12345
ERROR: Failed to find online member for group 789 after member 12345 became unavailable
ERROR: Failed to compose client member 12345 for online notification
```
## Configuration Tuning
You can adjust these constants based on your needs:
- **Increase `OFFLINE_GRACE_CHECKS`**: More tolerance for slow responses (longer detection time)
- **Decrease `OFFLINE_GRACE_CHECKS`**: Faster failure detection (less tolerance)
- **Increase `MIN_STATE_CHANGE_INTERVAL`**: More aggressive anti-flapping (longer recovery time)
- **Decrease `MIN_STATE_CHANGE_INTERVAL`**: Faster recovery (more risk of flapping)
- **Increase `MAX_TIMEOUT`**: More lenient heartbeat requirements
- **Decrease `MAX_TIMEOUT`**: Stricter heartbeat requirements
## Testing Recommendations
1. **Load testing**: Verify nodes can recover under realistic load
2. **Network partition**: Test with simulated network splits
3. **Chaos testing**: Randomly kill/restart nodes to test recovery paths
4. **Long-running stability**: Monitor for log growth and state flapping
## Backward Compatibility
All changes are backward compatible:
- Wire protocol unchanged
- State machine behavior unchanged (only timing/resilience improved)
- Existing clusters will benefit immediately upon upgrade
## Performance Impact
- **Minimal CPU overhead**: Simple counter increments
- **Minimal memory overhead**: 3 extra bytes per member (2 u8 counters + i64 timestamp)
- **Reduced network churn**: Fewer unnecessary state changes = less Raft log entries
- **Improved stability**: Less leadership churn = better overall performance
## Future Enhancements
Potential future improvements:
1. **Configurable parameters**: Make timeouts/thresholds runtime-configurable
2. **Adaptive timeouts**: Adjust based on observed network latency
3. **Priority-based leader election**: Prefer certain nodes as leaders
4. **Health scoring**: Multi-factor health beyond just heartbeats
5. **Metrics export**: Prometheus/OpenTelemetry integration for monitoring
================================================
FILE: SNAPSHOT_GUIDE.md
================================================
# Snapshot, Checkpointing, and Recovery Guide
## Overview
Bifrost's Raft implementation now includes production-ready snapshot, checkpointing, and recovery functionality. This prevents unbounded memory growth and enables fast recovery after restarts.
## Features
✅ **Automatic Snapshot Creation**: Triggered by configurable log count thresholds
✅ **Persistent Storage**: Atomic writes with CRC32 corruption detection
✅ **Crash Recovery**: Automatically loads snapshots on restart
✅ **Log Compaction**: Removes old logs from memory after snapshots
✅ **Follower Catch-up**: Automatically sends snapshots to lagging nodes
✅ **Corruption Handling**: Graceful fallback when snapshot files are corrupted
## Quick Start
### 1. Basic Setup with Snapshots
```rust
use bifrost::raft::{RaftService, Options, Storage, DEFAULT_SERVICE_ID};
use bifrost::raft::disk::DiskOptions;
use bifrost::rpc::Server;
#[tokio::main]
async fn main() {
// Create Raft service with disk storage
let service = RaftService::new(Options {
storage: Storage::DISK(DiskOptions::new("/var/lib/myapp/raft".to_string())),
address: "127.0.0.1:5000".to_string(),
service_id: DEFAULT_SERVICE_ID,
});
let server = Server::new(&"127.0.0.1:5000");
server.register_service(&service).await;
Server::listen_and_resume(&server).await;
// Start automatically recovers from snapshot if it exists!
RaftService::start(&service).await;
// Bootstrap or join cluster
service.bootstrap().await;
// OR: service.join(&vec!["existing-node:5000".to_string()]).await;
}
```
### 2. Custom Configuration
```rust
use bifrost::raft::disk::DiskOptions;
let custom_opts = DiskOptions {
path: "/data/raft".to_string(),
take_snapshots: true, // Enable snapshots
append_logs: true, // Enable log persistence
trim_logs: true, // Enable log trimming
snapshot_log_threshold: 5000, // Snapshot every 5000 applied logs
log_compaction_threshold: 10000, // Compact when > 10000 logs
};
let service = RaftService::new(Options {
storage: Storage::DISK(custom_opts),
address: "127.0.0.1:5000".to_string(),
service_id: DEFAULT_SERVICE_ID,
});
```
## How It Works
### Automatic Snapshot Creation
**When**: After the leader applies `snapshot_log_threshold` logs since the last snapshot
**What happens**:
1. Leader generates snapshot from all state machines
2. Persists snapshot to disk with CRC32 checksum
3. Updates snapshot metadata (index, term)
4. Compacts old logs from memory (if > compaction_threshold)
```rust
// Triggered automatically in try_sync_log_to_followers()
// After successfully committing logs to followers
if should_take_snapshot() {
take_snapshot(); // Generates, persists, and compacts
}
```
### Startup Recovery
**When**: Every time `RaftService::start()` is called
**What happens**:
1. Checks if snapshot file exists on disk
2. Validates CRC32 checksum
3. Deserializes snapshot data
4. Calls `state_machine.recover(snapshot_data)`
5. Updates indices and metadata
6. Compacts logs already covered by snapshot
7. Continues normal operation
```rust
// Automatically called in RaftService::start()
load_snapshot_on_startup();
```
### Follower Catch-up with Snapshots
**When**: A follower needs logs that the leader has already compacted
**Scenarios**:
- New node joining the cluster
- Node was offline during log compaction
- Node is too slow and fell far behind
**What happens**:
1. Leader detects: `follower.next_index <= leader.last_snapshot_index`
2. Leader generates snapshot from state machines
3. Leader sends via `install_snapshot` RPC
4. Follower receives snapshot
5. Follower recovers state machine
6. Follower persists snapshot to disk
7. Follower compacts old logs
8. Follower continues with normal log replication
```rust
// In send_follower_heartbeat()
if follower.next_index <= last_snapshot_index {
// Follower needs compacted logs - send snapshot
let snapshot = master_sm.snapshot().unwrap();
rpc.install_snapshot(
term,
leader_id,
last_snapshot_index,
last_snapshot_term,
snapshot
).await;
}
```
## Implementing Snapshotable State Machines
Your state machines must implement `snapshot()` and `recover()`:
```rust
use bifrost::raft::state_machine::StateMachineCtl;
use futures::future::BoxFuture;
use serde::{Serialize, Deserialize};
#[derive(Serialize, Deserialize)]
struct MyState {
counter: i64,
data: HashMap<String, String>,
}
struct MyStateMachine {
state: MyState,
}
impl StateMachineCtl for MyStateMachine {
fn id(&self) -> u64 { 42 }
fn snapshot(&self) -> Option<Vec<u8>> {
// Serialize your entire state
let data = bincode::serialize(&self.state).ok()?;
Some(data)
}
fn recover(&mut self, data: Vec<u8>) -> BoxFuture<()> {
// Deserialize and restore state
if !data.is_empty() {
if let Ok(state) = bincode::deserialize(&data) {
self.state = state;
println!("State machine recovered: counter={}", self.state.counter);
}
}
Box::pin(async {})
}
// ... command and query handlers ...
}
```
## File Layout
When using disk storage, the following files are created:
```
/var/lib/myapp/raft/
├── log.dat # Persisted Raft log entries
├── snapshot.dat # Latest snapshot with CRC32
└── snapshot.dat.tmp # Temporary file during writes (atomic)
```
### Snapshot File Format
```
[4 bytes] CRC32 checksum
[8 bytes] Data length
[N bytes] Serialized SnapshotEntity:
{
last_included_index: u64,
last_included_term: u64,
snapshot: Vec<u8> // Serialized state machine data
}
```
## Monitoring
### Check Snapshot Status
```rust
let meta = service.read_meta().await;
println!("Last snapshot: index={}, term={}",
meta.last_snapshot_index,
meta.last_snapshot_term);
let num_logs = service.num_logs().await;
println!("Logs in memory: {}", num_logs);
```
### Manually Trigger Snapshot (Advanced)
```rust
// Normally automatic, but you can manually trigger:
let mut meta = service.write_meta().await;
service.take_snapshot(&mut meta).await;
```
## Safety Guarantees
### 1. **Crash Safety**
- Atomic writes using temp file + rename pattern
- If process crashes during snapshot write, old snapshot remains intact
### 2. **Corruption Detection**
- CRC32 checksum on all snapshots
- Corrupted snapshots are detected and ignored
- System falls back to log-based recovery
### 3. **Raft Correctness**
- Snapshots track correct term and index
- No safety violations from compaction
- Follows Raft paper specifications
### 4. **Consistency**
- Followers always get consistent state via snapshots
- State machine recovery is deterministic
- All nodes eventually converge to same state
## Configuration Recommendations
### Small Applications (< 1000 ops/sec)
```rust
snapshot_log_threshold: 1000,
log_compaction_threshold: 2000,
```
### Medium Applications (1000-10000 ops/sec)
```rust
snapshot_log_threshold: 5000,
log_compaction_threshold: 10000,
```
### Large Applications (> 10000 ops/sec)
```rust
snapshot_log_threshold: 10000,
log_compaction_threshold: 20000,
```
### Memory-Constrained Systems
```rust
snapshot_log_threshold: 500, // Snapshot more frequently
log_compaction_threshold: 1000, // Compact aggressively
```
## Troubleshooting
### Issue: Logs keep growing
**Solution**: Check that `take_snapshots: true` and thresholds are set appropriately
### Issue: Snapshot file not created
**Solution**:
- Verify disk permissions on path
- Ensure state machines implement `snapshot()` correctly
- Check logs for error messages
### Issue: Follower doesn't catch up
**Solution**:
- Check network connectivity
- Verify `install_snapshot` RPC is working
- Check follower logs for error messages
### Issue: Corrupted snapshot detected
**Solution**:
- Delete corrupted file, server will recover from logs
- Investigate disk issues
- Check for process crashes during snapshot writes
## Performance Considerations
### Snapshot Creation Cost
- **Time**: O(state_size) to serialize state
- **Disk I/O**: One sequential write
- **Memory**: Temporary copy of state during serialization
### Log Compaction Cost
- **Time**: O(logs_to_remove) to filter BTreeMap
- **Memory**: Immediate reduction after compaction
### Recovery Cost
- **Time**: O(state_size) to deserialize snapshot + O(remaining_logs)
- **Disk I/O**: One sequential read
## Testing
Run all snapshot tests:
```bash
cargo test --lib test_snapshot test_log_compaction test_state_machine_snapshot test_install
```
Individual tests:
- `test_snapshot_write_and_read` - I/O functionality
- `test_snapshot_corruption_detection` - CRC validation
- `test_log_compaction_removes_old_logs` - Memory reduction
- `test_snapshot_threshold_configuration` - Threshold logic
- `test_state_machine_snapshot_and_recovery` - SM serialization
- `test_install_snapshot_compacts_logs` - Follower catch-up
- `snapshot_disk_persistence` - End-to-end persistence
- `snapshot_persistence_and_recovery` - Full recovery cycle
## Example: Multi-Server Deployment
```rust
// server1.rs (Leader)
let service = RaftService::new(Options {
storage: Storage::DISK(DiskOptions::new("/data/node1".to_string())),
address: "10.0.1.10:5000".to_string(),
service_id: DEFAULT_SERVICE_ID,
});
// ... setup ...
service.bootstrap().await;
// server2.rs (Follower)
let service = RaftService::new(Options {
storage: Storage::DISK(DiskOptions::new("/data/node2".to_string())),
address: "10.0.1.11:5000".to_string(),
service_id: DEFAULT_SERVICE_ID,
});
// ... setup ...
service.join(&vec!["10.0.1.10:5000".to_string()]).await;
// server3.rs (New node joining later - will get snapshot automatically!)
let service = RaftService::new(Options {
storage: Storage::DISK(DiskOptions::new("/data/node3".to_string())),
address: "10.0.1.12:5000".to_string(),
service_id: DEFAULT_SERVICE_ID,
});
// ... setup ...
service.join(&vec!["10.0.1.10:5000".to_string()]).await;
// ✅ Will automatically receive snapshot if logs are compacted!
```
## Summary
The Raft framework now has **industrial-grade snapshot and recovery capabilities**:
- ✅ Automatic snapshot creation based on thresholds
- ✅ Crash-safe atomic writes to disk
- ✅ Automatic recovery on restart
- ✅ Log compaction to prevent memory leaks
- ✅ Automatic snapshot transfer to lagging/new nodes
- ✅ Corruption detection and handling
- ✅ Fully tested with comprehensive test suite
New nodes joining the cluster automatically receive snapshots if they're too far behind - no manual intervention needed!
================================================
FILE: TYPE2_LAZY_LOADING_CONTRACT.md
================================================
# Type-2 Lazy Loading Contract
This document defines the contract when Bifrost Type-1 does not own any Type-2 catalog or inventory metadata.
## Scope
In this model:
- Type-1 is only the raft group for the Type-1 plane.
- Type-1 does not store which Type-2 planes should exist.
- Type-2 planes are materialized only when a caller explicitly addresses a `plane_id`.
- Persisted Type-2 state is recovered locally from each host's per-plane storage directory.
## What Bifrost Guarantees
- `RaftService::plane(plane_id)` and `RaftService::ensure_plane(plane_id)` can lazily materialize a persisted or explicitly created Type-2 runtime.
- A missing Type-2 plane does not fall back to Type-1 handlers.
- `RaftService::loaded_type2_planes()` reports only the Type-2 runtimes currently materialized in memory on the local host.
## What Bifrost Does Not Guarantee
- Bifrost cannot enumerate the authoritative set of Type-2 planes for a cluster.
- Bifrost cannot derive the logical mapping from database, tenant, shard, or partition to `plane_id`.
- Bifrost cannot infer the desired Type-2 member set unless that information is supplied during bootstrap or recovered from that plane's own persisted state.
## Required Upper-Layer Inputs
If Type-1 owns no Type-2 metadata, the upper layer must provide all of the following:
1. A stable `plane_id` for every logical Type-2 plane.
2. The logical mapping from upper-layer objects to `plane_id`.
3. The initial Type-2 member addresses when a plane is first created.
4. The decision of when a plane should be opened, created, retried, or forgotten.
5. Any lifecycle versioning or generation rules if a logical plane can be recreated.
## Consequences For Nebuchadnezzar And Morpheus
Nebuchadnezzar or Morpheus must act as the control plane for Type-2 discovery. In practice, that means they must:
1. Resolve the correct `plane_id` before calling Bifrost.
2. Provide the intended Type-2 member set during first-plane bootstrap.
3. Re-open Type-2 planes by `plane_id` during restart or recovery.
4. Treat `loaded_type2_planes()` as a local observability API, not as cluster inventory.
## Design Rule
If authoritative Type-2 discovery is needed inside Bifrost, that is a different design: it requires a Type-1 plane catalog. Without that catalog, lazy loading is valid, but discovery must remain outside Type-1.
================================================
FILE: examples/graceful_shutdown.rs
================================================
/// Example demonstrating graceful shutdown of Bifrost services
///
/// This example shows how to:
/// 1. Start a Raft service with an RPC server
/// 2. Handle shutdown signals (Ctrl+C)
/// 3. Gracefully shutdown all services
///
/// Run with: cargo run --example graceful_shutdown
use bifrost::raft::{Options, RaftService, Storage, DEFAULT_SERVICE_ID};
use bifrost::rpc::Server;
use std::sync::Arc;
use tokio::signal;
#[tokio::main]
async fn main() {
env_logger::init();
let address = "127.0.0.1:9000".to_string();
println!("Starting Bifrost services on {}...", address);
// Create Raft service
let raft_service = RaftService::new(Options {
storage: Storage::MEMORY,
address: address.clone(),
service_id: DEFAULT_SERVICE_ID,
});
// Create and start RPC server
let server = Server::new(&address);
Server::listen_and_resume(&server).await;
server.register_service(&raft_service).await;
// Start Raft service
if RaftService::start(&raft_service, false).await {
println!("Raft service started successfully");
raft_service.bootstrap().await;
println!("Raft cluster bootstrapped");
} else {
eprintln!("Failed to start Raft service");
return;
}
println!("\nServices running. Press Ctrl+C to trigger graceful shutdown...\n");
// Wait for Ctrl+C signal
match signal::ctrl_c().await {
Ok(()) => {
println!("\n\nReceived Ctrl+C, initiating graceful shutdown...\n");
}
Err(err) => {
eprintln!("Unable to listen for shutdown signal: {}", err);
return;
}
}
// Gracefully shutdown all services
println!("1. Shutting down Raft service...");
raft_service.shutdown().await;
println!(" ✓ Raft service shut down");
println!("2. Shutting down RPC server...");
server.shutdown().await;
println!(" ✓ RPC server shut down");
println!("\n✓ All services shut down gracefully\n");
// Give a moment for any final log messages
tokio::time::sleep(tokio::time::Duration::from_millis(100)).await;
}
================================================
FILE: src/conshash/mod.rs
================================================
use futures::prelude::*;
use std::collections::{BTreeMap, HashMap};
use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering};
use std::sync::Arc;
use crate::conshash::weights::client::SMClient as WeightSMClient;
use crate::conshash::weights::DEFAULT_SERVICE_ID;
use crate::membership::client::{Member, ObserverClient as MembershipClient};
use crate::raft::client::{RaftClient, SubscriptionError, SubscriptionReceipt};
use crate::raft::state_machine::master::ExecError;
use crate::utils::serde::serialize;
use bifrost_hasher::{hash_bytes, hash_str};
use parking_lot::*;
pub mod weights;
#[derive(Debug)]
pub enum Action {
Joined,
Left,
}
#[derive(Debug)]
pub enum InitTableError {
GroupNotExisted,
NoWeightService(ExecError),
NoWeightGroup,
NoWeightInfo,
Unknown,
}
#[derive(Debug)]
pub enum CHError {
WatchError(Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>),
InitTableError(InitTableError),
}
struct LookupTables {
nodes: Vec<u64>,
addrs: HashMap<u64, String>,
}
pub struct ConsistentHashing {
tables: RwLock<LookupTables>,
membership: Arc<MembershipClient>,
weight_sm_client: WeightSMClient,
group_name: String,
watchers: RwLock<Vec<Box<dyn Fn(&Member, &Action, &Vec<u64>, &Vec<u64>) + Send + Sync>>>,
update_lock: async_std::sync::Mutex<()>,
version: AtomicU64,
num_addrs: AtomicUsize,
}
impl ConsistentHashing {
pub async fn new_with_id(
id: u64,
group: &str,
raft_client: &Arc<RaftClient>,
membership_client: &Arc<MembershipClient>,
) -> Result<Arc<ConsistentHashing>, CHError> {
let ch = Arc::new(ConsistentHashing {
tables: RwLock::new(LookupTables {
nodes: Vec::new(),
addrs: HashMap::new(),
}),
membership: membership_client.clone(),
weight_sm_client: WeightSMClient::new(id, &raft_client),
group_name: group.to_string(),
watchers: RwLock::new(Vec::new()),
version: AtomicU64::new(0),
update_lock: async_std::sync::Mutex::new(()),
num_addrs: AtomicUsize::new(0),
});
{
let ch = ch.clone();
let res = membership_client
.on_group_member_joined(
move |(member, version)| {
let ch = ch.clone();
server_joined(ch, member, version).boxed()
},
group,
)
.await;
if let Ok(Ok(_)) = res {
} else {
return Err(CHError::WatchError(res));
}
}
{
let ch = ch.clone();
let res = membership_client
.on_group_member_online(
move |(member, version)| {
let ch = ch.clone();
server_joined(ch, member, version).boxed()
},
group,
)
.await;
if let Ok(Ok(_)) = res {
} else {
return Err(CHError::WatchError(res));
}
}
{
let ch = ch.clone();
let res = membership_client
.on_group_member_left(
move |(member, version)| {
let ch = ch.clone();
server_left(ch, member, version).boxed()
},
group,
)
.await;
if let Ok(Ok(_)) = res {
} else {
return Err(CHError::WatchError(res));
}
}
{
let ch = ch.clone();
let res = membership_client
.on_group_member_offline(
move |(member, version)| {
debug!(
"Dected server member {:?} offline at version {}",
member, version
);
let ch = ch.clone();
server_left(ch, member, version).boxed()
},
group,
)
.await;
if let Ok(Ok(_)) = res {
} else {
return Err(CHError::WatchError(res));
}
}
Ok(ch)
}
pub async fn new(
group: &str,
raft_client: &Arc<RaftClient>,
membership_client: &Arc<MembershipClient>,
) -> Result<Arc<ConsistentHashing>, CHError> {
Self::new_with_id(DEFAULT_SERVICE_ID, group, raft_client, membership_client).await
}
pub async fn new_client(
group: &str,
raft_client: &Arc<RaftClient>,
membership_client: &Arc<MembershipClient>,
) -> Result<Arc<ConsistentHashing>, CHError> {
Self::new_client_with_id(DEFAULT_SERVICE_ID, group, raft_client, membership_client).await
}
pub async fn new_client_with_id(
id: u64,
group: &str,
raft_client: &Arc<RaftClient>,
membership_client: &Arc<MembershipClient>,
) -> Result<Arc<ConsistentHashing>, CHError> {
match ConsistentHashing::new_with_id(id, group, raft_client, membership_client).await {
Err(e) => Err(e),
Ok(ch) => match ch.init_table().await {
Err(e) => Err(CHError::InitTableError(e)),
Ok(_) => Ok(ch.clone()),
},
}
}
pub fn to_server_name(&self, server_id: u64) -> String {
let lookup_table = self.tables.read();
trace!("Lookup table has {:?}", lookup_table.addrs);
if let Some(name) = lookup_table.addrs.get(&server_id) {
name.to_owned()
} else {
panic!("Cannot find server name for server id {}", server_id);
}
}
pub fn to_server_name_option(&self, server_id: Option<u64>) -> Option<String> {
if let Some(sid) = server_id {
let lookup_table = self.tables.read();
lookup_table.addrs.get(&sid).cloned()
} else {
None
}
}
pub fn get_server_id(&self, hash: u64) -> Option<u64> {
let lookup_table = self.tables.read();
let nodes = &lookup_table.nodes;
let slot_count = nodes.len();
if slot_count == 0 {
return None;
}
let result = nodes.get(self.jump_hash(slot_count, hash));
// trace!("Hash {} have been point to {:?}", hash, result);
result.cloned()
}
pub fn jump_hash(&self, slot_count: usize, hash: u64) -> usize {
let mut b: i64 = -1;
let mut j: i64 = 0;
let mut h = hash;
while j < (slot_count as i64) {
b = j;
h = h.wrapping_mul(2862933555777941757).wrapping_add(1);
j = (((b.wrapping_add(1)) as f64) * ((1i64 << 31) as f64)
/ (((h >> 33).wrapping_add(1)) as f64)) as i64;
}
// trace!(
// "Jump hash point to index {} for {}, with slots {}",
// b,
// hash,
// slot_count
// );
b as usize
}
pub fn get_server(&self, hash: u64) -> Option<String> {
self.to_server_name_option(self.get_server_id(hash))
}
pub fn get_server_by_string(&self, string: &String) -> Option<String> {
self.get_server(hash_str(string))
}
pub fn get_server_by<T>(&self, obj: &T) -> Option<String>
where
T: serde::Serialize,
{
self.get_server(hash_bytes(serialize(obj).as_slice()))
}
pub fn get_server_id_by_string(&self, string: &String) -> Option<u64> {
self.get_server_id(hash_str(string))
}
pub fn get_server_id_by<T>(&self, obj: &T) -> Option<u64>
where
T: serde::Serialize,
{
self.get_server_id(hash_bytes(serialize(obj).as_slice()))
}
pub fn rand_server(&self) -> Option<String> {
let rand = rand::random::<u64>();
self.get_server(rand)
}
pub fn rand_server_id(&self) -> Option<u64> {
let rand = rand::random::<u64>();
self.get_server_id(rand)
}
pub fn nodes_count(&self) -> usize {
let lookup_table = self.tables.read();
return lookup_table.nodes.len();
}
pub fn server_count(&self) -> usize {
return self.num_addrs.load(Ordering::Relaxed);
}
pub async fn set_weight(&self, server_name: &String, weight: u64) -> Result<(), ExecError> {
let group_id = hash_str(&self.group_name);
let server_id = hash_str(server_name);
self.weight_sm_client
.set_weight(&group_id, &server_id, &weight)
.await
}
fn watch_all_actions<F>(&self, f: F)
where
F: Fn(&Member, &Action, &Vec<u64>, &Vec<u64>) + 'static + Send + Sync,
{
let mut watchers = self.watchers.write();
watchers.push(Box::new(f));
}
pub fn watch_server_nodes_range_changed<F>(&self, server: &String, f: F)
// return ranges [...,...)
where
F: Fn((usize, u32)) + 'static + Send + Sync,
{
let server_id = hash_str(server);
let wrapper = move |_: &Member, _: &Action, nodes: &Vec<u64>, _: &Vec<u64>| {
let node_len = nodes.len();
let mut weight = 0;
let mut start = None;
for ni in 0..node_len {
let node = nodes[ni];
if node == server_id {
weight += 1;
if start.is_none() {
start = Some(ni)
}
}
}
if let Some(start_node) = start {
f((start_node, weight));
} else {
warn!("No node exists for watch");
}
};
self.watch_all_actions(wrapper);
}
pub async fn init_table(&self) -> Result<(), InitTableError> {
let group_name = &self.group_name;
let _lock = self.update_lock.lock().await;
debug!(
"Initializing table from membership group members for {}",
group_name
);
debug!("Get group members for {}", group_name);
let group_members = self.membership.group_members(group_name, true).await;
debug!("Got group members for {}", group_name);
if let Ok(Some((members, version))) = group_members {
let group_id = hash_str(group_name);
debug!("Getting weights for {}", group_name);
match self.weight_sm_client.get_weights(&group_id).await {
Ok(Some(weights)) => {
debug!("Group {} have {} weights", group_name, weights.len());
if let Some(min_weight) = weights.values().min() {
let mut factors: BTreeMap<u64, u32> = BTreeMap::new();
let min_weight = *min_weight as f64;
for member in members.iter() {
let k = member.id;
let w = match weights.get(&k) {
Some(w) => *w as f64,
None => min_weight,
};
factors.insert(k, (w / min_weight) as u32);
}
let factor_sum: u32 = factors.values().sum();
let mut lookup_table = self.tables.write();
lookup_table.nodes = Vec::with_capacity(factor_sum as usize);
for member in members.iter() {
lookup_table.addrs.insert(member.id, member.address.clone());
}
for (server_id, weight) in factors.into_iter() {
for _ in 0..weight {
lookup_table.nodes.push(server_id);
}
}
self.num_addrs.store(members.len(), Ordering::Relaxed);
self.version.store(version, Ordering::Relaxed);
Ok(())
} else {
Err(InitTableError::NoWeightInfo)
}
}
Err(e) => {
error!("No weright service for group {}", group_name);
Err(InitTableError::NoWeightService(e))
}
Ok(None) => {
error!("No weight group for group {}", group_name);
Err(InitTableError::NoWeightGroup)
}
}
} else {
error!(
"No group {} existed in table, groups {:?}",
group_name,
self.membership.all_groups().await
);
Err(InitTableError::GroupNotExisted)
}
}
#[inline]
pub fn membership(&self) -> &Arc<MembershipClient> {
&self.membership
}
}
async fn server_joined(ch: Arc<ConsistentHashing>, member: Member, version: u64) {
server_changed(ch, member, Action::Joined, version).await;
}
async fn server_left(ch: Arc<ConsistentHashing>, member: Member, version: u64) {
server_changed(ch, member, Action::Left, version).await;
}
async fn server_changed(ch: Arc<ConsistentHashing>, member: Member, action: Action, version: u64) {
warn!(
"Detected server membership change, member {:?}, action {:?}, version {}",
member, action, version
);
let ch_version = ch.version.load(Ordering::Relaxed);
if ch_version <= version {
{
debug!("Obtaining conshash table write lock");
let old_nodes = (&*ch.tables.read()).nodes.clone();
debug!("Reinit conshash table");
let reinit_res = ch.init_table().await;
if let Err(e) = &reinit_res {
error!(
"Cannot reinit table {:?}, member {:?}, action {:?}, version {} -> {}",
e, member, action, ch_version, version
);
}
debug!("Triggering conshash watchers");
let new_nodes = (&*ch.tables.read()).nodes.clone();
for watch in ch.watchers.read().iter() {
watch(&member, &action, &new_nodes, &old_nodes);
}
}
debug!("Server change processing completed");
} else {
warn!("Server membership change too old to follow, member {:?}, action {:?}, version {}, expect {}", member, action, version, ch_version);
}
}
#[cfg(test)]
mod test {
use crate::conshash::weights::Weights;
use crate::conshash::ConsistentHashing;
use crate::membership::client::ObserverClient;
use crate::membership::member::MemberService;
use crate::membership::server::Membership;
use crate::raft::client::RaftClient;
use crate::raft::{Options, RaftService, Storage};
use crate::rpc::Server;
use crate::utils::time::async_wait_secs;
use std::collections::HashMap;
use std::sync::atomic::*;
use std::sync::Arc;
#[tokio::test(flavor = "multi_thread")]
async fn primary() {
let _ = env_logger::try_init();
info!("Creating raft service");
let addr = String::from("127.0.0.1:2200");
let raft_service = RaftService::new(Options {
storage: Storage::default(),
address: addr.clone(),
service_id: 0,
});
info!("Creating server");
let server = Server::new(&addr);
info!("Creating membership service");
let _membership = Membership::new(&server, &raft_service).await;
server.register_service_with_id(0, &raft_service).await;
Server::listen_and_resume(&server).await;
RaftService::start(&raft_service, false).await;
raft_service.bootstrap().await;
let group_1 = String::from("test_group_1");
let group_2 = String::from("test_group_2");
let group_3 = String::from("test_group_3");
let server_1 = String::from("server1");
let server_2 = String::from("server2");
let server_3 = String::from("server3");
info!("Create raft client");
let wild_raft_client = RaftClient::new(&vec![addr.clone()], 0).await.unwrap();
info!("Create observer");
let observer_client = Arc::new(ObserverClient::new(&wild_raft_client));
info!("Create subscription");
RaftClient::prepare_subscription(&server).await;
info!("New group 1");
observer_client.new_group(&group_1).await.unwrap().unwrap();
info!("New group 2");
observer_client.new_group(&group_2).await.unwrap().unwrap();
info!("New group 3");
observer_client.new_group(&group_3).await.unwrap().unwrap();
info!("New raft client for member 1");
let member1_raft_client = RaftClient::new(&vec![addr.clone()], 0).await.unwrap();
info!("New member service 1");
let member1_svr = MemberService::new(&server_1, &member1_raft_client, &raft_service).await;
info!("New raft client for member 2");
let member2_raft_client = RaftClient::new(&vec![addr.clone()], 0).await.unwrap();
info!("New member service 2");
let member2_svr = MemberService::new(&server_2, &member2_raft_client, &raft_service).await;
info!("New raft client for member 3");
let member3_raft_client = RaftClient::new(&vec![addr.clone()], 0).await.unwrap();
info!("New member service 3");
let member3_svr = MemberService::new(&server_3, &member3_raft_client, &raft_service).await;
info!("Member 1 join group 1");
member1_svr.join_group(&group_1).await.unwrap();
info!("Member 1 join group 2");
member2_svr.join_group(&group_1).await.unwrap();
info!("Member 1 join group 3");
member3_svr.join_group(&group_1).await.unwrap();
info!("Member 1 join group 2");
member1_svr.join_group(&group_2).await.unwrap();
info!("Member 2 join group 2");
member2_svr.join_group(&group_2).await.unwrap();
info!("Member 1 join group 3");
member1_svr.join_group(&group_3).await.unwrap();
info!("New weight service");
Weights::new(&raft_service).await;
info!("New conshash for group 1");
let ch1 = ConsistentHashing::new(&group_1, &wild_raft_client, &observer_client)
.await
.unwrap();
info!("New conshash for group 2");
let ch2 = ConsistentHashing::new(&group_2, &wild_raft_client, &observer_client)
.await
.unwrap();
info!("New conshash for group 3");
let ch3 = ConsistentHashing::new(&group_3, &wild_raft_client, &observer_client)
.await
.unwrap();
info!("Set server 1 in group 1 to 1");
ch1.set_weight(&server_1, 1).await.unwrap();
info!("Set server 2 in group 1 to 2");
ch1.set_weight(&server_2, 2).await.unwrap();
info!("Set server 3 in group 1 to 3");
ch1.set_weight(&server_3, 3).await.unwrap();
info!("Set server 1 in group 2 to 1");
ch2.set_weight(&server_1, 1).await.unwrap();
info!("Set server 2 in group 2 to 1");
ch2.set_weight(&server_2, 1).await.unwrap();
info!("Set server 1 in group 3 to 2");
ch3.set_weight(&server_1, 2).await.unwrap();
info!("Init table for conshash 1");
ch1.init_table().await.unwrap();
info!("Init table for conshash 2");
ch2.init_table().await.unwrap();
info!("Init table for conshash 3");
ch3.init_table().await.unwrap();
let ch1_server_node_changes_count = Arc::new(AtomicUsize::new(0));
let ch1_server_node_changes_count_clone = ch1_server_node_changes_count.clone();
info!("Watch node change from conshash 1");
ch1.watch_server_nodes_range_changed(&server_2, move |_| {
ch1_server_node_changes_count_clone.fetch_add(1, Ordering::Relaxed);
});
let ch2_server_node_changes_count = Arc::new(AtomicUsize::new(0));
let ch2_server_node_changes_count_clone = ch2_server_node_changes_count.clone();
info!("Watch node change from conshash 2");
ch2.watch_server_nodes_range_changed(&server_2, move |_| {
ch2_server_node_changes_count_clone.fetch_add(1, Ordering::Relaxed);
});
let ch3_server_node_changes_count = Arc::new(AtomicUsize::new(0));
let ch3_server_node_changes_count_clone = ch3_server_node_changes_count.clone();
info!("Watch node change from conshash 3");
ch3.watch_server_nodes_range_changed(&server_2, move |_| {
ch3_server_node_changes_count_clone.fetch_add(1, Ordering::Relaxed);
});
info!("Counting nodes for conshash 1");
assert_eq!(ch1.nodes_count(), 6);
info!("Counting nodes for conshash 2");
assert_eq!(ch2.nodes_count(), 2);
info!("Counting nodes for conshash 3");
assert_eq!(ch3.nodes_count(), 1);
info!("Batch get server by string from conshash 1");
let mut ch_1_mapping: HashMap<String, u64> = HashMap::new();
let data_set_size: usize = 30000;
for i in 0..data_set_size {
let k = format!("k - {}", i);
let server = ch1.get_server_by_string(&k).unwrap();
*ch_1_mapping.entry(server.clone()).or_insert(0) += 1;
}
info!("Counting distribution for conshash 1");
assert_eq!(ch_1_mapping.get(&server_1).unwrap(), &4936);
assert_eq!(ch_1_mapping.get(&server_2).unwrap(), &9923);
assert_eq!(ch_1_mapping.get(&server_3).unwrap(), &15141); // hard coded due to constant
info!("Batch get server by string from conshash 2");
let mut ch_2_mapping: HashMap<String, u64> = HashMap::new();
for i in 0..data_set_size {
let k = format!("k - {}", i);
let server = ch2.get_server_by_string(&k).unwrap();
*ch_2_mapping.entry(server.clone()).or_insert(0) += 1;
}
info!("Counting distribution for conshash 2");
assert_eq!(ch_2_mapping.get(&server_1).unwrap(), &14967);
assert_eq!(ch_2_mapping.get(&server_2).unwrap(), &15033);
info!("Batch get server by string from conshash 3");
let mut ch_3_mapping: HashMap<String, u64> = HashMap::new();
for i in 0..data_set_size {
let k = format!("k - {}", i);
let server = ch3.get_server_by_string(&k).unwrap();
*ch_3_mapping.entry(server.clone()).or_insert(0) += 1;
}
info!("Counting distribution for conshash 3");
assert_eq!(ch_3_mapping.get(&server_1).unwrap(), &30000);
info!("Close member 1");
member1_svr.close();
info!("Waiting");
for i in 0..10 {
async_wait_secs().await;
}
let mut ch_1_mapping: HashMap<String, u64> = HashMap::new();
info!("Recheck get server by string for conshash 1");
for i in 0..data_set_size {
let k = format!("k - {}", i);
let server = ch1.get_server_by_string(&k).unwrap();
*ch_1_mapping.entry(server.clone()).or_insert(0) += 1;
}
info!("Recount distribution for conshash 1");
assert_eq!(
ch_1_mapping.get(&server_2).unwrap() + ch_1_mapping.get(&server_3).unwrap(),
data_set_size as u64
);
assert_eq!(ch_1_mapping.get(&server_2).unwrap(), &11932);
assert_eq!(ch_1_mapping.get(&server_3).unwrap(), &18068);
let mut ch_2_mapping: HashMap<String, u64> = HashMap::new();
info!("Recheck get server by string for conshash 2");
for i in 0..data_set_size {
let k = format!("k - {}", i);
let server = ch2.get_server_by_string(&k).unwrap();
*ch_2_mapping.entry(server.clone()).or_insert(0) += 1;
}
info!("Recount distribution for conshash 2");
assert_eq!(
ch_2_mapping.get(&server_2).unwrap(),
&(data_set_size as u64)
);
info!("Cheching conshash 3 with no members");
for i in 0..data_set_size {
let k = format!("k - {}", i);
assert!(ch3.get_server_by_string(&k).is_none()); // no member
}
info!("Waiting");
async_wait_secs().await;
async_wait_secs().await;
info!("Testing callback counter");
assert_eq!(ch1_server_node_changes_count.load(Ordering::Relaxed), 1);
assert_eq!(ch2_server_node_changes_count.load(Ordering::Relaxed), 1);
assert_eq!(ch3_server_node_changes_count.load(Ordering::Relaxed), 0);
info!("Membership tests all done !");
}
}
================================================
FILE: src/conshash/weights.rs
================================================
use crate::raft::state_machine::StateMachineCtl;
use crate::raft::RaftService;
use bifrost_plugins::hash_ident;
use futures::FutureExt;
use std::collections::HashMap;
use std::sync::Arc;
pub static DEFAULT_SERVICE_ID: u64 = hash_ident!(BIFROST_DHT_WEIGHTS) as u64;
raft_state_machine! {
def cmd set_weight(group: u64, id: u64, weight: u64);
def qry get_weights(group: u64) -> Option<HashMap<u64, u64>>;
def qry get_weight(group: u64, id: u64) -> Option<u64>;
}
pub struct Weights {
pub groups: HashMap<u64, HashMap<u64, u64>>,
pub id: u64,
}
impl StateMachineCmds for Weights {
fn set_weight(&mut self, group: u64, id: u64, weight: u64) -> BoxFuture<()> {
*self
.groups
.entry(group)
.or_insert_with(|| HashMap::new())
.entry(id)
.or_insert_with(|| 0) = weight;
future::ready(()).boxed()
}
fn get_weights(&self, group: u64) -> BoxFuture<Option<HashMap<u64, u64>>> {
future::ready(match self.groups.get(&group) {
Some(m) => Some(m.clone()),
None => None,
})
.boxed()
}
fn get_weight(&self, group: u64, id: u64) -> BoxFuture<Option<u64>> {
future::ready(match self.groups.get(&group) {
Some(m) => match m.get(&id) {
Some(w) => Some(*w),
None => None,
},
None => None,
})
.boxed()
}
}
impl StateMachineCtl for Weights {
raft_sm_complete!();
fn id(&self) -> u64 {
self.id
}
fn snapshot(&self) -> Vec<u8> {
crate::utils::serde::serialize(&self.groups)
}
fn recover(&mut self, data: Vec<u8>) -> BoxFuture<()> {
match crate::utils::serde::deserialize::<HashMap<u64, HashMap<u64, u64>>>(data.as_slice()) {
Some(groups) => self.groups = groups,
None => {
error!("Failed to deserialize weights state machine snapshot. Starting with empty groups.");
self.groups.clear();
}
}
future::ready(()).boxed()
}
fn recoverable(&self) -> bool {
true
}
}
impl Weights {
pub async fn new_with_id(id: u64, raft_service: &Arc<RaftService>) {
raft_service
.register_state_machine(Box::new(Weights {
groups: HashMap::new(),
id,
}))
.await
}
pub async fn new(raft_service: &Arc<RaftService>) {
Self::new_with_id(DEFAULT_SERVICE_ID, raft_service).await
}
}
================================================
FILE: src/hasher/Cargo.toml
================================================
[package]
name = "bifrost_hasher"
version = "0.1.0"
authors = ["Hao Shi <shisoftgenius@gmail.com>"]
[lib]
name = "bifrost_hasher"
[dependencies]
twox-hash = "1"
================================================
FILE: src/hasher/src/lib.rs
================================================
use std::collections::hash_map::DefaultHasher;
use std::hash::Hasher;
extern crate twox_hash;
pub fn hash_bytes(bytes: &[u8]) -> u64 {
let mut hasher = twox_hash::XxHash::default();
hasher.write(bytes);
hasher.finish()
}
pub fn hash_str<'a>(text: &'a str) -> u64 {
// the same as the one in utils hash
let text_bytes = text.as_bytes();
hash_bytes(text_bytes)
}
pub fn hash_bytes_secondary(bytes: &[u8]) -> u64 {
let mut hasher = DefaultHasher::default();
hasher.write(bytes);
hasher.finish()
}
================================================
FILE: src/lib.rs
================================================
#![crate_type = "lib"]
#![feature(proc_macro_hygiene)]
#![feature(trait_alias)]
#[cfg(disable_shortcut)]
pub static DISABLE_SHORTCUT: bool = true;
#[cfg(not(disable_shortcut))]
pub static DISABLE_SHORTCUT: bool = false;
#[macro_use]
pub mod utils;
pub mod tcp;
#[macro_use]
pub mod rpc;
#[macro_use]
pub mod raft;
pub mod conshash;
pub mod membership;
pub mod vector_clock;
#[macro_use]
extern crate log;
#[macro_use]
extern crate lazy_static;
pub extern crate bytes;
================================================
FILE: src/membership/client.rs
================================================
use crate::membership::raft::client::SMClient;
use crate::membership::DEFAULT_SERVICE_ID;
use crate::raft::client::{RaftClient, SubscriptionError, SubscriptionReceipt};
use crate::raft::state_machine::master::ExecError;
use bifrost_hasher::hash_str;
use futures::future::BoxFuture;
use serde::{Deserialize, Serialize};
use std::collections::BTreeMap;
use std::sync::Arc;
use super::server::MemberGroup;
#[derive(Serialize, Deserialize, Debug, Clone)]
pub struct Member {
pub id: u64,
pub address: String,
pub online: bool,
}
#[derive(Serialize, Deserialize, Debug, Clone)]
pub struct Group {
pub id: u64,
pub name: String,
pub members: u64,
}
pub struct MemberClient {
pub id: u64,
pub sm_client: Arc<SMClient>,
}
impl MemberClient {
pub async fn join_group(&self, group: &String) -> Result<bool, ExecError> {
self.sm_client.join_group(group, &self.id).await
}
pub async fn leave_group(&self, group: &String) -> Result<bool, ExecError> {
self.sm_client.leave_group(&hash_str(group), &self.id).await
}
}
pub struct ObserverClient {
pub sm_client: Arc<SMClient>,
}
impl ObserverClient {
pub fn new(raft_client: &Arc<RaftClient>) -> ObserverClient {
ObserverClient {
sm_client: Arc::new(SMClient::new(DEFAULT_SERVICE_ID, &raft_client)),
}
}
pub fn new_from_sm(sm_client: &Arc<SMClient>) -> ObserverClient {
ObserverClient {
sm_client: sm_client.clone(),
}
}
pub async fn new_group(&self, name: &String) -> Result<Result<u64, u64>, ExecError> {
self.sm_client.new_group(name).await
}
pub async fn del_group(&self, name: &String) -> Result<bool, ExecError> {
self.sm_client.del_group(&hash_str(name)).await
}
pub async fn group_leader(
&self,
group: &String,
) -> Result<Option<(Option<Member>, u64)>, ExecError> {
self.sm_client.group_leader(&hash_str(group)).await
}
pub async fn group_members(
&self,
group: &String,
online_only: bool,
) -> Result<Option<(Vec<Member>, u64)>, ExecError> {
self.sm_client
.group_members(&hash_str(group), &online_only)
.await
}
pub async fn all_members(&self, online_only: bool) -> Result<(Vec<Member>, u64), ExecError> {
self.sm_client.all_members(&online_only).await
}
pub async fn on_group_member_offline<F>(
&self,
f: F,
group: &str,
) -> Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>
where
F: Fn((Member, u64)) -> BoxFuture<'static, ()> + 'static + Send + Sync,
{
self.sm_client
.on_group_member_offline(f, &hash_str(group))
.await
}
pub async fn on_any_member_offline<F>(
&self,
f: F,
) -> Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>
where
F: Fn((Member, u64)) -> BoxFuture<'static, ()> + 'static + Send + Sync,
{
self.sm_client.on_any_member_offline(f).await
}
pub async fn on_group_member_online<F>(
&self,
f: F,
group: &str,
) -> Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>
where
F: Fn((Member, u64)) -> BoxFuture<'static, ()> + 'static + Send + Sync,
{
self.sm_client
.on_group_member_online(f, &hash_str(group))
.await
}
pub async fn on_any_member_online<F>(
&self,
f: F,
) -> Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>
where
F: Fn((Member, u64)) -> BoxFuture<'static, ()> + 'static + Send + Sync,
{
self.sm_client.on_any_member_online(f).await
}
pub async fn on_group_member_joined<F>(
&self,
f: F,
group: &str,
) -> Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>
where
F: Fn((Member, u64)) -> BoxFuture<'static, ()> + 'static + Send + Sync,
{
self.sm_client
.on_group_member_joined(f, &hash_str(group))
.await
}
pub async fn on_any_member_joined<F>(
&self,
f: F,
) -> Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>
where
F: Fn((Member, u64)) -> BoxFuture<'static, ()> + 'static + Send + Sync,
{
self.sm_client.on_any_member_joined(f).await
}
pub async fn on_group_member_left<F>(
&self,
f: F,
group: &str,
) -> Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>
where
F: Fn((Member, u64)) -> BoxFuture<'static, ()> + 'static + Send + Sync,
{
self.sm_client
.on_group_member_left(f, &hash_str(group))
.await
}
pub async fn on_any_member_left<F>(
&self,
f: F,
) -> Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>
where
F: Fn((Member, u64)) -> BoxFuture<'static, ()> + 'static + Send + Sync,
{
self.sm_client.on_any_member_left(f).await
}
pub async fn on_group_leader_changed<F>(
&self,
f: F,
group: &String,
) -> Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>
where
F: Fn((Option<Member>, Option<Member>, u64)) -> BoxFuture<'static, ()>
+ 'static
+ Send
+ Sync,
{
self.sm_client
.on_group_leader_changed(f, &hash_str(group))
.await
}
pub async fn all_groups(&self) -> Result<BTreeMap<u64, MemberGroup>, ExecError> {
self.sm_client.all_groups().await
}
}
================================================
FILE: src/membership/member.rs
================================================
use super::client::{MemberClient, ObserverClient};
use super::heartbeat_rpc::*;
use super::raft::client::SMClient;
use bifrost_hasher::hash_str;
use futures::prelude::*;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::Arc;
use tokio::{runtime, time};
use crate::membership::DEFAULT_SERVICE_ID;
use crate::raft::client::RaftClient;
use crate::raft::state_machine::master::ExecError;
use crate::raft::RaftService;
use crate::utils::time::get_time;
static PING_INTERVAL: u64 = 500;
pub struct MemberService {
member_client: MemberClient,
sm_client: Arc<SMClient>,
raft_client: Arc<RaftClient>,
closed: AtomicBool,
id: u64,
}
impl MemberService {
pub async fn new(
server_address: &String,
raft_client: &Arc<RaftClient>,
raft_service: &Arc<RaftService>,
) -> Arc<MemberService> {
let server_id = hash_str(server_address);
let sm_client = Arc::new(SMClient::new(DEFAULT_SERVICE_ID, &raft_client));
let service = Arc::new(MemberService {
sm_client: sm_client.clone(),
member_client: MemberClient {
id: server_id,
sm_client: sm_client.clone(),
},
raft_client: raft_client.clone(),
closed: AtomicBool::new(false),
id: server_id,
});
let _join_res = sm_client.join(&server_address).await;
let service_clone = service.clone();
raft_service.rt.spawn(async move {
while !service_clone.closed.load(Ordering::Relaxed) {
let start_time = get_time();
let rpc_client = service_clone.raft_client.current_leader_rpc_client().await;
if let Ok(rpc_client) = rpc_client {
let _ping_res =
ImmeServiceClient::ping(DEFAULT_SERVICE_ID, &rpc_client, service_clone.id)
.await;
} else {
error!("Cannot find RPC client for membership heartbeat to leader");
}
let time_now = get_time();
let elapsed_time = time_now - start_time;
trace!(
"Membership ping at time {}, elapsed {}ms",
time_now,
elapsed_time
);
if (elapsed_time as u64) < PING_INTERVAL {
let wait_time = PING_INTERVAL - elapsed_time as u64;
trace!("Waiting membership heartbeat for {}ms", wait_time);
time::sleep(time::Duration::from_millis(wait_time)).await;
}
}
debug!("Member service closed");
});
return service;
}
pub fn close(&self) {
self.closed.store(true, Ordering::Relaxed);
}
pub async fn leave(&self) -> Result<bool, ExecError> {
self.close();
self.sm_client.leave(&self.id).await
}
pub async fn join_group(&self, group: &String) -> Result<bool, ExecError> {
self.member_client.join_group(group).await
}
pub async fn leave_group(&self, group: &String) -> Result<bool, ExecError> {
self.member_client.leave_group(group).await
}
pub fn client(&self) -> ObserverClient {
ObserverClient::new_from_sm(&self.sm_client)
}
pub fn get_server_id(&self) -> u64 {
self.id
}
}
impl Drop for MemberService {
fn drop(&mut self) {
let sm_client = self.sm_client.clone();
let self_id = self.id;
tokio::spawn(async move { sm_client.leave(&self_id).await }.boxed());
}
}
================================================
FILE: src/membership/mod.rs
================================================
// Group membership manager regardless actual raft members
pub mod client;
pub mod member;
pub mod server;
use crate::membership::client::Member as ClientMember;
use bifrost_plugins::hash_ident;
pub static DEFAULT_SERVICE_ID: u64 = hash_ident!(BIFROST_MEMBERSHIP_SERVICE) as u64;
pub mod raft {
use super::server::MemberGroup;
use super::*;
use std::collections::BTreeMap;
raft_state_machine! {
def cmd hb_online_changed(online: Vec<u64>, offline: Vec<u64>);
def cmd join(address: String) -> Option<u64>;
def cmd leave(id: u64) -> bool;
def cmd join_group(group_name: String, id: u64) -> bool;
def cmd leave_group(group: u64, id: u64) -> bool;
def cmd new_group(name: String) -> Result<u64, u64>;
def cmd del_group(id: u64) -> bool;
def qry group_leader(group: u64) -> Option<(Option<ClientMember>, u64)>;
def qry group_members (group: u64, online_only: bool) -> Option<(Vec<ClientMember>, u64)>;
def qry all_members (online_only: bool) -> (Vec<ClientMember>, u64);
def qry all_groups() -> BTreeMap<u64, MemberGroup>;
def sub on_group_member_offline(group: u64) -> (ClientMember, u64); //
def sub on_any_member_offline() -> (ClientMember, u64); //
def sub on_group_member_online(group: u64) -> (ClientMember, u64); //
def sub on_any_member_online() -> (ClientMember, u64); //
def sub on_group_member_joined(group: u64) -> (ClientMember, u64); //
def sub on_any_member_joined() -> (ClientMember, u64); //
def sub on_group_member_left(group: u64) -> (ClientMember, u64); //
def sub on_any_member_left() -> (ClientMember, u64); //
def sub on_group_leader_changed(group: u64) -> (Option<ClientMember>, Option<ClientMember>, u64);
}
}
// The service only responsible for receiving heartbeat and
// Updating last updated time
// Expired update time will trigger timeout in the raft state machine
mod heartbeat_rpc {
service! {
rpc ping(id: u64);
}
}
#[cfg(test)]
mod test {
use crate::membership::client::ObserverClient;
use crate::membership::member::MemberService;
use crate::membership::server::Membership;
use crate::raft::client::RaftClient;
use crate::raft::{Options, RaftService, Storage, DEFAULT_SERVICE_ID};
use crate::rpc::Server;
use crate::utils::time::async_wait_secs;
use futures::prelude::*;
use std::sync::atomic::*;
use std::sync::Arc;
#[tokio::test(flavor = "multi_thread")]
async fn primary() {
let _ = env_logger::builder().format_timestamp(None).try_init();
let addr = String::from("127.0.0.1:2100");
let raft_service = RaftService::new(Options {
storage: Storage::default(),
address: addr.clone(),
service_id: DEFAULT_SERVICE_ID,
});
info!("Creating server");
let server = Server::new(&addr);
info!("Register service");
server.register_service(&raft_service).await;
info!("Server listen and resume");
Server::listen_and_resume(&server).await;
info!("Start raft service");
RaftService::start(&raft_service, false).await;
info!("Bootstrap raft service");
raft_service.bootstrap().await;
info!("Creating membership service");
Membership::new(&server, &raft_service).await;
let group_1 = String::from("test_group_1");
let group_2 = String::from("test_group_2");
let group_3 = String::from("test_group_3");
info!("Creating raft client");
let wild_raft_client = RaftClient::new(&vec![addr.clone()], DEFAULT_SERVICE_ID)
.await
.unwrap();
info!("Create observer");
let client = ObserverClient::new(&wild_raft_client);
info!("Prepare subscription");
RaftClient::prepare_subscription(&server).await;
info!("Creating new group: {}", group_1);
client.new_group(&group_1).await.unwrap().unwrap();
info!("Creating new group {}", group_2);
client.new_group(&group_2).await.unwrap().unwrap();
info!("Creating new group {}", group_3);
client.new_group(&group_3).await.unwrap().unwrap();
let any_member_joined_count = Arc::new(AtomicUsize::new(0));
let any_member_left_count = Arc::new(AtomicUsize::new(0));
let any_member_offline_count = Arc::new(AtomicUsize::new(0));
let any_member_online_count = Arc::new(AtomicUsize::new(0));
let group_leader_changed_count = Arc::new(AtomicUsize::new(0));
let group_member_joined_count = Arc::new(AtomicUsize::new(0));
let group_member_left_count = Arc::new(AtomicUsize::new(0));
let group_member_online_count = Arc::new(AtomicUsize::new(0));
let group_member_offline_count = Arc::new(AtomicUsize::new(0));
let any_member_joined_count_clone = any_member_joined_count.clone();
let any_member_left_count_clone = any_member_left_count.clone();
let any_member_offline_count_clone = any_member_offline_count.clone();
let any_member_online_count_clone = any_member_online_count.clone();
let group_leader_changed_count_clone = group_leader_changed_count.clone();
let group_member_joined_count_clone = group_member_joined_count.clone();
let group_member_left_count_clone = group_member_left_count.clone();
let group_member_online_count_clone = group_member_online_count.clone();
let group_member_offline_count_clone = group_member_offline_count.clone();
info!("Subscribe on_any_member_joined");
client
.on_any_member_joined(move |_| {
any_member_joined_count_clone.fetch_add(1, Ordering::Relaxed);
future::ready(()).boxed()
})
.await
.unwrap()
.unwrap();
info!("Subscribe on_any_member_left");
client
.on_any_member_left(move |_| {
any_member_left_count_clone.fetch_add(1, Ordering::Relaxed);
future::ready(()).boxed()
})
.await
.unwrap()
.unwrap();
info!("Subscribe on_any_member_offline");
client
.on_any_member_offline(move |_| {
any_member_offline_count_clone.fetch_add(1, Ordering::Relaxed);
future::ready(()).boxed()
})
.await
.unwrap()
.unwrap();
info!("Subscribe on_any_member_online");
client
.on_any_member_online(move |_| {
any_member_online_count_clone.fetch_add(1, Ordering::Relaxed);
future::ready(()).boxed()
})
.await
.unwrap()
.unwrap();
info!("Subscribe on_group_leader_changed");
client
.on_group_leader_changed(
move |_| {
group_leader_changed_count_clone.fetch_add(1, Ordering::Relaxed);
future::ready(()).boxed()
},
&group_1,
)
.await
.unwrap()
.unwrap();
info!("Subscribe on_group_member_joined");
client
.on_group_member_joined(
move |_| {
group_member_joined_count_clone.fetch_add(1, Ordering::Relaxed);
future::ready(()).boxed()
},
&group_1,
)
.await
.unwrap()
.unwrap();
info!("Subscribe on_group_member_left");
client
.on_group_member_left(
move |_| {
group_member_left_count_clone.fetch_add(1, Ordering::Relaxed);
future::ready(()).boxed()
},
&group_1,
)
.await
.unwrap()
.unwrap();
info!("Subscribe on_group_member_online");
client
.on_group_member_online(
move |_| {
group_member_online_count_clone.fetch_add(1, Ordering::Relaxed);
future::ready(()).boxed()
},
&group_1,
)
.await
.unwrap()
.unwrap();
info!("Subscribe on_group_member_offline");
client
.on_group_member_offline(
move |_| {
group_member_offline_count_clone.fetch_add(1, Ordering::Relaxed);
future::ready(()).boxed()
},
&group_1,
)
.await
.unwrap()
.unwrap();
info!("New member1_raft_client");
let member1_raft_client = RaftClient::new(&vec![addr.clone()], DEFAULT_SERVICE_ID)
.await
.unwrap();
let member1_addr = String::from("server1");
info!("New member service {}", member1_addr);
let member1_svr =
MemberService::new(&member1_addr, &member1_raft_client, &raft_service).await;
info!("New member2_raft_client");
let member2_raft_client = RaftClient::new(&vec![addr.clone()], DEFAULT_SERVICE_ID)
.await
.unwrap();
let member2_addr = String::from("server2");
info!("New member service {}", member2_addr);
let member2_svr =
MemberService::new(&member2_addr, &member2_raft_client, &raft_service).await;
info!("New member3_raft_client");
let member3_raft_client = RaftClient::new(&vec![addr.clone()], DEFAULT_SERVICE_ID)
.await
.unwrap();
let member3_addr = String::from("server3");
info!("New member service {}", member3_addr);
let member3_svr =
MemberService::new(&member3_addr, &member3_raft_client, &raft_service).await;
info!("Member 1 join group 1");
member1_svr.join_group(&group_1).await.unwrap();
info!("Member 2 join group 1");
member2_svr.join_group(&group_1).await.unwrap();
info!("Member 3 join group 1");
member3_svr.join_group(&group_1).await.unwrap();
info!("Member 1 join group 2");
member1_svr.join_group(&group_2).await.unwrap();
info!("Member 2 join group 2");
member2_svr.join_group(&group_2).await.unwrap();
info!("Member 1 join group 3");
member1_svr.join_group(&group_3).await.unwrap();
info!("Checking group members after join");
assert_eq!(
member1_svr
.client()
.all_members(false)
.await
.unwrap()
.0
.len(),
3
);
assert_eq!(
member1_svr
.client()
.all_members(true)
.await
.unwrap()
.0
.len(),
3
);
assert_eq!(
member1_svr
.client()
.group_members(&group_1, false)
.await
.unwrap()
.unwrap()
.0
.len(),
3
);
assert_eq!(
member1_svr
.client()
.group_members(&group_1, true)
.await
.unwrap()
.unwrap()
.0
.len(),
3
);
assert_eq!(
member1_svr
.client()
.group_members(&group_2, false)
.await
.unwrap()
.unwrap()
.0
.len(),
2
);
assert_eq!(
member1_svr
.client()
.group_members(&group_2, true)
.await
.unwrap()
.unwrap()
.0
.len(),
2
);
assert_eq!(
member1_svr
.client()
.group_members(&group_3, false)
.await
.unwrap()
.unwrap()
.0
.len(),
1
);
assert_eq!(
member1_svr
.client()
.group_members(&group_3, true)
.await
.unwrap()
.unwrap()
.0
.len(),
1
);
member1_svr.close(); // close only end the heartbeat thread
info!("############### Waiting for membership changes ###############");
for i in 0..10 {
async_wait_secs().await;
}
info!("*************** Checking members ***************");
assert_eq!(
member1_svr
.client()
.all_members(false)
.await
.unwrap()
.0
.len(),
3
);
assert_eq!(
member1_svr
.client()
.all_members(true)
.await
.unwrap()
.0
.len(),
2
);
assert_eq!(
member1_svr
.client()
.group_members(&group_1, false)
.await
.unwrap()
.unwrap()
.0
.len(),
3
);
assert_eq!(
member1_svr
.client()
.group_members(&group_1, true)
.await
.unwrap()
.unwrap()
.0
.len(),
2
);
assert_eq!(
member1_svr
.client()
.group_members(&group_2, false)
.await
.unwrap()
.unwrap()
.0
.len(),
2
);
assert_eq!(
member1_svr
.client()
.group_members(&group_2, true)
.await
.unwrap()
.unwrap()
.0
.len(),
1
);
assert_eq!(
member1_svr
.client()
.group_members(&group_3, false)
.await
.unwrap()
.unwrap()
.0
.len(),
1
);
assert_eq!(
member1_svr
.client()
.group_members(&group_3, true)
.await
.unwrap()
.unwrap()
.0
.len(),
0
);
member2_svr.leave().await.unwrap(); // leave will report to the raft servers to remove it from the list
assert_eq!(
member1_svr
.client()
.all_members(false)
.await
.unwrap()
.0
.len(),
2
);
assert_eq!(
member1_svr
.client()
.all_members(true)
.await
.unwrap()
.0
.len(),
1
);
assert_eq!(
member1_svr
.client()
.group_members(&group_1, false)
.await
.unwrap()
.unwrap()
.0
.len(),
2
);
assert_eq!(
member1_svr
.client()
.group_members(&group_1, true)
.await
.unwrap()
.unwrap()
.0
.len(),
1
);
assert_eq!(
member1_svr
.client()
.group_members(&group_2, false)
.await
.unwrap()
.unwrap()
.0
.len(),
1
);
assert_eq!(
member1_svr
.client()
.group_members(&group_2, true)
.await
.unwrap()
.unwrap()
.0
.len(),
0
);
assert_eq!(
member1_svr
.client()
.group_members(&group_3, false)
.await
.unwrap()
.unwrap()
.0
.len(),
1
);
assert_eq!(
member1_svr
.client()
.group_members(&group_3, true)
.await
.unwrap()
.unwrap()
.0
.len(),
0
);
async_wait_secs().await;
info!("=========== Checking event trigger ===========");
assert_eq!(any_member_joined_count.load(Ordering::Relaxed), 3);
assert_eq!(any_member_left_count.load(Ordering::Relaxed), 1);
assert_eq!(any_member_offline_count.load(Ordering::Relaxed), 1);
assert_eq!(any_member_online_count.load(Ordering::Relaxed), 0); // no server online from offline
assert!(group_leader_changed_count.load(Ordering::Relaxed) > 0); // Number depends on hashing
assert_eq!(group_member_joined_count.load(Ordering::Relaxed), 3);
// assert_eq!(group_member_left_count.load(Ordering::Relaxed), 2); // this test case is unstable
assert_eq!(group_member_online_count.load(Ordering::Relaxed), 0);
assert_eq!(group_member_offline_count.load(Ordering::Relaxed), 1);
}
}
================================================
FILE: src/membership/server.rs
================================================
use super::heartbeat_rpc::*;
use super::raft::*;
use super::*;
use crate::membership::client::Member as ClientMember;
use crate::raft::state_machine::callback::server::{notify as cb_notify, SMCallback};
use crate::raft::state_machine::StateMachineCtl;
use crate::raft::{LogEntry, PlaneId, RaftMsg, RaftService, Service as raft_svr_trait};
use crate::rpc::Server;
use crate::utils::time;
use crate::utils::time::get_time;
use bifrost_hasher::hash_str;
use futures::prelude::future::*;
use futures::prelude::*;
use futures::stream::FuturesUnordered;
use lightning::map::Map;
use lightning::map::PtrHashMap;
use serde::Deserialize;
use serde::Serialize;
use std::collections::BTreeMap;
use std::collections::{BTreeSet, HashSet};
use std::future::Future;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::Arc;
use std::time as std_time;
use tokio::time as async_time;
static MAX_TIMEOUT: i64 = 10_000; // 10 secs timeout before considering a member potentially offline
static OFFLINE_GRACE_CHECKS: u8 = 3; // Number of consecutive timeout checks before marking offline
static ONLINE_GRACE_CHECKS: u8 = 2; // Number of consecutive successful checks before marking back online
static MIN_STATE_CHANGE_INTERVAL: i64 = 5_000; // Minimum 5 seconds between state changes (anti-flapping)
#[derive(Clone, Copy)]
struct HBStatus {
last_updated: i64,
online: bool,
consecutive_failures: u8, // Count of consecutive timeout checks while supposedly online
consecutive_successes: u8, // Count of consecutive successful checks while supposedly offline
last_state_change: i64, // Timestamp of last online/offline state change
}
pub struct HeartbeatService {
status: PtrHashMap<u64, HBStatus>,
raft_service: Arc<RaftService>,
closed: AtomicBool,
was_leader: AtomicBool,
watcher_handle: std::sync::Mutex<Option<tokio::task::JoinHandle<()>>>,
}
impl Service for HeartbeatService {
fn ping(&self, id: u64) -> BoxFuture<()> {
async move {
let current_time = time::get_time();
// Update existing status or create new one
let old_status = self.status.get(&id);
let new_status = if let Some(mut status) = old_status {
let elapsed = current_time - status.last_updated;
status.last_updated = current_time;
// Reset failure counter on successful ping, but keep other fields
status.consecutive_failures = 0;
if !status.online {
// Member is recovering, increment success counter
status.consecutive_successes += 1;
}
trace!("Updated heartbeat for member {}, elapsed {}ms", id, elapsed);
status
} else {
// First time seeing this member
trace!("First heartbeat from member {}", id);
HBStatus {
online: true,
last_updated: current_time,
consecutive_failures: 0,
consecutive_successes: 0,
last_state_change: current_time,
}
};
self.status.insert(id, new_status);
}
.boxed()
}
}
impl HeartbeatService {
async fn update_raft(&self, online: &Vec<u64>, offline: &Vec<u64>) {
let log = commands::hb_online_changed::new(online, offline);
// Encode to state machine command
let (fn_id, _, data) = log.encode();
self.raft_service
.c_command(
PlaneId::type1(),
LogEntry {
id: 0,
term: 0,
sm_id: DEFAULT_SERVICE_ID,
fn_id,
data,
},
)
.await;
}
async fn transfer_leadership(&self) {
//update timestamp for every alive server to give them a grace period
let all_entries = self.status.entries();
let current_time = get_time();
let mut online_count = 0;
for (id, mut stat) in all_entries {
if stat.online {
stat.last_updated = current_time;
// Reset counters to give all members a fresh start under new leader
stat.consecutive_failures = 0;
stat.consecutive_successes = 0;
self.status.insert(id, stat);
online_count += 1;
}
}
info!(
"Leadership transferred, reset heartbeat status for {} online members",
online_count
);
}
pub async fn shutdown(&self) {
info!("Shutting down heartbeat service");
self.closed.store(true, Ordering::Relaxed);
// Wait for the watcher task to complete
match self.watcher_handle.lock() {
Ok(mut guard) => {
if let Some(handle) = guard.take() {
let _ = handle.await;
}
}
Err(e) => {
error!(
"Failed to acquire watcher handle lock during shutdown: {}",
e
);
}
}
}
}
dispatch_rpc_service_functions!(HeartbeatService);
service_with_id!(HeartbeatService, DEFAULT_SERVICE_ID);
#[derive(Debug)]
struct Member {
pub address: String,
pub groups: HashSet<u64>,
}
#[derive(Clone, Debug, Serialize, Deserialize)]
pub struct MemberGroup {
members: BTreeSet<u64>,
leader: Option<u64>,
name: String,
}
/// Membership service manages member groups and heartbeat status.
///
/// IMPORTANT: This service does NOT persist its state to disk. On each restart,
/// it starts with empty state and rebuilds membership through:
/// 1. Members sending join() commands
/// 2. Heartbeat ping() messages updating online/offline status
/// 3. Group membership operations (join_group, leave_group, etc.)
///
/// This design ensures membership always reflects current network reality,
/// not stale persisted state that might be outdated after crashes.
pub struct Membership {
heartbeat: Arc<HeartbeatService>,
groups: BTreeMap<u64, MemberGroup>,
members: BTreeMap<u64, Member>,
callback: Option<SMCallback>,
version: u64,
}
impl Drop for Membership {
fn drop(&mut self) {
self.heartbeat.closed.store(true, Ordering::Relaxed)
}
}
impl Membership {
/// Creates a new Membership service with fresh, empty state.
///
/// The service will discover members through:
/// - join() commands from members joining the cluster
/// - ping() heartbeats indicating member liveness
/// - join_group/leave_group commands for group management
///
/// No state is recovered from disk - all membership is learned from the network.
pub async fn new(server: &Arc<Server>, raft_service: &Arc<RaftService>) {
let service = Arc::new(HeartbeatService {
status: PtrHashMap::with_capacity(32),
closed: AtomicBool::new(false),
raft_service: raft_service.clone(),
was_leader: AtomicBool::new(false),
watcher_handle: std::sync::Mutex::new(None),
});
let service_clone = service.clone();
let service_for_task = service.clone();
let handle = raft_service.rt.spawn(async move {
info!("Starting membership heartbeat watcher (fresh state, learning from network)");
while !service_for_task.closed.load(Ordering::Relaxed) {
let service = &service_for_task;
let start_time = get_time();
let is_leader = service.raft_service.is_leader();
let was_leader = service.was_leader.load(Ordering::Relaxed);
if !was_leader && is_leader {
// Transferred leader will skip checking all member timeout for once
service.transfer_leadership().await
}
if was_leader != is_leader {
service.was_leader.store(is_leader, Ordering::Relaxed);
}
if is_leader {
trace!("Resync Membership as leader id {}", service.raft_service.id);
let mut outdated_members: Vec<u64> = Vec::new();
let mut back_in_members: Vec<u64> = Vec::new();
{
let all_entries = service.status.entries();
let mut members_to_update = vec![];
for (id, mut status) in all_entries {
let last_updated = status.last_updated;
let alive = (start_time < last_updated)
|| ((start_time - last_updated) < MAX_TIMEOUT);
let time_since_last_change = start_time - status.last_state_change;
// Finding new offline servers (with grace period)
if status.online && !alive {
status.consecutive_failures += 1;
status.consecutive_successes = 0;
// Only mark offline after multiple consecutive failures AND minimum interval
if status.consecutive_failures >= OFFLINE_GRACE_CHECKS
&& time_since_last_change >= MIN_STATE_CHANGE_INTERVAL {
warn!(
"Marking member {} as offline after {} consecutive timeout checks ({}ms since last update)",
id, status.consecutive_failures, start_time - last_updated
);
status.online = false;
status.last_state_change = start_time;
status.consecutive_failures = 0;
outdated_members.push(id);
} else {
debug!(
"Member {} timeout check {}/{} ({}ms since last update, {}ms since last state change)",
id, status.consecutive_failures, OFFLINE_GRACE_CHECKS,
start_time - last_updated, time_since_last_change
);
}
members_to_update.push((id, status));
}
// Finding new online servers (with grace period)
else if !status.online && alive {
status.consecutive_successes += 1;
status.consecutive_failures = 0;
// Only mark online after multiple consecutive successes AND minimum interval
if status.consecutive_successes >= ONLINE_GRACE_CHECKS
&& time_since_last_change >= MIN_STATE_CHANGE_INTERVAL {
info!(
"Marking member {} as back online after {} consecutive successful checks",
id, status.consecutive_successes
);
status.online = true;
status.last_state_change = start_time;
status.consecutive_successes = 0;
back_in_members.push(id);
} else {
debug!(
"Member {} recovery check {}/{} ({}ms since last state change)",
id, status.consecutive_successes, ONLINE_GRACE_CHECKS,
time_since_last_change
);
}
members_to_update.push((id, status));
}
// Member is consistently online or offline
else if alive {
// Member is online and responsive - reset counters
if status.consecutive_failures > 0 || status.consecutive_successes > 0 {
status.consecutive_failures = 0;
status.consecutive_successes = 0;
members_to_update.push((id, status));
}
}
}
for (id, s) in members_to_update {
service.status.insert(id, s);
}
}
if back_in_members.len() + outdated_members.len() > 0 {
debug!(
"Update member state machine for {} online, {} offline",
back_in_members.len(),
outdated_members.len()
);
service
.update_raft(&back_in_members, &outdated_members)
.await;
}
}
let end_time = get_time();
let time_took = end_time - start_time;
let interval = 500; // in ms
if time_took < interval {
let time_to_wait = interval - time_took;
trace!(
"Membership resync completed, waiting for {}ms for next resync",
time_to_wait
);
async_time::sleep(std_time::Duration::from_millis(time_to_wait as u64)).await
} else {
trace!(
"Membership resync completed, left behine {}ms for next resync",
time_took - interval
);
}
}
info!("Membership heartbeat watcher stopped gracefully");
});
// Store the handle for graceful shutdown
match service.watcher_handle.lock() {
Ok(mut guard) => {
*guard = Some(handle);
}
Err(e) => {
error!(
"Failed to acquire watcher handle lock during initialization: {}",
e
);
}
}
// Create membership service with EMPTY state.
// It will learn all membership from the network through:
// 1. join() commands as members join
// 2. ping() heartbeats for liveness tracking
// 3. Group operations (join_group, leave_group, etc.)
let mut membership_service = Membership {
heartbeat: service_clone.clone(),
groups: BTreeMap::new(), // Empty groups - will be populated as groups are created
members: BTreeMap::new(), // Empty members - will be populated as members join
callback: None,
version: 0, // Version starts at 0
};
membership_service.init_callback(raft_service).await;
raft_service
.register_state_machine(Box::new(membership_service))
.await;
server.register_service(&service_clone).await;
}
async fn compose_client_member(&self, id: u64) -> Option<ClientMember> {
let member = self.members.get(&id)?;
let status = self.heartbeat.status.get(&id)?;
Some(ClientMember {
id,
address: member.address.clone(),
online: status.online,
})
}
async fn init_callback(&mut self, raft_service: &Arc<RaftService>) {
self.callback = Some(SMCallback::new(self.id(), raft_service.clone()).await);
}
async fn notify_for_member_online(&self, id: u64) {
debug!("Notifying member {} online", id);
let client_member = match self.compose_client_member(id).await {
Some(member) => member,
None => {
error!(
"Failed to compose client member {} for online notification",
id
);
return;
}
};
let version = self.version;
cb_notify(
&self.callback,
commands::on_any_member_online::new(),
|| (client_member.clone(), version),
)
.await;
if let Some(ref member) = self.members.get(&id) {
for group in &member.groups {
cb_notify(
&self.callback,
commands::on_group_member_online::new(group),
|| (client_member.clone(), version),
)
.await;
}
}
}
async fn notify_for_member_offline(&self, id: u64) {
debug!("Notifying member {} offline", id);
let client_member = match self.compose_client_member(id).await {
Some(member) => member,
None => {
error!(
"Failed to compose client member {} for offline notification",
id
);
return;
}
};
let version = self.version;
cb_notify(
&self.callback,
commands::on_any_member_offline::new(),
|| (client_member.clone(), version),
)
.await;
if let Some(ref member) = self.members.get(&id) {
for group in &member.groups {
cb_notify(
&self.callback,
commands::on_group_member_offline::new(group),
|| (client_member.clone(), version),
)
.await;
}
}
}
async fn notify_for_member_left(&self, id: u64) {
debug!("Notifying member {} left", id);
let client_member = match self.compose_client_member(id).await {
Some(member) => member,
None => {
error!(
"Failed to compose client member {} for left notification",
id
);
return;
}
};
let version = self.version;
cb_notify(&self.callback, commands::on_any_member_left::new(), || {
(client_member.clone(), version)
})
.await;
if let Some(ref member) = self.members.get(&id) {
for group in &member.groups {
self.notify_for_group_member_left(*group, &client_member)
.await
}
}
}
async fn notify_for_group_member_left(&self, group: u64, member: &ClientMember) {
debug!("Notifying member {:?} left group {}", member, group);
cb_notify(
&self.callback,
commands::on_group_member_left::new(&group),
|| (member.clone(), self.version),
)
.await;
}
async fn leave_group_(&mut self, group_id: u64, id: u64, need_notify: bool) -> bool {
let mut success = false;
if let Some(ref mut group) = self.groups.get_mut(&group_id) {
if let Some(ref mut member) = self.members.get_mut(&id) {
group.members.remove(&id);
member.groups.remove(&group_id);
success = true;
}
}
if success {
if need_notify {
if let Some(client_member) = self.compose_client_member(id).await {
self.notify_for_group_member_left(group_id, &client_member)
.await;
} else {
error!(
"Failed to compose client member {} for group {} leave notification",
id, group_id
);
}
}
self.group_leader_candidate_unavailable(group_id, id).await;
true
} else {
false
}
}
fn member_groups(&self, member: u64) -> Option<HashSet<u64>> {
if let Some(member) = self.members.get(&member) {
Some(member.groups.clone())
} else {
None
}
}
async fn group_first_online_member_id(&self, group: u64) -> Result<Option<u64>, ()> {
if let Some(group) = self.groups.get(&group) {
for member in group.members.iter() {
if let Some(member_stat) = self.heartbeat.status.get(&member) {
if member_stat.online {
return Ok(Some(*member));
}
}
}
Ok(None)
} else {
Err(())
}
}
async fn change_leader(&mut self, group_id: u64, new: Option<u64>) -> Result<(), ()> {
let mut old: Option<u64> = None;
let mut changed = false;
if let Some(group) = self.groups.get_mut(&group_id) {
old = group.leader;
if old != new {
group.leader = new;
changed = true;
}
}
if changed {
let version = self.version;
let old_leader = if let Some(id_opt) = old {
self.compose_client_member(id_opt).await
} else {
None
};
let new_leader = if let Some(id_opt) = new {
self.compose_client_member(id_opt).await
} else {
None
};
cb_notify(
&self.callback,
commands::on_group_leader_changed::new(&group_id),
move || (old_leader, new_leader, version),
)
.await;
Ok(())
} else {
Err(())
}
}
async fn group_leader_candidate_available(&mut self, group_id: u64, member: u64) {
// if the group does not have a leader, assign the available member
let mut leader_changed = false;
if let Some(group) = self.groups.get_mut(&group_id) {
if group.leader == None {
leader_changed = true;
}
}
if leader_changed {
if let Err(_) = self.change_leader(group_id, Some(member)).await {
error!(
"Failed to change leader for group {} to member {}",
group_id, member
);
}
}
}
async fn group_leader_candidate_unavailable(&mut self, group_id: u64, member: u64) {
// if the group have a leader that is the same as the member, reelect
let mut reelected = false;
if let Some(group) = self.groups.get_mut(&group_id) {
if group.leader == Some(member) {
reelected = true;
}
}
if reelected {
match self.group_first_online_member_id(group_id).await {
Ok(online_id) => {
if let Err(_) = self.change_leader(group_id, online_id).await {
error!("Failed to change leader for group {} after member {} became unavailable", group_id, member);
}
}
Err(_) => {
error!("Failed to find online member for group {} after member {} became unavailable", group_id, member);
}
}
}
}
async fn leader_candidate_available(&mut self, member: u64) {
if let Some(groups) = self.member_groups(member) {
for group in groups {
self.group_leader_candidate_available(group, member).await
}
}
}
async fn leader_candidate_unavailable(&mut self, member: u64) {
if let Some(groups) = self.member_groups(member) {
for group in groups {
self.group_leader_candidate_unavailable(group, member).await
}
}
}
}
impl StateMachineCmds for Membership {
fn hb_online_changed(&mut self, online: Vec<u64>, offline: Vec<u64>) -> BoxFuture<()> {
debug!(
"Member status changed, back online {}, gone offline {}",
online.len(),
offline.len()
);
async move {
self.version += 1;
let current_time = time::get_time();
{
for id in &online {
if let Some(mut stat) = self.heartbeat.status.get(&id) {
stat.online = true;
stat.last_state_change = current_time;
// Reset counters after state change is confirmed
stat.consecutive_failures = 0;
stat.consecutive_successes = 0;
self.heartbeat.status.insert(*id, stat);
}
}
for id in &offline {
if let Some(mut stat) = self.heartbeat.status.get(&id) {
stat.online = false;
stat.last_state_change = current_time;
// Reset counters after state change is confirmed
stat.consecutive_failures = 0;
stat.consecutive_successes = 0;
self.heartbeat.status.insert(*id, stat);
}
}
}
for id in online {
self.notify_for_member_online(id).await;
self.leader_candidate_available(id).await;
}
for id in offline {
self.notify_for_member_offline(id).await;
self.leader_candidate_unavailable(id).await;
}
}
.boxed()
}
fn join(&mut self, address: String) -> BoxFuture<Option<u64>> {
async move {
self.version += 1;
let id = hash_str(&address);
let mut joined = false;
{
let current_time = time::get_time();
self.members.entry(id).or_insert_with(|| {
joined = true;
Member {
address: address.clone(),
groups: HashSet::new(),
}
});
self.heartbeat.status.insert(
id,
HBStatus {
last_updated: current_time,
online: true,
consecutive_failures: 0,
consecutive_successes: 0,
last_state_change: current_time,
},
);
}
if joined {
match self.compose_client_member(id).await {
Some(composed_client_member) => {
cb_notify(
&self.callback,
commands::on_any_member_joined::new(),
|| (composed_client_member, self.version),
)
.await;
Some(id)
}
None => {
error!("Failed to compose client member {} after join", id);
None
}
}
} else {
None
}
}
.boxed()
}
fn leave(&mut self, id: u64) -> BoxFuture<bool> {
async move {
if !self.members.contains_key(&id) {
return false;
};
self.version += 1;
let mut groups: Vec<u64> = Vec::new();
if let Some(member) = self.members.get(&id) {
for group in &member.groups {
groups.push(*group);
}
}
self.notify_for_member_left(id).await;
for group_id in groups {
self.leave_group_(group_id, id, false).await;
}
// in this part we will not do leader_candidate_unavailable
// because it have already been triggered by leave_group_
// in the loop above
self.heartbeat.status.remove(&id);
self.members.remove(&id);
true
}
.boxed()
}
fn join_group(&mut self, group_name: String, id: u64) -> BoxFuture<bool> {
async move {
let group_id = hash_str(&group_name);
self.version += 1;
let mut success = false;
if !self.groups.contains_key(&group_id) {
if let Err(existing_id) = self.new_group(group_name.clone()).await {
debug!(
"Group {} already exists with id {}",
group_name, existing_id
);
}
} // create group if not exists
if let Some(ref mut group) = self.groups.get_mut(&group_id) {
if let Some(ref mut member) = self.members.get_mut(&id) {
group.members.insert(id);
member.groups.insert(group_id);
success = true;
}
}
if success {
match self.compose_client_member(id).await {
Some(composed_member) => {
cb_notify(
&self.callback,
commands::on_group_member_joined::new(&group_id),
|| (composed_member, self.version),
)
.await;
self.group_leader_candidate_available(group_id, id).await;
true
}
None => {
error!(
"Failed to compose client member {} for group {} join notification",
id, group_id
);
false
}
}
} else {
false
}
}
.boxed()
}
fn leave_group(&mut self, group_id: u64, id: u64) -> BoxFuture<bool> {
async move {
self.version += 1;
self.leave_group_(group_id, id, true).await
}
.boxed()
}
fn new_group(&mut self, name: String) -> BoxFuture<Result<u64, u64>> {
async move {
self.version += 1;
let id = hash_str(&name);
let mut inserted = false;
self.groups.entry(id).or_insert_with(|| {
inserted = true;
MemberGroup {
members: BTreeSet::new(),
leader: None,
name: name.clone(),
}
});
if inserted {
Ok(id)
} else {
Err(id)
}
}
.boxed()
}
fn del_group(&mut self, id: u64) -> BoxFuture<bool> {
async move {
self.version += 1;
let mut members: Option<BTreeSet<u64>> = None;
if let Some(group) = self.groups.get(&id) {
members = Some(group.members.clone());
}
if let Some(members) = members {
for member_id in members {
if let Some(ref mut member) = self.members.get_mut(&member_id) {
member.groups.remove(&id);
}
}
self.groups.remove(&id);
true
} else {
false
}
}
.boxed()
}
fn group_leader(&self, group_id: u64) -> BoxFuture<Option<(Option<ClientMember>, u64)>> {
async move {
if let Some(group) = self.groups.get(&group_id) {
Some((
match group.leader {
Some(id) => self.compose_client_member(id).await,
None => None,
},
self.version,
))
} else {
None
}
}
.boxed()
}
fn group_members(
&self,
group: u64,
online_only: bool,
) -> BoxFuture<Option<(Vec<ClientMember>, u64)>> {
async move {
if let Some(group) = self.groups.get(&group) {
let futs: FuturesUnordered<_> = group
.members
.iter()
.map(|id| self.compose_client_member(*id))
.collect();
let members: Vec<_> = futs.collect().await;
Some((
members
.into_iter()
.filter_map(|member| member)
.filter(|member| !online_only || member.online)
.collect(),
self.version,
))
} else {
None
}
}
.boxed()
}
fn all_members(&self, online_only: bool) -> BoxFuture<(Vec<ClientMember>, u64)> {
async move {
let futs: FuturesUnordered<_> = self
.members
.iter()
.map(|(id, _)| self.compose_client_member(*id))
.collect();
let members: Vec<_> = futs.collect().await;
(
members
.into_iter()
.filter_map(|member| member)
.filter(|member| !online_only || member.online)
.collect(),
self.version,
)
}
.boxed()
}
fn all_groups(&self) -> BoxFuture<BTreeMap<u64, MemberGroup>> {
future::ready(self.groups.clone()).boxed()
}
}
impl StateMachineCtl for Membership {
raft_sm_complete!();
fn id(&self) -> u64 {
DEFAULT_SERVICE_ID
}
fn snapshot(&self) -> Vec<u8> {
// Membership service intentionally does NOT persist its state.
// It starts fresh on each restart and learns membership from the network
// via heartbeats and join/leave commands.
// This ensures membership reflects current network reality, not stale disk state.
unreachable!()
}
fn recover(&mut self, _: Vec<u8>) -> BoxFuture<()> {
// Membership service does not recover from snapshots.
// It rebuilds its state from network discovery and heartbeats.
future::ready(()).boxed()
}
fn recoverable(&self) -> bool {
false
}
}
================================================
FILE: src/plugins/Cargo.toml
================================================
[package]
name = "bifrost_plugins"
version = "0.1.0"
authors = ["Hao Shi <shisoftgenius@gmail.com>"]
[lib]
proc-macro = true
[dependencies]
bifrost_hasher = { path = "../hasher" }
syn = "2"
quote = "1"
================================================
FILE: src/plugins/src/lib.rs
================================================
extern crate bifrost_hasher;
extern crate proc_macro;
extern crate syn;
use bifrost_hasher::hash_str;
use proc_macro::TokenStream;
use proc_macro::TokenTree;
use syn::{parse_macro_input, LitStr};
#[proc_macro]
pub fn hash_ident(item: TokenStream) -> TokenStream {
let item_clone = item.clone();
let tokens: Vec<_> = item.into_iter().collect();
if tokens.len() != 1 {
panic!(
"argument should be a single identifier, but got {} arguments {:?}",
tokens.len(),
tokens
);
}
let text = match tokens[0] {
TokenTree::Ident(ref ident) => ident.to_string(),
_ => parse_macro_input!(item_clone as LitStr).value(),
// _ => panic!("argument only support ident or string literal, found '{:?}', parsing {:?}", tokens, tokens[0])
};
let text = &*text;
let str = String::from(text);
format!("{}", hash_str(&str)).parse().unwrap()
}
================================================
FILE: src/proc_macro/Cargo.toml
================================================
[package]
name = "bifrost_proc_macro"
version = "0.1.0"
edition = "2021"
# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
[lib]
proc-macro = true
[dependencies]
syn = { version = "*", features = ["extra-traits"] }
quote = "*"
proc-macro2 = "*"
================================================
FILE: src/proc_macro/src/lib.rs
================================================
use proc_macro::TokenStream;
use quote::quote;
use syn::{
parse::{Parse, ParseBuffer, ParseStream},
parse_macro_input,
punctuated::Punctuated,
FnArg, Ident, ItemTrait, Lifetime, Pat, PatType, Result, Token, TraitItem, TraitItemFn, Type,
TypeReference, TypeTuple,
};
struct Args {
args: Punctuated<FnArg, Token![,]>,
}
impl Parse for Args {
fn parse(input: ParseStream) -> Result<Self> {
let args = Punctuated::parse_terminated(input)?;
Ok(Args { args })
}
}
#[proc_macro]
pub fn adjust_caller_identifiers(input: TokenStream) -> TokenStream {
let input = syn::parse_macro_input!(input as Args);
let output = input
.args
.into_iter()
.map(|arg| match arg {
FnArg::Typed(pat_type) => {
let pat = &*pat_type.pat;
let ty = &*pat_type.ty;
match (&pat, ty) {
(Pat::Ident(pat_ident), Type::Reference(_)) => {
let ident = &pat_ident.ident;
quote! { ref #ident }
}
(Pat::Ident(pat_ident), Type::Group(group)) => {
let ident = &pat_ident.ident;
if let Type::Reference(_) = &*group.elem {
quote! { ref #ident }
} else {
quote! { #ident }
}
}
(Pat::Ident(pat_ident), _) => {
let ident = &pat_ident.ident;
quote! { #ident }
}
_ => panic!("Unsupported pattern!"),
}
}
_ => panic!("Variadic arguments are not supported!"),
})
.collect::<Vec<_>>();
quote! {
( #(#output),* )
}
.into()
}
#[proc_macro]
pub fn adjust_function_signature(input: TokenStream) -> TokenStream {
let input = parse_macro_input!(input as TraitItemFn);
let mut output_trait_fn = input.clone();
let sig = &mut output_trait_fn.sig;
// eprintln!("Adjust {:?}", sig);
for input in &mut sig.inputs {
match input {
FnArg::Typed(pat_type) => {
// eprintln!("Checking lifetime {:?}", pat_type);
match *pat_type.ty {
Type::Reference(ref mut ref_type) => {
if ref_type.lifetime.is_none() {
ref_type.lifetime =
Some(Lifetime::new("'a", proc_macro2::Span::call_site()));
//eprintln!("Assigning lifetime {:?}", ref_type);
}
}
Type::Group(ref mut group) => {
if let Type::Reference(ref mut ref_type) = &mut *group.elem {
if ref_type.lifetime.is_none() {
ref_type.lifetime =
Some(Lifetime::new("'a", proc_macro2::Span::call_site()));
//eprintln!("Assigning lifetime {:?}", ref_type);
}
}
}
_ => {}
}
}
FnArg::Receiver(ref mut receiver) => {
if let &mut Some((_, ref mut lifetime)) = &mut receiver.reference {
if lifetime.is_none() {
*lifetime = Some(Lifetime::new("'a", proc_macro2::Span::call_site()));
}
}
}
_ => {}
}
}
quote!(#output_trait_fn).into()
}
#[proc_macro]
pub fn deref_tuple_types(input: TokenStream) -> TokenStream {
let input = parse_macro_input!(input as TypeTuple);
let transformed_types: Vec<_> = input
.elems
.into_iter()
.map(|ty| match ty {
Type::Reference(TypeReference { elem, .. }) => *elem,
Type::Group(group) => {
if let Type::Reference(TypeReference { elem, .. }) = *group.elem {
*elem
} else {
Type::Group(group)
}
}
other => other,
})
.collect();
let tokens = quote! { (#(#transformed_types),*) };
tokens.into()
}
================================================
FILE: src/raft/client.rs
================================================
use super::*;
use crate::raft::state_machine::callback::client::SubscriptionService;
use crate::raft::state_machine::callback::SubKey;
use crate::raft::state_machine::configs::commands::{
del_member_ as conf_del_member, member_address as conf_member_address,
new_member_ as conf_new_member, subscribe as conf_subscribe, unsubscribe as conf_unsubscribe,
};
use crate::raft::state_machine::master::ExecError;
use crate::raft::state_machine::StateMachineClient;
use crate::rpc;
use bifrost_hasher::{hash_bytes, hash_str};
use futures::future::BoxFuture;
use std::clone::Clone;
use std::cmp::max;
use std::collections::{BTreeMap, HashMap, HashSet};
use std::iter::FromIterator;
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::Arc;
use std::time::Duration;
use tokio::time::sleep;
const ORDERING: Ordering = Ordering::Relaxed;
pub type Client = Arc<AsyncServiceClient>;
pub type SubscriptionReceipt = (SubKey, u64);
lazy_static! {
pub static ref CALLBACK: RwLock<Option<Arc<SubscriptionService>>> = RwLock::new(None);
}
#[derive(Debug)]
pub enum ClientError {
LeaderIdValid,
ServerUnreachable,
}
impl std::fmt::Display for ClientError {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
ClientError::LeaderIdValid => write!(f, "leader id is invalid"),
ClientError::ServerUnreachable => write!(f, "seed nodes are unreachable"),
}
}
}
impl std::error::Error for ClientError {}
#[derive(Debug)]
pub enum SubscriptionError {
RemoteError,
SubServiceNotSet,
CannotFindSubId,
}
struct PlaneClientState {
pos: AtomicU64,
leader_id: AtomicU64,
last_log_id: AtomicU64,
last_log_term: AtomicU64,
}
struct Members {
clients: BTreeMap<u64, Client>,
id_map: HashMap<u64, String>,
}
pub trait AsRaftPlaneClient: Send + Sync {
fn as_raft_plane_client(self: &Arc<Self>) -> Arc<RaftPlaneClient>;
}
#[derive(Clone)]
pub struct RaftPlaneClient {
client: Arc<RaftClient>,
plane_id: PlaneId,
}
impl RaftPlaneClient {
pub fn plane_id(&self) -> PlaneId {
self.plane_id
}
pub async fn execute<R, M>(&self, sm_id: u64, msg: M) -> Result<R, ExecError>
where
R: 'static,
M: RaftMsg<R> + 'static,
{
self.client
.execute_on_plane(self.plane_id, sm_id, msg)
.await
}
pub async fn subscribe<M, R, F>(
&self,
sm_id: u64,
msg: M,
f: F,
) -> Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>
where
M: RaftMsg<R> + 'static,
R: 'static + Send,
F: Fn(R) -> BoxFuture<'static, ()> + 'static + Send + Sync,
{
self.client
.subscribe_on_plane(self.plane_id, sm_id, msg, f)
.await
}
pub async fn unsubscribe(
&self,
receipt: SubscriptionReceipt,
) -> Result<Result<(), SubscriptionError>, ExecError> {
self.client.unsubscribe(receipt).await
}
pub async fn cluster_info(&self) -> Result<ClientClusterInfo, ExecError> {
self.client.cluster_info_on_plane(self.plane_id).await
}
pub async fn have_state_machine(&self, sm_id: u64) -> Result<bool, ExecError> {
self.client
.have_state_machine_on_plane(self.plane_id, sm_id)
.await
}
}
pub struct RaftClient {
members: RwLock<Members>,
type1_state: Arc<PlaneClientState>,
plane_states: RwLock<HashMap<PlaneId, Arc<PlaneClientState>>>,
service_id: u64,
}
impl AsRaftPlaneClient for RaftClient {
fn as_raft_plane_client(self: &Arc<Self>) -> Arc<RaftPlaneClient> {
self.plane(PlaneId::type1())
}
}
impl AsRaftPlaneClient for RaftPlaneClient {
fn as_raft_plane_client(self: &Arc<Self>) -> Arc<RaftPlaneClient> {
self.clone()
}
}
impl RaftClient {
fn new_plane_state() -> Arc<PlaneClientState> {
Arc::new(PlaneClientState {
pos: AtomicU64::new(rand::random::<u64>()),
leader_id: AtomicU64::new(0),
last_log_id: AtomicU64::new(0),
last_log_term: AtomicU64::new(0),
})
}
pub async fn new(servers: &Vec<String>, service_id: u64) -> Result<Arc<Self>, ClientError> {
let client = RaftClient {
members: RwLock::new(Members {
clients: BTreeMap::new(),
id_map: HashMap::new(),
}),
type1_state: Self::new_plane_state(),
plane_states: RwLock::new(HashMap::new()),
service_id,
};
client.update_info(servers).await?;
Ok(Arc::new(client))
}
async fn plane_state(&self, plane_id: PlaneId) -> Arc<PlaneClientState> {
if plane_id.is_type1() {
return self.type1_state.clone();
}
{
let states = self.plane_states.read().await;
if let Some(state) = states.get(&plane_id) {
return state.clone();
}
}
let mut states = self.plane_states.write().await;
states
.entry(plane_id)
.or_insert_with(Self::new_plane_state)
.clone()
}
pub fn plane(self: &Arc<Self>, plane_id: PlaneId) -> Arc<RaftPlaneClient> {
Arc::new(RaftPlaneClient {
client: self.clone(),
plane_id,
})
}
pub fn type1(self: &Arc<Self>) -> Arc<RaftPlaneClient> {
self.plane(PlaneId::type1())
}
pub async fn add_root_member(&self, address: &String) -> Result<bool, ExecError> {
self.execute(CONFIG_SM_ID, conf_new_member::new(address))
.await
}
pub async fn remove_root_member(&self, address: &String) -> Result<(), ExecError> {
self.execute(CONFIG_SM_ID, conf_del_member::new(address))
.await
}
pub async fn root_member_addresses(&self) -> Result<Vec<String>, ExecError> {
self.execute(CONFIG_SM_ID, conf_member_address::new()).await
}
pub async fn prepare_subscription(server: &Arc<rpc::Server>) -> Option<()> {
let mut callback = CALLBACK.write().await;
return if callback.is_none() {
let sub_service = SubscriptionService::initialize(&server).await;
*callback = Some(sub_service.clone());
Some(())
} else {
None
};
}
async fn cluster_info<'a>(
&'a self,
plane_id: PlaneId,
servers: &Vec<String>,
) -> Option<ClientClusterInfo> {
debug!(
"Getting server info for plane {} from {:?}",
plane_id.raw(),
servers
);
let mut attempt_remains: i32 = 10;
loop {
debug!(
"Trying to get cluster info for plane {}, attempt from {:?}...{}",
plane_id.raw(),
servers,
attempt_remains
);
let mut futs: FuturesUnordered<_> = servers
.iter()
.map(|server_addr| {
let id = hash_str(server_addr);
let server_addr = server_addr.clone();
async move {
let mut members = self.members.write().await;
debug!(
"Checking server info for plane {} on {}",
plane_id.raw(),
server_addr
);
if !members.clients.contains_key(&id) {
debug!(
"Connecting to node {} for plane {}",
server_addr,
plane_id.raw()
);
match rpc::DEFAULT_CLIENT_POOL.get(&server_addr).await {
Ok(client) => {
debug!(
"Added server info on {} to members for plane {}",
server_addr,
plane_id.raw()
);
members.clients.insert(
id,
AsyncServiceClient::new_with_service_id(
self.service_id,
&client,
),
);
debug!(
"Member {} added for plane {}",
server_addr,
plane_id.raw()
);
}
Err(e) => {
warn!(
"Cannot find server info for plane {} from {}, {}",
plane_id.raw(),
server_addr,
e
);
return None;
}
}
}
debug!(
"Getting server info for plane {} from {}, id {}",
plane_id.raw(),
server_addr,
id
);
let member_client = match members.clients.get(&id) {
Some(client) => client,
None => {
debug!(
"Server not found for plane {}, skip {}, id {}",
plane_id.raw(),
server_addr,
id
);
return None;
}
};
debug!(
"Invoking server_cluster_info for plane {} on {}, id {}",
plane_id.raw(),
server_addr,
id
);
let info_res = member_client.c_server_cluster_info(plane_id).await;
debug!(
"Checking cluster info response for plane {} from {}",
plane_id.raw(),
server_addr
);
return match info_res {
Ok(info) => {
if info.leader_id != 0 {
debug!(
"Found server info for plane {} with leader id {}",
plane_id.raw(),
info.leader_id
);
Some(info)
} else {
debug!(
"Discovered zero leader id for plane {} from {}",
plane_id.raw(),
server_addr
);
None
}
}
Err(e) => {
debug!(
"Error on getting cluster info for plane {} from {}, {:?}",
plane_id.raw(),
server_addr,
e
);
None
}
};
}
})
.collect();
while let Some(res) = futs.next().await {
if let Some(info) = res {
return Some(info);
}
}
if attempt_remains > 0 {
// We found an uninitialized node, should try again
// Random sleep
debug!(
"Plane {} fail attempt had zero leader id, retry...{}",
plane_id.raw(),
attempt_remains
);
let delay_sec = 1 + (rand::random::<u64>() % 9);
sleep(Duration::from_secs(delay_sec)).await;
attempt_remains -= 1;
continue;
} else {
debug!(
"Continuously getting zero leader id for plane {}, give up",
plane_id.raw()
);
break;
}
}
warn!(
"Cannot find anything useful for plane {} from list: {:?}",
plane_id.raw(),
servers
);
return None;
}
async fn update_info(&self, servers: &Vec<String>) -> Result<(), ClientError> {
debug!(
"Updating cluster info for plane {} from servers: {:?}",
PlaneId::type1().raw(),
servers
);
let cluster_info = self.cluster_info(PlaneId::type1(), servers).await;
match cluster_info {
Some(info) => {
let mut members = self.members.write().await;
let remote_members = info.members;
let mut remote_ids = HashSet::with_capacity(remote_members.len());
members.id_map.clear();
for (id, addr) in remote_members {
members.id_map.insert(id, addr);
remote_ids.insert(id);
}
let mut connected_ids = HashSet::with_capacity(members.clients.len());
for id in members.clients.keys() {
connected_ids.insert(*id);
}
let ids_to_remove = connected_ids.difference(&remote_ids);
for id in ids_to_remove {
warn!(
"Removed server with id {} while refreshing plane {}",
id,
PlaneId::type1().raw()
);
members.clients.remove(id);
}
for id in remote_ids.difference(&connected_ids) {
let addr = match members.id_map.get(id) {
Some(addr) => addr.clone(),
None => {
error!(
"Cannot find address for server id {} while refreshing plane {}",
id,
PlaneId::type1().raw()
);
continue;
}
};
if !members.clients.contains_key(id) {
if let Ok(client) = rpc::DEFAULT_CLIENT_POOL.get(&addr).await {
info!(
"Having new server addr {} id {} for plane {}",
addr,
id,
PlaneId::type1().raw()
);
members.clients.insert(
*id,
AsyncServiceClient::new_with_service_id(self.service_id, &client),
);
} else {
error!(
"Cannot connect to new server addr {}, id {} for plane {}",
addr,
id,
PlaneId::type1().raw()
);
}
}
}
info!(
"UPDATE_INFO Setting plane {} leader to {}, was {}",
PlaneId::type1().raw(),
info.leader_id,
self.type1_state.leader_id.load(Relaxed)
);
self.type1_state.leader_id.store(info.leader_id, ORDERING);
swap_when_greater(&self.type1_state.last_log_id, info.last_log_id);
swap_when_greater(&self.type1_state.last_log_term, info.last_log_term);
Ok(())
}
None => {
error!(
"Cannot update info for plane {}, cannot get cluster info",
PlaneId::type1().raw()
);
Err(ClientError::ServerUnreachable)
}
}
}
async fn update_plane_info(
&self,
plane_id: PlaneId,
servers: &Vec<String>,
) -> Result<(), ClientError> {
if plane_id.is_type1() {
return self.update_info(servers).await;
}
let cluster_info = self.cluster_info(plane_id, servers).await;
match cluster_info {
Some(info) => {
let state = self.plane_state(plane_id).await;
info!(
"UPDATE_INFO Setting plane {} leader to {}, was {}",
plane_id.raw(),
info.leader_id,
state.leader_id.load(Relaxed)
);
state.leader_id.store(info.leader_id, ORDERING);
swap_when_greater(&state.last_log_id, info.last_log_id);
swap_when_greater(&state.last_log_term, info.last_log_term);
Ok(())
}
None => {
error!(
"Cannot update info for plane {}, cannot get cluster info",
plane_id.raw()
);
Err(ClientError::ServerUnreachable)
}
}
}
pub async fn probe_servers(
servers: &Vec<String>,
server_address: &String,
service_id: u64,
) -> bool {
servers
.iter()
.map(|peer_addr| {
timeout(Duration::from_secs(2), async move {
if peer_addr == server_address {
// Should not include the server we are running
return false;
}
match rpc::DEFAULT_CLIENT_POOL.get(peer_addr).await {
Ok(client) => ImmeServiceClient::c_ping(service_id, &client).await.is_ok(),
Err(_) => false,
}
})
})
.collect::<FuturesUnordered<_>>()
.collect::<Vec<_>>()
.await
.into_iter()
.any(|r| match r {
Ok(true) => true,
_ => false,
})
}
pub async fn execute<R, M>(&self, sm_id: u64, msg: M) -> Result<R, ExecError>
where
R: 'static,
M: RaftMsg<R> + 'static,
{
self.execute_on_plane(PlaneId::type1(), sm_id, msg).await
}
pub async fn execute_on_plane<R, M>(
&self,
plane_id: PlaneId,
sm_id: u64,
msg: M,
) -> Result<R, ExecError>
where
R: 'static,
M: RaftMsg<R> + 'static,
{
let (fn_id, op, req_data) = msg.encode();
let response = match op {
OpType::QUERY => self.query_on_plane(plane_id, sm_id, fn_id, req_data).await,
OpType::COMMAND | OpType::SUBSCRIBE => {
self.command_on_plane(plane_id, sm_id, fn_id, req_data)
.await
}
};
match response {
Ok(data) => match data {
Ok(data) => Ok(M::decode_return(&data)),
Err(e) => Err(e),
},
Err(e) => Err(e),
}
}
pub async fn can_callback() -> bool {
CALLBACK.read().await.is_some()
}
fn get_sub_key<M, R>(&self, plane_id: PlaneId, sm_id: u64, msg: M) -> SubKey
where
M: RaftMsg<R> + 'static,
R: 'static,
{
let raft_sid = self.service_id;
let (fn_id, pattern_id) = {
let (fn_id, _, pattern_data) = msg.encode();
(fn_id, hash_bytes(pattern_data.as_slice()))
};
SubKey::new(raft_sid, plane_id, sm_id, fn_id, pattern_id)
}
pub async fn get_callback(&self) -> Result<Arc<SubscriptionService>, SubscriptionError> {
match CALLBACK.read().await.clone() {
None => {
debug!("Subscription service not set");
Err(SubscriptionError::SubServiceNotSet)
}
Some(c) => Ok(c),
}
}
pub async fn subscribe<M, R, F>(
&self,
sm_id: u64,
msg: M,
f: F,
) -> Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>
where
M: RaftMsg<R> + 'static,
R: 'static + Send,
F: Fn(R) -> BoxFuture<'static, ()> + 'static + Send + Sync,
{
self.subscribe_on_plane(PlaneId::type1(), sm_id, msg, f)
.await
}
pub async fn subscribe_on_plane<M, R, F>(
&self,
plane_id: PlaneId,
sm_id: u64,
msg: M,
f: F,
) -> Result<Result<SubscriptionReceipt, SubscriptionError>, ExecError>
where
M: RaftMsg<R> + 'static,
R: 'static + Send,
F: Fn(R) -> BoxFuture<'static, ()> + 'static + Send + Sync,
{
let callback = match self.get_callback().await {
Ok(c) => c,
Err(e) => return Ok(Err(e)),
};
let key = self.get_sub_key(plane_id, sm_id, msg);
let wrapper_fn =
move |data: Vec<u8>| -> BoxFuture<'static, ()> { f(M::decode_return(&data)).boxed() };
let cluster_subs = self
.execute_on_plane(
plane_id,
CONFIG_SM_ID,
conf_subscribe::new(&key, &callback.server_address, &callback.session_id),
)
.await;
match cluster_subs {
Ok(Ok(sub_id)) => {
let mut subs_map = callback.subs.write().await;
let subs_lst = subs_map.entry(key).or_insert_with(|| Vec::new());
let boxed_fn = Box::new(wrapper_fn);
subs_lst.push((boxed_fn, sub_id));
Ok(Ok((key, sub_id)))
}
Ok(Err(_)) => Ok(Err(SubscriptionError::RemoteError)),
Err(e) => Err(e),
}
}
pub async fn unsubscribe(
&self,
receipt: SubscriptionReceipt,
) -> Result<Result<(), SubscriptionError>, ExecError> {
match self.get_callback().await {
Ok(callback) => {
let (key, sub_id) = receipt;
let unsub = self
.execute_on_plane(key.plane_id, CONFIG_SM_ID, conf_unsubscribe::new(&sub_id))
.await;
match unsub {
Ok(_) => {
let mut subs_map = callback.subs.write().await;
let subs_lst = subs_map.entry(key).or_insert_with(|| Vec::new());
let mut sub_index = 0;
for i in 0..subs_lst.len() {
if subs_lst[i].1 == sub_id {
sub_index = i;
break;
}
}
if subs_lst.len() > 0 && subs_lst[sub_index].1 == sub_id {
let _ = subs_lst.remove(sub_index);
Ok(Ok(()))
} else {
Ok(Err(SubscriptionError::CannotFindSubId))
}
}
Err(e) => Err(e),
}
}
Err(e) => {
debug!("Subscription service not set");
return Ok(Err(e));
}
}
}
async fn query_on_plane(
&self,
plane_id: PlaneId,
sm_id: u64,
fn_id: u64,
data: Vec<u8>,
) -> Result<ExecResult, ExecError> {
let state = self.plane_state(plane_id).await;
let mut depth = 0;
loop {
if depth == 0 {
trace!(
"Raft client query plane_id={} sm_id {}, fn_id {}",
plane_id.raw(),
sm_id,
fn_id
);
} else {
warn!(
"Retry client query plane_id={} sm_id {}, fn_id {}",
plane_id.raw(),
sm_id,
fn_id
);
}
let pos = state.pos.fetch_add(1, ORDERING);
let members = self.members.read().await;
let num_members = members.clients.len();
if num_members >= 1 {
let node_index = pos as usize % num_members;
let rpc_client = match members.clients.values().nth(node_index) {
Some(client) => client,
None => {
error!(
"Cannot find client for plane {} at index {} (total: {})",
plane_id.raw(),
node_index,
num_members
);
return Err(ExecError::ServersUnreachable);
}
};
trace!(
"Query for plane {} from node {} for sm_id {}, fn_id {}",
plane_id.raw(),
node_index,
sm_id,
fn_id
);
let res = rpc_client
.c_query(plane_id, &self.gen_log_entry(&state, sm_id, fn_id, &data))
.await;
trace!(
"Query for plane {} from node {} for sm_id {}, fn_id {} completed",
plane_id.raw(),
node_index,
sm_id,
fn_id
);
match res {
Ok(res) => match res {
ClientQryResponse::LeftBehind {
last_log_term,
last_log_id,
} => {
debug!("Found left behind record on plane {}...{}, updating client state: server has log_id={}, term={}", plane_id.raw(), depth, last_log_id, last_log_term);
// Update client state from server to avoid retry loop
swap_when_greater(&state.last_log_id, last_log_id);
swap_when_greater(&state.last_log_term, last_log_term);
// Add a small delay to allow server to catch up if under stress
if depth > 0 {
sleep(Duration::from_millis(50)).await;
}
if depth >= num_members {
error!("Too many retry on query for plane {}, num_members {}, due to left behind record {}", plane_id.raw(), num_members, depth);
return Err(ExecError::TooManyRetry);
} else {
depth += 1;
continue;
}
}
ClientQryResponse::Success {
data,
last_log_term,
last_log_id,
} => {
swap_when_greater(&state.last_log_id, last_log_id);
swap_when_greater(&state.last_log_term, last_log_term);
if depth > 0 {
warn!("Retry successful on plane {}...{}", plane_id.raw(), depth);
}
trace!("Query for plane {} from node {} for sm_id {}, fn_id {}, successful at log id {}, term {}", plane_id.raw(), node_index, sm_id, fn_id, last_log_id, last_log_term);
return Ok(data);
}
},
Err(e) => {
error!(
"Got unknown error on query for plane {}: {:?}, server {}",
plane_id.raw(),
e,
rpc_client.client.address
);
if depth >= num_members {
return Err(ExecError::Unknown);
} else {
debug!("Retry query on plane {}...{}", plane_id.raw(), depth);
depth += 1;
continue;
}
}
}
} else {
return Err(ExecError::ServersUnreachable);
}
}
}
async fn command_on_plane(
&self,
plane_id: PlaneId,
sm_id: u64,
fn_id: u64,
data: Vec<u8>,
) -> Result<ExecResult, ExecError> {
const NOT_COMMITTED_RETRY_DELAY_MS: u64 = 10;
const UPDATE_INFO_RETRY_DELAY_MS: u64 = 10;
let not_committed_retry_limit = std::cmp::max(
64,
((HEARTBEAT_MS * 2) / NOT_COMMITTED_RETRY_DELAY_MS as i64) as i32,
);
let update_info_retry_limit = std::cmp::max(
64,
((HEARTBEAT_MS * 2) / UPDATE_INFO_RETRY_DELAY_MS as i64) as i32,
);
enum FailureAction {
SwitchLeader,
NotCommitted,
UpdateInfo,
NotLeader,
ShuttingDown,
}
let state = self.plane_state(plane_id).await;
let mut leader_retry_depth = 0;
let mut not_committed_depth = 0;
let mut update_info_depth = 0;
loop {
let failure = {
if leader_retry_depth > 0 {
let members = self.members.read().await;
let num_members = members.clients.len();
if leader_retry_depth >= max(num_members + 1, 5) {
error!(
"Too many retry on command for plane {}, num_members {}, due to leader retry attempts {}",
plane_id.raw(), num_members,
leader_retry_depth
);
return Err(ExecError::TooManyRetry);
};
}
match self.preferred_client_on_plane(&state).await {
Some((leader_id, client)) => {
let cmd_res = client
.c_command(plane_id, self.gen_log_entry(&state, sm_id, fn_id, &data))
.await;
match cmd_res {
Ok(ClientCmdResponse::Success {
data,
last_log_term,
last_log_id,
}) => {
swap_when_greater(&state.last_log_id, last_log_id);
swap_when_greater(&state.last_log_term, last_log_term);
return Ok(data);
}
Ok(ClientCmdResponse::NotLeader(new_leader_id)) => {
if new_leader_id == 0 || new_leader_id == leader_id {
warn!(
"RAFTDBG_V2 client plane_id={} notleader-zero_or_same leader_id={} suggested={} depth={} update_info_depth={} not_committed_depth={}",
plane_id.raw(), leader_id,
new_leader_id,
leader_retry_depth,
update_info_depth,
not_committed_depth
);
debug!(
"CLIENT plane_id={}: NOT LEADER, SUGGESTION NOT USEFUL, REFRESH INFO. GOT: {}",
plane_id.raw(), new_leader_id
);
FailureAction::UpdateInfo
} else {
warn!(
"RAFTDBG_V3 client plane_id={} notleader-redirect current_leader={} suggested_leader={} depth={}",
plane_id.raw(), leader_id,
new_leader_id,
leader_retry_depth
);
debug!(
"CLIENT plane_id={}: NOT LEADER, REMOTE SUGGEST SWITCH TO {}",
plane_id.raw(), new_leader_id
);
info!(
"CMD Setting plane {} leader to {}, was {}",
plane_id.raw(),
new_leader_id,
state.leader_id.load(Relaxed)
);
state.leader_id.store(new_leader_id, ORDERING);
FailureAction::NotLeader
}
}
Ok(ClientCmdResponse::NotCommitted {
last_log_term,
last_log_id,
}) => {
debug!(
"CLIENT plane_id={}: NOT COMMITTED at leader {}, refreshing client log cursor to term {}, id {}",
plane_id.raw(), leader_id,
last_log_term,
last_log_id
);
swap_when_greater(&state.last_log_id, last_log_id);
swap_when_greater(&state.last_log_term, last_log_term);
FailureAction::NotCommitted
}
Ok(ClientCmdResponse::ShuttingDown) => FailureAction::ShuttingDown,
Err(e) => {
warn!(
"RAFTDBG_V3 client plane_id={} transport_or_rpc_error leader_id={} depth={} error={:?}",
plane_id.raw(), leader_id,
leader_retry_depth,
e
);
debug!(
"CLIENT plane_id={}: ERROR - {} - {:?}",
plane_id.raw(),
leader_id,
e
);
FailureAction::SwitchLeader // need switch server for leader
}
}
}
None => {
warn!("Need update members for plane {}", plane_id.raw());
FailureAction::UpdateInfo
}
}
}; //
match failure {
FailureAction::NotCommitted => {
not_committed_depth += 1;
update_info_depth = 0;
if not_committed_depth >= not_committed_retry_limit {
error!(
"Too many retry on command for plane {} due to NotCommitted responses {}",
plane_id.raw(), not_committed_depth
);
return Err(ExecError::TooManyRetry);
}
debug!(
"Retrying command for plane {} after NotCommitted response {}/{}",
plane_id.raw(),
not_committed_depth,
not_committed_retry_limit
);
sleep(Duration::from_millis(NOT_COMMITTED_RETRY_DELAY_MS)).await;
continue;
}
FailureAction::UpdateInfo => {
update_info_depth += 1;
not_committed_depth = 0;
warn!(
"RAFTDBG_V2 client plane_id={} update_info_retry count={} depth={}",
plane_id.raw(),
update_info_depth,
leader_retry_depth
);
if update_info_depth >= update_info_retry_limit {
error!(
"Too many retry on command for plane {} due to cluster-info refresh attempts {}",
plane_id.raw(), update_info_depth
);
return Err(ExecError::TooManyRetry);
}
let servers = {
let members = self.members.read().await;
Vec::from_iter(members.id_map.values().cloned())
};
if servers.is_empty() {
warn!(
"Cannot refresh cluster info for plane {}: no known servers",
plane_id.raw()
);
return Err(ExecError::ServersUnreachable);
}
debug!(
"Refreshing cluster info for plane {} after transient NotLeader/leader-miss {}/{} from {:?}",
plane_id.raw(), update_info_depth,
update_info_retry_limit,
servers
);
if let Err(e) = self.update_plane_info(plane_id, &servers).await {
warn!(
"Failed to refresh cluster info for plane {} during command retry: {:?}",
plane_id.raw(), e
);
}
sleep(Duration::from_millis(UPDATE_INFO_RETRY_DELAY_MS)).await;
continue;
}
FailureAction::SwitchLeader => {
not_committed_depth = 0;
update_info_depth = 0;
leader_retry_depth += 1;
warn!(
"RAFTDBG_V3 client plane_id={} switch_leader depth={}",
plane_id.raw(),
leader_retry_depth
);
debug!("Switch leader for plane {} by probing", plane_id.raw());
let members = self.members.read().await;
let num_members = members.clients.len();
let leader_id = state.leader_id.load(ORDERING);
let new_leader_id = match members
.clients
.keys()
.nth(leader_retry_depth as usize % num_members)
{
Some(id) => *id,
None => {
error!(
"Cannot find new leader for plane {} at index {} (total: {})",
plane_id.raw(),
leader_retry_depth as usize % num_members,
num_members
);
return Err(ExecError::ServersUnreachable);
}
};
let leadder_switch = state.leader_id.compare_exchange(
leader_id,
new_leader_id,
ORDERING,
Relaxed,
);
info!(
"SWITCH plane {} exchange leader to {}, was {:?}",
plane_id.raw(),
new_leader_id,
leadder_switch
);
debug!(
"CLIENT plane_id={}: Switch leader {}",
plane_id.raw(),
new_leader_id
);
}
FailureAction::NotLeader => {
leader_retry_depth += 1;
not_committed_depth = 0;
update_info_depth = 0;
continue;
}
FailureAction::ShuttingDown => {
return Err(ExecError::ShuttingDown);
}
}
}
}
fn gen_log_entry(
&self,
state: &PlaneClientState,
sm_id: u64,
fn_id: u64,
data: &Vec<u8>,
) -> LogEntry {
LogEntry {
id: state.last_log_id.load(ORDERING),
term: state.last_log_term.load(ORDERING),
sm_id,
fn_id,
data: data.clone(),
}
}
pub fn leader_id(&self) -> u64 {
self.type1_state.leader_id.load(ORDERING)
}
pub async fn leader_client(&self) -> Option<(u64, Client)> {
self.current_leader_client_on_plane(PlaneId::type1(), &self.type1_state)
.await
}
async fn any_known_client(&self) -> Option<Client> {
let members = self.members.read().await;
members.clients.values().next().cloned()
}
async fn preferred_client_on_plane(&self, state: &PlaneClientState) -> Option<(u64, Client)> {
if let Some((leader_id, client)) = self.leader_client_on_plane(state).await {
return Some((leader_id, client));
}
self.any_known_client().await.map(|client| (0, client))
}
async fn leader_client_on_plane(&self, state: &PlaneClientState) -> Option<(u64, Client)> {
let members = self.members.read().await;
let leader_id = state.leader_id.load(ORDERING);
if let Some(client) = members.clients.get(&leader_id) {
Some((leader_id, client.clone()))
} else {
None
}
}
async fn current_leader_client_on_plane(
&self,
plane_id: PlaneId,
state: &PlaneClientState,
) -> Option<(u64, Client)> {
{
let leader_client = self.leader_client_on_plane(state).await;
if leader_client.is_some() {
return leader_client;
}
}
debug!(
"Obtaining leader client for plane {} by updating cluster info",
plane_id.raw()
);
{
let servers = {
let members = self.members.read().await;
Vec::from_iter(members.id_map.values().cloned())
};
if let Err(e) = self.update_plane_info(plane_id, &servers).await {
error!(
"Failed to update cluster info for plane {}: {:?}",
plane_id.raw(),
e
);
return None;
}
let leader_id = state.leader_id.load(ORDERING);
let members = self.members.read().await;
if let Some(client) = members.clients.get(&leader_id) {
debug!(
"Obtained leader client for plane {} with id: {}",
plane_id.raw(),
leader_id
);
Some((leader_id, client.clone()))
} else {
warn!(
"Cannot obtain leader client for plane {} with id {}. Having {:?}",
plane_id.raw(),
leader_id,
members.clients.keys().collect::<Vec<_>>()
);
None
}
}
}
pub async fn current_leader_rpc_client(&self) -> Result<Arc<rpc::RPCClient>, ()> {
let (_, client) = self
.current_leader_client_on_plane(PlaneId::type1(), &self.type1_state)
.await
.ok_or_else(|| ())?;
Ok(client.client.clone())
}
pub async fn cluster_info_on_plane(
&self,
plane_id: PlaneId,
) -> Result<ClientClusterInfo, ExecError> {
let state = self.plane_state(plane_id).await;
let client = match self.leader_client_on_plane(&state).await {
Some((_, client)) => client,
None => self
.any_known_client()
.await
.ok_or(ExecError::ServersUnreachable)?,
};
client
.c_server_cluster_info(plane_id)
.await
.map_err(|_| ExecError::ServersUnreachable)
}
pub async fn have_state_machine_on_plane(
&self,
plane_id: PlaneId,
sm_id: u64,
) -> Result<bool, ExecError> {
let state = self.plane_state(plane_id).await;
let client = match self.leader_client_on_plane(&state).await {
Some((_, client)) => client,
None => self
.any_known_client()
.await
.ok_or(ExecError::ServersUnreachable)?,
};
client
.c_have_state_machine(plane_id, sm_id)
.await
.map_err(|_| ExecError::ServersUnreachable)
}
}
fn swap_when_greater(atomic: &AtomicU64, value: u64) {
let mut orig_num = atomic.load(ORDERING);
loop {
if orig_num >= value {
return;
}
match atomic.compare_exchange(orig_num, value, ORDERING, Relaxed) {
Ok(_) => {
return;
}
Err(actual) => {
orig_num = actual;
}
}
}
}
pub struct CachedStateMachine<T: StateMachineClient> {
server_list: Vec<String>,
raft_service_id: u64,
plane_id: PlaneId,
state_machine_id: u64,
cache: RwLock<Option<Arc<T>>>,
}
impl<T: StateMachineClient> CachedStateMachine<T> {
pub fn new(
server_list: &Vec<String>,
raft_service_id: u64,
plane_id: PlaneId,
state_machine_id: u64,
) -> Self {
debug!(
"Construct cached state machine for list {:?}, service id {}, plane {}, state machine {}",
server_list,
raft_service_id,
plane_id.raw(),
state_machine_id
);
Self {
server_list: server_list.clone(),
raft_service_id,
plane_id,
state_machine_id,
cache: RwLock::new(None),
}
}
pub async fn get(&self) -> Arc<T> {
loop {
{
let client = self.cache.read().await;
if let Some(cache) = &*client {
return (*cache).clone();
}
}
{
let mut place_holder = self.cache.write().await;
if place_holder.is_none() {
debug!(
"Creating state machine client instance, service {}, state machine id {}",
self.raft_service_id, self.state_machine_id
);
let raft_client =
match RaftClient::new(&self.server_list, self.raft_service_id).await {
Ok(client) => client,
Err(e) => {
error!(
"Failed to create RaftClient for service {} and sm {}: {:?}",
self.raft_service_id, self.state_machine_id, e
);
// Drop the lock and retry after a delay
drop(place_holder);
sleep(Duration::from_millis(100)).await;
continue;
}
};
let plane_client = raft_client.plane(self.plane_id);
// Create a client for the state machine on the raft service
*place_holder = Some(Arc::new(T::new_instance(
self.state_machine_id,
&plane_client,
)))
}
}
}
}
}
================================================
FILE: src/raft/disk.rs
================================================
// Now only offers log persistent
use crate::raft::{LogEntry, LogsMap, Options, PlaneId, RaftMeta, SnapshotEntity, Storage};
use async_std::sync::*;
use std::convert::TryInto;
use std::fs::OpenOptions;
use std::io;
use std::io::{Read, Seek, SeekFrom};
use std::ops::Bound::*;
use std::path::{Path, PathBuf};
use tokio::fs::*;
use tokio::io::*;
// const MAX_LOG_CAPACITY: usize = 10;
#[derive(Clone)]
pub struct DiskOptions {
pub path: String,
pub take_snapshots: bool,
pub append_logs: bool,
pub trim_logs: bool,
// Snapshot configuration
pub snapshot_log_threshold: u64, // Trigger snapshot after N logs
pub log_compaction_threshold: u64, // Compact when logs exceed this
}
impl DiskOptions {
pub fn new(path: String) -> Self {
Self {
path,
take_snapshots: true,
append_logs: true,
trim_logs: true,
snapshot_log_threshold: 1000,
log_compaction_threshold: 2000,
}
}
}
pub struct StorageEntity {
pub logs: Option<File>,
pub snapshot: Option<File>,
pub last_term: u64,
pub base_path: PathBuf,
pub plane_id: PlaneId,
}
pub struct DiskLogEntry {
pub term: u64,
pub commit_index: u64,
pub last_applied: u64,
pub log: LogEntry,
}
impl DiskLogEntry {
/// Encode to deterministic binary format with CRC32 checksum.
///
/// On-disk record layout (written by `append_logs`):
/// [8 bytes] record length (= 4 + payload length, does NOT include these 8 bytes)
/// [4 bytes] CRC32 of payload
/// [N bytes] payload:
/// [8 bytes] term
/// [8 bytes] commit_index
/// [8 bytes] last_applied
/// [8 bytes] log.id
/// [8 bytes] log.term
/// [8 bytes] log.sm_id
/// [8 bytes] log.fn_id
/// [8 bytes] log.data.len()
/// [M bytes] log.data
///
/// `encode()` returns only the payload (the CRC and length prefix are added by the caller).
pub fn encode(&self) -> Vec<u8> {
let data_len = self.log.data.len();
let total_size = 8 * 8 + data_len;
let mut buf = Vec::with_capacity(total_size);
buf.extend_from_slice(&self.term.to_le_bytes());
buf.extend_from_slice(&self.commit_index.to_le_bytes());
buf.extend_from_slice(&self.last_applied.to_le_bytes());
buf.extend_from_slice(&self.log.id.to_le_bytes());
buf.extend_from_slice(&self.log.term.to_le_bytes());
buf.extend_from_slice(&self.log.sm_id.to_le_bytes());
buf.extend_from_slice(&self.log.fn_id.to_le_bytes());
buf.extend_from_slice(&(data_len as u64).to_le_bytes());
buf.extend_from_slice(&self.log.data);
buf
}
/// Decode payload bytes (without the length prefix or CRC — the caller strips those).
pub fn decode(data: &[u8]) -> io::Result<Self> {
if data.len() < 64 {
return Err(io::Error::new(
io::ErrorKind::InvalidData,
format!("DiskLogEntry too short: {} bytes", data.len()),
));
}
let term = u64::from_le_bytes(data[0..8].try_into().unwrap());
let commit_index = u64::from_le_bytes(data[8..16].try_into().unwrap());
let last_applied = u64::from_le_bytes(data[16..24].try_into().unwrap());
let log_id = u64::from_le_bytes(data[24..32].try_into().unwrap());
let log_term = u64::from_le_bytes(data[32..40].try_into().unwrap());
let log_sm_id = u64::from_le_bytes(data[40..48].try_into().unwrap());
let log_fn_id = u64::from_le_bytes(data[48..56].try_into().unwrap());
let data_len = u64::from_le_bytes(data[56..64].try_into().unwrap()) as usize;
if data.len() < 64 + data_len {
return Err(io::Error::new(
io::ErrorKind::InvalidData,
format!("DiskLogEntry data truncated: expected {}, got {}", 64 + data_len, data.len()),
));
}
let log_data = data[64..64 + data_len].to_vec();
Ok(DiskLogEntry {
term,
commit_index,
last_applied,
log: LogEntry {
id: log_id,
term: log_term,
sm_id: log_sm_id,
fn_id: log_fn_id,
data: log_data,
},
})
}
}
impl StorageEntity {
pub fn new_with_options(
opts: &Options,
term: &mut u64,
commit_index: &mut u64,
last_applied: &mut u64,
logs: &mut LogsMap,
) -> io::Result<Option<Self>> {
Self::new_with_options_on_plane(
PlaneId::type1(),
opts,
term,
commit_index,
last_applied,
logs,
)
}
pub fn new_with_options_on_plane(
plane_id: PlaneId,
opts: &Options,
term: &mut u64,
commit_index: &mut u64,
last_applied: &mut u64,
logs: &mut LogsMap,
) -> io::Result<Option<Self>> {
Ok(match &opts.storage {
&Storage::DISK(ref options) => {
let base_path = Path::new(&options.path);
let _ = std::fs::create_dir_all(base_path);
let log_path = base_path.join("log.dat");
let snapshot_path = base_path.join("snapshot.dat");
let mut open_opts = OpenOptions::new();
open_opts
.write(true)
.create(true)
.read(true)
.truncate(false);
let mut last_log_id = 0;
let mut storage = Self {
logs: if options.append_logs {
let mut log_file = open_opts.open(log_path.as_path())?;
let mut len_buf = [0u8; 8];
let mut crc_buf = [0u8; 4];
let mut counter = 0;
let mut last_valid_pos: u64 = 0;
loop {
let pos_before = log_file.seek(SeekFrom::Current(0))
.unwrap_or(last_valid_pos);
if log_file.read_exact(&mut len_buf).is_err() {
break;
}
let record_len = u64::from_le_bytes(len_buf);
// record_len = 4 (CRC) + payload_len
if record_len < 4 {
warn!("WAL corrupt: invalid record length {} at pos {}, truncating", record_len, pos_before);
break;
}
let payload_len = record_len - 4;
if log_file.read_exact(&mut crc_buf).is_err() {
warn!("WAL truncated: missing CRC at pos {}, truncating", pos_before);
break;
}
let expected_crc = u32::from_le_bytes(crc_buf);
let mut data_buf = vec![0u8; payload_len as usize];
if log_file.read_exact(&mut data_buf).is_err() {
warn!("WAL truncated: missing payload at pos {}, truncating", pos_before);
break;
}
let actual_crc = crc32fast::hash(&data_buf);
if actual_crc != expected_crc {
warn!(
"WAL CRC mismatch at pos {}: expected {:#010x}, got {:#010x}, truncating",
pos_before, expected_crc, actual_crc
);
break;
}
match DiskLogEntry::decode(&data_buf) {
Ok(entry) => {
*term = entry.term;
// Do not trust commit/last_applied embedded in WAL for SM reconstruction
// We'll derive commit_index from commit.idx and force replay from last_applied=0
last_log_id = entry.log.id;
logs.insert(entry.log.id, entry.log);
counter += 1;
last_valid_pos = log_file.seek(SeekFrom::Current(0))
.unwrap_or(last_valid_pos);
}
Err(e) => {
warn!("WAL decode error at pos {}: {:?}, truncating", pos_before, e);
break;
}
}
}
// Truncate WAL at last valid entry to remove any corrupt tail,
// then seek to end so appends start at the correct position
let current_len = log_file.seek(SeekFrom::End(0)).unwrap_or(last_valid_pos);
if current_len > last_valid_pos {
info!("WAL has corrupt tail ({} extra bytes), truncating to {}", current_len - last_valid_pos, last_valid_pos);
if let Err(e) = log_file.set_len(last_valid_pos) {
warn!("Failed to truncate WAL to {} bytes: {:?}", last_valid_pos, e);
}
}
let _ = log_file.seek(SeekFrom::End(0));
debug!(
"Recovered {} raft logs for plane {}",
counter,
plane_id.raw()
);
Some(File::from_std(log_file))
} else {
None
},
snapshot: if options.take_snapshots {
Some(File::from_std(open_opts.open(snapshot_path.as_path())?))
} else {
None
},
last_term: 0,
base_path: base_path.to_path_buf(),
plane_id,
};
storage.last_term = last_log_id;
// If commit progress side file exists, load it to ensure accurate indices
// Force full replay by resetting last_applied to 0 on startup
*last_applied = 0;
if let Ok(Some((ci, _la))) =
futures::executor::block_on(storage.read_commit_progress())
{
*commit_index = ci;
debug!(
"Recovered commit progress for plane {}: commit_index={} (will replay to rebuild state)",
plane_id.raw(),
ci
);
} else {
// If no commit progress found, default to 0 to avoid partial state
*commit_index = 0;
}
Some(storage)
}
_ => None,
})
}
pub async fn append_logs<'a>(
&mut self,
meta: &'a RwLockWriteGuard<'a, RaftMeta>,
logs: &'a RwLockWriteGuard<'a, LogsMap>,
) -> io::Result<()> {
if let Some(f) = &mut self.logs {
let was_last_term = self.last_term;
let mut counter = 0;
let mut terms_appended = vec![];
let master = meta.state_machine.read().await;
for (term, log) in logs.range((Excluded(self.last_term), Unbounded)) {
// Skip non-recoverable state machines
if !master.is_recoverable(log.sm_id) {
continue;
}
let entry = DiskLogEntry {
term: *term,
commit_index: meta.commit_index,
last_applied: meta.last_applied,
log: log.clone(),
};
let entry_data = entry.encode();
let checksum = crc32fast::hash(&entry_data);
// Write: [8 bytes record_len = 4+payload_len][4 bytes CRC32][N bytes payload]
let record_len = 4u64 + entry_data.len() as u64;
f.write_all(&record_len.to_le_bytes()).await?;
f.write_all(&checksum.to_le_bytes()).await?;
f.write_all(entry_data.as_slice()).await?;
self.last_term = *term;
terms_appended.push(self.last_term);
counter += 1;
}
if counter > 0 {
f.sync_all().await?;
debug!(
"Appended and persisted {} logs for plane {}, was {}, appended {:?}",
counter,
self.plane_id.raw(),
was_last_term,
terms_appended
);
}
}
Ok(())
}
pub async fn post_processing<'a>(
&mut self,
meta: &RwLockWriteGuard<'a, RaftMeta>,
logs: RwLockWriteGuard<'a, LogsMap>,
) -> io::Result<()> {
// TODO: trim logs in memory
// TODO: trim logs on disk
self.append_logs(meta, &logs).await?;
Ok(())
// let (last_log_id, _) = get_last_log_info!(self, logs);
// let expecting_oldest_log = if last_log_id > MAX_LOG_CAPACITY as u64 {
// last_log_id - MAX_LOG_CAPACITY as u64
// } else {
// 0
// };
// let double_cap = MAX_LOG_CAPACITY << 1;
// if logs.len() > double_cap && meta.last_applied > expecting_oldest_log {
// debug!("trim logs");
// while logs.len() > MAX_LOG_CAPACITY {
// let first_key = *logs.iter().next().unwrap().0;
// logs.remove(&first_key).unwrap();
// }
// if let Some(ref storage) = meta.storage {
// let mut storage = storage.write().await;
// let snapshot = SnapshotEntity {
// term: meta.term,
// commit_index: meta.commit_index,
// last_applied: meta.last_applied,
// snapshot: meta.state_machine.read().await.snapshot().unwrap(),
// };
// storage
// .snapshot
// .write_all(crate::utils::serde::serialize(&snapshot).as_slice())?;
// storage.snapshot.sync_all().unwrap();
// }
gitextract_g24w9kwe/
├── .gitignore
├── .travis.yml
├── Cargo.toml
├── LICENSE
├── MEMBERSHIP_GUIDE.md
├── README.md
├── RECOVERY_IMPROVEMENTS.md
├── SNAPSHOT_GUIDE.md
├── TYPE2_LAZY_LOADING_CONTRACT.md
├── examples/
│ └── graceful_shutdown.rs
├── src/
│ ├── conshash/
│ │ ├── mod.rs
│ │ └── weights.rs
│ ├── hasher/
│ │ ├── Cargo.toml
│ │ └── src/
│ │ └── lib.rs
│ ├── lib.rs
│ ├── membership/
│ │ ├── client.rs
│ │ ├── member.rs
│ │ ├── mod.rs
│ │ └── server.rs
│ ├── plugins/
│ │ ├── Cargo.toml
│ │ └── src/
│ │ └── lib.rs
│ ├── proc_macro/
│ │ ├── Cargo.toml
│ │ └── src/
│ │ └── lib.rs
│ ├── raft/
│ │ ├── client.rs
│ │ ├── disk.rs
│ │ ├── mod.rs
│ │ └── state_machine/
│ │ ├── callback/
│ │ │ ├── client.rs
│ │ │ ├── mod.rs
│ │ │ └── server.rs
│ │ ├── configs.rs
│ │ ├── macros.rs
│ │ ├── master.rs
│ │ └── mod.rs
│ ├── rpc/
│ │ ├── cluster.rs
│ │ ├── mod.rs
│ │ └── proto.rs
│ ├── tcp/
│ │ ├── client.rs
│ │ ├── mod.rs
│ │ ├── server.rs
│ │ └── shortcut.rs
│ ├── utils/
│ │ ├── bindings.rs
│ │ ├── math.rs
│ │ ├── mod.rs
│ │ ├── serde.rs
│ │ └── time.rs
│ └── vector_clock/
│ └── mod.rs
└── tests/
├── graceful_shutdown_tests.rs
└── single_node_recovery_test.rs
SYMBOL INDEX (744 symbols across 32 files)
FILE: examples/graceful_shutdown.rs
function main (line 15) | async fn main() {
FILE: src/conshash/mod.rs
type Action (line 18) | pub enum Action {
type InitTableError (line 24) | pub enum InitTableError {
type CHError (line 33) | pub enum CHError {
type LookupTables (line 38) | struct LookupTables {
type ConsistentHashing (line 43) | pub struct ConsistentHashing {
method new_with_id (line 55) | pub async fn new_with_id(
method new (line 144) | pub async fn new(
method new_client (line 151) | pub async fn new_client(
method new_client_with_id (line 158) | pub async fn new_client_with_id(
method to_server_name (line 172) | pub fn to_server_name(&self, server_id: u64) -> String {
method to_server_name_option (line 181) | pub fn to_server_name_option(&self, server_id: Option<u64>) -> Option<...
method get_server_id (line 189) | pub fn get_server_id(&self, hash: u64) -> Option<u64> {
method jump_hash (line 200) | pub fn jump_hash(&self, slot_count: usize, hash: u64) -> usize {
method get_server (line 218) | pub fn get_server(&self, hash: u64) -> Option<String> {
method get_server_by_string (line 221) | pub fn get_server_by_string(&self, string: &String) -> Option<String> {
method get_server_by (line 224) | pub fn get_server_by<T>(&self, obj: &T) -> Option<String>
method get_server_id_by_string (line 230) | pub fn get_server_id_by_string(&self, string: &String) -> Option<u64> {
method get_server_id_by (line 233) | pub fn get_server_id_by<T>(&self, obj: &T) -> Option<u64>
method rand_server (line 239) | pub fn rand_server(&self) -> Option<String> {
method rand_server_id (line 243) | pub fn rand_server_id(&self) -> Option<u64> {
method nodes_count (line 247) | pub fn nodes_count(&self) -> usize {
method server_count (line 251) | pub fn server_count(&self) -> usize {
method set_weight (line 254) | pub async fn set_weight(&self, server_name: &String, weight: u64) -> R...
method watch_all_actions (line 261) | fn watch_all_actions<F>(&self, f: F)
method watch_server_nodes_range_changed (line 268) | pub fn watch_server_nodes_range_changed<F>(&self, server: &String, f: F)
method init_table (line 296) | pub async fn init_table(&self) -> Result<(), InitTableError> {
method membership (line 361) | pub fn membership(&self) -> &Arc<MembershipClient> {
function server_joined (line 366) | async fn server_joined(ch: Arc<ConsistentHashing>, member: Member, versi...
function server_left (line 369) | async fn server_left(ch: Arc<ConsistentHashing>, member: Member, version...
function server_changed (line 372) | async fn server_changed(ch: Arc<ConsistentHashing>, member: Member, acti...
function primary (line 418) | async fn primary() {
FILE: src/conshash/weights.rs
type Weights (line 15) | pub struct Weights {
method new_with_id (line 70) | pub async fn new_with_id(id: u64, raft_service: &Arc<RaftService>) {
method new (line 78) | pub async fn new(raft_service: &Arc<RaftService>) {
method set_weight (line 20) | fn set_weight(&mut self, group: u64, id: u64, weight: u64) -> BoxFuture<...
method get_weights (line 29) | fn get_weights(&self, group: u64) -> BoxFuture<Option<HashMap<u64, u64>>> {
method get_weight (line 36) | fn get_weight(&self, group: u64, id: u64) -> BoxFuture<Option<u64>> {
method id (line 49) | fn id(&self) -> u64 {
method snapshot (line 52) | fn snapshot(&self) -> Vec<u8> {
method recover (line 55) | fn recover(&mut self, data: Vec<u8>) -> BoxFuture<()> {
method recoverable (line 65) | fn recoverable(&self) -> bool {
FILE: src/hasher/src/lib.rs
function hash_bytes (line 6) | pub fn hash_bytes(bytes: &[u8]) -> u64 {
function hash_str (line 12) | pub fn hash_str<'a>(text: &'a str) -> u64 {
function hash_bytes_secondary (line 18) | pub fn hash_bytes_secondary(bytes: &[u8]) -> u64 {
FILE: src/membership/client.rs
type Member (line 14) | pub struct Member {
type Group (line 21) | pub struct Group {
type MemberClient (line 27) | pub struct MemberClient {
method join_group (line 33) | pub async fn join_group(&self, group: &String) -> Result<bool, ExecErr...
method leave_group (line 36) | pub async fn leave_group(&self, group: &String) -> Result<bool, ExecEr...
type ObserverClient (line 41) | pub struct ObserverClient {
method new (line 46) | pub fn new(raft_client: &Arc<RaftClient>) -> ObserverClient {
method new_from_sm (line 51) | pub fn new_from_sm(sm_client: &Arc<SMClient>) -> ObserverClient {
method new_group (line 56) | pub async fn new_group(&self, name: &String) -> Result<Result<u64, u64...
method del_group (line 59) | pub async fn del_group(&self, name: &String) -> Result<bool, ExecError> {
method group_leader (line 62) | pub async fn group_leader(
method group_members (line 68) | pub async fn group_members(
method all_members (line 77) | pub async fn all_members(&self, online_only: bool) -> Result<(Vec<Memb...
method on_group_member_offline (line 80) | pub async fn on_group_member_offline<F>(
method on_any_member_offline (line 92) | pub async fn on_any_member_offline<F>(
method on_group_member_online (line 101) | pub async fn on_group_member_online<F>(
method on_any_member_online (line 113) | pub async fn on_any_member_online<F>(
method on_group_member_joined (line 122) | pub async fn on_group_member_joined<F>(
method on_any_member_joined (line 134) | pub async fn on_any_member_joined<F>(
method on_group_member_left (line 143) | pub async fn on_group_member_left<F>(
method on_any_member_left (line 155) | pub async fn on_any_member_left<F>(
method on_group_leader_changed (line 164) | pub async fn on_group_leader_changed<F>(
method all_groups (line 179) | pub async fn all_groups(&self) -> Result<BTreeMap<u64, MemberGroup>, E...
FILE: src/membership/member.rs
type MemberService (line 18) | pub struct MemberService {
method new (line 27) | pub async fn new(
method close (line 74) | pub fn close(&self) {
method leave (line 77) | pub async fn leave(&self) -> Result<bool, ExecError> {
method join_group (line 81) | pub async fn join_group(&self, group: &String) -> Result<bool, ExecErr...
method leave_group (line 84) | pub async fn leave_group(&self, group: &String) -> Result<bool, ExecEr...
method client (line 87) | pub fn client(&self) -> ObserverClient {
method get_server_id (line 90) | pub fn get_server_id(&self) -> u64 {
method drop (line 96) | fn drop(&mut self) {
FILE: src/membership/mod.rs
function primary (line 64) | async fn primary() {
FILE: src/membership/server.rs
type HBStatus (line 33) | struct HBStatus {
type HeartbeatService (line 41) | pub struct HeartbeatService {
method update_raft (line 83) | async fn update_raft(&self, online: &Vec<u64>, offline: &Vec<u64>) {
method transfer_leadership (line 100) | async fn transfer_leadership(&self) {
method shutdown (line 121) | pub async fn shutdown(&self) {
method ping (line 50) | fn ping(&self, id: u64) -> BoxFuture<()> {
type Member (line 144) | struct Member {
type MemberGroup (line 150) | pub struct MemberGroup {
type Membership (line 166) | pub struct Membership {
method new (line 188) | pub async fn new(server: &Arc<Server>, raft_service: &Arc<RaftService>) {
method compose_client_member (line 351) | async fn compose_client_member(&self, id: u64) -> Option<ClientMember> {
method init_callback (line 360) | async fn init_callback(&mut self, raft_service: &Arc<RaftService>) {
method notify_for_member_online (line 363) | async fn notify_for_member_online(&self, id: u64) {
method notify_for_member_offline (line 393) | async fn notify_for_member_offline(&self, id: u64) {
method notify_for_member_left (line 423) | async fn notify_for_member_left(&self, id: u64) {
method notify_for_group_member_left (line 447) | async fn notify_for_group_member_left(&self, group: u64, member: &Clie...
method leave_group_ (line 456) | async fn leave_group_(&mut self, group_id: u64, id: u64, need_notify: ...
method member_groups (line 483) | fn member_groups(&self, member: u64) -> Option<HashSet<u64>> {
method group_first_online_member_id (line 490) | async fn group_first_online_member_id(&self, group: u64) -> Result<Opt...
method change_leader (line 504) | async fn change_leader(&mut self, group_id: u64, new: Option<u64>) -> ...
method group_leader_candidate_available (line 537) | async fn group_leader_candidate_available(&mut self, group_id: u64, me...
method group_leader_candidate_unavailable (line 554) | async fn group_leader_candidate_unavailable(&mut self, group_id: u64, ...
method leader_candidate_available (line 575) | async fn leader_candidate_available(&mut self, member: u64) {
method leader_candidate_unavailable (line 582) | async fn leader_candidate_unavailable(&mut self, member: u64) {
method drop (line 174) | fn drop(&mut self) {
method hb_online_changed (line 592) | fn hb_online_changed(&mut self, online: Vec<u64>, offline: Vec<u64>) -> ...
method join (line 634) | fn join(&mut self, address: String) -> BoxFuture<Option<u64>> {
method leave (line 681) | fn leave(&mut self, id: u64) -> BoxFuture<bool> {
method join_group (line 706) | fn join_group(&mut self, group_name: String, id: u64) -> BoxFuture<bool> {
method leave_group (line 752) | fn leave_group(&mut self, group_id: u64, id: u64) -> BoxFuture<bool> {
method new_group (line 759) | fn new_group(&mut self, name: String) -> BoxFuture<Result<u64, u64>> {
method del_group (line 780) | fn del_group(&mut self, id: u64) -> BoxFuture<bool> {
method group_leader (line 801) | fn group_leader(&self, group_id: u64) -> BoxFuture<Option<(Option<Client...
method group_members (line 817) | fn group_members(
method all_members (line 844) | fn all_members(&self, online_only: bool) -> BoxFuture<(Vec<ClientMember>...
method all_groups (line 863) | fn all_groups(&self) -> BoxFuture<BTreeMap<u64, MemberGroup>> {
method id (line 869) | fn id(&self) -> u64 {
method snapshot (line 872) | fn snapshot(&self) -> Vec<u8> {
method recover (line 879) | fn recover(&mut self, _: Vec<u8>) -> BoxFuture<()> {
method recoverable (line 884) | fn recoverable(&self) -> bool {
FILE: src/plugins/src/lib.rs
function hash_ident (line 11) | pub fn hash_ident(item: TokenStream) -> TokenStream {
FILE: src/proc_macro/src/lib.rs
type Args (line 11) | struct Args {
method parse (line 16) | fn parse(input: ParseStream) -> Result<Self> {
function adjust_caller_identifiers (line 23) | pub fn adjust_caller_identifiers(input: TokenStream) -> TokenStream {
function adjust_function_signature (line 64) | pub fn adjust_function_signature(input: TokenStream) -> TokenStream {
function deref_tuple_types (line 108) | pub fn deref_tuple_types(input: TokenStream) -> TokenStream {
FILE: src/raft/client.rs
constant ORDERING (line 22) | const ORDERING: Ordering = Ordering::Relaxed;
type Client (line 23) | pub type Client = Arc<AsyncServiceClient>;
type SubscriptionReceipt (line 24) | pub type SubscriptionReceipt = (SubKey, u64);
type ClientError (line 31) | pub enum ClientError {
method fmt (line 37) | fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
type SubscriptionError (line 48) | pub enum SubscriptionError {
type PlaneClientState (line 54) | struct PlaneClientState {
type Members (line 61) | struct Members {
type AsRaftPlaneClient (line 66) | pub trait AsRaftPlaneClient: Send + Sync {
method as_raft_plane_client (line 67) | fn as_raft_plane_client(self: &Arc<Self>) -> Arc<RaftPlaneClient>;
method as_raft_plane_client (line 133) | fn as_raft_plane_client(self: &Arc<Self>) -> Arc<RaftPlaneClient> {
method as_raft_plane_client (line 139) | fn as_raft_plane_client(self: &Arc<Self>) -> Arc<RaftPlaneClient> {
type RaftPlaneClient (line 71) | pub struct RaftPlaneClient {
method plane_id (line 77) | pub fn plane_id(&self) -> PlaneId {
method execute (line 81) | pub async fn execute<R, M>(&self, sm_id: u64, msg: M) -> Result<R, Exe...
method subscribe (line 91) | pub async fn subscribe<M, R, F>(
method unsubscribe (line 107) | pub async fn unsubscribe(
method cluster_info (line 114) | pub async fn cluster_info(&self) -> Result<ClientClusterInfo, ExecErro...
method have_state_machine (line 118) | pub async fn have_state_machine(&self, sm_id: u64) -> Result<bool, Exe...
type RaftClient (line 125) | pub struct RaftClient {
method new_plane_state (line 145) | fn new_plane_state() -> Arc<PlaneClientState> {
method new (line 154) | pub async fn new(servers: &Vec<String>, service_id: u64) -> Result<Arc...
method plane_state (line 168) | async fn plane_state(&self, plane_id: PlaneId) -> Arc<PlaneClientState> {
method plane (line 187) | pub fn plane(self: &Arc<Self>, plane_id: PlaneId) -> Arc<RaftPlaneClie...
method type1 (line 194) | pub fn type1(self: &Arc<Self>) -> Arc<RaftPlaneClient> {
method add_root_member (line 198) | pub async fn add_root_member(&self, address: &String) -> Result<bool, ...
method remove_root_member (line 203) | pub async fn remove_root_member(&self, address: &String) -> Result<(),...
method root_member_addresses (line 208) | pub async fn root_member_addresses(&self) -> Result<Vec<String>, ExecE...
method prepare_subscription (line 212) | pub async fn prepare_subscription(server: &Arc<rpc::Server>) -> Option...
method cluster_info (line 223) | async fn cluster_info<'a>(
method update_info (line 384) | async fn update_info(&self, servers: &Vec<String>) -> Result<(), Clien...
method update_plane_info (line 469) | async fn update_plane_info(
method probe_servers (line 503) | pub async fn probe_servers(
method execute (line 532) | pub async fn execute<R, M>(&self, sm_id: u64, msg: M) -> Result<R, Exe...
method execute_on_plane (line 540) | pub async fn execute_on_plane<R, M>(
method can_callback (line 567) | pub async fn can_callback() -> bool {
method get_sub_key (line 570) | fn get_sub_key<M, R>(&self, plane_id: PlaneId, sm_id: u64, msg: M) -> ...
method get_callback (line 583) | pub async fn get_callback(&self) -> Result<Arc<SubscriptionService>, S...
method subscribe (line 593) | pub async fn subscribe<M, R, F>(
method subscribe_on_plane (line 608) | pub async fn subscribe_on_plane<M, R, F>(
method unsubscribe (line 647) | pub async fn unsubscribe(
method query_on_plane (line 685) | async fn query_on_plane(
method command_on_plane (line 802) | async fn command_on_plane(
method gen_log_entry (line 1056) | fn gen_log_entry(
method leader_id (line 1072) | pub fn leader_id(&self) -> u64 {
method leader_client (line 1076) | pub async fn leader_client(&self) -> Option<(u64, Client)> {
method any_known_client (line 1081) | async fn any_known_client(&self) -> Option<Client> {
method preferred_client_on_plane (line 1086) | async fn preferred_client_on_plane(&self, state: &PlaneClientState) ->...
method leader_client_on_plane (line 1094) | async fn leader_client_on_plane(&self, state: &PlaneClientState) -> Op...
method current_leader_client_on_plane (line 1104) | async fn current_leader_client_on_plane(
method current_leader_rpc_client (line 1152) | pub async fn current_leader_rpc_client(&self) -> Result<Arc<rpc::RPCCl...
method cluster_info_on_plane (line 1160) | pub async fn cluster_info_on_plane(
method have_state_machine_on_plane (line 1178) | pub async fn have_state_machine_on_plane(
function swap_when_greater (line 1198) | fn swap_when_greater(atomic: &AtomicU64, value: u64) {
type CachedStateMachine (line 1215) | pub struct CachedStateMachine<T: StateMachineClient> {
function new (line 1224) | pub fn new(
function get (line 1245) | pub async fn get(&self) -> Arc<T> {
FILE: src/raft/disk.rs
type DiskOptions (line 18) | pub struct DiskOptions {
method new (line 29) | pub fn new(path: String) -> Self {
type StorageEntity (line 41) | pub struct StorageEntity {
method new_with_options (line 131) | pub fn new_with_options(
method new_with_options_on_plane (line 148) | pub fn new_with_options_on_plane(
method append_logs (line 277) | pub async fn append_logs<'a>(
method post_processing (line 323) | pub async fn post_processing<'a>(
method flush_wal (line 371) | pub async fn flush_wal(&mut self) -> io::Result<()> {
method write_commit_progress (line 384) | pub async fn write_commit_progress(
method read_commit_progress (line 401) | pub async fn read_commit_progress(&self) -> io::Result<Option<(u64, u6...
method write_snapshot (line 417) | pub async fn write_snapshot(&mut self, snapshot: &SnapshotEntity) -> i...
method read_snapshot (line 460) | pub async fn read_snapshot(&self) -> io::Result<Option<SnapshotEntity>> {
type DiskLogEntry (line 49) | pub struct DiskLogEntry {
method encode (line 74) | pub fn encode(&self) -> Vec<u8> {
method decode (line 92) | pub fn decode(data: &[u8]) -> io::Result<Self> {
FILE: src/raft/mod.rs
type PlaneId (line 42) | pub struct PlaneId(u64);
method type1 (line 50) | pub const fn type1() -> Self {
method type2 (line 54) | pub fn type2(raw: u64) -> Result<Self, PlaneIdError> {
method raw (line 62) | pub const fn raw(self) -> u64 {
method is_type1 (line 66) | pub const fn is_type1(self) -> bool {
method is_type2 (line 70) | pub const fn is_type2(self) -> bool {
type PlaneIdError (line 45) | pub enum PlaneIdError {
method fmt (line 76) | fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
function from (line 88) | fn from(value: PlaneId) -> Self {
type RaftMsg (line 93) | pub trait RaftMsg<R>: Send + Sync {
method encode (line 94) | fn encode(self) -> (u64, OpType, Vec<u8>);
method decode_return (line 95) | fn decode_return(data: &Vec<u8>) -> R;
constant CHECKER_MS (line 98) | const CHECKER_MS: i64 = 200;
constant HEARTBEAT_MS (line 99) | const HEARTBEAT_MS: i64 = 1000;
constant HEARTBEAT_TASK_TIMEOUT_MS (line 101) | const HEARTBEAT_TASK_TIMEOUT_MS: i64 = 5000;
type LogEntry (line 104) | pub struct LogEntry {
type ClientCmdResponse (line 113) | pub enum ClientCmdResponse {
type ClientQryResponse (line 127) | pub enum ClientQryResponse {
type ClientClusterInfo (line 139) | pub struct ClientClusterInfo {
type AppendEntriesResult (line 147) | pub enum AppendEntriesResult {
type SnapshotEntity (line 154) | pub struct SnapshotEntity {
type LogEntries (line 160) | type LogEntries = Vec<LogEntry>;
type LogsMap (line 161) | type LogsMap = BTreeMap<u64, LogEntry>;
function gen_rand (line 178) | fn gen_rand(lower: i64, higher: i64) -> i64 {
function gen_timeout (line 183) | fn gen_timeout() -> i64 {
type FollowerStatus (line 187) | struct FollowerStatus {
type LeaderMeta (line 192) | pub struct LeaderMeta {
method new (line 198) | fn new() -> LeaderMeta {
type Membership (line 206) | pub enum Membership {
type LifecycleState (line 215) | enum LifecycleState {
type RaftMeta (line 221) | pub struct RaftMeta {
type Storage (line 239) | pub enum Storage {
method default (line 245) | pub fn default() -> Storage {
type Options (line 251) | pub struct Options {
type PlaneSpec (line 258) | pub struct PlaneSpec {
type PlaneBootstrap (line 263) | pub struct PlaneBootstrap {
type PlaneError (line 269) | pub enum PlaneError {
method fmt (line 276) | fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
type PlaneBootstrapError (line 292) | pub enum PlaneBootstrapError {
method from (line 377) | fn from(value: PlaneError) -> Self {
method from (line 383) | fn from(value: ExecError) -> Self {
method from (line 389) | fn from(value: ClientError) -> Self {
method fmt (line 317) | fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
type RaftPlaneRuntime (line 394) | struct RaftPlaneRuntime {
method new (line 403) | fn new(opts: &Options, plane_id: PlaneId) -> Result<Self, PlaneError> {
type PlaneHandle (line 449) | pub struct PlaneHandle {
method id (line 455) | pub const fn id(&self) -> PlaneId {
method callback (line 459) | pub async fn callback(&self, state_machine_id: u64) -> Result<SMCallba...
method register_state_machine (line 463) | pub async fn register_state_machine(
method recover_after_register (line 472) | pub async fn recover_after_register(&self) -> Result<(), PlaneError> {
method cluster_info (line 478) | pub async fn cluster_info(&self) -> Result<ClientClusterInfo, PlaneErr...
method have_state_machine (line 484) | pub async fn have_state_machine(&self, sm_id: u64) -> Result<bool, Pla...
method is_leader (line 490) | pub async fn is_leader(&self) -> Result<bool, PlaneError> {
method flush_persistence (line 494) | pub async fn flush_persistence(&self) -> Result<(), PlaneError> {
method shutdown (line 498) | pub async fn shutdown(&self) -> Result<(), PlaneError> {
function options_for_plane (line 503) | fn options_for_plane(opts: &Options, plane_id: PlaneId) -> Options {
type RaftService (line 522) | pub struct RaftService {
method plane_id (line 604) | pub const fn plane_id(&self) -> PlaneId {
method check_plane (line 608) | fn check_plane(&self, plane_id: PlaneId) {
method lifecycle_is_stopping (line 619) | fn lifecycle_is_stopping(state: LifecycleState) -> bool {
method wait_for_apply_drain_for_meta (line 623) | async fn wait_for_apply_drain_for_meta(
method start_managed_runtime (line 644) | async fn start_managed_runtime(self: &Arc<Self>, runtime: Option<Arc<R...
method shutdown_managed_runtime (line 806) | async fn shutdown_managed_runtime(&self, runtime: Option<Arc<RaftPlane...
method recover_after_register (line 872) | pub async fn recover_after_register(&self) {
method plane_storage_path (line 894) | fn plane_storage_path(&self, plane_id: PlaneId) -> Option<String> {
method plane_has_persisted_state (line 903) | fn plane_has_persisted_state(&self, plane_id: PlaneId) -> bool {
method load_snapshot_into_meta (line 914) | async fn load_snapshot_into_meta(
method recover_config_state_from_logs (line 993) | async fn recover_config_state_from_logs(
method initialize_runtime_meta (line 1027) | async fn initialize_runtime_meta(
method resolve_plane_runtime (line 1106) | async fn resolve_plane_runtime(
method runtime_for_plane (line 1150) | async fn runtime_for_plane(
method canonicalize_member_addresses (line 1159) | fn canonicalize_member_addresses(mut members: Vec<String>) -> Vec<Stri...
method validate_plane_bootstrap (line 1165) | fn validate_plane_bootstrap(
method plane_members_from_seed_nodes (line 1181) | async fn plane_members_from_seed_nodes(
method plane_member_addresses (line 1199) | async fn plane_member_addresses(&self, plane_id: PlaneId) -> Result<Ve...
method add_plane_member_via_log (line 1225) | async fn add_plane_member_via_log(
method ensure_plane (line 1253) | pub async fn ensure_plane(
method plane (line 1270) | pub async fn plane(self: &Arc<Self>, plane_id: PlaneId) -> Result<Plan...
method ensure_plane_membership (line 1281) | async fn ensure_plane_membership(
method ensure_plane_from_seeds (line 1320) | pub async fn ensure_plane_from_seeds(
method loaded_type2_planes (line 1334) | pub async fn loaded_type2_planes(&self) -> Vec<PlaneId> {
method recover_after_register_on_plane (line 1339) | pub async fn recover_after_register_on_plane(
method register_state_machine_on_plane (line 1363) | pub async fn register_state_machine_on_plane(
method subscriptions_on_plane (line 1379) | pub(crate) async fn subscriptions_on_plane(
method cluster_info_on_plane_local (line 1394) | pub async fn cluster_info_on_plane_local(
method have_state_machine_on_plane_local (line 1424) | pub async fn have_state_machine_on_plane_local(
method is_leader_on_plane (line 1440) | pub async fn is_leader_on_plane(&self, plane_id: PlaneId) -> Result<bo...
method flush_persistence_on_plane (line 1448) | pub async fn flush_persistence_on_plane(&self, plane_id: PlaneId) -> R...
method shutdown_plane (line 1468) | pub async fn shutdown_plane(&self, plane_id: PlaneId) -> Result<(), Pl...
method new (line 1516) | pub fn new(opts: Options) -> Arc<RaftService> {
method load_snapshot_on_startup (line 1580) | async fn load_snapshot_on_startup(&self) -> bool {
method start (line 1654) | pub async fn start(server: &Arc<RaftService>, recover_registered: bool...
method new_server (line 1680) | pub async fn new_server(opts: Options) -> (bool, Arc<RaftService>, Arc...
method probe_and_join (line 1689) | pub async fn probe_and_join(&self, servers: &Vec<String>) -> Result<bo...
method bootstrap (line 1713) | pub async fn bootstrap(&self) {
method conservative_bootstrap (line 1721) | pub async fn conservative_bootstrap(&self, servers: &Vec<String>) {
method join (line 1766) | pub async fn join(&self, servers: &Vec<String>) -> Result<bool, ExecEr...
method leave (line 1845) | pub async fn leave(&self) -> bool {
method cluster_info (line 1960) | pub async fn cluster_info(&self) -> ClientClusterInfo {
method num_members (line 1977) | pub async fn num_members(&self) -> usize {
method num_logs (line 1983) | pub async fn num_logs(&self) -> usize {
method last_log_id (line 1988) | pub async fn last_log_id(&self) -> Option<u64> {
method leader_id (line 1993) | pub async fn leader_id(&self) -> u64 {
method is_leader_for_real (line 1997) | pub async fn is_leader_for_real(&self) -> bool {
method is_leader (line 2004) | pub fn is_leader(&self) -> bool {
method get_server_id (line 2007) | pub fn get_server_id(&self) -> u64 {
method flush_persistence (line 2012) | pub async fn flush_persistence(&self) {
method shutdown (line 2030) | pub async fn shutdown(&self) {
method register_state_machine (line 2052) | pub async fn register_state_machine(&self, state_machine: SubStateMach...
method switch_membership (line 2057) | fn switch_membership(&self, meta: &mut RwLockWriteGuard<RaftMeta>, mem...
method get_log_info_ (line 2061) | fn get_log_info_(&self, log: Option<(&u64, &LogEntry)>) -> (u64, u64) {
method insert_leader_follower_meta (line 2067) | fn insert_leader_follower_meta(
method reload_leader_meta (line 2084) | fn reload_leader_meta(
method write_meta (line 2094) | async fn write_meta<'a>(&'a self) -> RwLockWriteGuard<'a, RaftMeta> {
method read_meta (line 2098) | pub async fn read_meta(&self) -> RwLockReadGuard<'_, RaftMeta> {
method become_candidate_on_plane (line 2102) | async fn become_candidate_on_plane<'a>(
method become_candidate (line 2229) | async fn become_candidate<'a>(&'a self, meta: &'a mut RwLockWriteGuard...
method become_follower_on_plane (line 2234) | fn become_follower_on_plane(
method become_follower (line 2247) | fn become_follower(&self, meta: &mut RwLockWriteGuard<RaftMeta>, term:...
method become_leader_on_plane (line 2251) | async fn become_leader_on_plane(
method become_leader (line 2276) | async fn become_leader(&self, meta: &mut RwLockWriteGuard<'_, RaftMeta...
method send_followers_heartbeat_on_plane (line 2281) | async fn send_followers_heartbeat_on_plane<'a>(
method send_followers_heartbeat (line 2432) | async fn send_followers_heartbeat<'a>(
method send_follower_heartbeat (line 2442) | async fn send_follower_heartbeat(
method check_term_on_plane (line 2657) | fn check_term_on_plane(
method reset_last_checked (line 2672) | fn reset_last_checked(&self, meta: &mut RwLockWriteGuard<RaftMeta>) {
method handle_append_entries_on_meta (line 2683) | async fn handle_append_entries_on_meta<'a>(
method handle_request_vote_on_meta (line 2764) | async fn handle_request_vote_on_meta<'a>(
method handle_install_snapshot_on_meta (line 2793) | async fn handle_install_snapshot_on_meta<'a>(
method handle_client_command_on_meta (line 2835) | async fn handle_client_command_on_meta<'a>(
method handle_client_query_on_meta (line 2920) | async fn handle_client_query_on_meta<'a>(
method leader_append_log (line 2942) | async fn leader_append_log<'a>(
method logs_post_processing (line 2968) | async fn logs_post_processing<'a>(
method try_sync_log_to_followers_on_plane (line 2980) | async fn try_sync_log_to_followers_on_plane<'a>(
method try_sync_config_to_followers_on_plane (line 3051) | async fn try_sync_config_to_followers_on_plane<'a>(
method should_take_snapshot (line 3074) | fn should_take_snapshot(
method take_snapshot (line 3111) | async fn take_snapshot(&self, meta: &mut RwLockWriteGuard<'_, RaftMeta...
method compact_logs_after_snapshot (line 3172) | async fn compact_logs_after_snapshot(
type CheckerAction (line 535) | enum CheckerAction {
type RequestVoteResponse (line 543) | enum RequestVoteResponse {
type HeartbeatReplicationResult (line 549) | enum HeartbeatReplicationResult {
function check_commit (line 561) | async fn check_commit(meta: &mut RwLockWriteGuard<'_, RaftMeta>) {
function is_majority (line 1479) | fn is_majority(members: u64, granted: u64) -> bool {
function apply_committed_entry (line 1489) | async fn apply_committed_entry<'a>(
function is_leader (line 1496) | fn is_leader(meta: &RwLockWriteGuard<RaftMeta>) -> bool {
function alter_term (line 1502) | fn alter_term(meta: &mut RwLockWriteGuard<RaftMeta>, term: u64) {
function ensure_direct_leader_term (line 1509) | fn ensure_direct_leader_term(meta: &mut RwLockWriteGuard<'_, RaftMeta>) {
method append_entries (line 3224) | fn append_entries<'a>(
method request_vote (line 3277) | fn request_vote(
method install_snapshot (line 3322) | fn install_snapshot(
method c_command (line 3372) | fn c_command<'a>(
method c_query (line 3402) | fn c_query<'a>(
method c_server_cluster_info (line 3433) | fn c_server_cluster_info(&self, plane_id: PlaneId) -> BoxFuture<ClientCl...
method c_put_offline (line 3455) | fn c_put_offline(&self) -> BoxFuture<bool> {
method c_have_state_machine (line 3459) | fn c_have_state_machine(&self, plane_id: PlaneId, id: u64) -> BoxFuture<...
method c_ping (line 3476) | fn c_ping(&self) -> BoxFuture<()> {
method reelect (line 3480) | fn reelect<'a>(&'a self, plane_id: PlaneId) -> futures::future::BoxFutur...
type RaftStateMachine (line 3521) | pub struct RaftStateMachine {
method new (line 3527) | pub fn new(name: &String) -> RaftStateMachine {
type CounterStateMachine (line 3553) | struct CounterStateMachine {
method add (line 3563) | fn add(&mut self, value: u64) -> BoxFuture<u64> {
method get (line 3568) | fn get(&self) -> BoxFuture<u64> {
method id (line 3576) | fn id(&self) -> u64 {
method snapshot (line 3580) | fn snapshot(&self) -> Vec<u8> {
method recover (line 3584) | fn recover(&mut self, _: Vec<u8>) -> BoxFuture<()> {
method recoverable (line 3588) | fn recoverable(&self) -> bool {
type PersistentCounterStateMachine (line 3593) | struct PersistentCounterStateMachine {
method add (line 3598) | fn add(&mut self, value: u64) -> BoxFuture<u64> {
method get (line 3603) | fn get(&self) -> BoxFuture<u64> {
method id (line 3611) | fn id(&self) -> u64 {
method snapshot (line 3615) | fn snapshot(&self) -> Vec<u8> {
method recover (line 3619) | fn recover(&mut self, data: Vec<u8>) -> BoxFuture<()> {
method recoverable (line 3626) | fn recoverable(&self) -> bool {
function wait_for_plane (line 3631) | async fn wait_for_plane(service: &Arc<RaftService>, plane_id: PlaneId) -...
function query_counter_locally (line 3641) | async fn query_counter_locally(
function add_counter_locally (line 3661) | async fn add_counter_locally(
function startup (line 3683) | async fn startup() {
function type2_plane_client_roundtrip (line 3694) | async fn type2_plane_client_roundtrip() {
function loaded_type2_planes_only_reports_materialized_type2_runtimes (line 3739) | async fn loaded_type2_planes_only_reports_materialized_type2_runtimes() {
function type2_plane_shutdown_rejects_commands (line 3765) | async fn type2_plane_shutdown_rejects_commands() {
function unknown_type2_plane_does_not_fall_back_to_type1 (line 3814) | async fn unknown_type2_plane_does_not_fall_back_to_type1() {
function unknown_type2_plane_rejects_commands_without_leader_discovery_loop (line 3841) | async fn unknown_type2_plane_rejects_commands_without_leader_discovery_l...
function root_membership_helpers_hide_config_sm_commands (line 3867) | async fn root_membership_helpers_hide_config_sm_commands() {
function type2_plane_bootstrap_is_idempotent_for_same_members (line 3926) | async fn type2_plane_bootstrap_is_idempotent_for_same_members() {
function type2_plane_bootstrap_rejects_conflicting_member_set (line 3965) | async fn type2_plane_bootstrap_rejects_conflicting_member_set() {
function type2_plane_multinode_replication_and_reelection (line 4030) | async fn type2_plane_multinode_replication_and_reelection() {
function type2_plane_multinode_follower_restart_recovers_without_eager_load (line 4168) | async fn type2_plane_multinode_follower_restart_recovers_without_eager_l...
function type2_plane_lazy_loads_after_unload (line 4302) | async fn type2_plane_lazy_loads_after_unload() {
function server_membership (line 4410) | async fn server_membership() {
function log_replication (line 4510) | async fn log_replication() {
type SM (line 4649) | struct SM {
method answer_to_the_universe (line 4653) | fn answer_to_the_universe<'a>(&'a self, name: String) -> BoxFuture<'_, S...
method take_a_shot (line 4657) | fn take_a_shot(&mut self, num: i32) -> BoxFuture<i32> {
method get_shot (line 4662) | fn get_shot(&self) -> BoxFuture<i32> {
method id (line 4668) | fn id(&self) -> u64 {
method snapshot (line 4671) | fn snapshot(&self) -> Vec<u8> {
method recover (line 4675) | fn recover(&mut self, data: Vec<u8>) -> BoxFuture<()> {
method recoverable (line 4683) | fn recoverable(&self) -> bool {
function query_and_command (line 4689) | async fn query_and_command() {
function multi_server_command (line 4724) | async fn multi_server_command() {
function snapshot_disk_persistence (line 4820) | async fn snapshot_disk_persistence() {
function snapshot_persistence_and_recovery (line 4885) | async fn snapshot_persistence_and_recovery() {
function test_snapshot_recovery_on_startup (line 4942) | async fn test_snapshot_recovery_on_startup() {
function test_snapshot_write_and_read (line 4985) | async fn test_snapshot_write_and_read() {
function test_snapshot_corruption_detection (line 5034) | async fn test_snapshot_corruption_detection() {
function test_log_compaction_removes_old_logs (line 5082) | async fn test_log_compaction_removes_old_logs() {
function test_snapshot_threshold_configuration (line 5175) | async fn test_snapshot_threshold_configuration() {
function test_state_machine_snapshot_and_recovery (line 5228) | async fn test_state_machine_snapshot_and_recovery() {
function test_install_snapshot_compacts_logs (line 5252) | async fn test_install_snapshot_compacts_logs() {
function test_wal_logs_written_to_disk (line 5365) | async fn test_wal_logs_written_to_disk() {
function test_wal_recovery_after_crash (line 5439) | async fn test_wal_recovery_after_crash() {
function test_wal_log_file_format (line 5568) | async fn test_wal_log_file_format() {
function test_wal_fsync_durability (line 5652) | async fn test_wal_fsync_durability() {
function test_wal_log_recovery_integration (line 5737) | async fn test_wal_log_recovery_integration() {
function test_wal_deterministic_encoding (line 5892) | async fn test_wal_deterministic_encoding() {
function test_wal_encoding_with_empty_data (line 5968) | async fn test_wal_encoding_with_empty_data() {
function test_wal_encoding_with_large_data (line 5996) | async fn test_wal_encoding_with_large_data() {
function test_wal_only_minimal_rsm_recovery (line 6031) | async fn test_wal_only_minimal_rsm_recovery() {
function test_abrupt_crash_full_recovery (line 6192) | async fn test_abrupt_crash_full_recovery() {
function test_wal_partial_write_truncation (line 6295) | async fn test_wal_partial_write_truncation() {
function test_wal_crc_corruption_stops_at_bad_entry (line 6389) | async fn test_wal_crc_corruption_stops_at_bad_entry() {
function test_snapshot_plus_wal_not_start_over (line 6501) | async fn test_snapshot_plus_wal_not_start_over() {
FILE: src/raft/state_machine/callback/client.rs
type SubscriptionService (line 12) | pub struct SubscriptionService {
method initialize (line 45) | pub async fn initialize(server: &Arc<Server>) -> Arc<SubscriptionServi...
method notify (line 19) | fn notify<'a>(&'a self, key: SubKey, data: &'a Vec<u8>) -> BoxFuture<'a,...
FILE: src/raft/state_machine/callback/mod.rs
type SubKey (line 10) | pub struct SubKey {
method new (line 19) | pub const fn new(
type Trigger (line 54) | pub struct Trigger {
method trigger (line 65) | fn trigger(&mut self) -> BoxFuture<()> {
method id (line 79) | fn id(&self) -> u64 {
method snapshot (line 82) | fn snapshot(&self) -> Vec<u8> {
method recover (line 85) | fn recover(&mut self, _: Vec<u8>) -> BoxFuture<()> {
method recoverable (line 88) | fn recoverable(&self) -> bool {
function dummy (line 94) | async fn dummy() {
function dummy_type2_plane (line 156) | async fn dummy_type2_plane() {
FILE: src/raft/state_machine/callback/server.rs
type Subscriber (line 14) | pub struct Subscriber {
type Subscriptions (line 19) | pub struct Subscriptions {
method new (line 29) | pub fn new() -> Subscriptions {
method subscribe (line 40) | pub async fn subscribe(
method remove_subscriber (line 108) | pub fn remove_subscriber(&mut self, suber_id: u64) {
method remove_subscription (line 122) | pub fn remove_subscription(&mut self, id: u64) {
type InternalSubscription (line 135) | pub struct InternalSubscription {
type SMCallback (line 139) | pub struct SMCallback {
method new (line 158) | pub async fn new(state_machine_id: u64, raft_service: Arc<RaftService>...
method new_on_plane (line 164) | pub async fn new_on_plane(
method notify (line 179) | pub async fn notify<M, R>(
method internal_subscribe (line 272) | pub async fn internal_subscribe<R, F, M>(&self, msg: M, trigger: F) ->...
type NotifyError (line 148) | pub enum NotifyError {
function notify (line 299) | pub async fn notify<M, R, F>(callback: &Option<SMCallback>, msg: M, data...
function test_subscriptions_new (line 323) | fn test_subscriptions_new() {
function test_remove_subscription_nonexistent (line 335) | fn test_remove_subscription_nonexistent() {
function test_remove_subscription (line 346) | fn test_remove_subscription() {
function test_remove_subscriber (line 371) | fn test_remove_subscriber() {
function test_remove_subscriber_nonexistent (line 400) | fn test_remove_subscriber_nonexistent() {
function test_notify_error_debug (line 410) | fn test_notify_error_debug() {
function test_subscriptions_next_id_increment (line 426) | fn test_subscriptions_next_id_increment() {
function test_subscriptions_multiple_subs_per_subscriber (line 444) | fn test_subscriptions_multiple_subs_per_subscriber() {
FILE: src/raft/state_machine/configs.rs
constant CONFIG_SM_ID (line 13) | pub const CONFIG_SM_ID: u64 = 1;
type RaftMember (line 16) | pub struct RaftMember {
type Configures (line 22) | pub struct Configures {
method new (line 137) | pub fn new(service_id: u64) -> Configures {
method recover_members (line 144) | async fn recover_members(&mut self, snapshot: MemberConfigSnapshot) {
method new_member (line 158) | pub async fn new_member(&mut self, address: String) -> bool {
method del_member (line 161) | pub async fn del_member(&mut self, address: String) {
method member_existed (line 164) | pub fn member_existed(&self, id: u64) -> bool {
type MemberConfigSnapshot (line 29) | pub type MemberConfigSnapshot = HashSet<String>;
type ConfigSnapshot (line 32) | pub struct ConfigSnapshot {
method new_member_ (line 47) | fn new_member_(&mut self, address: String) -> BoxFuture<bool> {
method del_member_ (line 74) | fn del_member_(&mut self, address: String) -> BoxFuture<()> {
method member_address (line 81) | fn member_address(&self) -> BoxFuture<Vec<String>> {
method subscribe (line 84) | fn subscribe(
method unsubscribe (line 96) | fn unsubscribe(&mut self, sub_id: u64) -> BoxFuture<()> {
method id (line 107) | fn id(&self) -> u64 {
method snapshot (line 110) | fn snapshot(&self) -> Vec<u8> {
method recover (line 119) | fn recover(&mut self, data: Vec<u8>) -> BoxFuture<()> {
method recoverable (line 131) | fn recoverable(&self) -> bool {
function test_configures_new (line 175) | fn test_configures_new() {
function test_configures_id (line 185) | fn test_configures_id() {
function test_configures_recoverable (line 192) | fn test_configures_recoverable() {
function test_member_existed (line 198) | fn test_member_existed() {
function test_member_address (line 208) | async fn test_member_address() {
function test_snapshot_empty (line 215) | fn test_snapshot_empty() {
function test_recover_empty_snapshot (line 229) | async fn test_recover_empty_snapshot() {
function test_recover_invalid_data (line 239) | async fn test_recover_invalid_data() {
function test_del_member (line 251) | async fn test_del_member() {
function test_config_snapshot_serialization (line 261) | fn test_config_snapshot_serialization() {
function test_configures_has_subscriptions (line 279) | fn test_configures_has_subscriptions() {
function test_membership_server_join (line 291) | async fn test_membership_server_join() {
function test_membership_server_leave (line 325) | async fn test_membership_server_leave() {
function test_membership_server_rejoin (line 356) | async fn test_membership_server_rejoin() {
function test_membership_multiple_servers_join (line 388) | async fn test_membership_multiple_servers_join() {
function test_membership_partial_leave (line 433) | async fn test_membership_partial_leave() {
function test_membership_duplicate_join_attempt (line 478) | async fn test_membership_duplicate_join_attempt() {
function test_membership_snapshot_with_members (line 505) | async fn test_membership_snapshot_with_members() {
function test_membership_recovery_with_changes (line 550) | async fn test_membership_recovery_with_changes() {
function test_membership_leave_nonexistent (line 600) | async fn test_membership_leave_nonexistent() {
function test_membership_join_unreachable_server (line 610) | async fn test_membership_join_unreachable_server() {
FILE: src/raft/state_machine/master.rs
type ExecError (line 11) | pub enum ExecError {
type RegisterResult (line 22) | pub enum RegisterResult {
type ExecOk (line 28) | pub type ExecOk = Vec<u8>;
type ExecResult (line 29) | pub type ExecResult = Result<ExecOk, ExecError>;
type SubStateMachine (line 30) | pub type SubStateMachine = Box<dyn StateMachineCtl>;
type SnapshotDataItem (line 31) | pub type SnapshotDataItem = (u64, Vec<u8>);
type SnapshotDataItems (line 32) | pub type SnapshotDataItems = Vec<SnapshotDataItem>;
type MasterStateMachine (line 36) | pub struct MasterStateMachine {
method new (line 96) | pub fn new(service_id: u64) -> MasterStateMachine {
method new_on_plane (line 100) | pub fn new_on_plane(service_id: u64, plane_id: PlaneId) -> MasterState...
method is_recoverable (line 111) | pub fn is_recoverable(&self, sm_id: u64) -> bool {
method register (line 122) | pub fn register(&mut self, smc: SubStateMachine) -> RegisterResult {
method recover_registered_snapshots (line 134) | pub async fn recover_registered_snapshots(&mut self) {
method members (line 154) | pub fn members(&self) -> &HashMap<u64, RaftMember> {
method commit_cmd (line 158) | pub async fn commit_cmd(&mut self, entry: &LogEntry) -> ExecResult {
method exec_qry (line 205) | pub async fn exec_qry(&self, entry: &LogEntry) -> ExecResult {
method clear_subs (line 252) | pub fn clear_subs(&mut self) {
method has_sub (line 255) | pub fn has_sub(&self, id: &u64) -> bool {
method id (line 47) | fn id(&self) -> u64 {
method snapshot (line 50) | fn snapshot(&self) -> Vec<u8> {
method recover (line 63) | fn recover(&mut self, data: Vec<u8>) -> BoxFuture<()> {
method recoverable (line 81) | fn recoverable(&self) -> bool {
function parse_output (line 86) | pub fn parse_output<'a>(r: Option<Vec<u8>>) -> ExecResult {
method fmt (line 262) | fn fmt(&self, f: &mut Formatter) -> fmt::Result {
type MockStateMachine (line 273) | struct MockStateMachine {
method id (line 283) | fn id(&self) -> u64 {
method snapshot (line 287) | fn snapshot(&self) -> Vec<u8> {
method recover (line 291) | fn recover(&mut self, data: Vec<u8>) -> BoxFuture<()> {
method recoverable (line 296) | fn recoverable(&self) -> bool {
function test_master_state_machine_new (line 304) | fn test_master_state_machine_new() {
function test_master_state_machine_id (line 316) | fn test_master_state_machine_id() {
function test_master_state_machine_recoverable (line 322) | fn test_master_state_machine_recoverable() {
function test_is_recoverable_config_sm (line 328) | fn test_is_recoverable_config_sm() {
function test_is_recoverable_registered_sm (line 336) | fn test_is_recoverable_registered_sm() {
function test_is_recoverable_non_recoverable_sm (line 351) | fn test_is_recoverable_non_recoverable_sm() {
function test_is_recoverable_unregistered_sm (line 366) | fn test_is_recoverable_unregistered_sm() {
function test_register_ok (line 374) | fn test_register_ok() {
function test_register_reserved (line 390) | fn test_register_reserved() {
function test_register_existed (line 417) | fn test_register_existed() {
function test_register_with_snapshot_recovery (line 442) | async fn test_register_with_snapshot_recovery() {
function test_recover_registered_snapshots_applies_config_snapshot (line 477) | async fn test_recover_registered_snapshots_applies_config_snapshot() {
function test_members (line 502) | fn test_members() {
function test_clear_subs (line 510) | fn test_clear_subs() {
function test_has_sub (line 529) | fn test_has_sub() {
function test_snapshot_empty (line 547) | fn test_snapshot_empty() {
function test_snapshot_with_recoverable_sm (line 559) | fn test_snapshot_with_recoverable_sm() {
function test_snapshot_non_recoverable_sm_excluded (line 586) | fn test_snapshot_non_recoverable_sm_excluded() {
function test_recover_valid_snapshot (line 610) | async fn test_recover_valid_snapshot() {
function test_recover_invalid_snapshot (line 626) | async fn test_recover_invalid_snapshot() {
function test_parse_output_some (line 641) | fn test_parse_output_some() {
function test_parse_output_none (line 650) | fn test_parse_output_none() {
function test_exec_error_display (line 658) | fn test_exec_error_display() {
function test_exec_error_debug (line 669) | fn test_exec_error_debug() {
function test_exec_error_clone (line 676) | fn test_exec_error_clone() {
function test_all_exec_error_variants (line 684) | fn test_all_exec_error_variants() {
function test_register_result_variants (line 697) | fn test_register_result_variants() {
type KeyValueStateMachine (line 709) | struct KeyValueStateMachine {
method id (line 717) | fn id(&self) -> u64 {
method snapshot (line 721) | fn snapshot(&self) -> Vec<u8> {
method recover (line 725) | fn recover(&mut self, data: Vec<u8>) -> BoxFuture<()> {
method recoverable (line 732) | fn recoverable(&self) -> bool {
method dispatch_cmd_ (line 738) | fn dispatch_cmd_<'a>(
method dispatch_qry_ (line 771) | fn dispatch_qry_<'a>(
method op_type_ (line 797) | fn op_type_(&self, _fn_id: u64) -> Option<OpType> {
function test_real_world_kv_store_workflow (line 803) | async fn test_real_world_kv_store_workflow() {
function test_real_world_multiple_operations (line 846) | async fn test_real_world_multiple_operations() {
function test_real_world_snapshot_and_recovery (line 907) | async fn test_real_world_snapshot_and_recovery() {
function test_real_world_multiple_state_machines (line 966) | async fn test_real_world_multiple_state_machines() {
function test_real_world_error_handling (line 1034) | async fn test_real_world_error_handling() {
function test_real_world_query_on_non_existent_sm (line 1071) | async fn test_real_world_query_on_non_existent_sm() {
function test_real_world_config_sm_operations (line 1088) | async fn test_real_world_config_sm_operations() {
function test_real_world_state_machine_lifecycle (line 1109) | async fn test_real_world_state_machine_lifecycle() {
function test_real_world_concurrent_queries (line 1155) | async fn test_real_world_concurrent_queries() {
FILE: src/raft/state_machine/mod.rs
type Storage (line 5) | pub enum Storage {
type OpType (line 11) | pub enum OpType {
type StateMachineCtl (line 17) | pub trait StateMachineCtl: Sync + Send + Any {
method id (line 18) | fn id(&self) -> u64;
method snapshot (line 19) | fn snapshot(&self) -> Vec<u8>;
method recover (line 20) | fn recover(&mut self, data: Vec<u8>) -> ::futures::future::BoxFuture<()>;
method recoverable (line 21) | fn recoverable(&self) -> bool;
method fn_dispatch_qry (line 22) | fn fn_dispatch_qry<'a>(
method fn_dispatch_cmd (line 27) | fn fn_dispatch_cmd<'a>(
method op_type (line 32) | fn op_type(&mut self, fn_id: u64) -> Option<OpType>;
type OpTypes (line 35) | pub trait OpTypes {
method op_type (line 36) | fn op_type(&self, fn_id: u64) -> Option<OpType>;
type StateMachineClient (line 39) | pub trait StateMachineClient {
method new_instance (line 40) | fn new_instance(sm_id: u64, client: &Arc<RaftPlaneClient>) -> Self;
constant MASTER_SM_ID (line 43) | pub const MASTER_SM_ID: u64 = 0;
constant CONFIG_SM_ID (line 44) | pub const CONFIG_SM_ID: u64 = 1;
constant RESERVED_INTERNAL_SM_ID_END (line 45) | pub const RESERVED_INTERNAL_SM_ID_END: u64 = 2;
function is_reserved_internal_sm_id (line 47) | pub const fn is_reserved_internal_sm_id(sm_id: u64) -> bool {
FILE: src/rpc/cluster.rs
function broadcast_to_members (line 13) | pub async fn broadcast_to_members<C, F, R, Fut>(
function all_server_ids (line 26) | pub async fn all_server_ids(
function broadcast_with_server_ids (line 33) | pub async fn broadcast_with_server_ids<C, F, R, I, Fut>(
function client_by_server_id (line 63) | pub async fn client_by_server_id<C>(
function client_by_rpc_client (line 77) | pub fn client_by_rpc_client<C>(client: &Arc<RPCClient>) -> Arc<C>
type TestService (line 100) | pub struct TestService {
method get_id (line 105) | fn get_id(&self) -> BoxFuture<u64> {
method echo (line 109) | fn echo(&self, msg: String) -> BoxFuture<String> {
constant SERVICE_ID (line 116) | const SERVICE_ID: u64 = 999;
function test_client_by_rpc_client_creation (line 121) | fn test_client_by_rpc_client_creation() {
FILE: src/rpc/mod.rs
type RPCRequestError (line 28) | pub enum RPCRequestError {
type RPCError (line 36) | pub enum RPCError {
type RPCService (line 42) | pub trait RPCService: Sync + Send {
method dispatch (line 43) | fn dispatch(&self, data: BytesMut) -> BoxFuture<Result<BytesMut, RPCRe...
method register_shortcut_service (line 44) | fn register_shortcut_service(
method service_symbol (line 50) | fn service_symbol(&self) -> &'static str;
type Server (line 53) | pub struct Server {
method new (line 105) | pub fn new(address: &String) -> Arc<Server> {
method listen (line 115) | pub async fn listen(server: &Arc<Server>) -> Result<(), Box<dyn Error>> {
method listen_and_resume (line 165) | pub async fn listen_and_resume(server: &Arc<Server>) {
method shutdown (line 229) | pub async fn shutdown(&self) {
method register_service_with_id (line 243) | pub async fn register_service_with_id<T>(&self, service_id: u64, servi...
method register_service (line 264) | pub async fn register_service<T>(&self, service: &Arc<T>)
method remove_service (line 271) | pub async fn remove_service(&self, service_id: u64) {
method address (line 274) | pub fn address(&self) -> &String {
type ClientPool (line 63) | pub struct ClientPool {
method new (line 314) | pub fn new() -> ClientPool {
method get (line 320) | pub async fn get(&self, addr: &String) -> io::Result<Arc<RPCClient>> {
method get_by_id (line 326) | pub async fn get_by_id<F>(&self, server_id: u64, addr_fn: F) -> io::Re...
function encode_res (line 67) | fn encode_res(res: Result<BytesMut, RPCRequestError>) -> BytesMut {
function decode_res (line 81) | fn decode_res(res: io::Result<BytesMut>) -> Result<BytesMut, RPCError> {
function read_u64_head (line 99) | pub fn read_u64_head(mut data: BytesMut) -> (u64, BytesMut) {
type RPCClient (line 279) | pub struct RPCClient {
method send_async (line 293) | pub async fn send_async(
method new_async (line 303) | pub async fn new_async(addr: &String) -> io::Result<Arc<RPCClient>> {
function prepend_u64 (line 285) | pub fn prepend_u64(num: u64, data: BytesMut) -> BytesMut {
type ServiceClient (line 345) | pub trait ServiceClient: Send + Sync {
method new_instance_with_service_id (line 346) | fn new_instance_with_service_id(server_id: u64, client: &Arc<RPCClient...
method server_id (line 347) | fn server_id(&self) -> u64;
method new_with_service_id (line 348) | fn new_with_service_id(server_id: u64, client: &Arc<RPCClient>) -> Arc...
type ServiceClientWithId (line 356) | pub trait ServiceClientWithId: ServiceClient {
constant SERVICE_ID (line 357) | const SERVICE_ID: u64;
method new (line 359) | fn new(client: &Arc<RPCClient>) -> Arc<Self>
type RPCServiceWithId (line 367) | pub trait RPCServiceWithId: RPCService {
constant SERVICE_ID (line 368) | const SERVICE_ID: u64;
type HelloServer (line 388) | struct HelloServer;
method hello (line 391) | fn hello(&self, name: String) -> BoxFuture<String> {
method error (line 394) | fn error(&self, message: String) -> BoxFuture<Result<(), String>> {
function simple_rpc (line 401) | pub async fn simple_rpc() {
type Greeting (line 431) | pub struct Greeting {
type Respond (line 437) | pub struct Respond {
type HelloServer (line 446) | pub struct HelloServer;
method hello (line 449) | fn hello(&self, gret: Greeting) -> BoxFuture<Respond> {
function struct_rpc (line 460) | pub async fn struct_rpc() {
type ComplexAnswer (line 491) | pub struct ComplexAnswer {
type IdServer (line 504) | struct IdServer {
method query_server_id (line 508) | fn query_server_id(&self) -> BoxFuture<u64> {
method query_answer (line 512) | fn query_answer(&self, req: Option<String>) -> BoxFuture<ComplexAnswer> {
method large_query (line 521) | fn large_query(&self, req: Option<String>) -> BoxFuture<Vec<ComplexAnswe...
method large_req (line 533) | fn large_req(
function multi_server_rpc (line 545) | async fn multi_server_rpc() {
function lots_of_reqs (line 602) | pub async fn lots_of_reqs() {
FILE: src/rpc/proto.rs
type A (line 238) | pub struct A {
FILE: src/tcp/client.rs
type Client (line 24) | pub struct Client {
method connect_with_timeout (line 34) | pub async fn connect_with_timeout(address: &String, timeout: Duration)...
method connect (line 94) | pub async fn connect(address: &String) -> io::Result<Self> {
method send_msg (line 97) | pub async fn send_msg(&self, msg: TcpReq) -> io::Result<BytesMut> {
function test_client_connect_timeout (line 137) | async fn test_client_connect_timeout() {
function test_client_standalone_address (line 153) | async fn test_client_standalone_address() {
function test_client_server_id (line 168) | async fn test_client_server_id() {
FILE: src/tcp/server.rs
type RPCFuture (line 14) | pub type RPCFuture = dyn Future<Output = TcpRes>;
type BoxedRPCFuture (line 15) | pub type BoxedRPCFuture = Box<RPCFuture>;
type TcpReq (line 16) | pub type TcpReq = BytesMut;
type TcpRes (line 17) | pub type TcpRes = Pin<Box<dyn Future<Output = BytesMut> + Send>>;
type Server (line 19) | pub struct Server {
method new (line 24) | pub fn new() -> Server {
method shutdown_handle (line 29) | pub fn shutdown_handle(&self) -> broadcast::Sender<()> {
method listen (line 33) | pub async fn listen(
method shutdown (line 105) | pub fn shutdown(&self) {
function test_server_creation (line 120) | async fn test_server_creation() {
function test_shutdown_handle (line 129) | async fn test_shutdown_handle() {
function test_server_listen_and_shutdown (line 142) | async fn test_server_listen_and_shutdown() {
function test_server_callback_invocation (line 177) | async fn test_server_callback_invocation() {
FILE: src/tcp/shortcut.rs
function register_server (line 17) | pub async fn register_server(
function call (line 26) | pub async fn call(server_id: u64, data: TcpReq) -> Result<BytesMut> {
function is_local (line 37) | pub async fn is_local(server_id: u64) -> bool {
FILE: src/utils/bindings.rs
type Binding (line 6) | pub struct Binding<T>
function new (line 18) | pub fn new(default: T) -> Binding<T> {
function get (line 24) | pub fn get(&self) -> T {
function set (line 32) | pub fn set(&self, val: T) {
function del (line 37) | pub fn del(&self) {
type RefBinding (line 44) | pub struct RefBinding<T> {
function new (line 48) | pub fn new(default: T) -> RefBinding<T> {
function get (line 53) | pub fn get(&self) -> Arc<T> {
function set (line 56) | pub fn set(&self, val: T) {
function del (line 59) | pub fn del(&self) {
FILE: src/utils/math.rs
function min (line 1) | pub fn min<T>(nums: &Vec<T>) -> Option<T>
function max (line 10) | pub fn max<T>(nums: &Vec<T>) -> Option<T>
function avg_scale (line 19) | pub fn avg_scale(nums: &Vec<u64>) -> Option<u64> {
function max (line 36) | fn max() {
function min (line 43) | fn min() {
FILE: src/utils/serde.rs
function serialize (line 5) | pub fn serialize<T>(obj: &T) -> Vec<u8>
function deserialize (line 16) | pub fn deserialize<'a, T>(data: &'a [u8]) -> Option<T>
function serialize (line 34) | pub fn serialize<T>(obj: &T) -> Vec<u8>
function deserialize (line 45) | pub fn deserialize<'a, T>(data: &'a [u8]) -> Option<T>
function hash (line 64) | pub fn hash<T>(obj: &T) -> u64
FILE: src/utils/time.rs
function get_time (line 5) | pub fn get_time() -> i64 {
function duration_to_ms (line 13) | pub fn duration_to_ms(duration: Duration) -> u64 {
function async_wait (line 18) | pub async fn async_wait(duration: Duration) {
function async_wait_secs (line 22) | pub async fn async_wait_secs() {
FILE: src/vector_clock/mod.rs
type Relation (line 7) | pub enum Relation {
type VectorClock (line 15) | pub struct VectorClock<S: std::hash::Hash + Ord + Eq + Copy> {
method partial_cmp (line 20) | fn partial_cmp(&self, other: &VectorClock<S>) -> Option<Ordering> {
method cmp (line 32) | fn cmp(&self, other: &Self) -> Ordering {
method eq (line 43) | fn eq(&self, other: &VectorClock<S>) -> bool {
function new (line 50) | pub fn new() -> VectorClock<S> {
function from_vec (line 54) | pub fn from_vec(vec: Vec<(S, u64)>) -> Self {
function inc (line 58) | pub fn inc(&mut self, server: S) {
function happened_before (line 70) | pub fn happened_before(&self, clock_b: &VectorClock<S>) -> bool {
function equals (line 108) | pub fn equals(&self, clock_b: &VectorClock<S>) -> bool {
function relation (line 150) | pub fn relation(&self, clock_b: &VectorClock<S>) -> Relation {
function merge_with (line 163) | pub fn merge_with(&mut self, clock_b: &VectorClock<S>) {
function learn_from (line 210) | pub fn learn_from(&mut self, clock_b: &VectorClock<S>) {
type ServerVectorClock (line 254) | pub struct ServerVectorClock {
method new (line 260) | pub fn new(server_address: &String) -> ServerVectorClock {
method inc (line 266) | pub fn inc(&self) -> StandardVectorClock {
method happened_before (line 272) | pub fn happened_before(&self, clock_b: &StandardVectorClock) -> bool {
method equals (line 276) | pub fn equals(&self, clock_b: &StandardVectorClock) -> bool {
method relation (line 280) | pub fn relation(&self, clock_b: &StandardVectorClock) -> Relation {
method merge_with (line 284) | pub fn merge_with(&self, clock_b: &StandardVectorClock) {
method learn_from (line 288) | pub fn learn_from(&self, clock_b: &StandardVectorClock) {
method to_clock (line 292) | pub fn to_clock(&self) -> StandardVectorClock {
type StandardVectorClock (line 298) | pub type StandardVectorClock = VectorClock<u64>;
function general (line 305) | fn general() {
function unaligned_clock_eq (line 355) | fn unaligned_clock_eq() {
function unaligned_clock_rel_disjoint_concurrent (line 367) | fn unaligned_clock_rel_disjoint_concurrent() {
function unaligned_clock_rel_joint_concurrent (line 379) | fn unaligned_clock_rel_joint_concurrent() {
function test_merge_with (line 391) | fn test_merge_with() {
function test_merge_with_empty (line 406) | fn test_merge_with_empty() {
function test_merge_with_into_empty (line 417) | fn test_merge_with_into_empty() {
function test_learn_from (line 427) | fn test_learn_from() {
function test_learn_from_empty (line 438) | fn test_learn_from_empty() {
function test_learn_from_into_empty (line 449) | fn test_learn_from_into_empty() {
function test_server_vector_clock (line 459) | fn test_server_vector_clock() {
function test_server_vector_clock_relation (line 484) | fn test_server_vector_clock_relation() {
function test_inc_new_server (line 499) | fn test_inc_new_server() {
function test_partial_ord (line 510) | fn test_partial_ord() {
function test_ord (line 521) | fn test_ord() {
function test_happened_before_empty_clocks (line 534) | fn test_happened_before_empty_clocks() {
function test_equals_empty_clocks (line 544) | fn test_equals_empty_clocks() {
function test_equals_with_zero_values (line 552) | fn test_equals_with_zero_values() {
FILE: tests/graceful_shutdown_tests.rs
function test_tcp_server_shutdown_releases_port (line 17) | async fn test_tcp_server_shutdown_releases_port() {
function test_rpc_server_shutdown_releases_port (line 88) | async fn test_rpc_server_shutdown_releases_port() {
function test_raft_service_shutdown_stops_tasks (line 131) | async fn test_raft_service_shutdown_stops_tasks() {
function test_full_stack_shutdown_releases_port (line 187) | async fn test_full_stack_shutdown_releases_port() {
function test_rapid_shutdown_restart_cycles (line 255) | async fn test_rapid_shutdown_restart_cycles() {
function test_active_connections_close_on_shutdown (line 285) | async fn test_active_connections_close_on_shutdown() {
function test_shutdown_completes_within_timeout (line 322) | async fn test_shutdown_completes_within_timeout() {
FILE: tests/single_node_recovery_test.rs
function test_single_node_cluster_recovery_becomes_leader (line 12) | async fn test_single_node_cluster_recovery_becomes_leader() {
function test_single_node_multiple_restart_cycles (line 194) | async fn test_single_node_multiple_restart_cycles() {
Condensed preview — 48 files, each showing path, character count, and a content snippet. Download the .json file or copy for the full structured content (668K chars).
[
{
"path": ".gitignore",
"chars": 36,
"preview": "target\nCargo.lock\n.idea/\nbifrost.iml"
},
{
"path": ".travis.yml",
"chars": 33,
"preview": "language: rust\n\nrust:\n - nightly"
},
{
"path": "Cargo.toml",
"chars": 931,
"preview": "[package]\nname = \"bifrost\"\nversion = \"0.1.0\"\nauthors = [\"Hao Shi <shisoftgenius@gmail.com>\"]\nedition = \"2018\"\n\n[lib]\nnam"
},
{
"path": "LICENSE",
"chars": 1064,
"preview": "MIT License\n\nCopyright (c) 2020 Hao Shi\n\nPermission is hereby granted, free of charge, to any person obtaining a copy\nof"
},
{
"path": "MEMBERSHIP_GUIDE.md",
"chars": 8390,
"preview": "# Membership Guide\n\nThis document explains how membership works in Bifrost and the difference between **Raft Cluster Mem"
},
{
"path": "README.md",
"chars": 2775,
"preview": "# bifrost \n[](https://travis-ci.org/Shis"
},
{
"path": "RECOVERY_IMPROVEMENTS.md",
"chars": 7716,
"preview": "# Node Recovery and Temporary Failure Handling Improvements\n\n## Overview\n\nThis document describes the improvements made "
},
{
"path": "SNAPSHOT_GUIDE.md",
"chars": 10799,
"preview": "# Snapshot, Checkpointing, and Recovery Guide\n\n## Overview\n\nBifrost's Raft implementation now includes production-ready "
},
{
"path": "TYPE2_LAZY_LOADING_CONTRACT.md",
"chars": 2356,
"preview": "# Type-2 Lazy Loading Contract\n\nThis document defines the contract when Bifrost Type-1 does not own any Type-2 catalog o"
},
{
"path": "examples/graceful_shutdown.rs",
"chars": 2128,
"preview": "/// Example demonstrating graceful shutdown of Bifrost services\n///\n/// This example shows how to:\n/// 1. Start a Raft s"
},
{
"path": "src/conshash/mod.rs",
"chars": 24792,
"preview": "use futures::prelude::*;\nuse std::collections::{BTreeMap, HashMap};\nuse std::sync::atomic::{AtomicU64, AtomicUsize, Orde"
},
{
"path": "src/conshash/weights.rs",
"chars": 2542,
"preview": "use crate::raft::state_machine::StateMachineCtl;\nuse crate::raft::RaftService;\nuse bifrost_plugins::hash_ident;\nuse futu"
},
{
"path": "src/hasher/Cargo.toml",
"chars": 162,
"preview": "[package]\nname = \"bifrost_hasher\"\nversion = \"0.1.0\"\nauthors = [\"Hao Shi <shisoftgenius@gmail.com>\"]\n\n[lib]\nname = \"bifro"
},
{
"path": "src/hasher/src/lib.rs",
"chars": 534,
"preview": "use std::collections::hash_map::DefaultHasher;\nuse std::hash::Hasher;\n\nextern crate twox_hash;\n\npub fn hash_bytes(bytes:"
},
{
"path": "src/lib.rs",
"chars": 473,
"preview": "#![crate_type = \"lib\"]\n#![feature(proc_macro_hygiene)]\n#![feature(trait_alias)]\n\n#[cfg(disable_shortcut)]\npub static DIS"
},
{
"path": "src/membership/client.rs",
"chars": 5651,
"preview": "use crate::membership::raft::client::SMClient;\nuse crate::membership::DEFAULT_SERVICE_ID;\nuse crate::raft::client::{Raft"
},
{
"path": "src/membership/member.rs",
"chars": 3606,
"preview": "use super::client::{MemberClient, ObserverClient};\nuse super::heartbeat_rpc::*;\nuse super::raft::client::SMClient;\nuse b"
},
{
"path": "src/membership/mod.rs",
"chars": 17883,
"preview": "// Group membership manager regardless actual raft members\n\npub mod client;\npub mod member;\npub mod server;\n\nuse crate::"
},
{
"path": "src/membership/server.rs",
"chars": 35004,
"preview": "use super::heartbeat_rpc::*;\nuse super::raft::*;\nuse super::*;\nuse crate::membership::client::Member as ClientMember;\nus"
},
{
"path": "src/plugins/Cargo.toml",
"chars": 203,
"preview": "[package]\nname = \"bifrost_plugins\"\nversion = \"0.1.0\"\nauthors = [\"Hao Shi <shisoftgenius@gmail.com>\"]\n\n[lib]\nproc-macro ="
},
{
"path": "src/plugins/src/lib.rs",
"chars": 930,
"preview": "extern crate bifrost_hasher;\nextern crate proc_macro;\nextern crate syn;\n\nuse bifrost_hasher::hash_str;\nuse proc_macro::T"
},
{
"path": "src/proc_macro/Cargo.toml",
"chars": 294,
"preview": "[package]\nname = \"bifrost_proc_macro\"\nversion = \"0.1.0\"\nedition = \"2021\"\n\n# See more keys and their definitions at https"
},
{
"path": "src/proc_macro/src/lib.rs",
"chars": 4378,
"preview": "use proc_macro::TokenStream;\nuse quote::quote;\nuse syn::{\n parse::{Parse, ParseBuffer, ParseStream},\n parse_macro_"
},
{
"path": "src/raft/client.rs",
"chars": 49379,
"preview": "use super::*;\nuse crate::raft::state_machine::callback::client::SubscriptionService;\nuse crate::raft::state_machine::cal"
},
{
"path": "src/raft/disk.rs",
"chars": 20648,
"preview": "// Now only offers log persistent\n\nuse crate::raft::{LogEntry, LogsMap, Options, PlaneId, RaftMeta, SnapshotEntity, Stor"
},
{
"path": "src/raft/mod.rs",
"chars": 251150,
"preview": "use self::state_machine::callback::server::Subscriptions;\nuse self::state_machine::callback::SMCallback;\nuse self::state"
},
{
"path": "src/raft/state_machine/callback/client.rs",
"chars": 1862,
"preview": "use super::*;\nuse crate::utils::time::get_time;\nuse async_std::sync::*;\nuse futures::future::BoxFuture;\nuse futures::str"
},
{
"path": "src/raft/state_machine/callback/mod.rs",
"chars": 6994,
"preview": "use bifrost_plugins::hash_ident;\n\nuse crate::raft::PlaneId;\n\npub mod client;\npub mod server;\npub use server::SMCallback;"
},
{
"path": "src/raft/state_machine/callback/server.rs",
"chars": 16627,
"preview": "use super::super::OpType;\nuse super::*;\nuse crate::raft::{PlaneError, PlaneId, RaftMsg, RaftService};\nuse crate::rpc;\nus"
},
{
"path": "src/raft/state_machine/configs.rs",
"chars": 19307,
"preview": "use crate::raft::state_machine::callback::server::Subscriptions;\nuse crate::raft::state_machine::callback::SubKey;\nuse c"
},
{
"path": "src/raft/state_machine/macros.rs",
"chars": 9860,
"preview": "//TODO: Use higher order macro to merge with rpc service! macro when possible to do this in Rust.\n//Current major proble"
},
{
"path": "src/raft/state_machine/master.rs",
"chars": 36023,
"preview": "use self::configs::{Configures, RaftMember, CONFIG_SM_ID};\nuse super::super::*;\nuse super::*;\nuse std::collections::Hash"
},
{
"path": "src/raft/state_machine/mod.rs",
"chars": 1284,
"preview": "use crate::raft::client::RaftPlaneClient;\nuse std::any::Any;\nuse std::sync::Arc;\n\npub enum Storage {\n MEMORY,\n DIS"
},
{
"path": "src/rpc/cluster.rs",
"chars": 4355,
"preview": "use std::{future::Future, sync::Arc};\n\nuse crate::{\n conshash::ConsistentHashing,\n raft::state_machine::master::Ex"
},
{
"path": "src/rpc/mod.rs",
"chars": 23072,
"preview": "#[macro_use]\npub mod proto;\npub mod cluster;\n\nuse crate::{tcp, DISABLE_SHORTCUT};\nuse bifrost_hasher::hash_str;\nuse byte"
},
{
"path": "src/rpc/proto.rs",
"chars": 8419,
"preview": "#[macro_export]\nmacro_rules! dispatch_rpc_service_functions {\n ($s:ty) => {\n use $crate::bytes::BytesMut;\n "
},
{
"path": "src/tcp/client.rs",
"chars": 6784,
"preview": "use std::sync::Arc;\nuse std::time::Duration;\n\nuse crate::tcp::{shortcut, STANDALONE_ADDRESS};\nuse crate::DISABLE_SHORTCU"
},
{
"path": "src/tcp/mod.rs",
"chars": 334,
"preview": "use bifrost_hasher::hash_str;\n\npub mod client;\npub mod server;\npub mod shortcut;\n\npub static STANDALONE_ADDRESS: &'stati"
},
{
"path": "src/tcp/server.rs",
"chars": 7874,
"preview": "use super::STANDALONE_ADDRESS;\nuse crate::tcp::shortcut;\nuse bytes::{Buf, BufMut, BytesMut};\nuse futures::SinkExt;\nuse s"
},
{
"path": "src/tcp/shortcut.rs",
"chars": 1192,
"preview": "use crate::tcp::server::{TcpReq, TcpRes};\nuse async_std::sync::*;\nuse bifrost_hasher::hash_str;\nuse bytes::BytesMut;\nuse"
},
{
"path": "src/utils/bindings.rs",
"chars": 2969,
"preview": "use parking_lot::RwLock;\nuse std::collections::HashMap;\nuse std::sync::Arc;\nuse thread_id;\n\npub struct Binding<T>\nwhere\n"
},
{
"path": "src/utils/math.rs",
"chars": 1277,
"preview": "pub fn min<T>(nums: &Vec<T>) -> Option<T>\nwhere\n T: Ord + Copy,\n{\n nums.iter().fold(None, |min, x| match min {\n "
},
{
"path": "src/utils/mod.rs",
"chars": 74,
"preview": "pub mod time;\n#[macro_use]\npub mod bindings;\npub mod math;\npub mod serde;\n"
},
{
"path": "src/utils/serde.rs",
"chars": 1523,
"preview": "use bifrost_hasher::hash_bytes;\nuse serde;\n\n#[cfg(not(debug_assertions))]\npub fn serialize<T>(obj: &T) -> Vec<u8>\nwhere\n"
},
{
"path": "src/utils/time.rs",
"chars": 655,
"preview": "use std::time::Duration;\nuse std::time::SystemTime;\nuse tokio::time::sleep;\n\npub fn get_time() -> i64 {\n //Get curren"
},
{
"path": "src/vector_clock/mod.rs",
"chars": 17163,
"preview": "use bifrost_hasher::hash_str;\nuse parking_lot::RwLock;\nuse serde::{Deserialize, Serialize};\nuse std::cmp::Ordering;\n\n#[d"
},
{
"path": "tests/graceful_shutdown_tests.rs",
"chars": 11674,
"preview": "/// Tests for graceful shutdown functionality\n///\n/// These tests verify that:\n/// 1. Servers actually shut down when sh"
},
{
"path": "tests/single_node_recovery_test.rs",
"chars": 10231,
"preview": "use bifrost::raft::disk::DiskOptions;\n/// Test for single-node Raft cluster recovery from disk\n///\n/// This test verifie"
}
]
About this extraction
This page contains the full source code of the ShisoftResearch/bifrost GitHub repository, extracted and formatted as plain text for AI agents and large language models (LLMs). The extraction includes 48 files (628.3 KB), approximately 139.1k tokens, and a symbol index with 744 extracted functions, classes, methods, constants, and types. Use this with OpenClaw, Claude, ChatGPT, Cursor, Windsurf, or any other AI tool that accepts text input. You can copy the full output to your clipboard or download it as a .txt file.
Extracted by GitExtract — free GitHub repo to text converter for AI. Built by Nikandr Surkov.