risingwave_frontend/optimizer/plan_rewriter/
share_source_rewriter.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
// 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 std::collections::{HashMap, HashSet};

use itertools::Itertools;

use crate::catalog::SourceId;
use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::{
    LogicalShare, LogicalSource, PlanNodeId, PlanTreeNode, StreamShare,
};
use crate::optimizer::plan_visitor::{DefaultBehavior, DefaultValue};
use crate::optimizer::{PlanRewriter, PlanVisitor};
use crate::PlanRef;

#[derive(Debug, Clone, Default)]
pub struct ShareSourceRewriter {
    /// Source id need to be shared.
    share_ids: HashSet<SourceId>,
    /// Source id to share node.
    share_source: HashMap<SourceId, PlanRef>,
    /// Original share node plan id to new share node.
    /// Rewriter will rewrite all nodes, but we need to keep the shape of the DAG.
    share_map: HashMap<PlanNodeId, PlanRef>,
}

#[derive(Debug, Clone, Default)]
struct SourceCounter {
    /// Source id to count.
    source_counter: HashMap<SourceId, usize>,
}

impl ShareSourceRewriter {
    pub fn share_source(plan: PlanRef) -> PlanRef {
        // Find which sources occurred more than once.
        let mut source_counter = SourceCounter::default();
        source_counter.visit(plan.clone());

        let mut share_source_rewriter = ShareSourceRewriter {
            share_ids: source_counter
                .source_counter
                .into_iter()
                .filter(|(_, v)| *v > 1)
                .map(|(k, _)| k)
                .collect(),
            share_source: Default::default(),
            share_map: Default::default(),
        };
        // Rewrite source to share source
        share_source_rewriter.rewrite(plan)
    }
}

impl PlanRewriter for ShareSourceRewriter {
    fn rewrite_logical_source(&mut self, source: &LogicalSource) -> PlanRef {
        let source_id = match &source.core.catalog {
            Some(s) => s.id,
            None => {
                return source.clone().into();
            }
        };
        if !self.share_ids.contains(&source_id) {
            let source_ref = source.clone().into();
            return source_ref;
        }
        match self.share_source.get(&source_id) {
            None => {
                let source_ref = source.clone().into();
                let share_source = LogicalShare::create(source_ref);
                self.share_source.insert(source_id, share_source.clone());
                share_source
            }
            Some(share_source) => share_source.clone(),
        }
    }

    fn rewrite_logical_share(&mut self, share: &LogicalShare) -> PlanRef {
        // When we use the plan rewriter, we need to take care of the share operator,
        // because our plan is a DAG rather than a tree.
        match self.share_map.get(&share.id()) {
            None => {
                let new_inputs = share
                    .inputs()
                    .into_iter()
                    .map(|input| self.rewrite(input))
                    .collect_vec();
                let new_share = share.clone_with_inputs(&new_inputs);
                self.share_map.insert(share.id(), new_share.clone());
                new_share
            }
            Some(new_share) => new_share.clone(),
        }
    }

    fn rewrite_stream_share(&mut self, _share: &StreamShare) -> PlanRef {
        // We only access logical node here, so stream share is unreachable.
        unreachable!()
    }
}

impl PlanVisitor for SourceCounter {
    type Result = ();

    type DefaultBehavior = impl DefaultBehavior<Self::Result>;

    fn default_behavior() -> Self::DefaultBehavior {
        DefaultValue
    }

    fn visit_logical_source(&mut self, source: &LogicalSource) {
        if let Some(source) = &source.core.catalog {
            self.source_counter
                .entry(source.id)
                .and_modify(|count| *count += 1)
                .or_insert(1);
        }
    }
}