risingwave_frontend/optimizer/plan_visitor/
temporal_join_validator.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 risingwave_sqlparser::ast::AsOf;
16
17use super::{DefaultBehavior, Merge};
18use crate::PlanRef;
19use crate::optimizer::plan_node::generic::GenericPlanRef;
20use crate::optimizer::plan_node::{
21    BatchSeqScan, LogicalScan, PlanTreeNodeBinary, StreamTableScan, StreamTemporalJoin,
22};
23use crate::optimizer::plan_visitor::PlanVisitor;
24
25#[derive(Debug, Clone, Default)]
26pub struct TemporalJoinValidator {
27    found_non_append_only_temporal_join: bool,
28}
29
30impl TemporalJoinValidator {
31    pub fn exist_dangling_temporal_scan(plan: PlanRef) -> bool {
32        let mut decider = TemporalJoinValidator {
33            found_non_append_only_temporal_join: false,
34        };
35        let ctx = plan.ctx();
36        let has_dangling_temporal_scan = decider.visit(plan);
37        if decider.found_non_append_only_temporal_join {
38            ctx.session_ctx().notice_to_user("A non-append-only temporal join is used in the query. It would introduce a additional memo-table comparing to append-only temporal join.");
39        }
40        has_dangling_temporal_scan
41    }
42}
43
44impl PlanVisitor for TemporalJoinValidator {
45    type Result = bool;
46
47    type DefaultBehavior = impl DefaultBehavior<Self::Result>;
48
49    fn default_behavior() -> Self::DefaultBehavior {
50        Merge(|a, b| a | b)
51    }
52
53    fn visit_stream_table_scan(&mut self, stream_table_scan: &StreamTableScan) -> bool {
54        matches!(stream_table_scan.core().as_of, Some(AsOf::ProcessTime))
55    }
56
57    fn visit_batch_seq_scan(&mut self, batch_seq_scan: &BatchSeqScan) -> bool {
58        matches!(batch_seq_scan.core().as_of, Some(AsOf::ProcessTime))
59    }
60
61    fn visit_logical_scan(&mut self, logical_scan: &LogicalScan) -> bool {
62        matches!(logical_scan.as_of(), Some(AsOf::ProcessTime))
63    }
64
65    fn visit_stream_temporal_join(&mut self, stream_temporal_join: &StreamTemporalJoin) -> bool {
66        if !stream_temporal_join.append_only() {
67            self.found_non_append_only_temporal_join = true;
68        }
69        self.visit(stream_temporal_join.left())
70    }
71}