risingwave_frontend/scheduler/
streaming_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
// 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;
use std::fmt::{Debug, Formatter};
use std::sync::Arc;

use itertools::Itertools;
use parking_lot::RwLock;
use pgwire::pg_server::SessionId;
use risingwave_pb::meta::cancel_creating_jobs_request::{
    CreatingJobInfo, CreatingJobInfos, PbJobs,
};
use uuid::Uuid;

use crate::catalog::{DatabaseId, SchemaId};
use crate::meta_client::FrontendMetaClient;

#[derive(Clone, Debug, Hash, Eq, PartialEq)]
pub struct TaskId {
    pub id: String,
}

impl std::fmt::Display for TaskId {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        write!(f, "TaskId:{}", self.id)
    }
}

impl Default for TaskId {
    fn default() -> Self {
        Self {
            id: Uuid::new_v4().to_string(),
        }
    }
}

pub type StreamingJobTrackerRef = Arc<StreamingJobTracker>;

pub struct StreamingJobTracker {
    creating_streaming_job: RwLock<HashMap<TaskId, CreatingStreamingJobInfo>>,
    meta_client: Arc<dyn FrontendMetaClient>,
}

impl StreamingJobTracker {
    pub fn new(meta_client: Arc<dyn FrontendMetaClient>) -> Self {
        Self {
            creating_streaming_job: RwLock::new(HashMap::default()),
            meta_client,
        }
    }
}

#[derive(Clone, Default)]
pub struct CreatingStreamingJobInfo {
    /// Identified by `process_id`, `secret_key`.
    session_id: SessionId,
    info: CreatingJobInfo,
}

impl CreatingStreamingJobInfo {
    pub fn new(
        session_id: SessionId,
        database_id: DatabaseId,
        schema_id: SchemaId,
        name: String,
    ) -> Self {
        Self {
            session_id,
            info: CreatingJobInfo {
                database_id,
                schema_id,
                name,
            },
        }
    }
}

pub struct StreamingJobGuard<'a> {
    task_id: TaskId,
    tracker: &'a StreamingJobTracker,
}

impl Drop for StreamingJobGuard<'_> {
    fn drop(&mut self) {
        self.tracker.delete_job(&self.task_id);
    }
}

impl StreamingJobTracker {
    pub fn guard(&self, task_info: CreatingStreamingJobInfo) -> StreamingJobGuard<'_> {
        let task_id = TaskId::default();
        self.add_job(task_id.clone(), task_info);
        StreamingJobGuard {
            task_id,
            tracker: self,
        }
    }

    fn add_job(&self, task_id: TaskId, info: CreatingStreamingJobInfo) {
        self.creating_streaming_job.write().insert(task_id, info);
    }

    fn delete_job(&self, task_id: &TaskId) {
        self.creating_streaming_job.write().remove(task_id);
    }

    pub fn abort_jobs(&self, session_id: SessionId) {
        let jobs = self
            .creating_streaming_job
            .read()
            .values()
            .filter(|job| job.session_id == session_id)
            .cloned()
            .collect_vec();

        let client = self.meta_client.clone();
        tokio::spawn(async move {
            client
                .cancel_creating_jobs(PbJobs::Infos(CreatingJobInfos {
                    infos: jobs.into_iter().map(|job| job.info).collect_vec(),
                }))
                .await
        });
    }
}