1use std::collections::BTreeSet;
16use std::sync::Arc;
17use std::sync::atomic::{AtomicU32, Ordering};
18use std::time::SystemTime;
19
20use anyhow::anyhow;
21use async_trait::async_trait;
22use fail::fail_point;
23use futures::stream::BoxStream;
24use futures::{Stream, StreamExt};
25use itertools::Itertools;
26use risingwave_common::catalog::TableId;
27use risingwave_hummock_sdk::change_log::build_table_change_log_delta;
28use risingwave_hummock_sdk::compact_task::CompactTask;
29use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId;
30use risingwave_hummock_sdk::sstable_info::SstableInfo;
31use risingwave_hummock_sdk::version::HummockVersion;
32use risingwave_hummock_sdk::{
33 HummockContextId, HummockEpoch, HummockVersionId, LocalSstableInfo, ObjectIdRange, SyncResult,
34};
35use risingwave_pb::common::{HostAddress, WorkerType};
36use risingwave_pb::hummock::compact_task::{TaskStatus, TaskType};
37use risingwave_pb::hummock::subscribe_compaction_event_request::{Event, ReportTask};
38use risingwave_pb::hummock::subscribe_compaction_event_response::Event as ResponseEvent;
39use risingwave_pb::hummock::{
40 PbHummockVersion, SubscribeCompactionEventRequest, SubscribeCompactionEventResponse,
41 compact_task,
42};
43use risingwave_pb::iceberg_compaction::SubscribeIcebergCompactionEventRequest;
44use risingwave_rpc_client::error::{Result, RpcError};
45use risingwave_rpc_client::{
46 CompactionEventItem, HummockMetaClient, HummockMetaClientChangeLogInfo,
47 IcebergCompactionEventItem,
48};
49use thiserror_ext::AsReport;
50use tokio::sync::mpsc::{UnboundedSender, unbounded_channel};
51use tokio::task::JoinHandle;
52use tokio_stream::wrappers::UnboundedReceiverStream;
53
54use crate::hummock::compaction::selector::{
55 CompactionSelector, SpaceReclaimCompactionSelector, default_compaction_selector,
56};
57use crate::hummock::error::Error;
58use crate::hummock::{CommitEpochInfo, HummockManager, NewTableFragmentInfo};
59
60pub struct MockHummockMetaClient {
61 hummock_manager: Arc<HummockManager>,
62 context_id: HummockContextId,
63 compact_context_id: AtomicU32,
64 sst_offset: u64,
66}
67
68impl MockHummockMetaClient {
69 pub fn new(
70 hummock_manager: Arc<HummockManager>,
71 context_id: HummockContextId,
72 ) -> MockHummockMetaClient {
73 MockHummockMetaClient {
74 hummock_manager,
75 context_id,
76 compact_context_id: AtomicU32::new(context_id),
77 sst_offset: 0,
78 }
79 }
80
81 pub fn with_sst_offset(
82 hummock_manager: Arc<HummockManager>,
83 context_id: HummockContextId,
84 sst_offset: u64,
85 ) -> Self {
86 Self {
87 hummock_manager,
88 context_id,
89 compact_context_id: AtomicU32::new(context_id),
90 sst_offset,
91 }
92 }
93
94 pub async fn get_compact_task(&self) -> Option<CompactTask> {
95 self.hummock_manager
96 .get_compact_task(
97 StaticCompactionGroupId::StateDefault.into(),
98 &mut default_compaction_selector(),
99 )
100 .await
101 .unwrap_or(None)
102 }
103
104 pub fn context_id(&self) -> HummockContextId {
105 self.context_id
106 }
107}
108
109fn mock_err(error: super::error::Error) -> RpcError {
110 anyhow!(error).context("mock error").into()
111}
112
113#[async_trait]
114impl HummockMetaClient for MockHummockMetaClient {
115 async fn unpin_version_before(&self, unpin_version_before: HummockVersionId) -> Result<()> {
116 self.hummock_manager
117 .unpin_version_before(self.context_id, unpin_version_before)
118 .await
119 .map_err(mock_err)
120 }
121
122 async fn get_current_version(&self) -> Result<HummockVersion> {
123 Ok(self.hummock_manager.get_current_version().await)
124 }
125
126 async fn get_new_object_ids(&self, number: u32) -> Result<ObjectIdRange> {
127 fail_point!("get_new_sst_ids_err", |_| Err(anyhow!(
128 "failpoint get_new_sst_ids_err"
129 )
130 .into()));
131 self.hummock_manager
132 .get_new_object_ids(number)
133 .await
134 .map_err(mock_err)
135 .map(|range| ObjectIdRange {
136 start_id: range.start_id + self.sst_offset,
137 end_id: range.end_id + self.sst_offset,
138 })
139 }
140
141 async fn commit_epoch_with_change_log(
142 &self,
143 epoch: HummockEpoch,
144 sync_result: SyncResult,
145 change_log_info: Option<HummockMetaClientChangeLogInfo>,
146 ) -> Result<()> {
147 let version: HummockVersion = self.hummock_manager.get_current_version().await;
148 let table_ids = version
149 .state_table_info
150 .info()
151 .keys()
152 .map(|table_id| table_id.table_id)
153 .collect::<BTreeSet<_>>();
154
155 let commit_table_ids = sync_result
156 .uncommitted_ssts
157 .iter()
158 .flat_map(|sstable| sstable.sst_info.table_ids.clone())
159 .chain({
160 sync_result
161 .old_value_ssts
162 .iter()
163 .flat_map(|sstable| sstable.sst_info.table_ids.clone())
164 })
165 .chain(
166 sync_result
167 .table_watermarks
168 .keys()
169 .map(|table_id| table_id.table_id),
170 )
171 .chain(table_ids.iter().cloned())
172 .collect::<BTreeSet<_>>();
173
174 let new_table_fragment_infos = if commit_table_ids
175 .iter()
176 .all(|table_id| table_ids.contains(table_id))
177 {
178 vec![]
179 } else {
180 vec![NewTableFragmentInfo {
181 table_ids: commit_table_ids
182 .iter()
183 .cloned()
184 .map(TableId::from)
185 .collect(),
186 }]
187 };
188
189 let sst_to_context = sync_result
190 .uncommitted_ssts
191 .iter()
192 .map(|LocalSstableInfo { sst_info, .. }| (sst_info.object_id, self.context_id))
193 .collect();
194 let new_table_watermark = sync_result.table_watermarks;
195 let table_change_log = match change_log_info {
196 Some(epochs) => {
197 assert_eq!(*epochs.last().expect("non-empty"), epoch);
198 build_table_change_log_delta(
199 sync_result
200 .old_value_ssts
201 .into_iter()
202 .map(|sst| sst.sst_info),
203 sync_result.uncommitted_ssts.iter().map(|sst| &sst.sst_info),
204 &epochs,
205 commit_table_ids.iter().map(|&table_id| (table_id, 0)),
206 )
207 }
208 None => Default::default(),
209 };
210
211 self.hummock_manager
212 .commit_epoch(CommitEpochInfo {
213 sstables: sync_result.uncommitted_ssts,
214 new_table_watermarks: new_table_watermark,
215 sst_to_context,
216 new_table_fragment_infos,
217 change_log_delta: table_change_log,
218 tables_to_commit: commit_table_ids
219 .iter()
220 .cloned()
221 .map(|table_id| (TableId::new(table_id), epoch))
222 .collect(),
223 })
224 .await
225 .map_err(mock_err)?;
226 Ok(())
227 }
228
229 async fn trigger_manual_compaction(
230 &self,
231 _compaction_group_id: u64,
232 _table_id: u32,
233 _level: u32,
234 _sst_ids: Vec<u64>,
235 ) -> Result<()> {
236 todo!()
237 }
238
239 async fn trigger_full_gc(
240 &self,
241 _sst_retention_time_sec: u64,
242 _prefix: Option<String>,
243 ) -> Result<()> {
244 unimplemented!()
245 }
246
247 async fn subscribe_compaction_event(
248 &self,
249 ) -> Result<(
250 UnboundedSender<SubscribeCompactionEventRequest>,
251 BoxStream<'static, CompactionEventItem>,
252 )> {
253 let context_id = self
254 .hummock_manager
255 .metadata_manager()
256 .add_worker_node(
257 WorkerType::Compactor,
258 HostAddress {
259 host: "compactor".to_owned(),
260 port: 0,
261 },
262 Default::default(),
263 Default::default(),
264 )
265 .await
266 .unwrap();
267 let _compactor_rx = self
268 .hummock_manager
269 .compactor_manager
270 .clone()
271 .add_compactor(context_id as _);
272
273 let (request_sender, mut request_receiver) =
274 unbounded_channel::<SubscribeCompactionEventRequest>();
275
276 self.compact_context_id
277 .store(context_id as _, Ordering::Release);
278
279 let (task_tx, task_rx) = tokio::sync::mpsc::unbounded_channel();
280
281 let hummock_manager_compact = self.hummock_manager.clone();
282 let mut join_handle_vec = vec![];
283
284 let handle = tokio::spawn(async move {
285 loop {
286 let group_and_type = hummock_manager_compact
287 .auto_pick_compaction_group_and_type()
288 .await;
289
290 if group_and_type.is_none() {
291 break;
292 }
293
294 let (group, task_type) = group_and_type.unwrap();
295
296 if let TaskType::Ttl = task_type {
297 match hummock_manager_compact
298 .metadata_manager_ref()
299 .get_all_table_options()
300 .await
301 .map_err(|err| Error::MetaStore(err.into()))
302 {
303 Ok(table_options) => {
304 hummock_manager_compact.update_table_id_to_table_option(table_options);
305 }
306 Err(e) => {
307 tracing::error!(error = %e.as_report(), "get_all_table_options fail");
308 }
309 }
310 }
311
312 let mut selector: Box<dyn CompactionSelector> = match task_type {
313 compact_task::TaskType::Dynamic => default_compaction_selector(),
314 compact_task::TaskType::SpaceReclaim => {
315 Box::<SpaceReclaimCompactionSelector>::default()
316 }
317
318 _ => panic!("Error type when mock_hummock_meta_client subscribe_compact_tasks"),
319 };
320 if let Some(task) = hummock_manager_compact
321 .get_compact_task(group, &mut selector)
322 .await
323 .unwrap()
324 {
325 let resp = SubscribeCompactionEventResponse {
326 event: Some(ResponseEvent::CompactTask(task.into())),
327 create_at: SystemTime::now()
328 .duration_since(std::time::UNIX_EPOCH)
329 .expect("Clock may have gone backwards")
330 .as_millis() as u64,
331 };
332
333 let _ = task_tx.send(Ok(resp));
334 }
335 }
336 });
337
338 join_handle_vec.push(handle);
339
340 let hummock_manager_compact = self.hummock_manager.clone();
341 let report_handle = tokio::spawn(async move {
342 tracing::info!("report_handle start");
343 loop {
344 if let Some(item) = request_receiver.recv().await {
345 if let Event::ReportTask(ReportTask {
346 task_id,
347 task_status,
348 sorted_output_ssts,
349 table_stats_change,
350 object_timestamps,
351 }) = item.event.unwrap()
352 {
353 if let Err(e) = hummock_manager_compact
354 .report_compact_task(
355 task_id,
356 TaskStatus::try_from(task_status).unwrap(),
357 sorted_output_ssts
358 .into_iter()
359 .map(SstableInfo::from)
360 .collect_vec(),
361 Some(table_stats_change),
362 object_timestamps
363 .into_iter()
364 .map(|(id, ts)| (id.into(), ts))
365 .collect(),
366 )
367 .await
368 {
369 tracing::error!(error = %e.as_report(), "report compact_tack fail");
370 }
371 }
372 }
373 }
374 });
375
376 join_handle_vec.push(report_handle);
377
378 Ok((
379 request_sender,
380 Box::pin(CompactionEventItemStream {
381 inner: UnboundedReceiverStream::new(task_rx),
382 _handle: join_handle_vec,
383 }),
384 ))
385 }
386
387 async fn get_version_by_epoch(
388 &self,
389 _epoch: HummockEpoch,
390 _table_id: u32,
391 ) -> Result<PbHummockVersion> {
392 unimplemented!()
393 }
394
395 async fn subscribe_iceberg_compaction_event(
396 &self,
397 ) -> Result<(
398 UnboundedSender<SubscribeIcebergCompactionEventRequest>,
399 BoxStream<'static, IcebergCompactionEventItem>,
400 )> {
401 unimplemented!()
402 }
403}
404
405impl MockHummockMetaClient {
406 pub fn hummock_manager_ref(&self) -> Arc<HummockManager> {
407 self.hummock_manager.clone()
408 }
409}
410
411pub struct CompactionEventItemStream {
412 inner: UnboundedReceiverStream<CompactionEventItem>,
413 _handle: Vec<JoinHandle<()>>,
414}
415
416impl Drop for CompactionEventItemStream {
417 fn drop(&mut self) {
418 self.inner.close();
419 }
420}
421
422impl Stream for CompactionEventItemStream {
423 type Item = CompactionEventItem;
424
425 fn poll_next(
426 mut self: std::pin::Pin<&mut Self>,
427 cx: &mut std::task::Context<'_>,
428 ) -> std::task::Poll<Option<Self::Item>> {
429 self.inner.poll_next_unpin(cx)
430 }
431}