risingwave_frontend/optimizer/plan_visitor/
relation_collector_visitor.rsuse std::collections::HashSet;
use risingwave_common::catalog::TableId;
use super::{DefaultBehavior, DefaultValue};
use crate::optimizer::plan_node::{BatchSource, LogicalScan, StreamSource, StreamTableScan};
use crate::optimizer::plan_visitor::PlanVisitor;
use crate::PlanRef;
#[derive(Debug, Clone, Default)]
pub struct RelationCollectorVisitor {
relations: HashSet<TableId>,
}
impl RelationCollectorVisitor {
fn new_with(relations: HashSet<TableId>) -> Self {
Self { relations }
}
pub fn collect_with(relations: HashSet<TableId>, plan: PlanRef) -> HashSet<TableId> {
let mut visitor = Self::new_with(relations);
visitor.visit(plan);
visitor.relations
}
}
impl PlanVisitor for RelationCollectorVisitor {
type Result = ();
type DefaultBehavior = impl DefaultBehavior<Self::Result>;
fn default_behavior() -> Self::DefaultBehavior {
DefaultValue
}
fn visit_batch_seq_scan(&mut self, plan: &crate::optimizer::plan_node::BatchSeqScan) {
self.relations.insert(plan.core().table_desc.table_id);
}
fn visit_logical_scan(&mut self, plan: &LogicalScan) {
self.relations.insert(plan.table_desc().table_id);
}
fn visit_stream_table_scan(&mut self, plan: &StreamTableScan) {
let logical = plan.core();
self.relations.insert(logical.table_desc.table_id);
}
fn visit_batch_source(&mut self, plan: &BatchSource) {
if let Some(catalog) = plan.source_catalog() {
self.relations.insert(catalog.id.into());
}
}
fn visit_stream_source(&mut self, plan: &StreamSource) {
if let Some(catalog) = plan.source_catalog() {
self.relations.insert(catalog.id.into());
}
}
}