risingwave_frontend/optimizer/plan_visitor/
relation_collector_visitor.rs1use std::collections::HashSet;
16
17use risingwave_common::catalog::TableId;
18
19use super::{DefaultBehavior, DefaultValue};
20use crate::PlanRef;
21use crate::optimizer::plan_node::{BatchSource, LogicalScan, StreamSource, StreamTableScan};
22use crate::optimizer::plan_visitor::PlanVisitor;
23
24#[derive(Debug, Clone, Default)]
26pub struct RelationCollectorVisitor {
27 relations: HashSet<TableId>,
28}
29
30impl RelationCollectorVisitor {
31 fn new_with(relations: HashSet<TableId>) -> Self {
32 Self { relations }
33 }
34
35 pub fn collect_with(relations: HashSet<TableId>, plan: PlanRef) -> HashSet<TableId> {
40 let mut visitor = Self::new_with(relations);
41 visitor.visit(plan);
42 visitor.relations
43 }
44}
45
46impl PlanVisitor for RelationCollectorVisitor {
47 type Result = ();
48
49 type DefaultBehavior = impl DefaultBehavior<Self::Result>;
50
51 fn default_behavior() -> Self::DefaultBehavior {
52 DefaultValue
53 }
54
55 fn visit_batch_seq_scan(&mut self, plan: &crate::optimizer::plan_node::BatchSeqScan) {
56 self.relations.insert(plan.core().table_desc.table_id);
57 }
58
59 fn visit_logical_scan(&mut self, plan: &LogicalScan) {
60 self.relations.insert(plan.table_desc().table_id);
61 }
62
63 fn visit_stream_table_scan(&mut self, plan: &StreamTableScan) {
64 let logical = plan.core();
65 self.relations.insert(logical.table_desc.table_id);
66 }
67
68 fn visit_batch_source(&mut self, plan: &BatchSource) {
69 if let Some(catalog) = plan.source_catalog() {
70 self.relations.insert(catalog.id.into());
71 }
72 }
73
74 fn visit_stream_source(&mut self, plan: &StreamSource) {
75 if let Some(catalog) = plan.source_catalog() {
76 self.relations.insert(catalog.id.into());
77 }
78 }
79}