1mod compaction_executor;
16mod compaction_filter;
17pub mod compaction_utils;
18mod iceberg_compaction;
19use parquet::basic::Compression;
20use parquet::file::properties::WriterProperties;
21use risingwave_hummock_sdk::compact_task::{CompactTask, ValidationTask};
22use risingwave_pb::compactor::{DispatchCompactionTaskRequest, dispatch_compaction_task_request};
23use risingwave_pb::hummock::PbCompactTask;
24use risingwave_pb::hummock::report_compaction_task_request::{
25 Event as ReportCompactionTaskEvent, HeartBeat as SharedHeartBeat,
26 ReportTask as ReportSharedTask,
27};
28use risingwave_pb::iceberg_compaction::{
29 SubscribeIcebergCompactionEventRequest, SubscribeIcebergCompactionEventResponse,
30 subscribe_iceberg_compaction_event_request,
31};
32use risingwave_rpc_client::GrpcCompactorProxyClient;
33use thiserror_ext::AsReport;
34use tokio::sync::mpsc;
35use tonic::Request;
36
37pub mod compactor_runner;
38mod context;
39pub mod fast_compactor_runner;
40mod iterator;
41mod shared_buffer_compact;
42pub(super) mod task_progress;
43
44use std::collections::{HashMap, VecDeque};
45use std::marker::PhantomData;
46use std::sync::atomic::{AtomicU32, AtomicU64, Ordering};
47use std::sync::{Arc, Mutex};
48use std::time::{Duration, SystemTime};
49
50use await_tree::{InstrumentAwait, SpanExt};
51pub use compaction_executor::CompactionExecutor;
52pub use compaction_filter::{
53 CompactionFilter, DummyCompactionFilter, MultiCompactionFilter, StateCleanUpCompactionFilter,
54 TtlCompactionFilter,
55};
56pub use context::{
57 CompactionAwaitTreeRegRef, CompactorContext, await_tree_key, new_compaction_await_tree_reg_ref,
58};
59use futures::{StreamExt, pin_mut};
60use iceberg_compaction::iceberg_compactor_runner::{
62 IcebergCompactorRunnerConfigBuilder, create_plan_runners,
63};
64use iceberg_compaction::{IcebergTaskQueue, PushResult};
65pub use iterator::{ConcatSstableIterator, SstableStreamIterator};
66use more_asserts::assert_ge;
67use risingwave_hummock_sdk::table_stats::{TableStatsMap, to_prost_table_stats_map};
68use risingwave_hummock_sdk::{
69 HummockCompactionTaskId, HummockSstableObjectId, LocalSstableInfo, compact_task_to_string,
70};
71use risingwave_pb::hummock::compact_task::TaskStatus;
72use risingwave_pb::hummock::subscribe_compaction_event_request::{
73 Event as RequestEvent, HeartBeat, PullTask, ReportTask,
74};
75use risingwave_pb::hummock::subscribe_compaction_event_response::Event as ResponseEvent;
76use risingwave_pb::hummock::{
77 CompactTaskProgress, ReportCompactionTaskRequest, SubscribeCompactionEventRequest,
78 SubscribeCompactionEventResponse,
79};
80use risingwave_rpc_client::HummockMetaClient;
81pub use shared_buffer_compact::{compact, merge_imms_in_memory};
82use tokio::sync::oneshot::Sender;
83use tokio::task::JoinHandle;
84use tokio::time::Instant;
85
86pub use self::compaction_utils::{
87 CompactionStatistics, RemoteBuilderFactory, TaskConfig, check_compaction_result,
88 check_flush_result,
89};
90pub use self::task_progress::TaskProgress;
91use super::multi_builder::CapacitySplitTableBuilder;
92use super::{
93 GetObjectId, HummockResult, ObjectIdManager, SstableBuilderOptions, Xor16FilterBuilder,
94};
95use crate::compaction_catalog_manager::{
96 CompactionCatalogAgentRef, CompactionCatalogManager, CompactionCatalogManagerRef,
97};
98use crate::hummock::compactor::compaction_utils::calculate_task_parallelism;
99use crate::hummock::compactor::compactor_runner::{compact_and_build_sst, compact_done};
100use crate::hummock::iterator::{Forward, HummockIterator};
101use crate::hummock::{
102 BlockedXor16FilterBuilder, FilterBuilder, SharedComapctorObjectIdManager, SstableWriterFactory,
103 UnifiedSstableWriterFactory, validate_ssts,
104};
105use crate::monitor::CompactorMetrics;
106
107const COMPACTION_HEARTBEAT_LOG_INTERVAL: Duration = Duration::from_secs(60);
109
110#[derive(Debug, Clone, PartialEq, Eq)]
112struct CompactionLogState {
113 running_parallelism: u32,
114 pull_task_ack: bool,
115 pending_pull_task_count: u32,
116}
117
118#[derive(Debug, Clone, PartialEq, Eq)]
120struct IcebergCompactionLogState {
121 running_parallelism: u32,
122 waiting_parallelism: u32,
123 available_parallelism: u32,
124 pull_task_ack: bool,
125 pending_pull_task_count: u32,
126}
127
128struct LogThrottler<T: PartialEq> {
130 last_logged_state: Option<T>,
131 last_heartbeat: Instant,
132 heartbeat_interval: Duration,
133}
134
135impl<T: PartialEq> LogThrottler<T> {
136 fn new(heartbeat_interval: Duration) -> Self {
137 Self {
138 last_logged_state: None,
139 last_heartbeat: Instant::now(),
140 heartbeat_interval,
141 }
142 }
143
144 fn should_log(&self, current_state: &T) -> bool {
146 self.last_logged_state.as_ref() != Some(current_state)
147 || self.last_heartbeat.elapsed() >= self.heartbeat_interval
148 }
149
150 fn update(&mut self, current_state: T) {
152 self.last_logged_state = Some(current_state);
153 self.last_heartbeat = Instant::now();
154 }
155}
156
157pub struct Compactor {
159 context: CompactorContext,
161 object_id_getter: Arc<dyn GetObjectId>,
162 task_config: TaskConfig,
163 options: SstableBuilderOptions,
164 get_id_time: Arc<AtomicU64>,
165}
166
167pub type CompactOutput = (usize, Vec<LocalSstableInfo>, CompactionStatistics);
168
169impl Compactor {
170 pub fn new(
172 context: CompactorContext,
173 options: SstableBuilderOptions,
174 task_config: TaskConfig,
175 object_id_getter: Arc<dyn GetObjectId>,
176 ) -> Self {
177 Self {
178 context,
179 options,
180 task_config,
181 get_id_time: Arc::new(AtomicU64::new(0)),
182 object_id_getter,
183 }
184 }
185
186 async fn compact_key_range(
191 &self,
192 iter: impl HummockIterator<Direction = Forward>,
193 compaction_filter: impl CompactionFilter,
194 compaction_catalog_agent_ref: CompactionCatalogAgentRef,
195 task_progress: Option<Arc<TaskProgress>>,
196 task_id: Option<HummockCompactionTaskId>,
197 split_index: Option<usize>,
198 ) -> HummockResult<(Vec<LocalSstableInfo>, CompactionStatistics)> {
199 let compact_timer = if self.context.is_share_buffer_compact {
201 self.context
202 .compactor_metrics
203 .write_build_l0_sst_duration
204 .start_timer()
205 } else {
206 self.context
207 .compactor_metrics
208 .compact_sst_duration
209 .start_timer()
210 };
211
212 let (split_table_outputs, table_stats_map) = {
213 let factory = UnifiedSstableWriterFactory::new(self.context.sstable_store.clone());
214 if self.task_config.use_block_based_filter {
215 self.compact_key_range_impl::<_, BlockedXor16FilterBuilder>(
216 factory,
217 iter,
218 compaction_filter,
219 compaction_catalog_agent_ref,
220 task_progress.clone(),
221 self.object_id_getter.clone(),
222 )
223 .instrument_await("compact".verbose())
224 .await?
225 } else {
226 self.compact_key_range_impl::<_, Xor16FilterBuilder>(
227 factory,
228 iter,
229 compaction_filter,
230 compaction_catalog_agent_ref,
231 task_progress.clone(),
232 self.object_id_getter.clone(),
233 )
234 .instrument_await("compact".verbose())
235 .await?
236 }
237 };
238
239 compact_timer.observe_duration();
240
241 Self::report_progress(
242 self.context.compactor_metrics.clone(),
243 task_progress,
244 &split_table_outputs,
245 self.context.is_share_buffer_compact,
246 );
247
248 self.context
249 .compactor_metrics
250 .get_table_id_total_time_duration
251 .observe(self.get_id_time.load(Ordering::Relaxed) as f64 / 1000.0 / 1000.0);
252
253 debug_assert!(
254 split_table_outputs
255 .iter()
256 .all(|table_info| table_info.sst_info.table_ids.is_sorted())
257 );
258
259 if task_id.is_some() {
260 tracing::info!(
262 "Finish Task {:?} split_index {:?} sst count {}",
263 task_id,
264 split_index,
265 split_table_outputs.len()
266 );
267 }
268 Ok((split_table_outputs, table_stats_map))
269 }
270
271 pub fn report_progress(
272 metrics: Arc<CompactorMetrics>,
273 task_progress: Option<Arc<TaskProgress>>,
274 ssts: &Vec<LocalSstableInfo>,
275 is_share_buffer_compact: bool,
276 ) {
277 for sst_info in ssts {
278 let sst_size = sst_info.file_size();
279 if let Some(tracker) = &task_progress {
280 tracker.inc_ssts_uploaded();
281 tracker.dec_num_pending_write_io();
282 }
283 if is_share_buffer_compact {
284 metrics.shared_buffer_to_sstable_size.observe(sst_size as _);
285 } else {
286 metrics.compaction_upload_sst_counts.inc();
287 }
288 }
289 }
290
291 async fn compact_key_range_impl<F: SstableWriterFactory, B: FilterBuilder>(
292 &self,
293 writer_factory: F,
294 iter: impl HummockIterator<Direction = Forward>,
295 compaction_filter: impl CompactionFilter,
296 compaction_catalog_agent_ref: CompactionCatalogAgentRef,
297 task_progress: Option<Arc<TaskProgress>>,
298 object_id_getter: Arc<dyn GetObjectId>,
299 ) -> HummockResult<(Vec<LocalSstableInfo>, CompactionStatistics)> {
300 let builder_factory = RemoteBuilderFactory::<F, B> {
301 object_id_getter,
302 limiter: self.context.memory_limiter.clone(),
303 options: self.options.clone(),
304 policy: self.task_config.cache_policy,
305 remote_rpc_cost: self.get_id_time.clone(),
306 compaction_catalog_agent_ref: compaction_catalog_agent_ref.clone(),
307 sstable_writer_factory: writer_factory,
308 _phantom: PhantomData,
309 };
310
311 let mut sst_builder = CapacitySplitTableBuilder::new(
312 builder_factory,
313 self.context.compactor_metrics.clone(),
314 task_progress.clone(),
315 self.task_config.table_vnode_partition.clone(),
316 self.context
317 .storage_opts
318 .compactor_concurrent_uploading_sst_count,
319 compaction_catalog_agent_ref,
320 );
321 let compaction_statistics = compact_and_build_sst(
322 &mut sst_builder,
323 &self.task_config,
324 self.context.compactor_metrics.clone(),
325 iter,
326 compaction_filter,
327 )
328 .instrument_await("compact_and_build_sst".verbose())
329 .await?;
330
331 let ssts = sst_builder
332 .finish()
333 .instrument_await("builder_finish".verbose())
334 .await?;
335
336 Ok((ssts, compaction_statistics))
337 }
338}
339
340#[must_use]
343pub fn start_iceberg_compactor(
344 compactor_context: CompactorContext,
345 hummock_meta_client: Arc<dyn HummockMetaClient>,
346) -> (JoinHandle<()>, Sender<()>) {
347 let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel();
348 let stream_retry_interval = Duration::from_secs(30);
349 let periodic_event_update_interval = Duration::from_millis(1000);
350 let worker_num = compactor_context.compaction_executor.worker_num();
351
352 let max_task_parallelism: u32 = (worker_num as f32
353 * compactor_context.storage_opts.compactor_max_task_multiplier)
354 .ceil() as u32;
355
356 const MAX_PULL_TASK_COUNT: u32 = 4;
357 let max_pull_task_count = std::cmp::min(max_task_parallelism, MAX_PULL_TASK_COUNT);
358
359 assert_ge!(
360 compactor_context.storage_opts.compactor_max_task_multiplier,
361 0.0
362 );
363
364 let join_handle = tokio::spawn(async move {
365 let pending_parallelism_budget = (max_task_parallelism as f32
367 * compactor_context
368 .storage_opts
369 .iceberg_compaction_pending_parallelism_budget_multiplier)
370 .ceil() as u32;
371 let (mut task_queue, _schedule_notify) =
372 IcebergTaskQueue::new_with_notify(max_task_parallelism, pending_parallelism_budget);
373
374 let shutdown_map = Arc::new(Mutex::new(
376 HashMap::<u64, tokio::sync::oneshot::Sender<()>>::new(),
377 ));
378
379 let (task_completion_tx, mut task_completion_rx) =
381 tokio::sync::mpsc::unbounded_channel::<u64>();
382
383 let mut min_interval = tokio::time::interval(stream_retry_interval);
384 let mut periodic_event_interval = tokio::time::interval(periodic_event_update_interval);
385
386 let mut log_throttler =
388 LogThrottler::<IcebergCompactionLogState>::new(COMPACTION_HEARTBEAT_LOG_INTERVAL);
389
390 'start_stream: loop {
392 let mut pull_task_ack = true;
395 tokio::select! {
396 _ = min_interval.tick() => {},
398 _ = &mut shutdown_rx => {
400 tracing::info!("Compactor is shutting down");
401 return;
402 }
403 }
404
405 let (request_sender, response_event_stream) = match hummock_meta_client
406 .subscribe_iceberg_compaction_event()
407 .await
408 {
409 Ok((request_sender, response_event_stream)) => {
410 tracing::debug!("Succeeded subscribe_iceberg_compaction_event.");
411 (request_sender, response_event_stream)
412 }
413
414 Err(e) => {
415 tracing::warn!(
416 error = %e.as_report(),
417 "Subscribing to iceberg compaction tasks failed with error. Will retry.",
418 );
419 continue 'start_stream;
420 }
421 };
422
423 pin_mut!(response_event_stream);
424
425 let _executor = compactor_context.compaction_executor.clone();
426
427 let mut event_loop_iteration_now = Instant::now();
429 'consume_stream: loop {
430 {
431 compactor_context
433 .compactor_metrics
434 .compaction_event_loop_iteration_latency
435 .observe(event_loop_iteration_now.elapsed().as_millis() as _);
436 event_loop_iteration_now = Instant::now();
437 }
438
439 let request_sender = request_sender.clone();
440 let event: Option<Result<SubscribeIcebergCompactionEventResponse, _>> = tokio::select! {
441 Some(completed_task_id) = task_completion_rx.recv() => {
443 tracing::debug!(task_id = completed_task_id, "Task completed, updating queue state");
444 task_queue.finish_running(completed_task_id);
445 continue 'consume_stream;
446 }
447
448 _ = task_queue.wait_schedulable() => {
450 schedule_queued_tasks(
451 &mut task_queue,
452 &compactor_context,
453 &shutdown_map,
454 &task_completion_tx,
455 );
456 continue 'consume_stream;
457 }
458
459 _ = periodic_event_interval.tick() => {
460 let should_restart_stream = handle_meta_task_pulling(
462 &mut pull_task_ack,
463 &task_queue,
464 max_task_parallelism,
465 max_pull_task_count,
466 &request_sender,
467 &mut log_throttler,
468 );
469
470 if should_restart_stream {
471 continue 'start_stream;
472 }
473 continue;
474 }
475 event = response_event_stream.next() => {
476 event
477 }
478
479 _ = &mut shutdown_rx => {
480 tracing::info!("Iceberg Compactor is shutting down");
481 return
482 }
483 };
484
485 match event {
486 Some(Ok(SubscribeIcebergCompactionEventResponse {
487 event,
488 create_at: _create_at,
489 })) => {
490 let event = match event {
491 Some(event) => event,
492 None => continue 'consume_stream,
493 };
494
495 match event {
496 risingwave_pb::iceberg_compaction::subscribe_iceberg_compaction_event_response::Event::CompactTask(iceberg_compaction_task) => {
497 let task_id = iceberg_compaction_task.task_id;
498 let write_parquet_properties = WriterProperties::builder()
499 .set_created_by(concat!(
500 "risingwave version ",
501 env!("CARGO_PKG_VERSION")
502 )
503 .to_owned())
504 .set_max_row_group_size(
505 compactor_context.storage_opts.iceberg_compaction_write_parquet_max_row_group_rows
506 )
507 .set_compression(Compression::SNAPPY) .build();
509
510 let compactor_runner_config = match IcebergCompactorRunnerConfigBuilder::default()
511 .max_parallelism((worker_num as f32 * compactor_context.storage_opts.iceberg_compaction_task_parallelism_ratio) as u32)
512 .min_size_per_partition(compactor_context.storage_opts.iceberg_compaction_min_size_per_partition_mb as u64 * 1024 * 1024)
513 .max_file_count_per_partition(compactor_context.storage_opts.iceberg_compaction_max_file_count_per_partition)
514 .enable_validate_compaction(compactor_context.storage_opts.iceberg_compaction_enable_validate)
515 .max_record_batch_rows(compactor_context.storage_opts.iceberg_compaction_max_record_batch_rows)
516 .write_parquet_properties(write_parquet_properties)
517 .enable_heuristic_output_parallelism(compactor_context.storage_opts.iceberg_compaction_enable_heuristic_output_parallelism)
518 .max_concurrent_closes(compactor_context.storage_opts.iceberg_compaction_max_concurrent_closes)
519 .enable_dynamic_size_estimation(compactor_context.storage_opts.iceberg_compaction_enable_dynamic_size_estimation)
520 .size_estimation_smoothing_factor(compactor_context.storage_opts.iceberg_compaction_size_estimation_smoothing_factor)
521 .target_binpack_group_size_mb(
522 compactor_context.storage_opts.iceberg_compaction_target_binpack_group_size_mb
523 )
524 .min_group_size_mb(
525 compactor_context.storage_opts.iceberg_compaction_min_group_size_mb
526 )
527 .min_group_file_count(
528 compactor_context.storage_opts.iceberg_compaction_min_group_file_count
529 )
530 .build() {
531 Ok(config) => config,
532 Err(e) => {
533 tracing::warn!(error = %e.as_report(), "Failed to build iceberg compactor runner config {}", task_id);
534 continue 'consume_stream;
535 }
536 };
537
538 let plan_runners = match create_plan_runners(
540 iceberg_compaction_task,
541 compactor_runner_config,
542 compactor_context.compactor_metrics.clone(),
543 ).await {
544 Ok(runners) => runners,
545 Err(e) => {
546 tracing::warn!(error = %e.as_report(), task_id, "Failed to create plan runners");
547 continue 'consume_stream;
548 }
549 };
550
551 if plan_runners.is_empty() {
552 tracing::info!(task_id, "No plans to execute");
553 continue 'consume_stream;
554 }
555
556 let total_plans = plan_runners.len();
558 let mut enqueued_count = 0;
559
560 for runner in plan_runners {
561 let meta = runner.to_meta();
562 let required_parallelism = runner.required_parallelism();
563 let push_result = task_queue.push(meta.clone(), Some(runner));
564
565 match push_result {
566 PushResult::Added => {
567 enqueued_count += 1;
568 tracing::debug!(
569 task_id = task_id,
570 plan_index = enqueued_count - 1,
571 required_parallelism = required_parallelism,
572 "Iceberg plan runner added to queue"
573 );
574 },
575 PushResult::RejectedCapacity => {
576 tracing::warn!(
577 task_id = task_id,
578 required_parallelism = required_parallelism,
579 pending_budget = pending_parallelism_budget,
580 enqueued_count = enqueued_count,
581 total_plans = total_plans,
582 "Iceberg plan runner rejected - queue capacity exceeded"
583 );
584 break;
586 },
587 PushResult::RejectedTooLarge => {
588 tracing::error!(
589 task_id = task_id,
590 required_parallelism = required_parallelism,
591 max_parallelism = max_task_parallelism,
592 "Iceberg plan runner rejected - parallelism exceeds max"
593 );
594 },
595 PushResult::RejectedInvalidParallelism => {
596 tracing::error!(
597 task_id = task_id,
598 required_parallelism = required_parallelism,
599 "Iceberg plan runner rejected - invalid parallelism"
600 );
601 }
602 }
603 }
604
605 tracing::info!(
606 task_id = task_id,
607 total_plans = total_plans,
608 enqueued_count = enqueued_count,
609 "Enqueued {} of {} Iceberg plan runners",
610 enqueued_count,
611 total_plans
612 );
613 },
614 risingwave_pb::iceberg_compaction::subscribe_iceberg_compaction_event_response::Event::PullTaskAck(_) => {
615 pull_task_ack = true;
617 },
618 }
619 }
620 Some(Err(e)) => {
621 tracing::warn!("Failed to consume stream. {}", e.message());
622 continue 'start_stream;
623 }
624 _ => {
625 continue 'start_stream;
627 }
628 }
629 }
630 }
631 });
632
633 (join_handle, shutdown_tx)
634}
635
636#[must_use]
639pub fn start_compactor(
640 compactor_context: CompactorContext,
641 hummock_meta_client: Arc<dyn HummockMetaClient>,
642 object_id_manager: Arc<ObjectIdManager>,
643 compaction_catalog_manager_ref: CompactionCatalogManagerRef,
644) -> (JoinHandle<()>, Sender<()>) {
645 type CompactionShutdownMap = Arc<Mutex<HashMap<u64, Sender<()>>>>;
646 let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel();
647 let stream_retry_interval = Duration::from_secs(30);
648 let task_progress = compactor_context.task_progress_manager.clone();
649 let periodic_event_update_interval = Duration::from_millis(1000);
650
651 let max_task_parallelism: u32 = (compactor_context.compaction_executor.worker_num() as f32
652 * compactor_context.storage_opts.compactor_max_task_multiplier)
653 .ceil() as u32;
654 let running_task_parallelism = Arc::new(AtomicU32::new(0));
655
656 const MAX_PULL_TASK_COUNT: u32 = 4;
657 let max_pull_task_count = std::cmp::min(max_task_parallelism, MAX_PULL_TASK_COUNT);
658
659 assert_ge!(
660 compactor_context.storage_opts.compactor_max_task_multiplier,
661 0.0
662 );
663
664 let join_handle = tokio::spawn(async move {
665 let shutdown_map = CompactionShutdownMap::default();
666 let mut min_interval = tokio::time::interval(stream_retry_interval);
667 let mut periodic_event_interval = tokio::time::interval(periodic_event_update_interval);
668
669 let mut log_throttler =
671 LogThrottler::<CompactionLogState>::new(COMPACTION_HEARTBEAT_LOG_INTERVAL);
672
673 'start_stream: loop {
675 let mut pull_task_ack = true;
678 tokio::select! {
679 _ = min_interval.tick() => {},
681 _ = &mut shutdown_rx => {
683 tracing::info!("Compactor is shutting down");
684 return;
685 }
686 }
687
688 let (request_sender, response_event_stream) =
689 match hummock_meta_client.subscribe_compaction_event().await {
690 Ok((request_sender, response_event_stream)) => {
691 tracing::debug!("Succeeded subscribe_compaction_event.");
692 (request_sender, response_event_stream)
693 }
694
695 Err(e) => {
696 tracing::warn!(
697 error = %e.as_report(),
698 "Subscribing to compaction tasks failed with error. Will retry.",
699 );
700 continue 'start_stream;
701 }
702 };
703
704 pin_mut!(response_event_stream);
705
706 let executor = compactor_context.compaction_executor.clone();
707 let object_id_manager = object_id_manager.clone();
708
709 let mut event_loop_iteration_now = Instant::now();
711 'consume_stream: loop {
712 {
713 compactor_context
715 .compactor_metrics
716 .compaction_event_loop_iteration_latency
717 .observe(event_loop_iteration_now.elapsed().as_millis() as _);
718 event_loop_iteration_now = Instant::now();
719 }
720
721 let running_task_parallelism = running_task_parallelism.clone();
722 let request_sender = request_sender.clone();
723 let event: Option<Result<SubscribeCompactionEventResponse, _>> = tokio::select! {
724 _ = periodic_event_interval.tick() => {
725 let progress_list = get_task_progress(task_progress.clone());
726
727 if let Err(e) = request_sender.send(SubscribeCompactionEventRequest {
728 event: Some(RequestEvent::HeartBeat(
729 HeartBeat {
730 progress: progress_list
731 }
732 )),
733 create_at: SystemTime::now()
734 .duration_since(std::time::UNIX_EPOCH)
735 .expect("Clock may have gone backwards")
736 .as_millis() as u64,
737 }) {
738 tracing::warn!(error = %e.as_report(), "Failed to report task progress");
739 continue 'start_stream;
741 }
742
743
744 let mut pending_pull_task_count = 0;
745 if pull_task_ack {
746 pending_pull_task_count = (max_task_parallelism - running_task_parallelism.load(Ordering::SeqCst)).min(max_pull_task_count);
748
749 if pending_pull_task_count > 0 {
750 if let Err(e) = request_sender.send(SubscribeCompactionEventRequest {
751 event: Some(RequestEvent::PullTask(
752 PullTask {
753 pull_task_count: pending_pull_task_count,
754 }
755 )),
756 create_at: SystemTime::now()
757 .duration_since(std::time::UNIX_EPOCH)
758 .expect("Clock may have gone backwards")
759 .as_millis() as u64,
760 }) {
761 tracing::warn!(error = %e.as_report(), "Failed to pull task");
762
763 continue 'start_stream;
765 } else {
766 pull_task_ack = false;
767 }
768 }
769 }
770
771 let running_count = running_task_parallelism.load(Ordering::SeqCst);
772 let current_state = CompactionLogState {
773 running_parallelism: running_count,
774 pull_task_ack,
775 pending_pull_task_count,
776 };
777
778 if log_throttler.should_log(¤t_state) {
780 tracing::info!(
781 running_parallelism_count = %current_state.running_parallelism,
782 pull_task_ack = %current_state.pull_task_ack,
783 pending_pull_task_count = %current_state.pending_pull_task_count
784 );
785 log_throttler.update(current_state);
786 }
787
788 continue;
789 }
790 event = response_event_stream.next() => {
791 event
792 }
793
794 _ = &mut shutdown_rx => {
795 tracing::info!("Compactor is shutting down");
796 return
797 }
798 };
799
800 fn send_report_task_event(
801 compact_task: &CompactTask,
802 table_stats: TableStatsMap,
803 object_timestamps: HashMap<HummockSstableObjectId, u64>,
804 request_sender: &mpsc::UnboundedSender<SubscribeCompactionEventRequest>,
805 ) {
806 if let Err(e) = request_sender.send(SubscribeCompactionEventRequest {
807 event: Some(RequestEvent::ReportTask(ReportTask {
808 task_id: compact_task.task_id,
809 task_status: compact_task.task_status.into(),
810 sorted_output_ssts: compact_task
811 .sorted_output_ssts
812 .iter()
813 .map(|sst| sst.into())
814 .collect(),
815 table_stats_change: to_prost_table_stats_map(table_stats),
816 object_timestamps: object_timestamps
817 .into_iter()
818 .map(|(object_id, timestamp)| (object_id.inner(), timestamp))
819 .collect(),
820 })),
821 create_at: SystemTime::now()
822 .duration_since(std::time::UNIX_EPOCH)
823 .expect("Clock may have gone backwards")
824 .as_millis() as u64,
825 }) {
826 let task_id = compact_task.task_id;
827 tracing::warn!(error = %e.as_report(), "Failed to report task {task_id:?}");
828 }
829 }
830
831 match event {
832 Some(Ok(SubscribeCompactionEventResponse { event, create_at })) => {
833 let event = match event {
834 Some(event) => event,
835 None => continue 'consume_stream,
836 };
837 let shutdown = shutdown_map.clone();
838 let context = compactor_context.clone();
839 let consumed_latency_ms = SystemTime::now()
840 .duration_since(std::time::UNIX_EPOCH)
841 .expect("Clock may have gone backwards")
842 .as_millis() as u64
843 - create_at;
844 context
845 .compactor_metrics
846 .compaction_event_consumed_latency
847 .observe(consumed_latency_ms as _);
848
849 let object_id_manager = object_id_manager.clone();
850 let compaction_catalog_manager_ref = compaction_catalog_manager_ref.clone();
851
852 match event {
853 ResponseEvent::CompactTask(compact_task) => {
854 let compact_task = CompactTask::from(compact_task);
855 let parallelism =
856 calculate_task_parallelism(&compact_task, &context);
857
858 assert_ne!(parallelism, 0, "splits cannot be empty");
859
860 if (max_task_parallelism
861 - running_task_parallelism.load(Ordering::SeqCst))
862 < parallelism as u32
863 {
864 tracing::warn!(
865 "Not enough core parallelism to serve the task {} task_parallelism {} running_task_parallelism {} max_task_parallelism {}",
866 compact_task.task_id,
867 parallelism,
868 max_task_parallelism,
869 running_task_parallelism.load(Ordering::Relaxed),
870 );
871 let (compact_task, table_stats, object_timestamps) =
872 compact_done(
873 compact_task,
874 context.clone(),
875 vec![],
876 TaskStatus::NoAvailCpuResourceCanceled,
877 );
878
879 send_report_task_event(
880 &compact_task,
881 table_stats,
882 object_timestamps,
883 &request_sender,
884 );
885
886 continue 'consume_stream;
887 }
888
889 running_task_parallelism
890 .fetch_add(parallelism as u32, Ordering::SeqCst);
891 executor.spawn(async move {
892 let (tx, rx) = tokio::sync::oneshot::channel();
893 let task_id = compact_task.task_id;
894 shutdown.lock().unwrap().insert(task_id, tx);
895
896 let ((compact_task, table_stats, object_timestamps), _memory_tracker)= compactor_runner::compact(
897 context.clone(),
898 compact_task,
899 rx,
900 object_id_manager.clone(),
901 compaction_catalog_manager_ref.clone(),
902 )
903 .await;
904
905 shutdown.lock().unwrap().remove(&task_id);
906 running_task_parallelism.fetch_sub(parallelism as u32, Ordering::SeqCst);
907
908 send_report_task_event(
909 &compact_task,
910 table_stats,
911 object_timestamps,
912 &request_sender,
913 );
914
915 let enable_check_compaction_result =
916 context.storage_opts.check_compaction_result;
917 let need_check_task = !compact_task.sorted_output_ssts.is_empty() && compact_task.task_status == TaskStatus::Success;
918
919 if enable_check_compaction_result && need_check_task {
920 let compact_table_ids = compact_task.build_compact_table_ids();
921 match compaction_catalog_manager_ref.acquire(compact_table_ids.into_iter().collect()).await {
922 Ok(compaction_catalog_agent_ref) => {
923 match check_compaction_result(&compact_task, context.clone(), compaction_catalog_agent_ref).await
924 {
925 Err(e) => {
926 tracing::warn!(error = %e.as_report(), "Failed to check compaction task {}",compact_task.task_id);
927 }
928 Ok(true) => (),
929 Ok(false) => {
930 panic!("Failed to pass consistency check for result of compaction task:\n{:?}", compact_task_to_string(&compact_task));
931 }
932 }
933 },
934 Err(e) => {
935 tracing::warn!(error = %e.as_report(), "failed to acquire compaction catalog agent");
936 }
937 }
938 }
939 });
940 }
941 ResponseEvent::VacuumTask(_) => {
942 unreachable!("unexpected vacuum task");
943 }
944 ResponseEvent::FullScanTask(_) => {
945 unreachable!("unexpected scan task");
946 }
947 ResponseEvent::ValidationTask(validation_task) => {
948 let validation_task = ValidationTask::from(validation_task);
949 executor.spawn(async move {
950 validate_ssts(validation_task, context.sstable_store.clone())
951 .await;
952 });
953 }
954 ResponseEvent::CancelCompactTask(cancel_compact_task) => match shutdown
955 .lock()
956 .unwrap()
957 .remove(&cancel_compact_task.task_id)
958 {
959 Some(tx) => {
960 if tx.send(()).is_err() {
961 tracing::warn!(
962 "Cancellation of compaction task failed. task_id: {}",
963 cancel_compact_task.task_id
964 );
965 }
966 }
967 _ => {
968 tracing::warn!(
969 "Attempting to cancel non-existent compaction task. task_id: {}",
970 cancel_compact_task.task_id
971 );
972 }
973 },
974
975 ResponseEvent::PullTaskAck(_pull_task_ack) => {
976 pull_task_ack = true;
978 }
979 }
980 }
981 Some(Err(e)) => {
982 tracing::warn!("Failed to consume stream. {}", e.message());
983 continue 'start_stream;
984 }
985 _ => {
986 continue 'start_stream;
988 }
989 }
990 }
991 }
992 });
993
994 (join_handle, shutdown_tx)
995}
996
997#[must_use]
1000pub fn start_shared_compactor(
1001 grpc_proxy_client: GrpcCompactorProxyClient,
1002 mut receiver: mpsc::UnboundedReceiver<Request<DispatchCompactionTaskRequest>>,
1003 context: CompactorContext,
1004) -> (JoinHandle<()>, Sender<()>) {
1005 type CompactionShutdownMap = Arc<Mutex<HashMap<u64, Sender<()>>>>;
1006 let task_progress = context.task_progress_manager.clone();
1007 let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel();
1008 let periodic_event_update_interval = Duration::from_millis(1000);
1009
1010 let join_handle = tokio::spawn(async move {
1011 let shutdown_map = CompactionShutdownMap::default();
1012
1013 let mut periodic_event_interval = tokio::time::interval(periodic_event_update_interval);
1014 let executor = context.compaction_executor.clone();
1015 let report_heartbeat_client = grpc_proxy_client.clone();
1016 'consume_stream: loop {
1017 let request: Option<Request<DispatchCompactionTaskRequest>> = tokio::select! {
1018 _ = periodic_event_interval.tick() => {
1019 let progress_list = get_task_progress(task_progress.clone());
1020 let report_compaction_task_request = ReportCompactionTaskRequest{
1021 event: Some(ReportCompactionTaskEvent::HeartBeat(
1022 SharedHeartBeat {
1023 progress: progress_list
1024 }
1025 )),
1026 };
1027 if let Err(e) = report_heartbeat_client.report_compaction_task(report_compaction_task_request).await{
1028 tracing::warn!(error = %e.as_report(), "Failed to report heartbeat");
1029 }
1030 continue
1031 }
1032
1033
1034 _ = &mut shutdown_rx => {
1035 tracing::info!("Compactor is shutting down");
1036 return
1037 }
1038
1039 request = receiver.recv() => {
1040 request
1041 }
1042
1043 };
1044 match request {
1045 Some(request) => {
1046 let context = context.clone();
1047 let shutdown = shutdown_map.clone();
1048
1049 let cloned_grpc_proxy_client = grpc_proxy_client.clone();
1050 executor.spawn(async move {
1051 let DispatchCompactionTaskRequest {
1052 tables,
1053 output_object_ids,
1054 task: dispatch_task,
1055 } = request.into_inner();
1056 let table_id_to_catalog = tables.into_iter().fold(HashMap::new(), |mut acc, table| {
1057 acc.insert(table.id, table);
1058 acc
1059 });
1060
1061 let mut output_object_ids_deque: VecDeque<_> = VecDeque::new();
1062 output_object_ids_deque.extend(output_object_ids.into_iter().map(Into::<HummockSstableObjectId>::into));
1063 let shared_compactor_object_id_manager =
1064 SharedComapctorObjectIdManager::new(output_object_ids_deque, cloned_grpc_proxy_client.clone(), context.storage_opts.sstable_id_remote_fetch_number);
1065 match dispatch_task.unwrap() {
1066 dispatch_compaction_task_request::Task::CompactTask(compact_task) => {
1067 let compact_task = CompactTask::from(&compact_task);
1068 let (tx, rx) = tokio::sync::oneshot::channel();
1069 let task_id = compact_task.task_id;
1070 shutdown.lock().unwrap().insert(task_id, tx);
1071
1072 let compaction_catalog_agent_ref = CompactionCatalogManager::build_compaction_catalog_agent(table_id_to_catalog);
1073 let ((compact_task, table_stats, object_timestamps), _memory_tracker)= compactor_runner::compact_with_agent(
1074 context.clone(),
1075 compact_task,
1076 rx,
1077 shared_compactor_object_id_manager,
1078 compaction_catalog_agent_ref.clone(),
1079 )
1080 .await;
1081 shutdown.lock().unwrap().remove(&task_id);
1082 let report_compaction_task_request = ReportCompactionTaskRequest {
1083 event: Some(ReportCompactionTaskEvent::ReportTask(ReportSharedTask {
1084 compact_task: Some(PbCompactTask::from(&compact_task)),
1085 table_stats_change: to_prost_table_stats_map(table_stats),
1086 object_timestamps: object_timestamps
1087 .into_iter()
1088 .map(|(object_id, timestamp)| (object_id.inner(), timestamp))
1089 .collect(),
1090 })),
1091 };
1092
1093 match cloned_grpc_proxy_client
1094 .report_compaction_task(report_compaction_task_request)
1095 .await
1096 {
1097 Ok(_) => {
1098 let enable_check_compaction_result = context.storage_opts.check_compaction_result;
1100 let need_check_task = !compact_task.sorted_output_ssts.is_empty() && compact_task.task_status == TaskStatus::Success;
1101 if enable_check_compaction_result && need_check_task {
1102 match check_compaction_result(&compact_task, context.clone(),compaction_catalog_agent_ref).await {
1103 Err(e) => {
1104 tracing::warn!(error = %e.as_report(), "Failed to check compaction task {}", task_id);
1105 },
1106 Ok(true) => (),
1107 Ok(false) => {
1108 panic!("Failed to pass consistency check for result of compaction task:\n{:?}", compact_task_to_string(&compact_task));
1109 }
1110 }
1111 }
1112 }
1113 Err(e) => tracing::warn!(error = %e.as_report(), "Failed to report task {task_id:?}"),
1114 }
1115
1116 }
1117 dispatch_compaction_task_request::Task::VacuumTask(_) => {
1118 unreachable!("unexpected vacuum task");
1119 }
1120 dispatch_compaction_task_request::Task::FullScanTask(_) => {
1121 unreachable!("unexpected scan task");
1122 }
1123 dispatch_compaction_task_request::Task::ValidationTask(validation_task) => {
1124 let validation_task = ValidationTask::from(validation_task);
1125 validate_ssts(validation_task, context.sstable_store.clone()).await;
1126 }
1127 dispatch_compaction_task_request::Task::CancelCompactTask(cancel_compact_task) => {
1128 match shutdown
1129 .lock()
1130 .unwrap()
1131 .remove(&cancel_compact_task.task_id)
1132 { Some(tx) => {
1133 if tx.send(()).is_err() {
1134 tracing::warn!(
1135 "Cancellation of compaction task failed. task_id: {}",
1136 cancel_compact_task.task_id
1137 );
1138 }
1139 } _ => {
1140 tracing::warn!(
1141 "Attempting to cancel non-existent compaction task. task_id: {}",
1142 cancel_compact_task.task_id
1143 );
1144 }}
1145 }
1146 }
1147 });
1148 }
1149 None => continue 'consume_stream,
1150 }
1151 }
1152 });
1153 (join_handle, shutdown_tx)
1154}
1155
1156fn get_task_progress(
1157 task_progress: Arc<
1158 parking_lot::lock_api::Mutex<parking_lot::RawMutex, HashMap<u64, Arc<TaskProgress>>>,
1159 >,
1160) -> Vec<CompactTaskProgress> {
1161 let mut progress_list = Vec::new();
1162 for (&task_id, progress) in &*task_progress.lock() {
1163 progress_list.push(progress.snapshot(task_id));
1164 }
1165 progress_list
1166}
1167
1168fn schedule_queued_tasks(
1170 task_queue: &mut IcebergTaskQueue,
1171 compactor_context: &CompactorContext,
1172 shutdown_map: &Arc<Mutex<HashMap<u64, tokio::sync::oneshot::Sender<()>>>>,
1173 task_completion_tx: &tokio::sync::mpsc::UnboundedSender<u64>,
1174) {
1175 while let Some(popped_task) = task_queue.pop() {
1176 let task_id = popped_task.meta.task_id;
1177
1178 let unique_ident = popped_task.runner.as_ref().map(|r| r.unique_ident());
1180
1181 let Some(runner) = popped_task.runner else {
1182 tracing::error!(
1183 task_id = task_id,
1184 "Popped task missing runner - this should not happen"
1185 );
1186 task_queue.finish_running(task_id);
1187 continue;
1188 };
1189
1190 let executor = compactor_context.compaction_executor.clone();
1191 let shutdown_map_clone = shutdown_map.clone();
1192 let completion_tx_clone = task_completion_tx.clone();
1193
1194 tracing::info!(
1195 task_id = task_id,
1196 unique_ident = ?unique_ident,
1197 required_parallelism = popped_task.meta.required_parallelism,
1198 "Starting iceberg compaction task from queue"
1199 );
1200
1201 executor.spawn(async move {
1202 let (tx, rx) = tokio::sync::oneshot::channel();
1203 {
1204 let mut shutdown_guard = shutdown_map_clone.lock().unwrap();
1205 shutdown_guard.insert(task_id, tx);
1206 }
1207
1208 let _cleanup_guard = scopeguard::guard(
1209 (task_id, shutdown_map_clone, completion_tx_clone),
1210 move |(task_id, shutdown_map, completion_tx)| {
1211 {
1212 let mut shutdown_guard = shutdown_map.lock().unwrap();
1213 shutdown_guard.remove(&task_id);
1214 }
1215 if completion_tx.send(task_id).is_err() {
1218 tracing::warn!(task_id = task_id, "Failed to notify task completion - main loop may have shut down");
1219 }
1220 },
1221 );
1222
1223 if let Err(e) = runner.compact(rx).await {
1224 tracing::warn!(error = %e.as_report(), "Failed to compact iceberg runner {}", task_id);
1225 }
1226 });
1227 }
1228}
1229
1230fn handle_meta_task_pulling(
1233 pull_task_ack: &mut bool,
1234 task_queue: &IcebergTaskQueue,
1235 max_task_parallelism: u32,
1236 max_pull_task_count: u32,
1237 request_sender: &mpsc::UnboundedSender<SubscribeIcebergCompactionEventRequest>,
1238 log_throttler: &mut LogThrottler<IcebergCompactionLogState>,
1239) -> bool {
1240 let mut pending_pull_task_count = 0;
1241 if *pull_task_ack {
1242 let current_running_parallelism = task_queue.running_parallelism_sum();
1244 pending_pull_task_count =
1245 (max_task_parallelism - current_running_parallelism).min(max_pull_task_count);
1246
1247 if pending_pull_task_count > 0 {
1248 if let Err(e) = request_sender.send(SubscribeIcebergCompactionEventRequest {
1249 event: Some(subscribe_iceberg_compaction_event_request::Event::PullTask(
1250 subscribe_iceberg_compaction_event_request::PullTask {
1251 pull_task_count: pending_pull_task_count,
1252 },
1253 )),
1254 create_at: SystemTime::now()
1255 .duration_since(std::time::UNIX_EPOCH)
1256 .expect("Clock may have gone backwards")
1257 .as_millis() as u64,
1258 }) {
1259 tracing::warn!(error = %e.as_report(), "Failed to pull task - will retry on stream restart");
1260 return true; } else {
1262 *pull_task_ack = false;
1263 }
1264 }
1265 }
1266
1267 let running_count = task_queue.running_parallelism_sum();
1268 let waiting_count = task_queue.waiting_parallelism_sum();
1269 let available_count = max_task_parallelism.saturating_sub(running_count);
1270 let current_state = IcebergCompactionLogState {
1271 running_parallelism: running_count,
1272 waiting_parallelism: waiting_count,
1273 available_parallelism: available_count,
1274 pull_task_ack: *pull_task_ack,
1275 pending_pull_task_count,
1276 };
1277
1278 if log_throttler.should_log(¤t_state) {
1280 tracing::info!(
1281 running_parallelism_count = %current_state.running_parallelism,
1282 waiting_parallelism_count = %current_state.waiting_parallelism,
1283 available_parallelism = %current_state.available_parallelism,
1284 pull_task_ack = %current_state.pull_task_ack,
1285 pending_pull_task_count = %current_state.pending_pull_task_count
1286 );
1287 log_throttler.update(current_state);
1288 }
1289
1290 false }
1292
1293#[cfg(test)]
1294mod tests {
1295 use super::*;
1296
1297 #[test]
1298 fn test_log_state_equality() {
1299 let state1 = CompactionLogState {
1301 running_parallelism: 10,
1302 pull_task_ack: true,
1303 pending_pull_task_count: 2,
1304 };
1305 let state2 = CompactionLogState {
1306 running_parallelism: 10,
1307 pull_task_ack: true,
1308 pending_pull_task_count: 2,
1309 };
1310 let state3 = CompactionLogState {
1311 running_parallelism: 11,
1312 pull_task_ack: true,
1313 pending_pull_task_count: 2,
1314 };
1315 assert_eq!(state1, state2);
1316 assert_ne!(state1, state3);
1317
1318 let ice_state1 = IcebergCompactionLogState {
1320 running_parallelism: 10,
1321 waiting_parallelism: 5,
1322 available_parallelism: 15,
1323 pull_task_ack: true,
1324 pending_pull_task_count: 2,
1325 };
1326 let ice_state2 = IcebergCompactionLogState {
1327 running_parallelism: 10,
1328 waiting_parallelism: 6,
1329 available_parallelism: 15,
1330 pull_task_ack: true,
1331 pending_pull_task_count: 2,
1332 };
1333 assert_ne!(ice_state1, ice_state2);
1334 }
1335
1336 #[test]
1337 fn test_log_throttler_state_change_detection() {
1338 let mut throttler = LogThrottler::<CompactionLogState>::new(Duration::from_secs(60));
1339 let state1 = CompactionLogState {
1340 running_parallelism: 10,
1341 pull_task_ack: true,
1342 pending_pull_task_count: 2,
1343 };
1344 let state2 = CompactionLogState {
1345 running_parallelism: 11,
1346 pull_task_ack: true,
1347 pending_pull_task_count: 2,
1348 };
1349
1350 assert!(throttler.should_log(&state1));
1352 throttler.update(state1.clone());
1353
1354 assert!(!throttler.should_log(&state1));
1356
1357 assert!(throttler.should_log(&state2));
1359 throttler.update(state2.clone());
1360
1361 assert!(!throttler.should_log(&state2));
1363 }
1364
1365 #[test]
1366 fn test_log_throttler_heartbeat() {
1367 let mut throttler = LogThrottler::<CompactionLogState>::new(Duration::from_millis(10));
1368 let state = CompactionLogState {
1369 running_parallelism: 10,
1370 pull_task_ack: true,
1371 pending_pull_task_count: 2,
1372 };
1373
1374 assert!(throttler.should_log(&state));
1376 throttler.update(state.clone());
1377
1378 assert!(!throttler.should_log(&state));
1380
1381 std::thread::sleep(Duration::from_millis(15));
1383
1384 assert!(throttler.should_log(&state));
1386 }
1387}