risingwave_frontend/optimizer/plan_node/
stream_materialize.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use std::assert_matches::assert_matches;
16use std::num::NonZeroU32;
17
18use fixedbitset::FixedBitSet;
19use itertools::Itertools;
20use pretty_xmlish::{Pretty, XmlNode};
21use risingwave_common::catalog::{
22    ColumnCatalog, ConflictBehavior, CreateType, Engine, StreamJobStatus, TableId,
23};
24use risingwave_common::hash::VnodeCount;
25use risingwave_common::id::FragmentId;
26use risingwave_common::types::DataType;
27use risingwave_common::util::column_index_mapping::ColIndexMapping;
28use risingwave_common::util::iter_util::ZipEqFast;
29use risingwave_common::util::sort_util::{ColumnOrder, OrderType};
30use risingwave_pb::catalog::PbWebhookSourceInfo;
31use risingwave_pb::stream_plan::stream_node::PbNodeBody;
32
33use super::derive::derive_columns;
34use super::stream::prelude::*;
35use super::utils::{Distill, TableCatalogBuilder, childless_record};
36use super::{
37    ExprRewritable, PlanTreeNodeUnary, StreamNode, StreamPlanRef as PlanRef, reorganize_elements_id,
38};
39use crate::catalog::table_catalog::{TableCatalog, TableType, TableVersion};
40use crate::catalog::{DatabaseId, SchemaId};
41use crate::error::Result;
42use crate::optimizer::StreamOptimizedLogicalPlanRoot;
43use crate::optimizer::plan_node::derive::derive_pk;
44use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
45use crate::optimizer::plan_node::utils::plan_can_use_background_ddl;
46use crate::optimizer::plan_node::{PlanBase, PlanNodeMeta};
47use crate::optimizer::property::{Cardinality, Distribution, Order, RequiredDist};
48use crate::stream_fragmenter::BuildFragmentGraphState;
49
50/// Materializes a stream.
51#[derive(Debug, Clone, PartialEq, Eq, Hash)]
52pub struct StreamMaterialize {
53    pub base: PlanBase<Stream>,
54    /// Child of Materialize plan
55    input: PlanRef,
56    table: TableCatalog,
57    /// For refreshable tables, staging table for collecting new data during refresh
58    staging_table: Option<TableCatalog>,
59    /// For refreshable tables, progress table for tracking refresh progress
60    refresh_progress_table: Option<TableCatalog>,
61}
62
63impl StreamMaterialize {
64    pub fn new(input: PlanRef, table: TableCatalog) -> Result<Self> {
65        Self::new_with_staging_and_progress(input, table, None, None)
66    }
67
68    pub fn new_with_staging_and_progress(
69        input: PlanRef,
70        table: TableCatalog,
71        staging_table: Option<TableCatalog>,
72        refresh_progress_table: Option<TableCatalog>,
73    ) -> Result<Self> {
74        let kind = match table.conflict_behavior() {
75            ConflictBehavior::NoCheck => {
76                reject_upsert_input!(input, "Materialize without conflict handling")
77            }
78
79            // When conflict handling is enabled, upsert stream can be converted to retract stream.
80            ConflictBehavior::Overwrite
81            | ConflictBehavior::IgnoreConflict
82            | ConflictBehavior::DoUpdateIfNotNull => match input.stream_kind() {
83                StreamKind::AppendOnly => StreamKind::AppendOnly,
84                StreamKind::Retract | StreamKind::Upsert => StreamKind::Retract,
85            },
86        };
87        let base = PlanBase::new_stream(
88            input.ctx(),
89            input.schema().clone(),
90            Some(table.stream_key()),
91            input.functional_dependency().clone(),
92            input.distribution().clone(),
93            kind,
94            input.emit_on_window_close(),
95            input.watermark_columns().clone(),
96            input.columns_monotonicity().clone(),
97        );
98
99        Ok(Self {
100            base,
101            input,
102            table,
103            staging_table,
104            refresh_progress_table,
105        })
106    }
107
108    /// Create a materialize node, for `MATERIALIZED VIEW` and `INDEX`.
109    ///
110    /// When creating index, `TableType` should be `Index`. Then, materialize will distribute keys
111    /// using `user_distributed_by`.
112    pub fn create(
113        StreamOptimizedLogicalPlanRoot {
114            plan: input,
115            required_dist: user_distributed_by,
116            required_order: user_order_by,
117            out_fields: user_cols,
118            out_names,
119            ..
120        }: StreamOptimizedLogicalPlanRoot,
121        name: String,
122        database_id: DatabaseId,
123        schema_id: SchemaId,
124        definition: String,
125        table_type: TableType,
126        cardinality: Cardinality,
127        retention_seconds: Option<NonZeroU32>,
128    ) -> Result<Self> {
129        let input = Self::rewrite_input(input, user_distributed_by.clone(), table_type)?;
130        // the hidden column name might refer some expr id
131        let input = reorganize_elements_id(input);
132        let columns = derive_columns(input.schema(), out_names, &user_cols)?;
133
134        let create_type = if matches!(table_type, TableType::MaterializedView)
135            && input.ctx().session_ctx().config().background_ddl()
136            && plan_can_use_background_ddl(&input)
137        {
138            CreateType::Background
139        } else {
140            CreateType::Foreground
141        };
142
143        // For upsert stream, use `Overwrite` conflict behavior to convert into retract stream.
144        let conflict_behavior = match input.stream_kind() {
145            StreamKind::Retract | StreamKind::AppendOnly => ConflictBehavior::NoCheck,
146            StreamKind::Upsert => ConflictBehavior::Overwrite,
147        };
148
149        let table = Self::derive_table_catalog(
150            input.clone(),
151            name,
152            database_id,
153            schema_id,
154            user_distributed_by,
155            user_order_by,
156            columns,
157            definition,
158            conflict_behavior,
159            vec![],
160            None,
161            None,
162            table_type,
163            None,
164            cardinality,
165            retention_seconds,
166            create_type,
167            None,
168            Engine::Hummock,
169            false,
170        )?;
171
172        Self::new(input, table)
173    }
174
175    /// Create a materialize node, for `TABLE`.
176    ///
177    /// Different from `create`, the `columns` are passed in directly, instead of being derived from
178    /// the input. So the column IDs are preserved from the SQL columns binding step and will be
179    /// consistent with the source node and DML node.
180    #[allow(clippy::too_many_arguments)]
181    pub fn create_for_table(
182        input: PlanRef,
183        name: String,
184        database_id: DatabaseId,
185        schema_id: SchemaId,
186        user_distributed_by: RequiredDist,
187        user_order_by: Order,
188        columns: Vec<ColumnCatalog>,
189        definition: String,
190        conflict_behavior: ConflictBehavior,
191        version_column_indices: Vec<usize>,
192        pk_column_indices: Vec<usize>,
193        row_id_index: Option<usize>,
194        version: TableVersion,
195        retention_seconds: Option<NonZeroU32>,
196        webhook_info: Option<PbWebhookSourceInfo>,
197        engine: Engine,
198        refreshable: bool,
199    ) -> Result<Self> {
200        let input = Self::rewrite_input(input, user_distributed_by.clone(), TableType::Table)?;
201
202        let table = Self::derive_table_catalog(
203            input.clone(),
204            name.clone(),
205            database_id,
206            schema_id,
207            user_distributed_by,
208            user_order_by,
209            columns,
210            definition,
211            conflict_behavior,
212            version_column_indices,
213            Some(pk_column_indices),
214            row_id_index,
215            TableType::Table,
216            Some(version),
217            Cardinality::unknown(), // unknown cardinality for tables
218            retention_seconds,
219            CreateType::Foreground,
220            webhook_info,
221            engine,
222            refreshable,
223        )?;
224
225        // For refreshable tables, create staging table and progress table
226        let (staging_table, refresh_progress_table) = if refreshable {
227            let staging = Some(Self::derive_staging_table_catalog(table.clone()));
228            let progress = Some(Self::derive_refresh_progress_table_catalog(table.clone()));
229            (staging, progress)
230        } else {
231            (None, None)
232        };
233
234        tracing::info!(
235            table_name = %name,
236            refreshable = %refreshable,
237            has_staging_table = %staging_table.is_some(),
238            has_progress_table = %refresh_progress_table.is_some(),
239            "Creating StreamMaterialize with staging and progress table info"
240        );
241
242        Self::new_with_staging_and_progress(input, table, staging_table, refresh_progress_table)
243    }
244
245    /// Rewrite the input to satisfy the required distribution if necessary, according to the type.
246    fn rewrite_input(
247        input: PlanRef,
248        user_distributed_by: RequiredDist,
249        table_type: TableType,
250    ) -> Result<PlanRef> {
251        let required_dist = match input.distribution() {
252            Distribution::Single => RequiredDist::single(),
253            _ => match table_type {
254                TableType::Table => {
255                    assert_matches!(
256                        user_distributed_by,
257                        RequiredDist::ShardByKey(_) | RequiredDist::ShardByExactKey(_)
258                    );
259                    user_distributed_by
260                }
261                TableType::MaterializedView => {
262                    assert_matches!(user_distributed_by, RequiredDist::Any);
263                    // ensure the same pk will not shuffle to different node
264                    let required_dist =
265                        RequiredDist::shard_by_key(input.schema().len(), input.expect_stream_key());
266
267                    // If the input is a stream join, enforce the stream key as the materialized
268                    // view distribution key to avoid slow backfilling caused by
269                    // data skew of the dimension table join key.
270                    // See <https://github.com/risingwavelabs/risingwave/issues/12824> for more information.
271                    let is_stream_join = matches!(input.as_stream_hash_join(), Some(_join))
272                        || matches!(input.as_stream_temporal_join(), Some(_join))
273                        || matches!(input.as_stream_delta_join(), Some(_join));
274
275                    if is_stream_join {
276                        return Ok(required_dist.stream_enforce(input));
277                    }
278
279                    required_dist
280                }
281                TableType::Index => {
282                    assert_matches!(
283                        user_distributed_by,
284                        RequiredDist::PhysicalDist(Distribution::HashShard(_))
285                    );
286                    user_distributed_by
287                }
288                TableType::VectorIndex => {
289                    unreachable!("VectorIndex should not be created by StreamMaterialize")
290                }
291                TableType::Internal => unreachable!(),
292            },
293        };
294
295        required_dist.streaming_enforce_if_not_satisfies(input)
296    }
297
298    /// Derive the table catalog with the given arguments.
299    ///
300    /// - The caller must ensure the validity of the given `columns`.
301    /// - The `rewritten_input` should be generated by `rewrite_input`.
302    #[expect(clippy::too_many_arguments)]
303    fn derive_table_catalog(
304        rewritten_input: PlanRef,
305        name: String,
306        database_id: DatabaseId,
307        schema_id: SchemaId,
308        user_distributed_by: RequiredDist,
309        user_order_by: Order,
310        columns: Vec<ColumnCatalog>,
311        definition: String,
312        conflict_behavior: ConflictBehavior,
313        version_column_indices: Vec<usize>,
314        pk_column_indices: Option<Vec<usize>>, // Is some when create table
315        row_id_index: Option<usize>,
316        table_type: TableType,
317        version: Option<TableVersion>,
318        cardinality: Cardinality,
319        retention_seconds: Option<NonZeroU32>,
320        create_type: CreateType,
321        webhook_info: Option<PbWebhookSourceInfo>,
322        engine: Engine,
323        refreshable: bool,
324    ) -> Result<TableCatalog> {
325        let input = rewritten_input;
326
327        let value_indices = (0..columns.len()).collect_vec();
328        let distribution_key = input.distribution().dist_column_indices().to_vec();
329        let append_only = input.append_only();
330        // TODO(rc): In `TableCatalog` we still use `FixedBitSet` for watermark columns, ignoring the watermark group information.
331        // We will record the watermark group information in `TableCatalog` in the future. For now, let's flatten the watermark columns.
332        let watermark_columns = input.watermark_columns().indices().collect();
333
334        let (table_pk, stream_key) = if let Some(pk_column_indices) = pk_column_indices {
335            let table_pk = pk_column_indices
336                .iter()
337                .map(|idx| ColumnOrder::new(*idx, OrderType::ascending()))
338                .collect();
339            // No order by for create table, so stream key is identical to table pk.
340            (table_pk, pk_column_indices)
341        } else {
342            derive_pk(input, user_distributed_by, user_order_by, &columns)
343        };
344        // assert: `stream_key` is a subset of `table_pk`
345
346        let read_prefix_len_hint = table_pk.len();
347        Ok(TableCatalog {
348            id: TableId::placeholder(),
349            schema_id,
350            database_id,
351            associated_source_id: None,
352            name,
353            columns,
354            pk: table_pk,
355            stream_key,
356            distribution_key,
357            table_type,
358            append_only,
359            owner: risingwave_common::catalog::DEFAULT_SUPER_USER_ID,
360            fragment_id: FragmentId::placeholder(),
361            dml_fragment_id: None,
362            vnode_col_index: None,
363            row_id_index,
364            value_indices,
365            definition,
366            conflict_behavior,
367            version_column_indices,
368            read_prefix_len_hint,
369            version,
370            watermark_columns,
371            dist_key_in_pk: vec![],
372            cardinality,
373            created_at_epoch: None,
374            initialized_at_epoch: None,
375            cleaned_by_watermark: false,
376            create_type,
377            stream_job_status: StreamJobStatus::Creating,
378            description: None,
379            initialized_at_cluster_version: None,
380            created_at_cluster_version: None,
381            retention_seconds: retention_seconds.map(|i| i.into()),
382            cdc_table_id: None,
383            vnode_count: VnodeCount::Placeholder, // will be filled in by the meta service later
384            webhook_info,
385            job_id: None,
386            engine: match table_type {
387                TableType::Table => engine,
388                TableType::MaterializedView
389                | TableType::Index
390                | TableType::Internal
391                | TableType::VectorIndex => {
392                    assert_eq!(engine, Engine::Hummock);
393                    engine
394                }
395            },
396            clean_watermark_index_in_pk: None, // TODO: fill this field
397            refreshable,
398            vector_index_info: None,
399            cdc_table_type: None,
400        })
401    }
402
403    /// The staging table is a pk-only table.
404    fn derive_staging_table_catalog(
405        TableCatalog {
406            id,
407            schema_id,
408            database_id,
409            associated_source_id,
410            name,
411            columns,
412            pk,
413            stream_key,
414            table_type: _,
415            distribution_key,
416            append_only,
417            cardinality,
418            owner,
419            retention_seconds,
420            fragment_id,
421            dml_fragment_id: _,
422            vnode_col_index,
423            row_id_index,
424            value_indices: _,
425            definition,
426            conflict_behavior,
427            version_column_indices,
428            read_prefix_len_hint,
429            version,
430            watermark_columns: _,
431            dist_key_in_pk,
432            created_at_epoch,
433            initialized_at_epoch,
434            cleaned_by_watermark,
435            create_type,
436            stream_job_status,
437            description,
438            created_at_cluster_version,
439            initialized_at_cluster_version,
440            cdc_table_id,
441            vnode_count,
442            webhook_info,
443            job_id,
444            engine,
445            clean_watermark_index_in_pk,
446            refreshable,
447            vector_index_info,
448            cdc_table_type,
449        }: TableCatalog,
450    ) -> TableCatalog {
451        tracing::info!(
452            table_name = %name,
453            "Creating staging table for refreshable table"
454        );
455
456        assert!(row_id_index.is_none());
457        assert!(retention_seconds.is_none());
458        assert!(refreshable);
459
460        // only keep pk columns
461        let mut pk_col_indices = vec![];
462        let mut pk_cols = vec![];
463        for (i, col) in columns.iter().enumerate() {
464            if pk.iter().any(|pk| pk.column_index == i) {
465                pk_col_indices.push(i);
466                pk_cols.push(col.clone());
467            }
468        }
469        let mapping = ColIndexMapping::with_remaining_columns(&pk_col_indices, columns.len());
470
471        TableCatalog {
472            id,
473            schema_id,
474            database_id,
475            associated_source_id,
476            name,
477            value_indices: (0..pk_cols.len()).collect(),
478            columns: pk_cols,
479            pk: pk
480                .iter()
481                .map(|pk| ColumnOrder::new(mapping.map(pk.column_index), pk.order_type))
482                .collect(),
483            stream_key: mapping.try_map_all(stream_key).unwrap(),
484            vnode_col_index: vnode_col_index.map(|i| mapping.map(i)),
485            dist_key_in_pk: mapping.try_map_all(dist_key_in_pk).unwrap(),
486            distribution_key: mapping.try_map_all(distribution_key).unwrap(),
487            table_type: TableType::Internal,
488            watermark_columns: FixedBitSet::new(),
489            append_only,
490            cardinality,
491            owner,
492            retention_seconds: None,
493            fragment_id,
494            dml_fragment_id: None,
495            row_id_index: None,
496            definition,
497            conflict_behavior,
498            version_column_indices,
499            read_prefix_len_hint,
500            version,
501            created_at_epoch,
502            initialized_at_epoch,
503            cleaned_by_watermark,
504            create_type,
505            stream_job_status,
506            description,
507            created_at_cluster_version,
508            initialized_at_cluster_version,
509            cdc_table_id,
510            vnode_count,
511            webhook_info,
512            job_id,
513            engine,
514            clean_watermark_index_in_pk,
515            refreshable: false,
516            vector_index_info,
517            cdc_table_type,
518        }
519    }
520
521    /// The refresh progress table is used to track refresh operation progress.
522    /// Simplified Schema: vnode (i32), `current_pos`... (variable PK from upstream),
523    /// `is_completed` (bool), `processed_rows` (i64)
524    fn derive_refresh_progress_table_catalog(table: TableCatalog) -> TableCatalog {
525        tracing::debug!(
526            table_name = %table.name,
527            "Creating refresh progress table for refreshable table"
528        );
529
530        // Define the simplified schema for the refresh progress table
531        // Schema: | vnode | current_pos... | is_completed | processed_rows |
532        let mut columns = vec![ColumnCatalog {
533            column_desc: risingwave_common::catalog::ColumnDesc::named(
534                "vnode",
535                0.into(),
536                DataType::Int16,
537            ),
538            is_hidden: false,
539        }];
540
541        // Add current_pos columns (mirror upstream table's primary key)
542        let mut col_index = 1;
543        for pk_col in &table.pk {
544            let upstream_col = &table.columns[pk_col.column_index];
545            columns.push(ColumnCatalog {
546                column_desc: risingwave_common::catalog::ColumnDesc::named(
547                    format!("pos_{}", upstream_col.name()),
548                    col_index.into(),
549                    upstream_col.data_type().clone(),
550                ),
551                is_hidden: false,
552            });
553            col_index += 1;
554        }
555
556        // Add metadata columns
557        for (name, data_type) in [
558            ("is_completed", DataType::Boolean),
559            ("processed_rows", DataType::Int64),
560        ] {
561            columns.push(ColumnCatalog {
562                column_desc: risingwave_common::catalog::ColumnDesc::named(
563                    name,
564                    col_index.into(),
565                    data_type,
566                ),
567                is_hidden: false,
568            });
569            col_index += 1;
570        }
571
572        let mut builder = TableCatalogBuilder::default();
573
574        // Add all columns to builder
575        for column in &columns {
576            builder.add_column(&(&column.column_desc).into());
577        }
578
579        // Primary key is vnode (column 0)
580        builder.add_order_column(0, OrderType::ascending());
581        builder.set_vnode_col_idx(0);
582        builder.set_value_indices((0..columns.len()).collect());
583        builder.set_dist_key_in_pk(vec![0]);
584
585        builder.build(vec![0], 1)
586    }
587
588    /// Get a reference to the stream materialize's table.
589    #[must_use]
590    pub fn table(&self) -> &TableCatalog {
591        &self.table
592    }
593
594    /// Get a reference to the stream materialize's staging table.
595    #[must_use]
596    pub fn staging_table(&self) -> Option<&TableCatalog> {
597        self.staging_table.as_ref()
598    }
599
600    /// Get a reference to the stream materialize's refresh progress table.
601    #[must_use]
602    pub fn refresh_progress_table(&self) -> Option<&TableCatalog> {
603        self.refresh_progress_table.as_ref()
604    }
605
606    pub fn name(&self) -> &str {
607        self.table.name()
608    }
609}
610
611impl Distill for StreamMaterialize {
612    fn distill<'a>(&self) -> XmlNode<'a> {
613        let table = self.table();
614
615        let column_names = (table.columns.iter())
616            .map(|col| col.name_with_hidden().to_string())
617            .map(Pretty::from)
618            .collect();
619
620        let stream_key = (table.stream_key().iter())
621            .map(|&k| table.columns[k].name().to_owned())
622            .map(Pretty::from)
623            .collect();
624
625        let pk_columns = (table.pk.iter())
626            .map(|o| table.columns[o.column_index].name().to_owned())
627            .map(Pretty::from)
628            .collect();
629        let mut vec = Vec::with_capacity(5);
630        vec.push(("columns", Pretty::Array(column_names)));
631        vec.push(("stream_key", Pretty::Array(stream_key)));
632        vec.push(("pk_columns", Pretty::Array(pk_columns)));
633        let pk_conflict_behavior = self.table.conflict_behavior().debug_to_string();
634
635        vec.push(("pk_conflict", Pretty::from(pk_conflict_behavior)));
636
637        let watermark_columns = &self.base.watermark_columns();
638        if self.base.watermark_columns().n_indices() > 0 {
639            // TODO(rc): we ignore the watermark group info here, will be fixed it later
640            let watermark_column_names = watermark_columns
641                .indices()
642                .map(|i| table.columns()[i].name_with_hidden().to_string())
643                .map(Pretty::from)
644                .collect();
645            vec.push(("watermark_columns", Pretty::Array(watermark_column_names)));
646        };
647        childless_record("StreamMaterialize", vec)
648    }
649}
650
651impl PlanTreeNodeUnary<Stream> for StreamMaterialize {
652    fn input(&self) -> PlanRef {
653        self.input.clone()
654    }
655
656    fn clone_with_input(&self, input: PlanRef) -> Self {
657        let new = Self::new_with_staging_and_progress(
658            input,
659            self.table().clone(),
660            self.staging_table.clone(),
661            self.refresh_progress_table.clone(),
662        )
663        .unwrap();
664        new.base
665            .schema()
666            .fields
667            .iter()
668            .zip_eq_fast(self.base.schema().fields.iter())
669            .for_each(|(a, b)| {
670                assert_eq!(a.data_type, b.data_type);
671            });
672        assert_eq!(new.plan_base().stream_key(), self.plan_base().stream_key());
673        new
674    }
675}
676
677impl_plan_tree_node_for_unary! { Stream, StreamMaterialize }
678
679impl StreamNode for StreamMaterialize {
680    fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> PbNodeBody {
681        use risingwave_pb::stream_plan::*;
682
683        tracing::debug!(
684            table_name = %self.table().name(),
685            refreshable = %self.table().refreshable,
686            has_staging_table = %self.staging_table.is_some(),
687            has_progress_table = %self.refresh_progress_table.is_some(),
688            staging_table_name = ?self.staging_table.as_ref().map(|t| (&t.id, &t.name)),
689            progress_table_name = ?self.refresh_progress_table.as_ref().map(|t| (&t.id, &t.name)),
690            "Converting StreamMaterialize to protobuf"
691        );
692
693        let staging_table_prost = self
694            .staging_table
695            .clone()
696            .map(|t| t.with_id(state.gen_table_id_wrapped()).to_prost());
697
698        let refresh_progress_table_prost = self
699            .refresh_progress_table
700            .clone()
701            .map(|t| t.with_id(state.gen_table_id_wrapped()).to_prost());
702
703        PbNodeBody::Materialize(Box::new(MaterializeNode {
704            // Do not fill `table` and `table_id` here to avoid duplication. It will be filled by
705            // meta service after global information is generated.
706            table_id: 0.into(),
707            table: None,
708            // Pass staging table catalog if available for refreshable tables
709            staging_table: staging_table_prost,
710            // Pass refresh progress table catalog if available for refreshable tables
711            refresh_progress_table: refresh_progress_table_prost,
712
713            column_orders: self
714                .table()
715                .pk()
716                .iter()
717                .copied()
718                .map(ColumnOrder::to_protobuf)
719                .collect(),
720        }))
721    }
722}
723
724impl ExprRewritable<Stream> for StreamMaterialize {}
725
726impl ExprVisitable for StreamMaterialize {}