Skip to content

Commit

Permalink
Refactor: use test_harness to build unittest harness
Browse files Browse the repository at this point in the history
`#[test_harness::test(harness = func)]` builds the harness for test
function in a more structural way.
  • Loading branch information
drmingdrmer committed Jul 22, 2024
1 parent 55f4e5a commit 4318f4e
Show file tree
Hide file tree
Showing 84 changed files with 355 additions and 199 deletions.
9 changes: 5 additions & 4 deletions Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ byte-unit = "4.0.12"
bytes = "1.0"
chrono = { version = "0.4" }
clap = { version = "4.1.11", features = ["derive", "env"] }
derive_more = { version="0.99.9" }
derive_more = { version = "0.99.9" }
futures = "0.3"
lazy_static = "1.4.0"
maplit = "1.0.2"
Expand All @@ -31,16 +31,17 @@ proc-macro2 = { version = ">=1.0.0,<1.0.80", features = [] }
quote = "1.0"
rand = "0.8"
semver = "1.0.14"
serde = { version="1.0.114", features=["derive", "rc"]}
serde = { version = "1.0.114", features = ["derive", "rc"] }
serde_json = "1.0.57"
syn = "2.0"
tempfile = { version = "3.4.0" }
test-harness = "0.3.0"
thiserror = "1.0.49"
tokio = { version="1.22", default-features=false, features=["io-util", "macros", "rt", "rt-multi-thread", "sync", "time"] }
tokio = { version = "1.22", default-features = false, features = ["io-util", "macros", "rt", "rt-multi-thread", "sync", "time"] }
tracing = { version = "0.1.40" }
tracing-appender = "0.2.0"
tracing-futures = "0.2.4"
tracing-subscriber = { version = "0.3.3", features=["env-filter"] }
tracing-subscriber = { version = "0.3.3", features = ["env-filter"] }
validit = { version = "0.2.2" }

[workspace]
Expand Down
11 changes: 7 additions & 4 deletions openraft/src/core/sm/worker.rs
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
use anyerror::AnyError;
use tracing_futures::Instrument;

use crate::async_runtime::MpscUnboundedReceiver;
use crate::async_runtime::MpscUnboundedSender;
Expand Down Expand Up @@ -59,6 +60,7 @@ where
state_machine: SM,
log_reader: LR,
resp_tx: MpscUnboundedSenderOf<C, Notification<C>>,
span: tracing::Span,
) -> Handle<C> {
let (cmd_tx, cmd_rx) = C::mpsc_unbounded();

Expand All @@ -69,13 +71,13 @@ where
resp_tx,
};

let join_handle = worker.do_spawn();
let join_handle = worker.do_spawn(span);

Handle { cmd_tx, join_handle }
}

fn do_spawn(mut self) -> JoinHandleOf<C, ()> {
C::spawn(async move {
fn do_spawn(mut self, span: tracing::Span) -> JoinHandleOf<C, ()> {
let fu = async move {
let res = self.worker_loop().await;

if let Err(err) = res {
Expand All @@ -85,7 +87,8 @@ where
command_result: CommandResult { result: Err(err) },
});
}
})
};
C::spawn(fu.instrument(span))
}

#[tracing::instrument(level = "debug", skip_all)]
Expand Down
9 changes: 8 additions & 1 deletion openraft/src/raft/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -271,7 +271,14 @@ where C: RaftTypeConfig

let engine = Engine::new(state, eng_config);

let sm_handle = worker::Worker::spawn(state_machine, log_store.get_log_reader().await, tx_notify.clone());
let sm_span = tracing::span!(parent: &core_span, Level::DEBUG, "sm_worker");

let sm_handle = worker::Worker::spawn(
state_machine,
log_store.get_log_reader().await,
tx_notify.clone(),
sm_span,
);

let core: RaftCore<C, N, LS> = RaftCore {
id,
Expand Down
3 changes: 3 additions & 0 deletions scripts/check.kdl
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,17 @@ layout {
pane {
command "cargo"
args "test" "--lib"
close_on_exit true
}
pane {
command "cargo"
args "test" "--test" "*"
close_on_exit true
}
pane {
command "cargo"
args "clippy" "--no-deps" "--all-targets" "--" "-D" "warnings"
close_on_exit true
}
}
// status-bar
Expand Down
1 change: 1 addition & 0 deletions tests/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ lazy_static = { workspace = true }
maplit = { workspace = true }
pretty_assertions = { workspace = true }
rand = { workspace = true }
test-harness = { workspace = true }
tokio = { workspace = true }
tracing = { workspace = true }
tracing-appender = { workspace = true }
Expand Down
5 changes: 3 additions & 2 deletions tests/tests/append_entries/t10_conflict_with_empty_entries.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ use openraft::LogId;
use openraft::Vote;
use openraft_memstore::ClientRequest;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// Cluster conflict_with_empty_entries test.
Expand All @@ -36,7 +36,8 @@ use crate::fixtures::RaftRouter;
///
/// - send `append_logs` message with conflicting prev_log_index and empty `entries`.
/// - asserts that a response with ConflictOpt set.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn conflict_with_empty_entries() -> Result<()> {
let config = Arc::new(
Config {
Expand Down
5 changes: 3 additions & 2 deletions tests/tests/append_entries/t10_see_higher_vote.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,12 @@ use openraft::Vote;
use openraft_memstore::ClientRequest;
use tokio::time::sleep;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// A leader reverts to follower if a higher vote is seen when append-entries.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn append_sees_higher_vote() -> Result<()> {
let config = Arc::new(
Config {
Expand Down
5 changes: 3 additions & 2 deletions tests/tests/append_entries/t11_append_conflicts.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,14 @@ use openraft::RaftTypeConfig;
use openraft::ServerState;
use openraft::Vote;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// Test append-entries response in every case.
///
/// - bring up a learner and send to it append_entries request. Check the response in every case.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn append_conflicts() -> Result<()> {
let config = Arc::new(
Config {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,15 +13,16 @@ use openraft::Config;
use openraft::LogId;
use openraft::Vote;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// append-entries should update the vote when adding new logs with greater vote.
///
/// - Bring up a learner and send to it append_entries request.
///
/// Check the vote updated.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn append_entries_with_bigger_term() -> Result<()> {
// Setup test dependencies.
let config = Arc::new(
Expand Down
5 changes: 3 additions & 2 deletions tests/tests/append_entries/t11_append_inconsistent_log.rs
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ use openraft::RaftLogReader;
use openraft::ServerState;
use openraft::Vote;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// Too many inconsistent log should not block replication.
Expand All @@ -31,7 +31,8 @@ use crate::fixtures::RaftRouter;
///
/// - Start the cluster and node 2 start to replicate logs.
/// - test the log should be replicated to node 0.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn append_inconsistent_log() -> Result<()> {
// Setup test dependencies.
let config = Arc::new(
Expand Down
5 changes: 3 additions & 2 deletions tests/tests/append_entries/t11_append_updates_membership.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,14 +14,15 @@ use openraft::Membership;
use openraft::ServerState;
use openraft::Vote;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// append-entries should update membership correctly when adding new logs and deleting
/// inconsistent logs.
///
/// - bring up a learner and send to it append_entries request. Check the membership updated.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn append_updates_membership() -> Result<()> {
let config = Arc::new(
Config {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ use anyhow::Result;
use maplit::btreeset;
use openraft::Config;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// Test replication to learner that is not in membership should not block.
Expand All @@ -15,7 +15,8 @@ use crate::fixtures::RaftRouter;
/// - bring on a cluster of 1 voter and 1 learner.
/// - isolate replication to node 1.
/// - client write should not be blocked.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn replication_1_voter_to_isolated_learner() -> Result<()> {
let config = Arc::new(
Config {
Expand Down
5 changes: 3 additions & 2 deletions tests/tests/append_entries/t60_enable_heartbeat.rs
Original file line number Diff line number Diff line change
Expand Up @@ -9,11 +9,12 @@ use openraft::type_config::TypeConfigExt;
use openraft::Config;
use openraft_memstore::TypeConfig;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// Enable heartbeat, heartbeat should be replicated.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn enable_heartbeat() -> Result<()> {
// Setup test dependencies.
let config = Arc::new(
Expand Down
5 changes: 3 additions & 2 deletions tests/tests/append_entries/t61_heartbeat_reject_vote.rs
Original file line number Diff line number Diff line change
Expand Up @@ -11,11 +11,12 @@ use openraft::TokioInstant;
use openraft::Vote;
use tokio::time::sleep;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// If a follower receives heartbeat, it should reject vote request until leader lease expired.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn heartbeat_reject_vote() -> Result<()> {
let config = Arc::new(
Config {
Expand Down
5 changes: 3 additions & 2 deletions tests/tests/append_entries/t61_large_heartbeat.rs
Original file line number Diff line number Diff line change
Expand Up @@ -5,12 +5,13 @@ use anyhow::Result;
use maplit::btreeset;
use openraft::Config;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// Large heartbeat should not block replication.
/// I.e., replication should not be driven by heartbeat.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn large_heartbeat() -> Result<()> {
// Setup test dependencies.
let config = Arc::new(
Expand Down
5 changes: 3 additions & 2 deletions tests/tests/append_entries/t90_issue_216_stale_last_log_id.rs
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ use anyhow::Result;
use maplit::btreeset;
use openraft::Config;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// Ensures the stale value of ReplicationCore.last_log_id won't affect replication.
Expand All @@ -15,7 +15,8 @@ use crate::fixtures::RaftRouter;
/// TODO(xp): `max_applied_log_to_keep` to be 0 makes it very easy to enter snapshot replication and
/// it will keeps replicating every log by snapshot and get timeout.
/// Thus it is disabled until we find another way to test it.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
#[ignore]
async fn stale_last_log_id() -> Result<()> {
// Setup test dependencies.
Expand Down
8 changes: 5 additions & 3 deletions tests/tests/client_api/t10_client_writes.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,13 +12,14 @@ use openraft_memstore::ClientRequest;
use openraft_memstore::IntoMemClientRequest;
use openraft_memstore::TypeConfig;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// - create a stable 3-node cluster.
/// - write a lot of data to it.
/// - assert that the cluster stayed stable and has all of the expected data.
#[async_entry::test(worker_threads = 4, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn client_writes() -> Result<()> {
let config = Arc::new(
Config {
Expand Down Expand Up @@ -67,7 +68,8 @@ async fn client_writes() -> Result<()> {
/// Test Raft::client_write_ff,
///
/// Manually receive the client-write response via the returned `Responder::Receiver`
#[async_entry::test(worker_threads = 4, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn client_write_ff() -> Result<()> {
let config = Arc::new(
Config {
Expand Down
8 changes: 5 additions & 3 deletions tests/tests/client_api/t11_client_reads.rs
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@ use openraft::Config;
use openraft::LogIdOptionExt;
use openraft::RPCTypes;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RPCRequest;
use crate::fixtures::RaftRouter;

Expand All @@ -21,7 +21,8 @@ use crate::fixtures::RaftRouter;
/// - create a stable 3-node cluster.
/// - call the ensure_linearizable interface on the leader, and assert success.
/// - call the ensure_linearizable interface on the followers, and assert failure.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn client_reads() -> Result<()> {
let config = Arc::new(
Config {
Expand Down Expand Up @@ -68,7 +69,8 @@ async fn client_reads() -> Result<()> {
/// - A leader that has not yet committed any log entries returns leader initialization log id(blank
/// log id).
/// - Return the last committed log id if the leader has committed any log entries.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn get_read_log_id() -> Result<()> {
let config = Arc::new(
Config {
Expand Down
5 changes: 3 additions & 2 deletions tests/tests/client_api/t12_trigger_purge_log.rs
Original file line number Diff line number Diff line change
Expand Up @@ -6,11 +6,12 @@ use openraft::testing::log_id;
use openraft::Config;
use openraft::SnapshotPolicy;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

/// Call `Raft::trigger_purged()` to purge logs.
#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn trigger_purge_log() -> anyhow::Result<()> {
let config = Arc::new(
Config {
Expand Down
5 changes: 3 additions & 2 deletions tests/tests/client_api/t13_begin_receiving_snapshot.rs
Original file line number Diff line number Diff line change
Expand Up @@ -4,10 +4,11 @@ use std::time::Duration;
use maplit::btreeset;
use openraft::Config;

use crate::fixtures::init_default_ut_tracing;
use crate::fixtures::ut_harness;
use crate::fixtures::RaftRouter;

#[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")]
#[tracing::instrument]
#[test_harness::test(harness = ut_harness)]
async fn begin_receiving_snapshot() -> anyhow::Result<()> {
let config = Arc::new(
Config {
Expand Down
Loading

0 comments on commit 4318f4e

Please sign in to comment.