Skip to content

[ENH]: Dead letter queuing for compaction jobs #5023

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 1 commit into from
Jul 20, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions rust/worker/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ async-trait = { workspace = true }
roaring = { workspace = true }
figment = { workspace = true }
futures = { workspace = true }
opentelemetry = { workspace = true }
parking_lot = { workspace = true }
tracing = { workspace = true }
tokio = { workspace = true }
Expand Down
53 changes: 38 additions & 15 deletions rust/worker/src/compactor/compaction_manager.rs
Original file line number Diff line number Diff line change
Expand Up @@ -49,14 +49,19 @@ use tracing::Instrument;
use tracing::Span;
use uuid::Uuid;

type BoxedFuture =
Pin<Box<dyn Future<Output = Result<CompactionResponse, Box<dyn ChromaError>>> + Send>>;
type CompactionOutput = Result<CompactionResponse, Box<dyn ChromaError>>;
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good cleanup ty

type BoxedFuture = Pin<Box<dyn Future<Output = CompactionOutput> + Send>>;

struct CompactionTask {
collection_id: CollectionUuid,
future: BoxedFuture,
}

struct CompactionTaskCompletion {
collection_id: CollectionUuid,
result: CompactionOutput,
}

#[derive(Clone)]
pub(crate) struct CompactionManagerContext {
system: System,
Expand Down Expand Up @@ -85,7 +90,7 @@ pub(crate) struct CompactionManager {
scheduler: Scheduler,
context: CompactionManagerContext,
compact_awaiter_channel: mpsc::Sender<CompactionTask>,
compact_awaiter_completion_channel: mpsc::UnboundedReceiver<CompactionResponse>,
compact_awaiter_completion_channel: mpsc::UnboundedReceiver<CompactionTaskCompletion>,
compact_awaiter: tokio::task::JoinHandle<()>,
on_next_memberlist_signal: Option<oneshot::Sender<()>>,
}
Expand Down Expand Up @@ -129,7 +134,7 @@ impl CompactionManager {
// Using unbounded channel for the completion channel as its size
// is bounded by max_concurrent_jobs. It's far more important for the
// completion channel to not block or drop messages.
let (completion_tx, completion_rx) = mpsc::unbounded_channel::<CompactionResponse>();
let (completion_tx, completion_rx) = mpsc::unbounded_channel::<CompactionTaskCompletion>();
let compact_awaiter = tokio::spawn(async {
compact_awaiter_loop(compact_awaiter_rx, completion_tx).await;
});
Expand Down Expand Up @@ -237,11 +242,18 @@ impl CompactionManager {
self.context.dispatcher = Some(dispatcher);
}

fn process_completions(&mut self) -> Vec<CompactionResponse> {
fn process_completions(&mut self) -> Vec<CompactionTaskCompletion> {
let compact_awaiter_completion_channel = &mut self.compact_awaiter_completion_channel;
let mut completed_collections = Vec::new();
while let Ok(resp) = compact_awaiter_completion_channel.try_recv() {
self.scheduler.complete_collection(resp.collection_id);
match resp.result {
Ok(_) => {
self.scheduler.succeed_collection(resp.collection_id);
}
Err(_) => {
self.scheduler.fail_collection(resp.collection_id);
}
}
completed_collections.push(resp);
}
completed_collections
Expand Down Expand Up @@ -349,6 +361,7 @@ impl Configurable<(CompactionServiceConfig, System)> for CompactionManager {
Box::<dyn AssignmentPolicy>::try_from_config(assignment_policy_config, registry)
.await?;
let job_expiry_seconds = config.compactor.job_expiry_seconds;
let max_failure_count = config.compactor.max_failure_count;
let scheduler = Scheduler::new(
my_ip,
log.clone(),
Expand All @@ -359,6 +372,7 @@ impl Configurable<(CompactionServiceConfig, System)> for CompactionManager {
assignment_policy,
disabled_collections,
job_expiry_seconds,
max_failure_count,
);

let blockfile_provider = BlockfileProvider::try_from_config(
Expand Down Expand Up @@ -405,25 +419,31 @@ impl Configurable<(CompactionServiceConfig, System)> for CompactionManager {

async fn compact_awaiter_loop(
mut job_rx: mpsc::Receiver<CompactionTask>,
completion_tx: mpsc::UnboundedSender<CompactionResponse>,
completion_tx: mpsc::UnboundedSender<CompactionTaskCompletion>,
) {
let mut futures = FuturesUnordered::new();
loop {
select! {
Some(job) = job_rx.recv() => {
futures.push(async move {
let _ = AssertUnwindSafe(job.future).catch_unwind().await;
CompactionResponse {
collection_id: job.collection_id,
let result = AssertUnwindSafe(job.future).catch_unwind().await;
match result {
Ok(response) => CompactionTaskCompletion {
collection_id: job.collection_id,
result: response,
},
Err(_) => CompactionTaskCompletion {
collection_id: job.collection_id,
result: Err(Box::new(CompactionError::FailedToCompact)),
},
}
});
}
Some(compaction_response) = futures.next() => {
match completion_tx.send(compaction_response) {
Some(completed_job) = futures.next() => {
let collection_id = completed_job.collection_id;
match completion_tx.send(completed_job) {
Ok(_) => {},
Err(_) => {
tracing::error!("Failed to send compaction response");
}
Err(_) => tracing::error!("Failed to record compaction result for collection {}", collection_id),
}
}
else => {
Expand Down Expand Up @@ -771,6 +791,7 @@ mod tests {
let fetch_log_batch_size = 100;
let purge_dirty_log_timeout_seconds = 60;
let job_expiry_seconds = 3600;
let max_failure_count = 3;

// Set assignment policy
let mut assignment_policy = Box::new(RendezvousHashingAssignmentPolicy::default());
Expand All @@ -786,6 +807,7 @@ mod tests {
assignment_policy,
HashSet::new(),
job_expiry_seconds,
max_failure_count,
);
// Set memberlist
scheduler.set_memberlist(vec![my_member.clone()]);
Expand Down Expand Up @@ -863,6 +885,7 @@ mod tests {
completed_compactions.extend(
completed
.iter()
.filter(|c| c.result.is_ok())
.map(|c| c.collection_id)
.collect::<Vec<CollectionUuid>>(),
);
Expand Down
7 changes: 7 additions & 0 deletions rust/worker/src/compactor/config.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@ pub struct CompactorConfig {
pub fetch_log_batch_size: u32,
#[serde(default = "CompactorConfig::default_purge_dirty_log_timeout_seconds")]
pub purge_dirty_log_timeout_seconds: u64,
#[serde(default = "CompactorConfig::default_max_failure_count")]
pub max_failure_count: u8,
}

impl CompactorConfig {
Expand Down Expand Up @@ -64,6 +66,10 @@ impl CompactorConfig {
fn default_purge_dirty_log_timeout_seconds() -> u64 {
60
}

fn default_max_failure_count() -> u8 {
5
}
}

impl Default for CompactorConfig {
Expand All @@ -80,6 +86,7 @@ impl Default for CompactorConfig {
fetch_log_batch_size: CompactorConfig::default_fetch_log_batch_size(),
purge_dirty_log_timeout_seconds:
CompactorConfig::default_purge_dirty_log_timeout_seconds(),
max_failure_count: CompactorConfig::default_max_failure_count(),
}
}
}
Loading
Loading