risingwave_frontend/optimizer/plan_node/
stream_cdc_table_scan.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
// 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 itertools::Itertools;
use pretty_xmlish::{Pretty, XmlNode};
use risingwave_common::catalog::Field;
use risingwave_common::types::DataType;
use risingwave_common::util::sort_util::OrderType;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
use risingwave_pb::stream_plan::PbStreamNode;

use super::stream::prelude::*;
use super::utils::{childless_record, Distill};
use super::{generic, ExprRewritable, PlanBase, PlanRef, StreamNode};
use crate::catalog::ColumnId;
use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor, FunctionCall, InputRef};
use crate::handler::create_source::debezium_cdc_source_schema;
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::plan_node::utils::{IndicesDisplay, TableCatalogBuilder};
use crate::optimizer::property::{Distribution, DistributionDisplay};
use crate::scheduler::SchedulerResult;
use crate::stream_fragmenter::BuildFragmentGraphState;
use crate::{Explain, TableCatalog};

/// `StreamCdcTableScan` is a virtual plan node to represent a stream cdc table scan.
/// It will be converted to cdc backfill + merge node (for upstream source)
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct StreamCdcTableScan {
    pub base: PlanBase<Stream>,
    core: generic::CdcScan,
}

impl StreamCdcTableScan {
    pub fn new(core: generic::CdcScan) -> Self {
        let distribution = Distribution::SomeShard;
        let base = PlanBase::new_stream_with_core(
            &core,
            distribution,
            core.append_only(),
            false,
            core.watermark_columns(),
            core.columns_monotonicity(),
        );
        Self { base, core }
    }

    pub fn table_name(&self) -> &str {
        &self.core.table_name
    }

    pub fn core(&self) -> &generic::CdcScan {
        &self.core
    }

    /// Build catalog for cdc backfill state
    /// Right now we only persist whether the backfill is finished and the corresponding cdc offset
    /// schema: | `split_id` | `pk...` | `backfill_finished` | `row_count` | `cdc_offset` |
    pub fn build_backfill_state_catalog(
        &self,
        state: &mut BuildFragmentGraphState,
    ) -> TableCatalog {
        let mut catalog_builder = TableCatalogBuilder::default();
        let upstream_schema = &self.core.get_table_columns();

        // Use `split_id` as primary key in state table.
        // Currently we only support single split for cdc backfill.
        catalog_builder.add_column(&Field::with_name(DataType::Varchar, "split_id"));
        catalog_builder.add_order_column(0, OrderType::ascending());

        // pk columns
        for col_order in self.core.primary_key() {
            let col = &upstream_schema[col_order.column_index];
            catalog_builder.add_column(&Field::from(col));
        }

        catalog_builder.add_column(&Field::with_name(DataType::Boolean, "backfill_finished"));

        // `row_count` column, the number of rows read from snapshot
        catalog_builder.add_column(&Field::with_name(DataType::Int64, "row_count"));

        // The offset is only for observability, not for recovery right now
        catalog_builder.add_column(&Field::with_name(DataType::Jsonb, "cdc_offset"));

        // leave dist key empty, since the cdc backfill executor is singleton
        catalog_builder
            .build(vec![], 1)
            .with_id(state.gen_table_id_wrapped())
    }
}

impl_plan_tree_node_for_leaf! { StreamCdcTableScan }

impl Distill for StreamCdcTableScan {
    fn distill<'a>(&self) -> XmlNode<'a> {
        let verbose = self.base.ctx().is_explain_verbose();
        let mut vec = Vec::with_capacity(4);
        vec.push(("table", Pretty::from(self.core.table_name.clone())));
        vec.push(("columns", self.core.columns_pretty(verbose)));

        if verbose {
            let pk = IndicesDisplay {
                indices: self.stream_key().unwrap_or_default(),
                schema: self.base.schema(),
            };
            vec.push(("pk", pk.distill()));
            let dist = Pretty::display(&DistributionDisplay {
                distribution: self.distribution(),
                input_schema: self.base.schema(),
            });
            vec.push(("dist", dist));
        }

        childless_record("StreamCdcTableScan", vec)
    }
}

impl StreamNode for StreamCdcTableScan {
    fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody {
        unreachable!("stream scan cannot be converted into a prost body -- call `adhoc_to_stream_prost` instead.")
    }
}

impl StreamCdcTableScan {
    /// plan: merge -> filter -> exchange(simple) -> `stream_scan`
    pub fn adhoc_to_stream_prost(
        &self,
        state: &mut BuildFragmentGraphState,
    ) -> SchedulerResult<PbStreamNode> {
        use risingwave_pb::stream_plan::*;

        let stream_key = self
            .stream_key()
            .unwrap_or_else(|| {
                panic!(
                    "should always have a stream key in the stream plan but not, sub plan: {}",
                    PlanRef::from(self.clone()).explain_to_string()
                )
            })
            .iter()
            .map(|x| *x as u32)
            .collect_vec();

        // The schema of the shared cdc source upstream is different from snapshot,
        // refer to `debezium_cdc_source_schema()` for details.
        let cdc_source_schema = {
            let columns = debezium_cdc_source_schema();
            columns
                .into_iter()
                .map(|c| Field::from(c.column_desc).to_prost())
                .collect_vec()
        };

        let catalog = self
            .build_backfill_state_catalog(state)
            .to_internal_table_prost();

        // We need to pass the id of upstream source job here
        let upstream_source_id = self.core.cdc_table_desc.source_id.table_id;

        // filter upstream source chunk by the value of `_rw_table_name` column
        let filter_expr =
            Self::build_cdc_filter_expr(self.core.cdc_table_desc.external_table_name.as_str());

        let filter_operator_id = self.core.ctx.next_plan_node_id();
        // The filter node receive chunks in `(payload, _rw_offset, _rw_table_name)` schema
        let filter_stream_node = StreamNode {
            operator_id: filter_operator_id.0 as _,
            input: vec![
                // The merge node body will be filled by the `ActorBuilder` on the meta service.
                PbStreamNode {
                    node_body: Some(PbNodeBody::Merge(Default::default())),
                    identity: "Upstream".into(),
                    fields: cdc_source_schema.clone(),
                    stream_key: vec![], // not used
                    ..Default::default()
                },
            ],
            stream_key: vec![], // not used
            append_only: true,
            identity: "StreamCdcFilter".to_string(),
            fields: cdc_source_schema.clone(),
            node_body: Some(PbNodeBody::CdcFilter(CdcFilterNode {
                search_condition: Some(filter_expr.to_expr_proto()),
                upstream_source_id,
            })),
        };

        let exchange_operator_id = self.core.ctx.next_plan_node_id();
        // Add a simple exchange node between filter and stream scan
        let exchange_stream_node = StreamNode {
            operator_id: exchange_operator_id.0 as _,
            input: vec![filter_stream_node],
            stream_key: vec![], // not used
            append_only: true,
            identity: "Exchange".to_string(),
            fields: cdc_source_schema.clone(),
            node_body: Some(PbNodeBody::Exchange(ExchangeNode {
                strategy: Some(DispatchStrategy {
                    r#type: DispatcherType::Simple as _,
                    dist_key_indices: vec![], // simple exchange doesn't need dist key
                    output_indices: (0..cdc_source_schema.len() as u32).collect(),
                }),
            })),
        };

        // The required columns from the external table
        let upstream_column_ids = self
            .core
            .output_and_pk_column_ids()
            .iter()
            .map(ColumnId::get_id)
            .collect_vec();

        let output_indices = self
            .core
            .output_column_ids()
            .iter()
            .map(|i| {
                upstream_column_ids
                    .iter()
                    .position(|&x| x == i.get_id())
                    .unwrap() as u32
            })
            .collect_vec();

        tracing::debug!(
            output_column_ids=?self.core.output_column_ids(),
            ?upstream_column_ids,
            ?output_indices,
            "stream cdc table scan output indices"
        );

        let options = self.core.options.to_proto();
        let stream_scan_body = PbNodeBody::StreamCdcScan(StreamCdcScanNode {
            table_id: upstream_source_id,
            upstream_column_ids,
            output_indices,
            // The table desc used by backfill executor
            state_table: Some(catalog),
            cdc_table_desc: Some(self.core.cdc_table_desc.to_protobuf()),
            rate_limit: self.base.ctx().overwrite_options().backfill_rate_limit,
            disable_backfill: options.disable_backfill,
            options: Some(options),
        });

        // plan: merge -> filter -> exchange(simple) -> stream_scan
        Ok(PbStreamNode {
            fields: self.schema().to_prost(),
            input: vec![exchange_stream_node],
            node_body: Some(stream_scan_body),
            stream_key,
            operator_id: self.base.id().0 as u64,
            identity: self.distill_to_string(),
            append_only: self.append_only(),
        })
    }

    // The filter node receive input chunks in `(payload, _rw_offset, _rw_table_name)` schema
    pub fn build_cdc_filter_expr(cdc_table_name: &str) -> ExprImpl {
        // filter by the `_rw_table_name` column
        FunctionCall::new(
            ExprType::Equal,
            vec![
                InputRef::new(2, DataType::Varchar).into(),
                ExprImpl::literal_varchar(cdc_table_name.into()),
            ],
        )
        .unwrap()
        .into()
    }
}

impl ExprRewritable for StreamCdcTableScan {
    fn has_rewritable_expr(&self) -> bool {
        true
    }

    fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef {
        let core = self.core.clone();
        core.rewrite_exprs(r);
        Self::new(core).into()
    }
}

impl ExprVisitable for StreamCdcTableScan {
    fn visit_exprs(&self, v: &mut dyn ExprVisitor) {
        self.core.visit_exprs(v);
    }
}

#[cfg(test)]
mod tests {
    use std::str::FromStr;

    use risingwave_common::row::OwnedRow;
    use risingwave_common::types::{JsonbVal, ScalarImpl};

    use super::*;

    #[tokio::test]
    async fn test_cdc_filter_expr() {
        let t1_json = JsonbVal::from_str(r#"{ "before": null, "after": { "v": 111, "v2": 222.2 }, "source": { "version": "2.2.0.Alpha3", "connector": "mysql", "name": "dbserver1", "ts_ms": 1678428689000, "snapshot": "false", "db": "inventory", "sequence": null, "table": "t1", "server_id": 223344, "gtid": null, "file": "mysql-bin.000003", "pos": 774, "row": 0, "thread": 8, "query": null }, "op": "c", "ts_ms": 1678428689389, "transaction": null }"#).unwrap();
        let t2_json = JsonbVal::from_str(r#"{ "before": null, "after": { "v": 333, "v2": 666.6 }, "source": { "version": "2.2.0.Alpha3", "connector": "mysql", "name": "dbserver1", "ts_ms": 1678428689000, "snapshot": "false", "db": "inventory", "sequence": null, "table": "t2", "server_id": 223344, "gtid": null, "file": "mysql-bin.000003", "pos": 884, "row": 0, "thread": 8, "query": null }, "op": "c", "ts_ms": 1678428689389, "transaction": null }"#).unwrap();

        // NOTES: transaction metadata column expects to be filtered out before going to cdc filter
        let trx_json = JsonbVal::from_str(r#"{"data_collections": null, "event_count": null, "id": "35319:3962662584", "status": "BEGIN", "ts_ms": 1704263537068}"#).unwrap();
        let row1 = OwnedRow::new(vec![
            Some(t1_json.into()),
            Some(r#"{"file": "1.binlog", "pos": 100}"#.into()),
            Some("public.t2".into()),
        ]);
        let row2 = OwnedRow::new(vec![
            Some(t2_json.into()),
            Some(r#"{"file": "2.binlog", "pos": 100}"#.into()),
            Some("abs.t2".into()),
        ]);

        let row3 = OwnedRow::new(vec![
            Some(trx_json.into()),
            Some(r#"{"file": "3.binlog", "pos": 100}"#.into()),
            Some("public.t2".into()),
        ]);

        let filter_expr = StreamCdcTableScan::build_cdc_filter_expr("public.t2");
        assert_eq!(
            filter_expr.eval_row(&row1).await.unwrap(),
            Some(ScalarImpl::Bool(true))
        );
        assert_eq!(
            filter_expr.eval_row(&row2).await.unwrap(),
            Some(ScalarImpl::Bool(false))
        );
        assert_eq!(
            filter_expr.eval_row(&row3).await.unwrap(),
            Some(ScalarImpl::Bool(true))
        )
    }
}