risingwave_frontend/optimizer/plan_node/
batch_postgres_query.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
// 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 pretty_xmlish::XmlNode;
use risingwave_pb::batch_plan::plan_node::NodeBody;
use risingwave_pb::batch_plan::PostgresQueryNode;

use super::batch::prelude::*;
use super::utils::{childless_record, column_names_pretty, Distill};
use super::{
    generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch, ToLocalBatch,
};
use crate::error::Result;
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::property::{Distribution, Order};

#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct BatchPostgresQuery {
    pub base: PlanBase<Batch>,
    pub core: generic::PostgresQuery,
}

impl BatchPostgresQuery {
    pub fn new(core: generic::PostgresQuery) -> Self {
        let base = PlanBase::new_batch_with_core(&core, Distribution::Single, Order::any());

        Self { base, core }
    }

    pub fn column_names(&self) -> Vec<&str> {
        self.schema().names_str()
    }

    pub fn clone_with_dist(&self) -> Self {
        let base = self.base.clone_with_new_distribution(Distribution::Single);
        Self {
            base,
            core: self.core.clone(),
        }
    }
}

impl_plan_tree_node_for_leaf! { BatchPostgresQuery }

impl Distill for BatchPostgresQuery {
    fn distill<'a>(&self) -> XmlNode<'a> {
        let fields = vec![("columns", column_names_pretty(self.schema()))];
        childless_record("BatchPostgresQuery", fields)
    }
}

impl ToLocalBatch for BatchPostgresQuery {
    fn to_local(&self) -> Result<PlanRef> {
        Ok(self.clone_with_dist().into())
    }
}

impl ToDistributedBatch for BatchPostgresQuery {
    fn to_distributed(&self) -> Result<PlanRef> {
        Ok(self.clone_with_dist().into())
    }
}

impl ToBatchPb for BatchPostgresQuery {
    fn to_batch_prost_body(&self) -> NodeBody {
        NodeBody::PostgresQuery(PostgresQueryNode {
            columns: self
                .core
                .columns()
                .iter()
                .map(|c| c.to_protobuf())
                .collect(),
            hostname: self.core.hostname.clone(),
            port: self.core.port.clone(),
            username: self.core.username.clone(),
            password: self.core.password.clone(),
            database: self.core.database.clone(),
            query: self.core.query.clone(),
        })
    }
}

impl ExprRewritable for BatchPostgresQuery {}

impl ExprVisitable for BatchPostgresQuery {}