Add client stats to Events (#3116)

* add stats alongside Event over the wire
This commit is contained in:
Romain Malmain 2025-04-01 16:51:52 +02:00 committed by GitHub
parent 184b69be8e
commit 9dff7a438d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
20 changed files with 464 additions and 341 deletions

View File

@ -1,5 +1,5 @@
use libafl::{
events::{Event, EventManagerHook},
events::{Event, EventManagerHook, EventWithStats},
state::Stoppable,
Error,
};
@ -18,9 +18,9 @@ where
&mut self,
state: &mut S,
_client_id: ClientId,
event: &Event<I>,
event: &EventWithStats<I>,
) -> Result<bool, Error> {
if self.exit_on_solution && matches!(event, Event::Objective { .. }) {
if self.exit_on_solution && matches!(event.event(), Event::Objective { .. }) {
// TODO: dump state
state.request_stop();
}

View File

@ -6,7 +6,7 @@ use core::{hash::Hash, marker::PhantomData};
use hashbrown::{HashMap, HashSet};
use libafl_bolts::{
AsIter, Named, current_time,
AsIter, Named,
tuples::{Handle, Handled},
};
use num_traits::ToPrimitive;
@ -15,7 +15,7 @@ use z3::{Config, Context, Optimize, ast::Bool};
use crate::{
Error, HasMetadata, HasScheduler,
corpus::Corpus,
events::{Event, EventFirer, LogSeverity},
events::{Event, EventFirer, EventWithStats, LogSeverity},
executors::{Executor, HasObservers},
inputs::Input,
monitors::stats::{AggregatorOps, UserStats, UserStatsValue},
@ -123,20 +123,17 @@ where
manager.fire(
state,
EventWithStats::with_current_time(
Event::UpdateUserStats {
name: Cow::from("minimisation exec pass"),
value: UserStats::new(UserStatsValue::Ratio(curr, total), AggregatorOps::None),
value: UserStats::new(
UserStatsValue::Ratio(curr, total),
AggregatorOps::None,
),
phantom: PhantomData,
},
)?;
manager.fire(
state,
Event::UpdateExecStats {
time: current_time(),
phantom: PhantomData,
executions,
},
),
)?;
let seed_expr = Bool::fresh_const(&ctx, "seed");

View File

@ -11,7 +11,7 @@ use serde::de::DeserializeOwned;
#[cfg(feature = "llmp_compression")]
use crate::events::COMPRESS_THRESHOLD;
use crate::events::{_LLMP_TAG_TO_MAIN, BrokerEventResult, Event};
use crate::events::{_LLMP_TAG_TO_MAIN, BrokerEventResult, Event, EventWithStats};
/// An LLMP-backed event manager for scalable multi-processed fuzzing
pub struct CentralizedLlmpHook<I> {
@ -47,7 +47,7 @@ where
} else {
&*msg
};
let event: Event<I> = postcard::from_bytes(event_bytes)?;
let event: EventWithStats<I> = postcard::from_bytes(event_bytes)?;
match Self::handle_in_broker(client_id, &event)? {
BrokerEventResult::Forward => Ok(LlmpMsgHookResult::ForwardToClients),
BrokerEventResult::Handled => Ok(LlmpMsgHookResult::Handled),
@ -85,9 +85,9 @@ impl<I> CentralizedLlmpHook<I> {
#[expect(clippy::unnecessary_wraps)]
fn handle_in_broker(
_client_id: ClientId,
event: &Event<I>,
event: &EventWithStats<I>,
) -> Result<BrokerEventResult, Error> {
match &event {
match event.event() {
Event::NewTestcase { .. } | Event::Stop => Ok(BrokerEventResult::Forward),
_ => Ok(BrokerEventResult::Handled),
}

View File

@ -22,7 +22,7 @@ use tokio::{
use crate::{
events::{
Event,
EventWithStats,
centralized::_LLMP_TAG_TO_MAIN,
multi_machine::{MultiMachineMsg, TcpMultiMachineState},
},
@ -128,7 +128,7 @@ where
#[cfg(feature = "llmp_compression")]
fn try_compress(
state_lock: &mut RwLockWriteGuard<TcpMultiMachineState<A>>,
event: &Event<I>,
event: &EventWithStats<I>,
) -> Result<(Flags, Vec<u8>), Error> {
let serialized = postcard::to_allocvec(&event)?;
@ -141,7 +141,7 @@ where
#[cfg(not(feature = "llmp_compression"))]
fn try_compress(
_state_lock: &mut RwLockWriteGuard<TcpMultiMachineState<A>>,
event: &Event<I>,
event: &EventWithStats<I>,
) -> Result<(Flags, Vec<u8>), Error> {
Ok((Flags(0), postcard::to_allocvec(&event)?))
}

View File

@ -30,6 +30,8 @@ pub mod centralized_multi_machine;
#[cfg(all(unix, feature = "multi_machine"))]
pub use centralized_multi_machine::*;
use super::EventWithStats;
/// An LLMP-backed event hook for scalable multi-processed fuzzing
#[derive(Debug)]
pub struct StdLlmpEventHook<I, MT> {
@ -71,7 +73,7 @@ where
} else {
&*msg
};
let event: Event<I> = postcard::from_bytes(event_bytes)?;
let event: EventWithStats<I> = postcard::from_bytes(event_bytes)?;
match Self::handle_in_broker(
monitor,
&mut self.client_stats_manager,
@ -117,8 +119,16 @@ where
monitor: &mut MT,
client_stats_manager: &mut ClientStatsManager,
client_id: ClientId,
event: &Event<I>,
event: &EventWithStats<I>,
) -> Result<BrokerEventResult, Error> {
let stats = event.stats();
client_stats_manager.client_stats_insert(ClientId(0));
client_stats_manager.update_client_stats_for(ClientId(0), |client_stat| {
client_stat.update_executions(stats.executions, stats.time);
});
let event = event.event();
match &event {
Event::NewTestcase {
corpus_size,
@ -138,19 +148,7 @@ where
monitor.display(client_stats_manager, event.name(), id);
Ok(BrokerEventResult::Forward)
}
Event::UpdateExecStats {
time,
executions,
phantom: _,
} => {
// TODO: The monitor buffer should be added on client add.
client_stats_manager.client_stats_insert(client_id);
client_stats_manager.update_client_stats_for(client_id, |client_stat| {
client_stat.update_executions(*executions, *time);
});
monitor.display(client_stats_manager, event.name(), client_id);
Ok(BrokerEventResult::Handled)
}
Event::Heartbeat => Ok(BrokerEventResult::Handled),
Event::UpdateUserStats { name, value, .. } => {
client_stats_manager.client_stats_insert(client_id);
client_stats_manager.update_client_stats_for(client_id, |client_stat| {
@ -162,8 +160,6 @@ where
}
#[cfg(feature = "introspection")]
Event::UpdatePerfMonitor {
time,
executions,
introspection_stats,
phantom: _,
} => {
@ -172,8 +168,6 @@ where
// Get the client for the staterestorer ID
client_stats_manager.client_stats_insert(client_id);
client_stats_manager.update_client_stats_for(client_id, |client_stat| {
// Update the normal monitor for this client
client_stat.update_executions(*executions, *time);
// Update the performance monitor for this client
client_stat.update_introspection_stats((**introspection_stats).clone());
});

View File

@ -22,7 +22,7 @@ use libafl_bolts::{
llmp::{LLMP_FLAG_COMPRESSED, LLMP_FLAG_INITIALIZED},
};
use super::AwaitRestartSafe;
use super::{AwaitRestartSafe, EventWithStats};
#[cfg(feature = "llmp_compression")]
use crate::events::llmp::COMPRESS_THRESHOLD;
use crate::{
@ -166,18 +166,18 @@ where
}
#[expect(clippy::match_same_arms)]
fn fire(&mut self, state: &mut S, mut event: Event<I>) -> Result<(), Error> {
fn fire(&mut self, state: &mut S, mut event: EventWithStats<I>) -> Result<(), Error> {
if !self.is_main {
// secondary node
let mut is_tc = false;
// Forward to main only if new tc, heartbeat, or optionally, a new objective
let should_be_forwarded = match &mut event {
let should_be_forwarded = match event.event_mut() {
Event::NewTestcase { forward_id, .. } => {
*forward_id = Some(ClientId(self.inner.mgr_id().0 as u32));
is_tc = true;
true
}
Event::UpdateExecStats { .. } => true, // send UpdateExecStats but this guy won't be handled. the only purpose is to keep this client alive else the broker thinks it is dead and will dc it
Event::Heartbeat => true, // the only purpose is to keep this client alive else the broker thinks it is dead and will dc it
Event::Objective { .. } => true,
Event::Stop => true,
_ => false,
@ -201,7 +201,10 @@ where
state: &mut S,
severity_level: LogSeverity,
message: String,
) -> Result<(), Error> {
) -> Result<(), Error>
where
S: HasExecutions,
{
self.inner.log(state, severity_level, message)
}
@ -261,7 +264,7 @@ where
SHM: ShMem,
SP: ShMemProvider<ShMem = SHM>,
{
fn try_receive(&mut self, state: &mut S) -> Result<Option<(Event<I>, bool)>, Error> {
fn try_receive(&mut self, state: &mut S) -> Result<Option<(EventWithStats<I>, bool)>, Error> {
if self.is_main {
// main node
self.receive_from_secondary(state)
@ -272,7 +275,7 @@ where
}
}
fn on_interesting(&mut self, state: &mut S, event: Event<I>) -> Result<(), Error> {
fn on_interesting(&mut self, state: &mut S, event: EventWithStats<I>) -> Result<(), Error> {
self.inner.fire(state, event)
}
}
@ -343,7 +346,7 @@ where
SP: ShMemProvider<ShMem = SHM>,
{
#[cfg(feature = "llmp_compression")]
fn forward_to_main(&mut self, event: &Event<I>) -> Result<(), Error> {
fn forward_to_main(&mut self, event: &EventWithStats<I>) -> Result<(), Error> {
let serialized = postcard::to_allocvec(event)?;
let flags = LLMP_FLAG_INITIALIZED;
@ -363,13 +366,16 @@ where
}
#[cfg(not(feature = "llmp_compression"))]
fn forward_to_main(&mut self, event: &Event<I>) -> Result<(), Error> {
fn forward_to_main(&mut self, event: &EventWithStats<I>) -> Result<(), Error> {
let serialized = postcard::to_allocvec(event)?;
self.client.send_buf(_LLMP_TAG_TO_MAIN, &serialized)?;
Ok(())
}
fn receive_from_secondary(&mut self, state: &mut S) -> Result<Option<(Event<I>, bool)>, Error> {
fn receive_from_secondary(
&mut self,
state: &mut S,
) -> Result<Option<(EventWithStats<I>, bool)>, Error> {
// TODO: Get around local event copy by moving handle_in_client
let self_id = self.client.sender().id();
while let Some((client_id, tag, _flags, msg)) = self.client.recv_buf_with_flags()? {
@ -392,15 +398,18 @@ where
} else {
msg
};
let event: Event<I> = postcard::from_bytes(event_bytes)?;
log::debug!("Processor received message {}", event.name_detailed());
let event: EventWithStats<I> = postcard::from_bytes(event_bytes)?;
log::debug!(
"Processor received message {}",
event.event().name_detailed()
);
let event_name = event.name_detailed();
let event_name = event.event().name_detailed();
match event {
match event.event() {
Event::NewTestcase {
client_config,
ref observers_buf,
observers_buf,
forward_id,
..
} => {
@ -425,7 +434,7 @@ where
_ => {
return Err(Error::illegal_state(format!(
"Received illegal message that message should not have arrived: {:?}.",
event.name()
event.event().name()
)));
}
}

View File

@ -4,7 +4,7 @@
//! other clients
use libafl_bolts::ClientId;
use crate::{Error, events::Event};
use crate::{Error, events::EventWithStats};
/// The `broker_hooks` that are run before and after the event manager calls `try_receive`
pub trait EventManagerHook<I, S> {
@ -14,7 +14,7 @@ pub trait EventManagerHook<I, S> {
&mut self,
state: &mut S,
client_id: ClientId,
event: &Event<I>,
event: &EventWithStats<I>,
) -> Result<bool, Error>;
}
@ -25,7 +25,7 @@ pub trait EventManagerHooksTuple<I, S> {
&mut self,
state: &mut S,
client_id: ClientId,
event: &Event<I>,
event: &EventWithStats<I>,
) -> Result<bool, Error>;
}
@ -35,7 +35,7 @@ impl<I, S> EventManagerHooksTuple<I, S> for () {
&mut self,
_state: &mut S,
_client_id: ClientId,
_event: &Event<I>,
_event: &EventWithStats<I>,
) -> Result<bool, Error> {
Ok(true)
}
@ -51,7 +51,7 @@ where
&mut self,
state: &mut S,
client_id: ClientId,
event: &Event<I>,
event: &EventWithStats<I>,
) -> Result<bool, Error> {
let first = self.0.pre_receive(state, client_id, event)?;
let second = self.1.pre_receive_all(state, client_id, event)?;

View File

@ -16,7 +16,7 @@ use serde::{Serialize, de::DeserializeOwned};
use crate::{
Error,
events::{Event, EventFirer},
events::{Event, EventFirer, EventWithStats},
fuzzer::EvaluatorObservers,
inputs::{Input, InputConverter, NopInput, NopInputConverter},
state::{HasCurrentTestcase, HasSolutions, NopState},
@ -385,20 +385,20 @@ where
}
#[cfg(feature = "llmp_compression")]
fn fire(&mut self, _state: &mut S, event: Event<I>) -> Result<(), Error> {
fn fire(&mut self, _state: &mut S, event: EventWithStats<I>) -> Result<(), Error> {
if self.converter.is_none() {
return Ok(());
}
// Filter out non interestign events and convert `NewTestcase`
let converted_event = match event {
let converted_event = EventWithStats::new(
match event.event {
Event::NewTestcase {
input,
client_config,
exit_kind,
corpus_size,
observers_buf,
time,
forward_id,
#[cfg(all(unix, feature = "std", feature = "multi_machine"))]
node_id,
@ -408,7 +408,6 @@ where
exit_kind,
corpus_size,
observers_buf,
time,
forward_id,
#[cfg(all(unix, feature = "std", feature = "multi_machine"))]
node_id,
@ -416,7 +415,10 @@ where
_ => {
return Ok(());
}
};
},
event.stats,
);
let serialized = postcard::to_allocvec(&converted_event)?;
let flags = LLMP_FLAG_INITIALIZED;
@ -437,20 +439,20 @@ where
}
#[cfg(not(feature = "llmp_compression"))]
fn fire(&mut self, _state: &mut S, event: Event<I>) -> Result<(), Error> {
fn fire(&mut self, _state: &mut S, event: EventWithStats<I>) -> Result<(), Error> {
if self.converter.is_none() {
return Ok(());
}
// Filter out non interestign events and convert `NewTestcase`
let converted_event = match event {
let converted_event = EventWithStats::new(
match event.event {
Event::NewTestcase {
input,
client_config,
exit_kind,
corpus_size,
observers_buf,
time,
forward_id,
#[cfg(all(unix, feature = "std", feature = "multi_machine"))]
node_id,
@ -460,7 +462,6 @@ where
exit_kind,
corpus_size,
observers_buf,
time,
forward_id,
#[cfg(all(unix, feature = "std", feature = "multi_machine"))]
node_id,
@ -468,7 +469,10 @@ where
_ => {
return Ok(());
}
};
},
event.stats,
);
let serialized = postcard::to_allocvec(&converted_event)?;
self.llmp.send_buf(LLMP_TAG_EVENT_TO_BOTH, &serialized)?;
Ok(())

View File

@ -55,9 +55,9 @@ use crate::{
common::HasMetadata,
events::{
_LLMP_TAG_EVENT_TO_BROKER, AwaitRestartSafe, Event, EventConfig, EventFirer,
EventManagerHooksTuple, EventManagerId, EventReceiver, EventRestarter, HasEventManagerId,
LLMP_TAG_EVENT_TO_BOTH, LlmpShouldSaveState, ProgressReporter, SendExiting,
StdLlmpEventHook, launcher::ClientDescription, std_maybe_report_progress,
EventManagerHooksTuple, EventManagerId, EventReceiver, EventRestarter, EventWithStats,
HasEventManagerId, LLMP_TAG_EVENT_TO_BOTH, LlmpShouldSaveState, ProgressReporter,
SendExiting, StdLlmpEventHook, launcher::ClientDescription, std_maybe_report_progress,
std_report_progress,
},
inputs::Input,
@ -121,7 +121,7 @@ where
SHM: ShMem,
SP: ShMemProvider<ShMem = SHM>,
{
fn fire(&mut self, _state: &mut S, event: Event<I>) -> Result<(), Error> {
fn fire(&mut self, _state: &mut S, event: EventWithStats<I>) -> Result<(), Error> {
// Check if we are going to crash in the event, in which case we store our current state for the next runner
#[cfg(feature = "llmp_compression")]
let flags = LLMP_FLAG_INITIALIZED;
@ -255,7 +255,7 @@ where
SHM: ShMem,
SP: ShMemProvider<ShMem = SHM>,
{
fn try_receive(&mut self, state: &mut S) -> Result<Option<(Event<I>, bool)>, Error> {
fn try_receive(&mut self, state: &mut S) -> Result<Option<(EventWithStats<I>, bool)>, Error> {
// TODO: Get around local event copy by moving handle_in_client
let self_id = self.llmp.sender().id();
while let Some((client_id, tag, flags, msg)) = self.llmp.recv_buf_with_flags()? {
@ -280,24 +280,31 @@ where
msg
};
let event: Event<I> = postcard::from_bytes(event_bytes)?;
log::debug!("Received event in normal llmp {}", event.name_detailed());
let event: EventWithStats<I> = postcard::from_bytes(event_bytes)?;
log::debug!(
"Received event in normal llmp {}",
event.event().name_detailed()
);
// If the message comes from another machine, do not
// consider other events than new testcase.
if !event.is_new_testcase() && (flags & LLMP_FLAG_FROM_MM == LLMP_FLAG_FROM_MM) {
if !event.event().is_new_testcase() && (flags & LLMP_FLAG_FROM_MM == LLMP_FLAG_FROM_MM)
{
continue;
}
log::trace!("Got event in client: {} from {client_id:?}", event.name());
log::trace!(
"Got event in client: {} from {client_id:?}",
event.event().name()
);
if !self.hooks.pre_receive_all(state, client_id, &event)? {
continue;
}
let evt_name = event.name_detailed();
match event {
let evt_name = event.event().name_detailed();
match event.event() {
Event::NewTestcase {
client_config,
ref observers_buf,
observers_buf,
#[cfg(feature = "std")]
forward_id,
..
@ -326,7 +333,7 @@ where
_ => {
return Err(Error::unknown(format!(
"Received illegal message that message should not have arrived: {:?}.",
event.name()
event.event().name()
)));
}
}
@ -334,7 +341,11 @@ where
Ok(None)
}
fn on_interesting(&mut self, _state: &mut S, _event_vec: Event<I>) -> Result<(), Error> {
fn on_interesting(
&mut self,
_state: &mut S,
_event_vec: EventWithStats<I>,
) -> Result<(), Error> {
Ok(())
}
}

View File

@ -238,6 +238,64 @@ where
}
*/
/// Basic statistics
#[derive(Serialize, Deserialize, Clone, Debug)]
pub struct ExecStats {
/// The time of generation of the [`Event`]
time: Duration,
/// The executions of this client
executions: u64,
}
impl ExecStats {
/// Create an new [`ExecStats`].
#[must_use]
pub fn new(time: Duration, executions: u64) -> Self {
Self { time, executions }
}
}
/// Event with associated stats
#[derive(Serialize, Deserialize, Clone, Debug)]
pub struct EventWithStats<I> {
/// The event
event: Event<I>,
/// Statistics on new event
stats: ExecStats,
}
impl<I> EventWithStats<I> {
/// Create a new [`EventWithStats`].
pub fn new(event: Event<I>, stats: ExecStats) -> Self {
Self { event, stats }
}
/// Create a new [`EventWithStats`], with the current time.
pub fn with_current_time(event: Event<I>, executions: u64) -> Self {
let time = current_time();
Self {
event,
stats: ExecStats { time, executions },
}
}
/// Get the inner ref to the [`Event`] in [`EventWithStats`].
pub fn event(&self) -> &Event<I> {
&self.event
}
/// Get the inner mutable ref to the [`Event`] in [`EventWithStats`].
pub fn event_mut(&mut self) -> &mut Event<I> {
&mut self.event
}
/// Get the inner ref to the [`ExecStats`] in [`EventWithStats`].
pub fn stats(&self) -> &ExecStats {
&self.stats
}
}
// TODO remove forward_id as not anymore needed for centralized
/// Events sent around in the library
#[derive(Serialize, Deserialize, Clone, Debug)]
@ -256,23 +314,14 @@ pub enum Event<I> {
corpus_size: usize,
/// The client config for this observers/testcase combination
client_config: EventConfig,
/// The time of generation of the event
time: Duration,
/// The original sender if, if forwarded
forward_id: Option<libafl_bolts::ClientId>,
/// The (multi-machine) node from which the tc is from, if any
#[cfg(all(unix, feature = "std", feature = "multi_machine"))]
node_id: Option<NodeId>,
},
/// New stats event to monitor.
UpdateExecStats {
/// The time of generation of the [`Event`]
time: Duration,
/// The executions of this client
executions: u64,
/// [`PhantomData`]
phantom: PhantomData<I>,
},
/// A hearbeat, to notice a fuzzer is still alive.
Heartbeat,
/// New user stats event to monitor.
UpdateUserStats {
/// Custom user monitor name
@ -285,10 +334,6 @@ pub enum Event<I> {
/// New monitor with performance monitor.
#[cfg(feature = "introspection")]
UpdatePerfMonitor {
/// The time of generation of the event
time: Duration,
/// The executions of this client
executions: u64,
/// Current performance statistics
introspection_stats: Box<ClientPerfStats>,
@ -301,8 +346,6 @@ pub enum Event<I> {
input: Option<I>,
/// Objective corpus size
objective_size: usize,
/// The time when this event was created
time: Duration,
},
/// Write a new log
Log {
@ -327,7 +370,7 @@ impl<I> Event<I> {
pub fn name(&self) -> &str {
match self {
Event::NewTestcase { .. } => "Testcase",
Event::UpdateExecStats { .. } => "Client Heartbeat",
Event::Heartbeat => "Client Heartbeat",
Event::UpdateUserStats { .. } => "UserStats",
#[cfg(feature = "introspection")]
Event::UpdatePerfMonitor { .. } => "PerfMonitor",
@ -349,7 +392,7 @@ impl<I> Event<I> {
Event::NewTestcase { input, .. } => {
Cow::Owned(format!("Testcase {}", input.generate_name(None)))
}
Event::UpdateExecStats { .. } => Cow::Borrowed("Client Heartbeat"),
Event::Heartbeat => Cow::Borrowed("Client Heartbeat"),
Event::UpdateUserStats { .. } => Cow::Borrowed("UserStats"),
#[cfg(feature = "introspection")]
Event::UpdatePerfMonitor { .. } => Cow::Borrowed("PerfMonitor"),
@ -378,7 +421,7 @@ pub trait EventFirer<I, S> {
/// (for example for each [`Input`], on multiple cores)
/// the [`llmp`] shared map may fill up and the client will eventually OOM or [`panic`].
/// This should not happen for a normal use-case.
fn fire(&mut self, state: &mut S, event: Event<I>) -> Result<(), Error>;
fn fire(&mut self, state: &mut S, event: EventWithStats<I>) -> Result<(), Error>;
/// Send off an [`Event::Log`] event to the broker.
/// This is a shortcut for [`EventFirer::fire`] with [`Event::Log`] as argument.
@ -387,14 +430,28 @@ pub trait EventFirer<I, S> {
state: &mut S,
severity_level: LogSeverity,
message: String,
) -> Result<(), Error> {
) -> Result<(), Error>
where
S: HasExecutions,
{
let executions = *state.executions();
let cur = current_time();
let stats = ExecStats {
executions,
time: cur,
};
self.fire(
state,
Event::Log {
EventWithStats {
event: Event::Log {
severity_level,
message,
phantom: PhantomData,
},
stats,
},
)
}
@ -442,14 +499,18 @@ where
let executions = *state.executions();
let cur = current_time();
let stats = ExecStats {
executions,
time: cur,
};
// Default no introspection implmentation
#[cfg(not(feature = "introspection"))]
reporter.fire(
state,
Event::UpdateExecStats {
executions,
time: cur,
phantom: PhantomData,
EventWithStats {
event: Event::Heartbeat,
stats,
},
)?;
@ -464,12 +525,13 @@ where
// costly as `ClientPerfStats` impls `Copy` since it only contains `u64`s
reporter.fire(
state,
EventWithStats::new(
Event::UpdatePerfMonitor {
executions,
time: cur,
introspection_stats: Box::new(state.introspection_stats().clone()),
phantom: PhantomData,
},
stats,
),
)?;
}
@ -538,11 +600,11 @@ pub trait AwaitRestartSafe {
pub trait EventReceiver<I, S> {
/// Lookup for incoming events and process them.
/// Return the event, if any, that needs to be evaluated
fn try_receive(&mut self, state: &mut S) -> Result<Option<(Event<I>, bool)>, Error>;
fn try_receive(&mut self, state: &mut S) -> Result<Option<(EventWithStats<I>, bool)>, Error>;
/// Run the post processing routine after the fuzzer deemed this event as interesting
/// For example, in centralized manager you wanna send this an event.
fn on_interesting(&mut self, state: &mut S, event: Event<I>) -> Result<(), Error>;
fn on_interesting(&mut self, state: &mut S, event: EventWithStats<I>) -> Result<(), Error>;
}
/// The id of this `EventManager`.
/// For multi processed `EventManagers`,
@ -570,7 +632,7 @@ impl<I, S> EventFirer<I, S> for NopEventManager {
true
}
fn fire(&mut self, _state: &mut S, _event: Event<I>) -> Result<(), Error> {
fn fire(&mut self, _state: &mut S, _event: EventWithStats<I>) -> Result<(), Error> {
Ok(())
}
}
@ -602,11 +664,15 @@ impl AwaitRestartSafe for NopEventManager {
}
impl<I, S> EventReceiver<I, S> for NopEventManager {
fn try_receive(&mut self, _state: &mut S) -> Result<Option<(Event<I>, bool)>, Error> {
fn try_receive(&mut self, _state: &mut S) -> Result<Option<(EventWithStats<I>, bool)>, Error> {
Ok(None)
}
fn on_interesting(&mut self, _state: &mut S, _event_vec: Event<I>) -> Result<(), Error> {
fn on_interesting(
&mut self,
_state: &mut S,
_event_vec: EventWithStats<I>,
) -> Result<(), Error> {
Ok(())
}
}
@ -659,7 +725,7 @@ where
}
#[inline]
fn fire(&mut self, state: &mut S, event: Event<I>) -> Result<(), Error> {
fn fire(&mut self, state: &mut S, event: EventWithStats<I>) -> Result<(), Error> {
self.inner.fire(state, event)
}
@ -669,7 +735,10 @@ where
state: &mut S,
severity_level: LogSeverity,
message: String,
) -> Result<(), Error> {
) -> Result<(), Error>
where
S: HasExecutions,
{
self.inner.log(state, severity_level, message)
}
@ -718,10 +787,14 @@ where
EM: EventReceiver<I, S>,
{
#[inline]
fn try_receive(&mut self, state: &mut S) -> Result<Option<(Event<I>, bool)>, Error> {
fn try_receive(&mut self, state: &mut S) -> Result<Option<(EventWithStats<I>, bool)>, Error> {
self.inner.try_receive(state)
}
fn on_interesting(&mut self, _state: &mut S, _event_vec: Event<I>) -> Result<(), Error> {
fn on_interesting(
&mut self,
_state: &mut S,
_event_vec: EventWithStats<I>,
) -> Result<(), Error> {
Ok(())
}
}
@ -758,7 +831,7 @@ where
#[cfg(test)]
mod tests {
use libafl_bolts::{Named, current_time, tuples::tuple_list};
use libafl_bolts::{Named, tuples::tuple_list};
use tuple_list::tuple_list_type;
use crate::{
@ -787,7 +860,6 @@ mod tests {
exit_kind: ExitKind::Ok,
corpus_size: 123,
client_config: EventConfig::AlwaysUnique,
time: current_time(),
forward_id: None,
#[cfg(all(unix, feature = "std", feature = "multi_machine"))]
node_id: None,

View File

@ -22,7 +22,7 @@ use tokio::{
use typed_builder::TypedBuilder;
use crate::{
events::{Event, TcpMultiMachineLlmpReceiverHook, TcpMultiMachineLlmpSenderHook},
events::{EventWithStats, TcpMultiMachineLlmpReceiverHook, TcpMultiMachineLlmpSenderHook},
inputs::{Input, NopInput},
};
@ -50,7 +50,7 @@ pub enum MultiMachineMsg<'a, I> {
LlmpMsg(OwnedRef<'a, [u8]>),
/// A `LibAFL` Event (already deserialized)
Event(OwnedRef<'a, Event<I>>),
Event(OwnedRef<'a, EventWithStats<I>>),
}
/// We do not use raw pointers, so no problem with thead-safety
@ -65,7 +65,7 @@ impl<'a, I> MultiMachineMsg<'a, I> {
/// `OwnedRef` should **never** be a raw pointer for thread-safety reasons.
/// We check this for debug builds, but not for release.
#[must_use]
pub unsafe fn event(event: OwnedRef<'a, Event<I>>) -> Self {
pub unsafe fn event(event: OwnedRef<'a, EventWithStats<I>>) -> Self {
debug_assert!(!event.is_raw());
MultiMachineMsg::Event(event)

View File

@ -23,7 +23,7 @@ use serde::Serialize;
#[cfg(feature = "std")]
use serde::de::DeserializeOwned;
use super::{AwaitRestartSafe, ProgressReporter, std_on_restart};
use super::{AwaitRestartSafe, EventWithStats, ProgressReporter, std_on_restart};
#[cfg(all(unix, feature = "std", not(miri)))]
use crate::events::EVENTMGR_SIGHANDLER_STATE;
use crate::{
@ -54,7 +54,7 @@ pub struct SimpleEventManager<I, MT, S> {
/// The monitor
monitor: MT,
/// The events that happened since the last `handle_in_broker`
events: Vec<Event<I>>,
events: Vec<EventWithStats<I>>,
phantom: PhantomData<S>,
client_stats_manager: ClientStatsManager,
}
@ -83,7 +83,7 @@ where
true
}
fn fire(&mut self, _state: &mut S, event: Event<I>) -> Result<(), Error> {
fn fire(&mut self, _state: &mut S, event: EventWithStats<I>) -> Result<(), Error> {
match Self::handle_in_broker(&mut self.monitor, &mut self.client_stats_manager, &event)? {
BrokerEventResult::Forward => self.events.push(event),
BrokerEventResult::Handled => (),
@ -121,9 +121,9 @@ where
MT: Monitor,
S: Stoppable,
{
fn try_receive(&mut self, state: &mut S) -> Result<Option<(Event<I>, bool)>, Error> {
fn try_receive(&mut self, state: &mut S) -> Result<Option<(EventWithStats<I>, bool)>, Error> {
while let Some(event) = self.events.pop() {
match event {
match event.event() {
Event::Stop => {
state.request_stop();
}
@ -136,7 +136,11 @@ where
}
Ok(None)
}
fn on_interesting(&mut self, _state: &mut S, _event_vec: Event<I>) -> Result<(), Error> {
fn on_interesting(
&mut self,
_state: &mut S,
_event_vec: EventWithStats<I>,
) -> Result<(), Error> {
Ok(())
}
}
@ -200,8 +204,16 @@ where
fn handle_in_broker(
monitor: &mut MT,
client_stats_manager: &mut ClientStatsManager,
event: &Event<I>,
event: &EventWithStats<I>,
) -> Result<BrokerEventResult, Error> {
let stats = event.stats();
client_stats_manager.client_stats_insert(ClientId(0));
client_stats_manager.update_client_stats_for(ClientId(0), |client_stat| {
client_stat.update_executions(stats.executions, stats.time);
});
let event = event.event();
match event {
Event::NewTestcase { corpus_size, .. } => {
client_stats_manager.client_stats_insert(ClientId(0));
@ -211,15 +223,7 @@ where
monitor.display(client_stats_manager, event.name(), ClientId(0));
Ok(BrokerEventResult::Handled)
}
Event::UpdateExecStats {
time, executions, ..
} => {
// TODO: The monitor buffer should be added on client add.
client_stats_manager.client_stats_insert(ClientId(0));
client_stats_manager.update_client_stats_for(ClientId(0), |client_stat| {
client_stat.update_executions(*executions, *time);
});
Event::Heartbeat => {
monitor.display(client_stats_manager, event.name(), ClientId(0));
Ok(BrokerEventResult::Handled)
}
@ -234,15 +238,11 @@ where
}
#[cfg(feature = "introspection")]
Event::UpdatePerfMonitor {
time,
executions,
introspection_stats,
..
} => {
// TODO: The monitor buffer should be added on client add.
client_stats_manager.client_stats_insert(ClientId(0));
client_stats_manager.update_client_stats_for(ClientId(0), |client_stat| {
client_stat.update_executions(*executions, *time);
client_stat.update_introspection_stats((**introspection_stats).clone());
});
monitor.display(client_stats_manager, event.name(), ClientId(0));
@ -295,7 +295,7 @@ where
true
}
fn fire(&mut self, _state: &mut S, event: Event<I>) -> Result<(), Error> {
fn fire(&mut self, _state: &mut S, event: EventWithStats<I>) -> Result<(), Error> {
self.inner.fire(_state, event)
}
}
@ -354,11 +354,15 @@ where
SHM: ShMem,
SP: ShMemProvider<ShMem = SHM>,
{
fn try_receive(&mut self, state: &mut S) -> Result<Option<(Event<I>, bool)>, Error> {
fn try_receive(&mut self, state: &mut S) -> Result<Option<(EventWithStats<I>, bool)>, Error> {
self.inner.try_receive(state)
}
fn on_interesting(&mut self, _state: &mut S, _event_vec: Event<I>) -> Result<(), Error> {
fn on_interesting(
&mut self,
_state: &mut S,
_event_vec: EventWithStats<I>,
) -> Result<(), Error> {
Ok(())
}
}

View File

@ -45,7 +45,8 @@ use crate::{
Error, HasMetadata,
events::{
BrokerEventResult, Event, EventConfig, EventFirer, EventManagerHooksTuple, EventManagerId,
EventReceiver, EventRestarter, HasEventManagerId, ProgressReporter, std_on_restart,
EventReceiver, EventRestarter, EventWithStats, HasEventManagerId, ProgressReporter,
std_on_restart,
},
inputs::Input,
monitors::{Monitor, stats::ClientStatsManager},
@ -292,7 +293,7 @@ where
#[cfg(feature = "tcp_compression")]
let event_bytes = &GzipCompressor::new().decompress(event_bytes)?;
let event: Event<I> = postcard::from_bytes(event_bytes)?;
let event: EventWithStats<I> = postcard::from_bytes(event_bytes)?;
match Self::handle_in_broker(
&mut self.monitor,
&mut self.client_stats_manager,
@ -321,9 +322,17 @@ where
monitor: &mut MT,
client_stats_manager: &mut ClientStatsManager,
client_id: ClientId,
event: &Event<I>,
event: &EventWithStats<I>,
) -> Result<BrokerEventResult, Error> {
match &event {
let stats = event.stats();
client_stats_manager.client_stats_insert(ClientId(0));
client_stats_manager.update_client_stats_for(ClientId(0), |client_stat| {
client_stat.update_executions(stats.executions, stats.time);
});
let event = event.event();
match event {
Event::NewTestcase {
corpus_size,
forward_id,
@ -341,16 +350,7 @@ where
monitor.display(client_stats_manager, event.name(), id);
Ok(BrokerEventResult::Forward)
}
Event::UpdateExecStats {
time,
executions,
phantom: _,
} => {
// TODO: The monitor buffer should be added on client add.
client_stats_manager.client_stats_insert(client_id);
client_stats_manager.update_client_stats_for(client_id, |client| {
client.update_executions(*executions, *time);
});
Event::Heartbeat => {
monitor.display(client_stats_manager, event.name(), client_id);
Ok(BrokerEventResult::Handled)
}
@ -369,8 +369,6 @@ where
}
#[cfg(feature = "introspection")]
Event::UpdatePerfMonitor {
time,
executions,
introspection_stats,
phantom: _,
} => {
@ -379,8 +377,6 @@ where
// Get the client for the staterestorer ID
client_stats_manager.client_stats_insert(client_id);
client_stats_manager.update_client_stats_for(client_id, |client| {
// Update the normal monitor for this client
client.update_executions(*executions, *time);
// Update the performance monitor for this client
client.update_introspection_stats((**introspection_stats).clone());
});
@ -608,7 +604,7 @@ where
}
}
fn fire(&mut self, _state: &mut S, event: Event<I>) -> Result<(), Error> {
fn fire(&mut self, _state: &mut S, event: EventWithStats<I>) -> Result<(), Error> {
let serialized = postcard::to_allocvec(&event)?;
#[cfg(feature = "tcp_compression")]
@ -648,7 +644,7 @@ where
+ Stoppable,
I: DeserializeOwned,
{
fn try_receive(&mut self, state: &mut S) -> Result<Option<(Event<I>, bool)>, Error> {
fn try_receive(&mut self, state: &mut S) -> Result<Option<(EventWithStats<I>, bool)>, Error> {
// TODO: Get around local event copy by moving handle_in_client
let self_id = self.client_id;
let mut len_buf = [0_u8; 4];
@ -678,15 +674,15 @@ where
let buf = &self.compressor.decompress(buf)?;
// make decompressed vec and slice compatible
let event = postcard::from_bytes(buf)?;
let event: EventWithStats<I> = postcard::from_bytes(buf)?;
if !self.hooks.pre_receive_all(state, other_client_id, &event)? {
continue;
}
match event {
match event.event() {
Event::NewTestcase {
client_config,
ref observers_buf,
observers_buf,
forward_id,
..
} => {
@ -710,7 +706,7 @@ where
_ => {
return Err(Error::unknown(format!(
"Received illegal message that message should not have arrived: {:?}.",
event.name()
event.event().name()
)));
}
}
@ -729,7 +725,7 @@ where
Ok(None)
}
fn on_interesting(&mut self, _state: &mut S, _event: Event<I>) -> Result<(), Error> {
fn on_interesting(&mut self, _state: &mut S, _event: EventWithStats<I>) -> Result<(), Error> {
Ok(())
}
}
@ -820,7 +816,7 @@ where
self.tcp_mgr.should_send()
}
fn fire(&mut self, state: &mut S, event: Event<I>) -> Result<(), Error> {
fn fire(&mut self, state: &mut S, event: EventWithStats<I>) -> Result<(), Error> {
// Check if we are going to crash in the event, in which case we store our current state for the next runner
self.tcp_mgr.fire(state, event)
}
@ -896,11 +892,11 @@ where
SHM: ShMem,
SP: ShMemProvider<ShMem = SHM>,
{
fn try_receive(&mut self, state: &mut S) -> Result<Option<(Event<I>, bool)>, Error> {
fn try_receive(&mut self, state: &mut S) -> Result<Option<(EventWithStats<I>, bool)>, Error> {
self.tcp_mgr.try_receive(state)
}
fn on_interesting(&mut self, state: &mut S, event: Event<I>) -> Result<(), Error> {
fn on_interesting(&mut self, state: &mut S, event: EventWithStats<I>) -> Result<(), Error> {
self.tcp_mgr.on_interesting(state, event)
}
}

View File

@ -17,7 +17,7 @@ use libafl_bolts::tuples::{RefIndexable, tuple_list};
use crate::{
Error, HasMetadata,
corpus::{Corpus, Testcase},
events::{Event, EventFirer, EventRestarter},
events::{Event, EventFirer, EventRestarter, EventWithStats},
executors::{
Executor, ExitKind, HasObservers,
hooks::{ExecutorHooksTuple, inprocess::InProcessHooks},
@ -360,14 +360,16 @@ pub fn run_observers_and_save_state<E, EM, I, OF, S, Z>(
.solutions_mut()
.add(new_testcase)
.expect("In run_observers_and_save_state solutions failure.");
let event = Event::Objective {
input: fuzzer.share_objectives().then_some(input.clone()),
objective_size: state.solutions().count(),
};
event_mgr
.fire(
state,
Event::Objective {
input: fuzzer.share_objectives().then_some(input.clone()),
objective_size: state.solutions().count(),
time: libafl_bolts::current_time(),
},
EventWithStats::with_current_time(event, *state.executions()),
)
.expect("Could not send off events in run_observers_and_save_state");
}

View File

@ -23,11 +23,12 @@ use crate::feedbacks::premature_last_result_err;
use crate::{
Error, HasMetadata, HasNamedMetadata,
corpus::Testcase,
events::{Event, EventFirer},
events::{Event, EventFirer, EventWithStats},
executors::ExitKind,
feedbacks::{Feedback, HasObserverHandle, StateInitializer},
monitors::stats::{AggregatorOps, UserStats, UserStatsValue},
observers::{CanTrack, MapObserver},
state::HasExecutions,
};
/// A [`MapFeedback`] that implements the AFL algorithm using an [`OrReducer`] combining the bits for the history map and the bit from (`HitcountsMapObserver`)[`crate::observers::HitcountsMapObserver`].
@ -390,7 +391,7 @@ where
O::Entry: 'static + Default + Debug + DeserializeOwned + Serialize,
OT: MatchName,
R: Reducer<O::Entry>,
S: HasNamedMetadata,
S: HasNamedMetadata + HasExecutions,
{
#[rustversion::nightly]
default fn is_interesting(
@ -512,6 +513,7 @@ where
// unnecessarily
manager.fire(
state,
EventWithStats::with_current_time(
Event::UpdateUserStats {
name: self.stats_name.clone(),
value: UserStats::new(
@ -520,6 +522,8 @@ where
),
phantom: PhantomData,
},
*state.executions(),
),
)?;
Ok(())
@ -534,7 +538,7 @@ where
EM: EventFirer<I, S>,
O: MapObserver<Entry = u8> + for<'a> AsSlice<'a, Entry = u8> + for<'a> AsIter<'a, Item = u8>,
OT: MatchName,
S: HasNamedMetadata,
S: HasNamedMetadata + HasExecutions,
{
fn is_interesting(
&mut self,

View File

@ -15,7 +15,10 @@ use crate::monitors::stats::PerfFeature;
use crate::{
Error, HasMetadata,
corpus::{Corpus, CorpusId, HasCurrentCorpusId, HasTestcase, Testcase},
events::{Event, EventConfig, EventFirer, EventReceiver, ProgressReporter, SendExiting},
events::{
Event, EventConfig, EventFirer, EventReceiver, EventWithStats, ProgressReporter,
SendExiting,
},
executors::{Executor, ExitKind, HasObservers},
feedbacks::Feedback,
inputs::Input,
@ -359,7 +362,8 @@ where
+ MaybeHasClientPerfMonitor
+ HasCurrentTestcase<I>
+ HasSolutions<I>
+ HasLastFoundTime,
+ HasLastFoundTime
+ HasExecutions,
{
fn check_results(
&mut self,
@ -484,27 +488,32 @@ where
if exec_res.is_corpus() {
manager.fire(
state,
EventWithStats::with_current_time(
Event::NewTestcase {
input: input.clone(),
observers_buf,
exit_kind: *exit_kind,
corpus_size: state.corpus().count(),
client_config: manager.configuration(),
time: current_time(),
forward_id: None,
#[cfg(all(unix, feature = "std", feature = "multi_machine"))]
node_id: None,
},
*state.executions(),
),
)?;
}
if exec_res.is_solution() {
manager.fire(
state,
EventWithStats::with_current_time(
Event::Objective {
input: self.share_objectives.then_some(input.clone()),
objective_size: state.solutions().count(),
time: current_time(),
},
*state.executions(),
),
)?;
}
}
@ -693,11 +702,13 @@ where
manager.fire(
state,
EventWithStats::with_current_time(
Event::Objective {
input: self.share_objectives.then_some(input.clone()),
objective_size: state.solutions().count(),
time: current_time(),
},
*state.executions(),
),
)?;
}
@ -733,17 +744,19 @@ where
};
manager.fire(
state,
EventWithStats::with_current_time(
Event::NewTestcase {
input,
observers_buf,
exit_kind,
corpus_size: state.corpus().count(),
client_config: manager.configuration(),
time: current_time(),
forward_id: None,
#[cfg(all(unix, feature = "std", feature = "multi_machine"))]
node_id: None,
},
*state.executions(),
),
)?;
Ok((id, ExecuteInputResult::new(corpus_worthy, is_solution)))
}
@ -785,32 +798,32 @@ where
while let Some((event, with_observers)) = manager.try_receive(state)? {
// at this point event is either newtestcase or objectives
let res = if with_observers {
match event {
match event.event() {
Event::NewTestcase {
ref input,
ref observers_buf,
input,
observers_buf,
exit_kind,
..
} => {
let observers: E::Observers =
postcard::from_bytes(observers_buf.as_ref().unwrap())?;
let res = self.evaluate_execution(
state, manager, input, &observers, &exit_kind, false,
state, manager, input, &observers, exit_kind, false,
)?;
res.1
}
_ => None,
}
} else {
match event {
Event::NewTestcase { ref input, .. } => {
match event.event() {
Event::NewTestcase { input, .. } => {
let res = self.evaluate_input_with_observers(
state, executor, manager, input, false,
)?;
res.1
}
Event::Objective {
input: Some(ref unwrapped_input),
input: Some(unwrapped_input),
..
} => {
let res = self.evaluate_input_with_observers(

View File

@ -23,7 +23,7 @@ use crate::feedbacks::{CRASH_FEEDBACK_NAME, TIMEOUT_FEEDBACK_NAME};
use crate::{
Error, HasMetadata, HasNamedMetadata, HasScheduler,
corpus::{Corpus, HasCurrentCorpusId, SchedulerTestcaseMetadata, Testcase},
events::{Event, EventFirer},
events::{Event, EventFirer, EventWithStats},
executors::HasObservers,
monitors::stats::{AggregatorOps, UserStats, UserStatsValue},
mutators::Tokens,
@ -418,6 +418,7 @@ where
manager.fire(
state,
EventWithStats::with_current_time(
Event::UpdateUserStats {
name: Cow::Borrowed("AflStats"),
value: UserStats::new(
@ -426,6 +427,8 @@ where
),
phantom: PhantomData,
},
*state.executions(),
),
)?;
Ok(())

View File

@ -15,7 +15,7 @@ use serde::{Deserialize, Serialize};
use crate::{
Error, HasMetadata, HasNamedMetadata,
corpus::{Corpus, HasCurrentCorpusId, SchedulerTestcaseMetadata},
events::{Event, EventFirer, LogSeverity},
events::{Event, EventFirer, EventWithStats, LogSeverity},
executors::{Executor, ExitKind, HasObservers},
feedbacks::{HasObserverHandle, map::MapFeedbackMetadata},
fuzzer::Evaluator,
@ -335,6 +335,7 @@ where
map_first_filled_count.saturating_sub(unstable_entries) as u64;
mgr.fire(
state,
EventWithStats::with_current_time(
Event::UpdateUserStats {
name: Cow::from("stability"),
value: UserStats::new(
@ -343,11 +344,14 @@ where
),
phantom: PhantomData,
},
*state.executions(),
),
)?;
}
} else if send_default_stability {
mgr.fire(
state,
EventWithStats::with_current_time(
Event::UpdateUserStats {
name: Cow::from("stability"),
value: UserStats::new(
@ -359,6 +363,8 @@ where
),
phantom: PhantomData,
},
*state.executions(),
),
)?;
}

View File

@ -17,7 +17,7 @@ use serde::{Deserialize, Serialize};
use crate::{
Error, HasMetadata, HasNamedMetadata,
corpus::{Corpus, CorpusId, HasCurrentCorpusId},
events::{Event, EventConfig, EventFirer, llmp::LlmpEventConverter},
events::{Event, EventConfig, EventFirer, EventWithStats, llmp::LlmpEventConverter},
executors::{Executor, ExitKind, HasObservers},
fuzzer::{Evaluator, EvaluatorObservers, ExecutionProcessor, HasObjective},
inputs::{Input, InputConverter},
@ -272,17 +272,19 @@ where
self.client.fire(
state,
EventWithStats::with_current_time(
Event::NewTestcase {
input,
observers_buf: None,
exit_kind: ExitKind::Ok,
corpus_size: 0, // TODO choose if sending 0 or the actual real value
client_config: EventConfig::AlwaysUnique,
time: current_time(),
forward_id: None,
#[cfg(all(unix, feature = "multi_machine"))]
node_id: None,
},
*state.executions(),
),
)?;
cur_id = state.corpus().next(id);

View File

@ -33,7 +33,7 @@ use crate::monitors::stats::ClientPerfStats;
use crate::{
Error, HasMetadata, HasNamedMetadata,
corpus::{Corpus, CorpusId, HasCurrentCorpusId, HasTestcase, InMemoryCorpus, Testcase},
events::{Event, EventFirer, LogSeverity},
events::{Event, EventFirer, EventWithStats, LogSeverity},
feedbacks::StateInitializer,
fuzzer::Evaluator,
generators::Generator,
@ -764,11 +764,14 @@ where
manager.fire(
self,
EventWithStats::with_current_time(
Event::Log {
severity_level: LogSeverity::Debug,
message: format!("Loaded {} initial testcases.", self.corpus().count()), // get corpus count
phantom: PhantomData::<I>,
},
*self.executions(),
),
)?;
Ok(())
}
@ -1062,11 +1065,14 @@ where
}
manager.fire(
self,
EventWithStats::with_current_time(
Event::Log {
severity_level: LogSeverity::Debug,
message: format!("Loaded {added} over {num} initial testcases"),
phantom: PhantomData,
},
*self.executions(),
),
)?;
Ok(())
}