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