openraft/
lib.rs

1#![doc = include_str!("lib_readme.md")]
2#![doc = include_str!("docs/docs.md")]
3#![cfg_attr(feature = "bt", feature(error_generic_member_access))]
4#![cfg_attr(feature = "bench", feature(test))]
5// TODO: `clippy::result-large-err`: StorageError is 136 bytes, try to reduce the size.
6#![allow(clippy::bool_assert_comparison, clippy::type_complexity, clippy::result_large_err)]
7#![deny(unused_qualifications)]
8// TODO: Enable this when doc is complete
9// #![warn(missing_docs)]
10
11macro_rules! func_name {
12    () => {{
13        fn f() {}
14        fn type_name_of<T>(_: T) -> &'static str {
15            std::any::type_name::<T>()
16        }
17        let name = type_name_of(f);
18        let n = &name[..name.len() - 3];
19        let nn = n.replace("::{{closure}}", "");
20        nn
21        // nn.split("::").last().unwrap_or_default()
22    }};
23}
24
25pub extern crate openraft_macros;
26
27mod change_members;
28mod config;
29mod core;
30mod defensive;
31mod display_ext;
32mod membership;
33mod node;
34mod progress;
35mod quorum;
36mod raft_types;
37mod replication;
38mod runtime;
39mod storage_error;
40mod summary;
41mod try_as_ref;
42mod vote;
43
44pub(crate) mod engine;
45pub(crate) mod log_id_range;
46pub(crate) mod proposer;
47pub(crate) mod raft_state;
48pub(crate) mod utime;
49
50pub mod async_runtime;
51#[cfg(feature = "compat")]
52pub mod compat;
53pub mod docs;
54pub mod entry;
55pub mod error;
56pub mod impls;
57pub mod instant;
58pub mod log_id;
59pub mod metrics;
60pub mod network;
61pub mod raft;
62pub mod storage;
63pub mod testing;
64pub mod type_config;
65
66#[cfg(test)]
67mod feature_serde_test;
68
69pub use anyerror;
70pub use anyerror::AnyError;
71pub use openraft_macros::add_async_trait;
72
73pub use crate::async_runtime::AsyncRuntime;
74pub use crate::async_runtime::TokioRuntime;
75pub use crate::change_members::ChangeMembers;
76pub use crate::config::Config;
77pub use crate::config::ConfigError;
78pub use crate::config::SnapshotPolicy;
79pub use crate::core::ServerState;
80pub use crate::entry::Entry;
81pub use crate::entry::EntryPayload;
82pub use crate::instant::Instant;
83pub use crate::instant::TokioInstant;
84pub use crate::log_id::LogId;
85pub use crate::log_id::LogIdOptionExt;
86pub use crate::log_id::LogIndexOptionExt;
87pub use crate::log_id::RaftLogId;
88pub use crate::membership::EffectiveMembership;
89pub use crate::membership::Membership;
90pub use crate::membership::StoredMembership;
91pub use crate::metrics::RaftMetrics;
92pub use crate::network::RPCTypes;
93pub use crate::network::RaftNetwork;
94pub use crate::network::RaftNetworkFactory;
95pub use crate::node::BasicNode;
96pub use crate::node::EmptyNode;
97pub use crate::node::Node;
98pub use crate::node::NodeId;
99pub use crate::raft::Raft;
100pub use crate::raft_state::MembershipState;
101pub use crate::raft_state::RaftState;
102pub use crate::raft_types::SnapshotId;
103pub use crate::raft_types::SnapshotSegmentId;
104pub use crate::storage::LogState;
105pub use crate::storage::RaftLogReader;
106pub use crate::storage::RaftSnapshotBuilder;
107#[cfg(not(feature = "storage-v2"))]
108pub use crate::storage::RaftStorage;
109pub use crate::storage::Snapshot;
110pub use crate::storage::SnapshotMeta;
111pub use crate::storage::StorageHelper;
112pub use crate::storage_error::DefensiveError;
113pub use crate::storage_error::ErrorSubject;
114pub use crate::storage_error::ErrorVerb;
115pub use crate::storage_error::StorageError;
116pub use crate::storage_error::StorageIOError;
117pub use crate::storage_error::ToStorageResult;
118pub use crate::storage_error::Violation;
119pub use crate::summary::MessageSummary;
120pub use crate::try_as_ref::TryAsRef;
121pub use crate::type_config::RaftTypeConfig;
122pub use crate::vote::CommittedLeaderId;
123pub use crate::vote::LeaderId;
124pub use crate::vote::Vote;
125
126#[cfg(feature = "serde")]
127#[doc(hidden)]
128pub trait OptionalSerde: serde::Serialize + for<'a> serde::Deserialize<'a> {}
129
130#[cfg(feature = "serde")]
131impl<T> OptionalSerde for T where T: serde::Serialize + for<'a> serde::Deserialize<'a> {}
132
133#[cfg(not(feature = "serde"))]
134#[doc(hidden)]
135pub trait OptionalSerde {}
136
137#[cfg(not(feature = "serde"))]
138impl<T> OptionalSerde for T {}
139
140#[cfg(feature = "singlethreaded")]
141pub trait OptionalSend {}
142
143#[cfg(feature = "singlethreaded")]
144pub trait OptionalSync {}
145
146#[cfg(feature = "singlethreaded")]
147impl<T: ?Sized> OptionalSend for T {}
148
149#[cfg(feature = "singlethreaded")]
150impl<T: ?Sized> OptionalSync for T {}
151
152#[cfg(not(feature = "singlethreaded"))]
153pub trait OptionalSend: Send {}
154
155#[cfg(not(feature = "singlethreaded"))]
156pub trait OptionalSync: Sync {}
157
158#[cfg(not(feature = "singlethreaded"))]
159impl<T: Send + ?Sized> OptionalSend for T {}
160
161#[cfg(not(feature = "singlethreaded"))]
162impl<T: Sync + ?Sized> OptionalSync for T {}
163
164/// A trait defining application specific data.
165///
166/// The intention of this trait is that applications which are using this crate will be able to
167/// use their own concrete data types throughout their application without having to serialize and
168/// deserialize their data as it goes through Raft. Instead, applications can present their data
169/// models as-is to Raft, Raft will present it to the application's `RaftStorage` impl when ready,
170/// and the application may then deal with the data directly in the storage engine without having
171/// to do a preliminary deserialization.
172///
173/// ## Note
174///
175/// The trait is automatically implemented for all types which satisfy its supertraits.
176pub trait AppData: OptionalSend + OptionalSync + 'static + OptionalSerde {}
177
178impl<T> AppData for T where T: OptionalSend + OptionalSync + 'static + OptionalSerde {}
179
180/// A trait defining application specific response data.
181///
182/// The intention of this trait is that applications which are using this crate will be able to
183/// use their own concrete data types for returning response data from the storage layer when an
184/// entry is applied to the state machine as part of a client request (this is not used during
185/// replication). This allows applications to seamlessly return application specific data from
186/// their storage layer, up through Raft, and back into their application for returning
187/// data to clients.
188///
189/// This type must encapsulate both success and error responses, as application specific logic
190/// related to the success or failure of a client request — application specific validation logic,
191/// enforcing of data constraints, and anything of that nature — are expressly out of the realm of
192/// the Raft consensus protocol.
193///
194/// ## Note
195///
196/// The trait is automatically implemented for all types which satisfy its supertraits.
197pub trait AppDataResponse: OptionalSend + OptionalSync + 'static + OptionalSerde {}
198
199impl<T> AppDataResponse for T where T: OptionalSend + OptionalSync + 'static + OptionalSerde {}