risingwave_meta/hummock/
compactor_manager.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use std::time::{Duration, Instant, SystemTime};

use fail::fail_point;
use parking_lot::RwLock;
use risingwave_hummock_sdk::compact::statistics_compact_task;
use risingwave_hummock_sdk::compact_task::CompactTask;
use risingwave_hummock_sdk::{HummockCompactionTaskId, HummockContextId};
use risingwave_pb::hummock::subscribe_compaction_event_response::Event as ResponseEvent;
use risingwave_pb::hummock::{
    CancelCompactTask, CompactTaskAssignment, CompactTaskProgress, SubscribeCompactionEventResponse,
};
use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender};

use crate::manager::MetaSrvEnv;
use crate::model::MetadataModelError;
use crate::MetaResult;

pub type CompactorManagerRef = Arc<CompactorManager>;

pub const TASK_RUN_TOO_LONG: &str = "running too long";
pub const TASK_NOT_FOUND: &str = "task not found";
pub const TASK_NORMAL: &str = "task is normal, please wait some time";

/// Wraps the stream between meta node and compactor node.
/// Compactor node will re-establish the stream when the previous one fails.
#[derive(Debug)]
pub struct Compactor {
    context_id: HummockContextId,
    sender: UnboundedSender<MetaResult<SubscribeCompactionEventResponse>>,
}

struct TaskHeartbeat {
    task: CompactTask,
    num_ssts_sealed: u32,
    num_ssts_uploaded: u32,
    num_progress_key: u64,
    num_pending_read_io: u64,
    num_pending_write_io: u64,
    create_time: Instant,
    expire_at: u64,

    update_at: u64,
}

impl Compactor {
    pub fn new(
        context_id: HummockContextId,
        sender: UnboundedSender<MetaResult<SubscribeCompactionEventResponse>>,
    ) -> Self {
        Self { context_id, sender }
    }

    pub fn send_event(&self, event: ResponseEvent) -> MetaResult<()> {
        fail_point!("compaction_send_task_fail", |_| Err(anyhow::anyhow!(
            "compaction_send_task_fail"
        )
        .into()));

        self.sender
            .send(Ok(SubscribeCompactionEventResponse {
                event: Some(event),
                create_at: SystemTime::now()
                    .duration_since(std::time::UNIX_EPOCH)
                    .expect("Clock may have gone backwards")
                    .as_millis() as u64,
            }))
            .map_err(|e| anyhow::anyhow!(e))?;

        Ok(())
    }

    pub fn cancel_task(&self, task_id: u64) -> MetaResult<()> {
        self.sender
            .send(Ok(SubscribeCompactionEventResponse {
                event: Some(ResponseEvent::CancelCompactTask(CancelCompactTask {
                    context_id: self.context_id,
                    task_id,
                })),
                create_at: SystemTime::now()
                    .duration_since(std::time::UNIX_EPOCH)
                    .expect("Clock may have gone backwards")
                    .as_millis() as u64,
            }))
            .map_err(|e| anyhow::anyhow!(e))?;
        Ok(())
    }

    pub fn cancel_tasks(&self, task_ids: &Vec<u64>) -> MetaResult<()> {
        for task_id in task_ids {
            self.cancel_task(*task_id)?;
        }
        Ok(())
    }

    pub fn context_id(&self) -> HummockContextId {
        self.context_id
    }
}

/// `CompactorManagerInner` maintains compactors which can process compact task.
/// A compact task is tracked in `HummockManager::Compaction` via both `CompactStatus` and
/// `CompactTaskAssignment`.
///
/// A compact task can be in one of these states:
/// 1. Success: an assigned task is reported as success via `CompactStatus::report_compact_task`.
///    It's the final state.
/// 2. Failed: an Failed task is reported as success via `CompactStatus::report_compact_task`.
///    It's the final state.
/// 3. Cancelled: a task is reported as cancelled via `CompactStatus::report_compact_task`. It's
///    the final state.
pub struct CompactorManagerInner {
    pub task_expired_seconds: u64,
    pub heartbeat_expired_seconds: u64,
    task_heartbeats: HashMap<HummockCompactionTaskId, TaskHeartbeat>,

    /// The outer lock is a `RwLock`, so we should still be able to modify each compactor
    pub compactor_map: HashMap<HummockContextId, Arc<Compactor>>,
}

impl CompactorManagerInner {
    pub async fn with_meta(env: MetaSrvEnv) -> MetaResult<Self> {
        use risingwave_meta_model::compaction_task;
        use sea_orm::EntityTrait;
        // Retrieve the existing task assignments from metastore.
        let task_assignment: Vec<CompactTaskAssignment> = compaction_task::Entity::find()
            .all(&env.meta_store_ref().conn)
            .await
            .map_err(MetadataModelError::from)?
            .into_iter()
            .map(Into::into)
            .collect();
        let mut manager = Self {
            task_expired_seconds: env.opts.compaction_task_max_progress_interval_secs,
            heartbeat_expired_seconds: env.opts.compaction_task_max_heartbeat_interval_secs,
            task_heartbeats: Default::default(),
            compactor_map: Default::default(),
        };
        // Initialize heartbeat for existing tasks.
        task_assignment.into_iter().for_each(|assignment| {
            manager.initiate_task_heartbeat(CompactTask::from(assignment.compact_task.unwrap()));
        });
        Ok(manager)
    }

    /// Only used for unit test.
    pub fn for_test() -> Self {
        Self {
            task_expired_seconds: 1,
            heartbeat_expired_seconds: 1,
            task_heartbeats: Default::default(),
            compactor_map: Default::default(),
        }
    }

    pub fn next_compactor(&self) -> Option<Arc<Compactor>> {
        use rand::Rng;

        if self.compactor_map.is_empty() {
            return None;
        }

        let rand_index = rand::thread_rng().gen_range(0..self.compactor_map.len());
        let compactor = self.compactor_map.values().nth(rand_index).unwrap().clone();

        Some(compactor)
    }

    /// Retrieve a receiver of tasks for the compactor identified by `context_id`. The sender should
    /// be obtained by calling one of the compactor getters.
    ///
    ///  If `add_compactor` is called with the same `context_id` more than once, the only cause
    /// would be compactor re-subscription, as `context_id` is a monotonically increasing
    /// sequence.
    pub fn add_compactor(
        &mut self,
        context_id: HummockContextId,
    ) -> UnboundedReceiver<MetaResult<SubscribeCompactionEventResponse>> {
        let (tx, rx) = tokio::sync::mpsc::unbounded_channel();
        self.compactor_map
            .insert(context_id, Arc::new(Compactor::new(context_id, tx)));

        tracing::info!("Added compactor session {}", context_id);
        rx
    }

    /// Used when meta exiting to support graceful shutdown.
    pub fn abort_all_compactors(&mut self) {
        while let Some(compactor) = self.next_compactor() {
            self.remove_compactor(compactor.context_id);
        }
    }

    pub fn remove_compactor(&mut self, context_id: HummockContextId) {
        self.compactor_map.remove(&context_id);

        // To remove the heartbeats, they need to be forcefully purged,
        // which is only safe when the context has been completely removed from meta.
        tracing::info!("Removed compactor session {}", context_id);
    }

    pub fn get_compactor(&self, context_id: HummockContextId) -> Option<Arc<Compactor>> {
        self.compactor_map.get(&context_id).cloned()
    }

    pub fn check_tasks_status(
        &self,
        tasks: &[HummockCompactionTaskId],
        slow_task_duration: Duration,
    ) -> HashMap<HummockCompactionTaskId, (Duration, &'static str)> {
        let tasks_ids: HashSet<u64> = HashSet::from_iter(tasks.to_vec());
        let mut ret = HashMap::default();
        for TaskHeartbeat {
            task, create_time, ..
        } in self.task_heartbeats.values()
        {
            if !tasks_ids.contains(&task.task_id) {
                continue;
            }
            let pending_time = create_time.elapsed();
            if pending_time > slow_task_duration {
                ret.insert(task.task_id, (pending_time, TASK_RUN_TOO_LONG));
            } else {
                ret.insert(task.task_id, (pending_time, TASK_NORMAL));
            }
        }

        for task_id in tasks {
            if !ret.contains_key(task_id) {
                ret.insert(*task_id, (Duration::from_secs(0), TASK_NOT_FOUND));
            }
        }
        ret
    }

    pub fn get_heartbeat_expired_tasks(&self) -> Vec<CompactTask> {
        let heartbeat_expired_ts: u64 = SystemTime::now()
            .duration_since(SystemTime::UNIX_EPOCH)
            .expect("Clock may have gone backwards")
            .as_secs()
            - self.heartbeat_expired_seconds;
        Self::get_heartbeat_expired_tasks_impl(&self.task_heartbeats, heartbeat_expired_ts)
    }

    fn get_heartbeat_expired_tasks_impl(
        task_heartbeats: &HashMap<HummockCompactionTaskId, TaskHeartbeat>,
        heartbeat_expired_ts: u64,
    ) -> Vec<CompactTask> {
        let mut cancellable_tasks = vec![];
        const MAX_TASK_DURATION_SEC: u64 = 2700;

        for TaskHeartbeat {
            expire_at,
            task,
            create_time,
            num_ssts_sealed,
            num_ssts_uploaded,
            num_progress_key,
            num_pending_read_io,
            num_pending_write_io,
            update_at,
        } in task_heartbeats.values()
        {
            if *update_at < heartbeat_expired_ts {
                cancellable_tasks.push(task.clone());
            }

            let task_duration_too_long = create_time.elapsed().as_secs() > MAX_TASK_DURATION_SEC;
            if task_duration_too_long {
                let compact_task_statistics = statistics_compact_task(task);
                tracing::info!(
                    "CompactionGroupId {} Task {} duration too long create_time {:?} expire_at {:?} num_ssts_sealed {} num_ssts_uploaded {} num_progress_key {} \
                        pending_read_io_count {} pending_write_io_count {} target_level {} \
                        base_level {} target_sub_level_id {} task_type {} compact_task_statistics {:?}",
                        task.compaction_group_id,
                        task.task_id,
                        create_time,
                        expire_at,
                        num_ssts_sealed,
                        num_ssts_uploaded,
                        num_progress_key,
                        num_pending_read_io,
                        num_pending_write_io,
                        task.target_level,
                        task.base_level,
                        task.target_sub_level_id,
                        task.task_type.as_str_name(),
                        compact_task_statistics
                );
            }
        }
        cancellable_tasks
    }

    pub fn initiate_task_heartbeat(&mut self, task: CompactTask) {
        let now = SystemTime::now()
            .duration_since(SystemTime::UNIX_EPOCH)
            .expect("Clock may have gone backwards")
            .as_secs();
        self.task_heartbeats.insert(
            task.task_id,
            TaskHeartbeat {
                task,
                num_ssts_sealed: 0,
                num_ssts_uploaded: 0,
                num_progress_key: 0,
                num_pending_read_io: 0,
                num_pending_write_io: 0,
                create_time: Instant::now(),
                expire_at: now + self.task_expired_seconds,
                update_at: now,
            },
        );
    }

    pub fn remove_task_heartbeat(&mut self, task_id: u64) {
        self.task_heartbeats.remove(&task_id).unwrap();
    }

    pub fn update_task_heartbeats(
        &mut self,
        progress_list: &Vec<CompactTaskProgress>,
    ) -> Vec<CompactTask> {
        let now = SystemTime::now()
            .duration_since(SystemTime::UNIX_EPOCH)
            .expect("Clock may have gone backwards")
            .as_secs();
        let mut cancel_tasks = vec![];
        for progress in progress_list {
            if let Some(task_ref) = self.task_heartbeats.get_mut(&progress.task_id) {
                task_ref.update_at = now;

                if task_ref.num_ssts_sealed < progress.num_ssts_sealed
                    || task_ref.num_ssts_uploaded < progress.num_ssts_uploaded
                    || task_ref.num_progress_key < progress.num_progress_key
                {
                    // Refresh the expired of the task as it is showing progress.
                    task_ref.expire_at = now + self.task_expired_seconds;
                    task_ref.num_ssts_sealed = progress.num_ssts_sealed;
                    task_ref.num_ssts_uploaded = progress.num_ssts_uploaded;
                    task_ref.num_progress_key = progress.num_progress_key;
                }
                task_ref.num_pending_read_io = progress.num_pending_read_io;
                task_ref.num_pending_write_io = progress.num_pending_write_io;

                // timeout check
                if task_ref.expire_at < now {
                    // cancel
                    cancel_tasks.push(task_ref.task.clone())
                }
            }
        }

        cancel_tasks
    }

    pub fn compactor_num(&self) -> usize {
        self.compactor_map.len()
    }

    pub fn get_progress(&self) -> Vec<CompactTaskProgress> {
        self.task_heartbeats
            .values()
            .map(|hb| CompactTaskProgress {
                task_id: hb.task.task_id,
                num_ssts_sealed: hb.num_ssts_sealed,
                num_ssts_uploaded: hb.num_ssts_uploaded,
                num_progress_key: hb.num_progress_key,
                num_pending_read_io: hb.num_pending_read_io,
                num_pending_write_io: hb.num_pending_write_io,
                compaction_group_id: Some(hb.task.compaction_group_id),
            })
            .collect()
    }
}

pub struct CompactorManager {
    inner: Arc<RwLock<CompactorManagerInner>>,
}

impl CompactorManager {
    pub async fn with_meta(env: MetaSrvEnv) -> MetaResult<Self> {
        let inner = CompactorManagerInner::with_meta(env).await?;

        Ok(Self {
            inner: Arc::new(RwLock::new(inner)),
        })
    }

    /// Only used for unit test.
    pub fn for_test() -> Self {
        let inner = CompactorManagerInner::for_test();
        Self {
            inner: Arc::new(RwLock::new(inner)),
        }
    }

    pub fn next_compactor(&self) -> Option<Arc<Compactor>> {
        self.inner.read().next_compactor()
    }

    pub fn add_compactor(
        &self,
        context_id: HummockContextId,
    ) -> UnboundedReceiver<MetaResult<SubscribeCompactionEventResponse>> {
        self.inner.write().add_compactor(context_id)
    }

    pub fn abort_all_compactors(&self) {
        self.inner.write().abort_all_compactors();
    }

    pub fn remove_compactor(&self, context_id: HummockContextId) {
        self.inner.write().remove_compactor(context_id)
    }

    pub fn get_compactor(&self, context_id: HummockContextId) -> Option<Arc<Compactor>> {
        self.inner.read().get_compactor(context_id)
    }

    pub fn check_tasks_status(
        &self,
        tasks: &[HummockCompactionTaskId],
        slow_task_duration: Duration,
    ) -> HashMap<HummockCompactionTaskId, (Duration, &'static str)> {
        self.inner
            .read()
            .check_tasks_status(tasks, slow_task_duration)
    }

    pub fn get_heartbeat_expired_tasks(&self) -> Vec<CompactTask> {
        self.inner.read().get_heartbeat_expired_tasks()
    }

    pub fn initiate_task_heartbeat(&self, task: CompactTask) {
        self.inner.write().initiate_task_heartbeat(task);
    }

    pub fn remove_task_heartbeat(&self, task_id: u64) {
        self.inner.write().remove_task_heartbeat(task_id);
    }

    pub fn update_task_heartbeats(
        &self,
        progress_list: &Vec<CompactTaskProgress>,
    ) -> Vec<CompactTask> {
        self.inner.write().update_task_heartbeats(progress_list)
    }

    pub fn compactor_num(&self) -> usize {
        self.inner.read().compactor_num()
    }

    pub fn get_progress(&self) -> Vec<CompactTaskProgress> {
        self.inner.read().get_progress()
    }
}

#[cfg(test)]
mod tests {
    use std::sync::Arc;
    use std::time::Duration;

    use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId;
    use risingwave_pb::hummock::CompactTaskProgress;
    use risingwave_rpc_client::HummockMetaClient;

    use crate::hummock::compaction::selector::default_compaction_selector;
    use crate::hummock::test_utils::{
        add_ssts, register_table_ids_to_compaction_group, setup_compute_env,
    };
    use crate::hummock::{CompactorManager, MockHummockMetaClient};

    #[tokio::test]
    async fn test_compactor_manager() {
        // Initialize metastore with task assignment.
        let (env, context_id) = {
            let (env, hummock_manager, _cluster_manager, worker_id) = setup_compute_env(80).await;
            let context_id = worker_id as _;
            let hummock_meta_client: Arc<dyn HummockMetaClient> = Arc::new(
                MockHummockMetaClient::new(hummock_manager.clone(), context_id),
            );
            let compactor_manager = hummock_manager.compactor_manager_ref_for_test();
            register_table_ids_to_compaction_group(
                hummock_manager.as_ref(),
                &[1],
                StaticCompactionGroupId::StateDefault.into(),
            )
            .await;
            let _sst_infos =
                add_ssts(1, hummock_manager.as_ref(), hummock_meta_client.clone()).await;
            let _receiver = compactor_manager.add_compactor(context_id);
            hummock_manager
                .get_compact_task(
                    StaticCompactionGroupId::StateDefault.into(),
                    &mut default_compaction_selector(),
                )
                .await
                .unwrap()
                .unwrap();
            (env, context_id)
        };

        // Restart. Set task_expired_seconds to 0 only to speed up test.
        let compactor_manager = CompactorManager::with_meta(env).await.unwrap();
        // Because task assignment exists.
        // Because compactor gRPC is not established yet.
        assert_eq!(compactor_manager.compactor_num(), 0);
        assert!(compactor_manager.get_compactor(context_id).is_none());

        // Ensure task is expired.
        tokio::time::sleep(Duration::from_secs(2)).await;
        let expired = compactor_manager.get_heartbeat_expired_tasks();
        assert_eq!(expired.len(), 1);

        // Mimic no-op compaction heartbeat
        assert_eq!(compactor_manager.get_heartbeat_expired_tasks().len(), 1);

        // Mimic compaction heartbeat with invalid task id
        compactor_manager.update_task_heartbeats(&vec![CompactTaskProgress {
            task_id: expired[0].task_id + 1,
            num_ssts_sealed: 1,
            num_ssts_uploaded: 1,
            num_progress_key: 100,
            ..Default::default()
        }]);
        assert_eq!(compactor_manager.get_heartbeat_expired_tasks().len(), 1);

        // Mimic effective compaction heartbeat
        compactor_manager.update_task_heartbeats(&vec![CompactTaskProgress {
            task_id: expired[0].task_id,
            num_ssts_sealed: 1,
            num_ssts_uploaded: 1,
            num_progress_key: 100,
            ..Default::default()
        }]);
        assert_eq!(compactor_manager.get_heartbeat_expired_tasks().len(), 0);

        // Test add
        assert_eq!(compactor_manager.compactor_num(), 0);
        assert!(compactor_manager.get_compactor(context_id).is_none());
        compactor_manager.add_compactor(context_id);
        assert_eq!(compactor_manager.compactor_num(), 1);
        assert_eq!(
            compactor_manager
                .get_compactor(context_id)
                .unwrap()
                .context_id(),
            context_id
        );
        // Test remove
        compactor_manager.remove_compactor(context_id);
        assert_eq!(compactor_manager.compactor_num(), 0);
        assert!(compactor_manager.get_compactor(context_id).is_none());
    }
}