1use std::collections::HashMap;
16use std::sync::Arc;
17use std::time::Instant;
18
19use anyhow::anyhow;
20use iceberg::spec::Operation;
21use iceberg::transaction::Transaction;
22use itertools::Itertools;
23use parking_lot::RwLock;
24use risingwave_common::bail;
25use risingwave_connector::connector_common::IcebergSinkCompactionUpdate;
26use risingwave_connector::sink::catalog::{SinkCatalog, SinkId, SinkType};
27use risingwave_connector::sink::iceberg::{IcebergConfig, should_enable_iceberg_cow};
28use risingwave_connector::sink::{SinkError, SinkParam};
29use risingwave_pb::catalog::PbSink;
30use risingwave_pb::iceberg_compaction::iceberg_compaction_task::TaskType;
31use risingwave_pb::iceberg_compaction::{
32 IcebergCompactionTask, SubscribeIcebergCompactionEventRequest,
33};
34use thiserror_ext::AsReport;
35use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender};
36use tokio::sync::oneshot::Sender;
37use tokio::task::JoinHandle;
38use tonic::Streaming;
39
40use super::MetaSrvEnv;
41use crate::MetaResult;
42use crate::hummock::{
43 IcebergCompactionEventDispatcher, IcebergCompactionEventHandler, IcebergCompactionEventLoop,
44 IcebergCompactor, IcebergCompactorManagerRef,
45};
46use crate::manager::MetadataManager;
47use crate::rpc::metrics::MetaMetrics;
48
49pub type IcebergCompactionManagerRef = std::sync::Arc<IcebergCompactionManager>;
50
51type CompactorChangeTx = UnboundedSender<(u32, Streaming<SubscribeIcebergCompactionEventRequest>)>;
52
53type CompactorChangeRx =
54 UnboundedReceiver<(u32, Streaming<SubscribeIcebergCompactionEventRequest>)>;
55
56#[derive(Debug, Clone)]
57struct CommitInfo {
58 count: usize,
59 next_compaction_time: Option<Instant>,
60 compaction_interval: u64,
61}
62
63impl CommitInfo {
64 fn set_processing(&mut self) {
65 self.count = 0;
66 self.next_compaction_time.take();
68 }
69
70 fn initialize(&mut self) {
71 self.count = 0;
72 self.next_compaction_time =
73 Some(Instant::now() + std::time::Duration::from_secs(self.compaction_interval));
74 }
75
76 fn replace(&mut self, commit_info: CommitInfo) {
77 self.count = commit_info.count;
78 self.next_compaction_time = commit_info.next_compaction_time;
79 self.compaction_interval = commit_info.compaction_interval;
80 }
81
82 fn increase_count(&mut self) {
83 self.count += 1;
84 }
85
86 fn update_compaction_interval(&mut self, compaction_interval: u64) {
87 self.compaction_interval = compaction_interval;
88
89 self.next_compaction_time =
91 Some(Instant::now() + std::time::Duration::from_secs(compaction_interval));
92 }
93}
94
95pub struct IcebergCompactionHandle {
96 sink_id: SinkId,
97 inner: Arc<RwLock<IcebergCompactionManagerInner>>,
98 metadata_manager: MetadataManager,
99 handle_success: bool,
100
101 commit_info: CommitInfo,
103}
104
105impl IcebergCompactionHandle {
106 fn new(
107 sink_id: SinkId,
108 inner: Arc<RwLock<IcebergCompactionManagerInner>>,
109 metadata_manager: MetadataManager,
110 commit_info: CommitInfo,
111 ) -> Self {
112 Self {
113 sink_id,
114 inner,
115 metadata_manager,
116 handle_success: false,
117 commit_info,
118 }
119 }
120
121 pub async fn send_compact_task(
122 mut self,
123 compactor: Arc<IcebergCompactor>,
124 task_id: u64,
125 ) -> MetaResult<()> {
126 use risingwave_pb::iceberg_compaction::subscribe_iceberg_compaction_event_response::Event as IcebergResponseEvent;
127 let mut sinks = self
128 .metadata_manager
129 .catalog_controller
130 .get_sink_by_ids(vec![self.sink_id.sink_id as i32])
131 .await?;
132 if sinks.is_empty() {
133 tracing::warn!("Sink not found: {}", self.sink_id.sink_id);
135 return Ok(());
136 }
137 let prost_sink_catalog: PbSink = sinks.remove(0);
138 let sink_catalog = SinkCatalog::from(prost_sink_catalog);
139 let param = SinkParam::try_from_sink_catalog(sink_catalog)?;
140 let task_type: TaskType = match param.sink_type {
141 SinkType::AppendOnly | SinkType::ForceAppendOnly => {
142 if risingwave_common::license::Feature::IcebergCompaction
143 .check_available()
144 .is_ok()
145 {
146 TaskType::SmallDataFileCompaction
147 } else {
148 TaskType::FullCompaction
149 }
150 }
151
152 _ => TaskType::FullCompaction,
153 };
154 let result =
155 compactor.send_event(IcebergResponseEvent::CompactTask(IcebergCompactionTask {
156 task_id,
157 props: param.properties,
158 task_type: task_type as i32,
159 }));
160
161 if result.is_ok() {
162 self.handle_success = true;
163 }
164
165 result
166 }
167
168 pub fn sink_id(&self) -> SinkId {
169 self.sink_id
170 }
171}
172
173impl Drop for IcebergCompactionHandle {
174 fn drop(&mut self) {
175 if self.handle_success {
176 let mut guard = self.inner.write();
177 if let Some(commit_info) = guard.iceberg_commits.get_mut(&self.sink_id) {
178 commit_info.initialize();
179 }
180 } else {
181 let mut guard = self.inner.write();
186 if let Some(commit_info) = guard.iceberg_commits.get_mut(&self.sink_id) {
187 commit_info.replace(self.commit_info.clone());
188 }
189 }
190 }
191}
192
193struct IcebergCompactionManagerInner {
194 pub iceberg_commits: HashMap<SinkId, CommitInfo>,
195}
196
197pub struct IcebergCompactionManager {
198 pub env: MetaSrvEnv,
199 inner: Arc<RwLock<IcebergCompactionManagerInner>>,
200
201 metadata_manager: MetadataManager,
202 pub iceberg_compactor_manager: IcebergCompactorManagerRef,
203
204 compactor_streams_change_tx: CompactorChangeTx,
205
206 pub metrics: Arc<MetaMetrics>,
207}
208
209impl IcebergCompactionManager {
210 pub fn build(
211 env: MetaSrvEnv,
212 metadata_manager: MetadataManager,
213 iceberg_compactor_manager: IcebergCompactorManagerRef,
214 metrics: Arc<MetaMetrics>,
215 ) -> (Arc<Self>, CompactorChangeRx) {
216 let (compactor_streams_change_tx, compactor_streams_change_rx) =
217 tokio::sync::mpsc::unbounded_channel();
218 (
219 Arc::new(Self {
220 env,
221 inner: Arc::new(RwLock::new(IcebergCompactionManagerInner {
222 iceberg_commits: HashMap::default(),
223 })),
224 metadata_manager,
225 iceberg_compactor_manager,
226 compactor_streams_change_tx,
227 metrics,
228 }),
229 compactor_streams_change_rx,
230 )
231 }
232
233 pub fn compaction_stat_loop(
234 manager: Arc<Self>,
235 mut rx: UnboundedReceiver<IcebergSinkCompactionUpdate>,
236 ) -> (JoinHandle<()>, Sender<()>) {
237 let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel();
238 let join_handle = tokio::spawn(async move {
239 loop {
240 tokio::select! {
241 Some(stat) = rx.recv() => {
242 manager.update_iceberg_commit_info(stat);
243 },
244 _ = &mut shutdown_rx => {
245 tracing::info!("Iceberg compaction manager is stopped");
246 return;
247 }
248 }
249 }
250 });
251
252 (join_handle, shutdown_tx)
253 }
254
255 pub fn update_iceberg_commit_info(&self, msg: IcebergSinkCompactionUpdate) {
256 let mut guard = self.inner.write();
257
258 let IcebergSinkCompactionUpdate {
259 sink_id,
260 compaction_interval,
261 force_compaction,
262 } = msg;
263
264 let compaction_interval = if force_compaction {
265 0
266 } else {
267 compaction_interval
268 };
269
270 let commit_info = guard.iceberg_commits.entry(sink_id).or_insert(CommitInfo {
272 count: 0,
273 next_compaction_time: Some(
274 Instant::now() + std::time::Duration::from_secs(compaction_interval),
275 ),
276 compaction_interval,
277 });
278
279 commit_info.increase_count();
280 if commit_info.compaction_interval != compaction_interval {
281 commit_info.update_compaction_interval(compaction_interval);
282 }
283 }
284
285 pub fn get_top_n_iceberg_commit_sink_ids(&self, n: usize) -> Vec<IcebergCompactionHandle> {
288 let now = Instant::now();
289 let mut guard = self.inner.write();
290 guard
291 .iceberg_commits
292 .iter_mut()
293 .filter(|(_, commit_info)| {
294 commit_info.count > 0
295 && if let Some(next_compaction_time) = commit_info.next_compaction_time {
296 next_compaction_time <= now
297 } else {
298 false
299 }
300 })
301 .sorted_by(|a, b| {
302 b.1.count
303 .cmp(&a.1.count)
304 .then_with(|| b.1.next_compaction_time.cmp(&a.1.next_compaction_time))
305 })
306 .take(n)
307 .map(|(sink_id, commit_info)| {
308 let handle = IcebergCompactionHandle::new(
310 *sink_id,
311 self.inner.clone(),
312 self.metadata_manager.clone(),
313 commit_info.clone(),
314 );
315
316 commit_info.set_processing();
317
318 handle
319 })
320 .collect::<Vec<_>>()
321 }
322
323 pub fn clear_iceberg_commits_by_sink_id(&self, sink_id: SinkId) {
324 let mut guard = self.inner.write();
325 guard.iceberg_commits.remove(&sink_id);
326 }
327
328 pub async fn get_sink_param(&self, sink_id: &SinkId) -> MetaResult<SinkParam> {
329 let mut sinks = self
330 .metadata_manager
331 .catalog_controller
332 .get_sink_by_ids(vec![sink_id.sink_id as i32])
333 .await?;
334 if sinks.is_empty() {
335 bail!("Sink not found: {}", sink_id.sink_id);
336 }
337 let prost_sink_catalog: PbSink = sinks.remove(0);
338 let sink_catalog = SinkCatalog::from(prost_sink_catalog);
339 let param = SinkParam::try_from_sink_catalog(sink_catalog)?;
340 Ok(param)
341 }
342
343 pub async fn load_iceberg_config(&self, sink_id: &SinkId) -> MetaResult<IcebergConfig> {
344 let sink_param = self.get_sink_param(sink_id).await?;
345 let iceberg_config = IcebergConfig::from_btreemap(sink_param.properties.clone())?;
346 Ok(iceberg_config)
347 }
348
349 pub fn add_compactor_stream(
350 &self,
351 context_id: u32,
352 req_stream: Streaming<SubscribeIcebergCompactionEventRequest>,
353 ) {
354 self.compactor_streams_change_tx
355 .send((context_id, req_stream))
356 .unwrap();
357 }
358
359 pub fn iceberg_compaction_event_loop(
360 iceberg_compaction_manager: Arc<Self>,
361 compactor_streams_change_rx: UnboundedReceiver<(
362 u32,
363 Streaming<SubscribeIcebergCompactionEventRequest>,
364 )>,
365 ) -> Vec<(JoinHandle<()>, Sender<()>)> {
366 let mut join_handle_vec = Vec::default();
367
368 let iceberg_compaction_event_handler =
369 IcebergCompactionEventHandler::new(iceberg_compaction_manager.clone());
370
371 let iceberg_compaction_event_dispatcher =
372 IcebergCompactionEventDispatcher::new(iceberg_compaction_event_handler);
373
374 let event_loop = IcebergCompactionEventLoop::new(
375 iceberg_compaction_event_dispatcher,
376 iceberg_compaction_manager.metrics.clone(),
377 compactor_streams_change_rx,
378 );
379
380 let (event_loop_join_handle, event_loop_shutdown_tx) = event_loop.run();
381 join_handle_vec.push((event_loop_join_handle, event_loop_shutdown_tx));
382
383 join_handle_vec
384 }
385
386 pub fn gc_loop(manager: Arc<Self>) -> (JoinHandle<()>, Sender<()>) {
390 let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel();
391 let join_handle = tokio::spawn(async move {
392 const GC_LOOP_INTERVAL_SECS: u64 = 3600;
394 let mut interval =
395 tokio::time::interval(std::time::Duration::from_secs(GC_LOOP_INTERVAL_SECS));
396
397 loop {
398 tokio::select! {
399 _ = interval.tick() => {
400 if let Err(e) = manager.perform_gc_operations().await {
401 tracing::error!(error = ?e.as_report(), "GC operations failed");
402 }
403 },
404 _ = &mut shutdown_rx => {
405 tracing::info!("Iceberg GC loop is stopped");
406 return;
407 }
408 }
409 }
410 });
411
412 (join_handle, shutdown_tx)
413 }
414
415 pub async fn trigger_manual_compaction(&self, sink_id: SinkId) -> MetaResult<u64> {
418 use risingwave_pb::iceberg_compaction::subscribe_iceberg_compaction_event_response::Event as IcebergResponseEvent;
419
420 let iceberg_config = self.load_iceberg_config(&sink_id).await?;
422 let initial_table = iceberg_config.load_table().await?;
423 let initial_snapshot_id = initial_table
424 .metadata()
425 .current_snapshot()
426 .map(|s| s.snapshot_id())
427 .unwrap_or(0); let initial_timestamp = chrono::Utc::now().timestamp_millis();
429
430 let compactor = self
432 .iceberg_compactor_manager
433 .next_compactor()
434 .ok_or_else(|| anyhow!("No iceberg compactor available"))?;
435
436 let task_id = self
438 .env
439 .hummock_seq
440 .next_interval("compaction_task", 1)
441 .await?;
442
443 let sink_param = self.get_sink_param(&sink_id).await?;
444
445 compactor.send_event(IcebergResponseEvent::CompactTask(IcebergCompactionTask {
446 task_id,
447 props: sink_param.properties,
448 task_type: TaskType::FullCompaction as i32, }))?;
450
451 tracing::info!(
452 "Manual compaction triggered for sink {} with task ID {}, waiting for completion...",
453 sink_id.sink_id,
454 task_id
455 );
456
457 self.wait_for_compaction_completion(
458 &sink_id,
459 iceberg_config,
460 initial_snapshot_id,
461 initial_timestamp,
462 task_id,
463 )
464 .await?;
465
466 Ok(task_id)
467 }
468
469 async fn wait_for_compaction_completion(
470 &self,
471 sink_id: &SinkId,
472 iceberg_config: IcebergConfig,
473 initial_snapshot_id: i64,
474 initial_timestamp: i64,
475 task_id: u64,
476 ) -> MetaResult<()> {
477 const INITIAL_POLL_INTERVAL_SECS: u64 = 2;
478 const MAX_POLL_INTERVAL_SECS: u64 = 60;
479 const MAX_WAIT_TIME_SECS: u64 = 1800;
480 const BACKOFF_MULTIPLIER: f64 = 1.5;
481
482 let mut elapsed_time = 0;
483 let mut current_interval_secs = INITIAL_POLL_INTERVAL_SECS;
484
485 let cow = should_enable_iceberg_cow(
486 iceberg_config.r#type.as_str(),
487 iceberg_config.write_mode.as_str(),
488 );
489
490 while elapsed_time < MAX_WAIT_TIME_SECS {
491 let poll_interval = std::time::Duration::from_secs(current_interval_secs);
492 tokio::time::sleep(poll_interval).await;
493 elapsed_time += current_interval_secs;
494
495 let current_table = iceberg_config.load_table().await?;
496
497 let metadata = current_table.metadata();
498 let new_snapshots: Vec<_> = metadata
499 .snapshots()
500 .filter(|snapshot| {
501 let snapshot_timestamp = snapshot.timestamp_ms();
502 let snapshot_id = snapshot.snapshot_id();
503 snapshot_timestamp > initial_timestamp && snapshot_id != initial_snapshot_id
504 })
505 .collect();
506
507 for snapshot in new_snapshots {
508 let summary = snapshot.summary();
509 if cow {
510 if matches!(summary.operation, Operation::Overwrite) {
511 return Ok(());
512 }
513 } else if matches!(summary.operation, Operation::Replace) {
514 return Ok(());
515 }
516 }
517
518 current_interval_secs = std::cmp::min(
519 MAX_POLL_INTERVAL_SECS,
520 ((current_interval_secs as f64) * BACKOFF_MULTIPLIER) as u64,
521 );
522 }
523
524 Err(anyhow!(
525 "Compaction did not complete within {} seconds for sink {} (task_id={})",
526 MAX_WAIT_TIME_SECS,
527 sink_id.sink_id,
528 task_id
529 )
530 .into())
531 }
532
533 async fn perform_gc_operations(&self) -> MetaResult<()> {
534 let sink_ids = {
535 let guard = self.inner.read();
536 guard.iceberg_commits.keys().cloned().collect::<Vec<_>>()
537 };
538
539 tracing::info!("Starting GC operations for {} tables", sink_ids.len());
540
541 for sink_id in sink_ids {
542 if let Err(e) = self.check_and_expire_snapshots(&sink_id).await {
543 tracing::error!(error = ?e.as_report(), "Failed to perform GC for sink {}", sink_id.sink_id);
544 }
545 }
546
547 tracing::info!("GC operations completed");
548 Ok(())
549 }
550
551 pub async fn check_and_expire_snapshots(&self, sink_id: &SinkId) -> MetaResult<()> {
552 const MAX_SNAPSHOT_AGE_MS_DEFAULT: i64 = 24 * 60 * 60 * 1000; let now = chrono::Utc::now().timestamp_millis();
554
555 let iceberg_config = self.load_iceberg_config(sink_id).await?;
556 if !iceberg_config.enable_snapshot_expiration {
557 return Ok(());
558 }
559
560 let catalog = iceberg_config.create_catalog().await?;
561 let table = catalog
562 .load_table(&iceberg_config.full_table_name()?)
563 .await
564 .map_err(|e| SinkError::Iceberg(e.into()))?;
565
566 let metadata = table.metadata();
567 let mut snapshots = metadata.snapshots().collect_vec();
568 snapshots.sort_by_key(|s| s.timestamp_ms());
569
570 let default_snapshot_expiration_timestamp_ms = now - MAX_SNAPSHOT_AGE_MS_DEFAULT;
571
572 let snapshot_expiration_timestamp_ms =
573 match iceberg_config.snapshot_expiration_timestamp_ms(now) {
574 Some(timestamp) => timestamp,
575 None => default_snapshot_expiration_timestamp_ms,
576 };
577
578 if snapshots.is_empty()
579 || snapshots.first().unwrap().timestamp_ms() > snapshot_expiration_timestamp_ms
580 {
581 return Ok(());
583 }
584
585 tracing::info!(
586 catalog_name = iceberg_config.catalog_name(),
587 table_name = iceberg_config.full_table_name()?.to_string(),
588 sink_id = sink_id.sink_id,
589 snapshots_len = snapshots.len(),
590 snapshot_expiration_timestamp_ms = snapshot_expiration_timestamp_ms,
591 snapshot_expiration_retain_last = ?iceberg_config.snapshot_expiration_retain_last,
592 clear_expired_files = ?iceberg_config.snapshot_expiration_clear_expired_files,
593 clear_expired_meta_data = ?iceberg_config.snapshot_expiration_clear_expired_meta_data,
594 "try trigger snapshots expiration",
595 );
596
597 let tx = Transaction::new(&table);
598
599 let mut expired_snapshots = tx.expire_snapshot();
600
601 expired_snapshots = expired_snapshots.expire_older_than(snapshot_expiration_timestamp_ms);
602
603 if let Some(retain_last) = iceberg_config.snapshot_expiration_retain_last {
604 expired_snapshots = expired_snapshots.retain_last(retain_last);
605 }
606
607 expired_snapshots = expired_snapshots
608 .clear_expired_files(iceberg_config.snapshot_expiration_clear_expired_files);
609
610 expired_snapshots = expired_snapshots
611 .clear_expired_meta_data(iceberg_config.snapshot_expiration_clear_expired_meta_data);
612
613 let tx = expired_snapshots
614 .apply()
615 .await
616 .map_err(|e| SinkError::Iceberg(e.into()))?;
617
618 tx.commit(catalog.as_ref())
619 .await
620 .map_err(|e| SinkError::Iceberg(e.into()))?;
621
622 tracing::info!(
623 catalog_name = iceberg_config.catalog_name(),
624 table_name = iceberg_config.full_table_name()?.to_string(),
625 sink_id = sink_id.sink_id,
626 "Expired snapshots for iceberg table",
627 );
628
629 Ok(())
630 }
631}