risingwave_meta/manager/
streaming_job.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
// Copyright 2025 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::HashSet;

use risingwave_common::catalog::TableVersionId;
use risingwave_common::util::epoch::Epoch;
use risingwave_common::{bail_not_implemented, current_cluster_version};
use risingwave_meta_model::object::ObjectType;
use risingwave_meta_model::prelude::{SourceModel, TableModel};
use risingwave_meta_model::{source, table, SourceId, TableId, TableVersion};
use risingwave_pb::catalog::{CreateType, Index, PbSource, Sink, Table};
use risingwave_pb::ddl_service::TableJobType;
use sea_orm::entity::prelude::*;
use sea_orm::{DatabaseTransaction, QuerySelect};
use strum::{EnumIs, EnumTryAs};

use super::{
    get_referred_connection_ids_from_sink, get_referred_connection_ids_from_source,
    get_referred_secret_ids_from_sink, get_referred_secret_ids_from_source,
};
use crate::stream::StreamFragmentGraph;
use crate::{MetaError, MetaResult};

// This enum is used in order to re-use code in `DdlServiceImpl` for creating MaterializedView and
// Sink.
#[derive(Debug, Clone, EnumIs, EnumTryAs)]
pub enum StreamingJob {
    MaterializedView(Table),
    Sink(Sink, Option<(Table, Option<PbSource>)>),
    Table(Option<PbSource>, Table, TableJobType),
    Index(Index, Table),
    Source(PbSource),
}

#[derive(Debug, Clone, Copy, PartialEq)]
pub enum StreamingJobType {
    MaterializedView,
    Sink,
    Table(TableJobType),
    Index,
    Source,
}

impl From<&StreamingJob> for StreamingJobType {
    fn from(job: &StreamingJob) -> Self {
        match job {
            StreamingJob::MaterializedView(_) => StreamingJobType::MaterializedView,
            StreamingJob::Sink(_, _) => StreamingJobType::Sink,
            StreamingJob::Table(_, _, ty) => StreamingJobType::Table(*ty),
            StreamingJob::Index(_, _) => StreamingJobType::Index,
            StreamingJob::Source(_) => StreamingJobType::Source,
        }
    }
}

#[cfg(test)]
#[allow(clippy::derivable_impls)]
impl Default for StreamingJobType {
    fn default() -> Self {
        // This should not be used by mock services,
        // so we can just pick an arbitrary default variant.
        StreamingJobType::MaterializedView
    }
}

impl StreamingJob {
    pub fn mark_created(&mut self) {
        let created_at_epoch = Some(Epoch::now().0);
        let created_at_cluster_version = Some(current_cluster_version());
        match self {
            StreamingJob::MaterializedView(table) => {
                table.created_at_epoch = created_at_epoch;
                table.created_at_cluster_version = created_at_cluster_version;
            }
            StreamingJob::Sink(table, _) => table.created_at_epoch = created_at_epoch,
            StreamingJob::Table(source, table, ..) => {
                table.created_at_epoch = created_at_epoch;
                table
                    .created_at_cluster_version
                    .clone_from(&created_at_cluster_version);
                if let Some(source) = source {
                    source.created_at_epoch = created_at_epoch;
                    source.created_at_cluster_version = created_at_cluster_version;
                }
            }
            StreamingJob::Index(index, _) => {
                index.created_at_epoch = created_at_epoch;
                index.created_at_cluster_version = created_at_cluster_version;
            }
            StreamingJob::Source(source) => {
                source.created_at_epoch = created_at_epoch;
                source.created_at_cluster_version = created_at_cluster_version;
            }
        }
    }

    pub fn mark_initialized(&mut self) {
        let initialized_at_epoch = Some(Epoch::now().0);
        let initialized_at_cluster_version = Some(current_cluster_version());
        match self {
            StreamingJob::MaterializedView(table) => {
                table.initialized_at_epoch = initialized_at_epoch;
                table.initialized_at_cluster_version = initialized_at_cluster_version;
            }
            StreamingJob::Sink(table, _) => {
                table.initialized_at_epoch = initialized_at_epoch;
                table.initialized_at_cluster_version = initialized_at_cluster_version;
            }
            StreamingJob::Table(source, table, ..) => {
                table.initialized_at_epoch = initialized_at_epoch;
                table
                    .initialized_at_cluster_version
                    .clone_from(&initialized_at_cluster_version);

                if let Some(source) = source {
                    source.initialized_at_epoch = initialized_at_epoch;
                    source.initialized_at_cluster_version = initialized_at_cluster_version;
                }
            }
            StreamingJob::Index(index, _) => {
                index.initialized_at_epoch = initialized_at_epoch;
                index.initialized_at_cluster_version = initialized_at_cluster_version;
            }
            StreamingJob::Source(source) => {
                source.initialized_at_epoch = initialized_at_epoch;
                source.initialized_at_cluster_version = initialized_at_cluster_version;
            }
        }
    }
}

// TODO: basically we want to ensure that the `Table` persisted in the catalog is the same as the
// one in the `Materialize` node in the actor. However, they are currently handled separately
// and can be out of sync. Shall we directly copy the whole struct from the actor to the catalog
// to avoid `set`ting each field separately?
impl StreamingJob {
    pub fn set_id(&mut self, id: u32) {
        match self {
            Self::MaterializedView(table) => table.id = id,
            Self::Sink(sink, _) => sink.id = id,
            Self::Table(_, table, ..) => table.id = id,
            Self::Index(index, index_table) => {
                index.id = id;
                index.index_table_id = id;
                index_table.id = id;
            }
            StreamingJob::Source(src) => {
                src.id = id;
            }
        }
    }

    /// Set the vnode count of the table.
    pub fn set_table_vnode_count(&mut self, vnode_count: usize) {
        match self {
            Self::MaterializedView(table) | Self::Index(_, table) | Self::Table(_, table, ..) => {
                table.maybe_vnode_count = Some(vnode_count as u32);
            }
            Self::Sink(_, _) | Self::Source(_) => {}
        }
    }

    /// Add some info which is only available in fragment graph to the catalog.
    pub fn set_info_from_graph(&mut self, graph: &StreamFragmentGraph) {
        match self {
            Self::Table(_, table, ..) => {
                table.fragment_id = graph.table_fragment_id();
                table.dml_fragment_id = graph.dml_fragment_id();
            }
            Self::MaterializedView(table) | Self::Index(_, table) => {
                table.fragment_id = graph.table_fragment_id();
            }
            Self::Sink(_, _) | Self::Source(_) => {}
        }
    }

    pub fn id(&self) -> u32 {
        match self {
            Self::MaterializedView(table) => table.id,
            Self::Sink(sink, _) => sink.id,
            Self::Table(_, table, ..) => table.id,
            Self::Index(index, _) => index.id,
            Self::Source(source) => source.id,
        }
    }

    pub fn mv_table(&self) -> Option<u32> {
        match self {
            Self::MaterializedView(table) => Some(table.id),
            Self::Sink(_, _) => None,
            Self::Table(_, table, ..) => Some(table.id),
            Self::Index(_, table) => Some(table.id),
            Self::Source(_) => None,
        }
    }

    /// Returns the reference to the [`Table`] of the job if it exists.
    pub fn table(&self) -> Option<&Table> {
        match self {
            Self::MaterializedView(table) | Self::Index(_, table) | Self::Table(_, table, ..) => {
                Some(table)
            }
            Self::Sink(_, _) | Self::Source(_) => None,
        }
    }

    pub fn schema_id(&self) -> u32 {
        match self {
            Self::MaterializedView(table) => table.schema_id,
            Self::Sink(sink, _) => sink.schema_id,
            Self::Table(_, table, ..) => table.schema_id,
            Self::Index(index, _) => index.schema_id,
            Self::Source(source) => source.schema_id,
        }
    }

    pub fn database_id(&self) -> u32 {
        match self {
            Self::MaterializedView(table) => table.database_id,
            Self::Sink(sink, _) => sink.database_id,
            Self::Table(_, table, ..) => table.database_id,
            Self::Index(index, _) => index.database_id,
            Self::Source(source) => source.database_id,
        }
    }

    pub fn name(&self) -> String {
        match self {
            Self::MaterializedView(table) => table.name.clone(),
            Self::Sink(sink, _) => sink.name.clone(),
            Self::Table(_, table, ..) => table.name.clone(),
            Self::Index(index, _) => index.name.clone(),
            Self::Source(source) => source.name.clone(),
        }
    }

    pub fn owner(&self) -> u32 {
        match self {
            StreamingJob::MaterializedView(mv) => mv.owner,
            StreamingJob::Sink(sink, _) => sink.owner,
            StreamingJob::Table(_, table, ..) => table.owner,
            StreamingJob::Index(index, _) => index.owner,
            StreamingJob::Source(source) => source.owner,
        }
    }

    pub fn job_type(&self) -> StreamingJobType {
        self.into()
    }

    pub fn job_type_str(&self) -> &'static str {
        match self {
            StreamingJob::MaterializedView(_) => "materialized view",
            StreamingJob::Sink(_, _) => "sink",
            StreamingJob::Table(_, _, _) => "table",
            StreamingJob::Index(_, _) => "index",
            StreamingJob::Source(_) => "source",
        }
    }

    pub fn definition(&self) -> String {
        match self {
            Self::MaterializedView(table) => table.definition.clone(),
            Self::Table(_, table, ..) => table.definition.clone(),
            Self::Index(_, table) => table.definition.clone(),
            Self::Sink(sink, _) => sink.definition.clone(),
            Self::Source(source) => source.definition.clone(),
        }
    }

    pub fn object_type(&self) -> ObjectType {
        match self {
            Self::MaterializedView(_) => ObjectType::Table, // Note MV is special.
            Self::Sink(_, _) => ObjectType::Sink,
            Self::Table(_, _, _) => ObjectType::Table,
            Self::Index(_, _) => ObjectType::Index,
            Self::Source(_) => ObjectType::Source,
        }
    }

    /// Returns the [`TableVersionId`] if this job is `Table`.
    pub fn table_version_id(&self) -> Option<TableVersionId> {
        if let Self::Table(_, table, ..) = self {
            Some(
                table
                    .get_version()
                    .expect("table must be versioned")
                    .version,
            )
        } else {
            None
        }
    }

    pub fn create_type(&self) -> CreateType {
        match self {
            Self::MaterializedView(table) => {
                table.get_create_type().unwrap_or(CreateType::Foreground)
            }
            Self::Sink(s, _) => s.get_create_type().unwrap_or(CreateType::Foreground),
            _ => CreateType::Foreground,
        }
    }

    // TODO: to be removed, pass all objects uniformly through `dependencies` field instead.
    pub fn dependent_relations(&self) -> Vec<u32> {
        match self {
            StreamingJob::MaterializedView(table) => table.dependent_relations.clone(),
            StreamingJob::Sink(_sink, _) => vec![], /* sink dependencies are now passed via `dependencies` field in `CreateSinkRequest` */
            StreamingJob::Table(_, table, _) => table.dependent_relations.clone(), /* TODO(rc): record table dependencies via `dependencies` field */
            StreamingJob::Index(index, index_table) => {
                // TODO(rc): record index dependencies via `dependencies` field
                assert_eq!(index.primary_table_id, index_table.dependent_relations[0]);
                vec![]
            }
            StreamingJob::Source(_) => vec![],
        }
    }

    pub fn dependent_connection_ids(&self) -> MetaResult<HashSet<u32>> {
        match self {
            StreamingJob::Source(source) => Ok(get_referred_connection_ids_from_source(source)),
            StreamingJob::Table(source, _, _) => {
                if let Some(source) = source {
                    Ok(get_referred_connection_ids_from_source(source))
                } else {
                    Ok(HashSet::new())
                }
            }
            StreamingJob::Sink(sink, _) => Ok(get_referred_connection_ids_from_sink(sink)),
            StreamingJob::MaterializedView(_) | StreamingJob::Index(_, _) => Ok(HashSet::new()),
        }
    }

    // Get the secret ids that are referenced by this job.
    pub fn dependent_secret_ids(&self) -> MetaResult<HashSet<u32>> {
        match self {
            StreamingJob::Sink(sink, _) => Ok(get_referred_secret_ids_from_sink(sink)),
            StreamingJob::Table(source, _, _) => {
                if let Some(source) = source {
                    get_referred_secret_ids_from_source(source)
                } else {
                    Ok(HashSet::new())
                }
            }
            StreamingJob::Source(source) => get_referred_secret_ids_from_source(source),
            StreamingJob::MaterializedView(_) | StreamingJob::Index(_, _) => Ok(HashSet::new()),
        }
    }

    /// Verify the new version is the next version of the original version.
    pub async fn verify_version_for_replace(&self, txn: &DatabaseTransaction) -> MetaResult<()> {
        let id = self.id();

        match self {
            StreamingJob::Table(_source, table, _table_job_type) => {
                let new_version = table.get_version()?.get_version();
                let original_version: Option<TableVersion> = TableModel::find_by_id(id as TableId)
                    .select_only()
                    .column(table::Column::Version)
                    .into_tuple()
                    .one(txn)
                    .await?
                    .ok_or_else(|| MetaError::catalog_id_not_found(self.job_type_str(), id))?;
                let original_version = original_version
                    .expect("version for table should exist")
                    .to_protobuf();
                if new_version != original_version.version + 1 {
                    return Err(MetaError::permission_denied("table version is stale"));
                }
            }
            StreamingJob::Source(source) => {
                let new_version = source.get_version();
                let original_version: Option<i64> = SourceModel::find_by_id(id as SourceId)
                    .select_only()
                    .column(source::Column::Version)
                    .into_tuple()
                    .one(txn)
                    .await?
                    .ok_or_else(|| MetaError::catalog_id_not_found(self.job_type_str(), id))?;
                let original_version = original_version.expect("version for source should exist");
                if new_version != original_version as u64 + 1 {
                    return Err(MetaError::permission_denied("source version is stale"));
                }
            }
            StreamingJob::MaterializedView(_)
            | StreamingJob::Sink(_, _)
            | StreamingJob::Index(_, _) => {
                bail_not_implemented!("schema change for {}", self.job_type_str())
            }
        }
        Ok(())
    }
}