risingwave_connector/source/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
// 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::fmt::Debug;
use risingwave_common::catalog::{
ColumnDesc, ColumnId, KAFKA_TIMESTAMP_COLUMN_NAME, OFFSET_COLUMN_NAME, ROWID_PREFIX,
TABLE_NAME_COLUMN_NAME,
};
use risingwave_common::types::DataType;
use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn;
use risingwave_pb::plan_common::{AdditionalColumn, ColumnDescVersion};
/// `SourceColumnDesc` is used to describe a column in the Source.
///
/// See the implementation of `From<&ColumnDesc>` for the difference between `SourceColumnDesc` and [`ColumnDesc`].
#[derive(Clone, Debug)]
pub struct SourceColumnDesc {
pub name: String,
pub data_type: DataType,
pub column_id: ColumnId,
pub fields: Vec<ColumnDesc>,
/// `additional_column` and `column_type` are orthogonal
/// `additional_column` is used to indicate the column is from which part of the message
/// `column_type` is used to indicate the type of the column, only used in cdc scenario
pub additional_column: AdditionalColumn,
// ------
// Fields above are the same in `ColumnDesc`.
// Fields below are specific to `SourceColumnDesc`.
// ------
pub column_type: SourceColumnType,
/// `is_pk` is used to indicate whether the column is part of the primary key columns.
pub is_pk: bool,
/// `is_hidden_addition_col` is used to indicate whether the column is a hidden addition column.
pub is_hidden_addition_col: bool,
}
/// `SourceColumnType` is used to indicate the type of a column emitted by the Source.
/// There are 4 types of columns:
/// - `Normal`: a visible column
/// - `RowId`: internal column to uniquely identify a row
/// - `Meta`: internal column to store source related metadata
/// - `Offset`: internal column to store upstream offset for a row, used in CDC source
#[derive(Clone, Debug, PartialEq)]
pub enum SourceColumnType {
Normal,
// internal columns
RowId,
Meta,
Offset,
}
impl SourceColumnType {
pub fn from_name(name: &str) -> Self {
if name.starts_with(KAFKA_TIMESTAMP_COLUMN_NAME) || name.starts_with(TABLE_NAME_COLUMN_NAME)
{
Self::Meta
} else if name == (ROWID_PREFIX) {
Self::RowId
} else if name == OFFSET_COLUMN_NAME {
Self::Offset
} else {
Self::Normal
}
}
}
impl SourceColumnDesc {
/// Create a [`SourceColumnDesc`] without composite types.
#[track_caller]
pub fn simple(name: impl Into<String>, data_type: DataType, column_id: ColumnId) -> Self {
assert!(
!matches!(data_type, DataType::List { .. } | DataType::Struct(..)),
"called `SourceColumnDesc::simple` with a composite type."
);
let name = name.into();
Self {
name,
data_type,
column_id,
fields: vec![],
column_type: SourceColumnType::Normal,
is_pk: false,
is_hidden_addition_col: false,
additional_column: AdditionalColumn { column_type: None },
}
}
pub fn hidden_addition_col_from_column_desc(c: &ColumnDesc) -> Self {
Self {
is_hidden_addition_col: true,
..c.into()
}
}
pub fn is_row_id(&self) -> bool {
self.column_type == SourceColumnType::RowId
}
pub fn is_meta(&self) -> bool {
self.column_type == SourceColumnType::Meta
}
pub fn is_offset(&self) -> bool {
self.column_type == SourceColumnType::Offset
}
#[inline]
pub fn is_visible(&self) -> bool {
!self.is_hidden_addition_col && self.column_type == SourceColumnType::Normal
}
}
impl From<&ColumnDesc> for SourceColumnDesc {
fn from(
ColumnDesc {
data_type,
column_id,
name,
field_descs,
additional_column,
// ignored fields below
generated_or_default_column,
type_name: _,
description: _,
version: _,
system_column: _,
}: &ColumnDesc,
) -> Self {
if let Some(option) = generated_or_default_column {
debug_assert!(
matches!(option, GeneratedOrDefaultColumn::DefaultColumn(_)),
"source column should not be generated: {:?}",
generated_or_default_column.as_ref().unwrap()
)
}
Self {
name: name.clone(),
data_type: data_type.clone(),
column_id: *column_id,
fields: field_descs.clone(),
additional_column: additional_column.clone(),
// additional fields below
column_type: SourceColumnType::from_name(name),
is_pk: false,
is_hidden_addition_col: false,
}
}
}
impl From<&SourceColumnDesc> for ColumnDesc {
fn from(
SourceColumnDesc {
name,
data_type,
column_id,
fields,
additional_column,
// ignored fields below
column_type: _,
is_pk: _,
is_hidden_addition_col: _,
}: &SourceColumnDesc,
) -> Self {
ColumnDesc {
data_type: data_type.clone(),
column_id: *column_id,
name: name.clone(),
field_descs: fields.clone(),
additional_column: additional_column.clone(),
// additional fields below
type_name: "".to_string(),
generated_or_default_column: None,
description: None,
version: ColumnDescVersion::Pr13707,
system_column: None,
}
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_is_visible() {
let mut c = SourceColumnDesc::simple("a", DataType::Int32, ColumnId::new(0));
assert!(c.is_visible());
c.column_type = SourceColumnType::RowId;
assert!(!c.is_visible());
c.column_type = SourceColumnType::Meta;
assert!(!c.is_visible());
}
}