risingwave_frontend/optimizer/plan_node/
batch_postgres_query.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 pretty_xmlish::XmlNode;
16use risingwave_pb::batch_plan::PostgresQueryNode;
17use risingwave_pb::batch_plan::plan_node::NodeBody;
18
19use super::batch::prelude::*;
20use super::utils::{Distill, childless_record, column_names_pretty};
21use super::{
22    ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch, ToLocalBatch, generic,
23};
24use crate::error::Result;
25use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
26use crate::optimizer::property::{Distribution, Order};
27
28#[derive(Debug, Clone, PartialEq, Eq, Hash)]
29pub struct BatchPostgresQuery {
30    pub base: PlanBase<Batch>,
31    pub core: generic::PostgresQuery,
32}
33
34impl BatchPostgresQuery {
35    pub fn new(core: generic::PostgresQuery) -> Self {
36        let base = PlanBase::new_batch_with_core(&core, Distribution::Single, Order::any());
37
38        Self { base, core }
39    }
40
41    pub fn column_names(&self) -> Vec<&str> {
42        self.schema().names_str()
43    }
44
45    pub fn clone_with_dist(&self) -> Self {
46        let base = self.base.clone_with_new_distribution(Distribution::Single);
47        Self {
48            base,
49            core: self.core.clone(),
50        }
51    }
52}
53
54impl_plan_tree_node_for_leaf! { BatchPostgresQuery }
55
56impl Distill for BatchPostgresQuery {
57    fn distill<'a>(&self) -> XmlNode<'a> {
58        let fields = vec![("columns", column_names_pretty(self.schema()))];
59        childless_record("BatchPostgresQuery", fields)
60    }
61}
62
63impl ToLocalBatch for BatchPostgresQuery {
64    fn to_local(&self) -> Result<PlanRef> {
65        Ok(self.clone_with_dist().into())
66    }
67}
68
69impl ToDistributedBatch for BatchPostgresQuery {
70    fn to_distributed(&self) -> Result<PlanRef> {
71        Ok(self.clone_with_dist().into())
72    }
73}
74
75impl ToBatchPb for BatchPostgresQuery {
76    fn to_batch_prost_body(&self) -> NodeBody {
77        NodeBody::PostgresQuery(PostgresQueryNode {
78            columns: self
79                .core
80                .columns()
81                .iter()
82                .map(|c| c.to_protobuf())
83                .collect(),
84            hostname: self.core.hostname.clone(),
85            port: self.core.port.clone(),
86            username: self.core.username.clone(),
87            password: self.core.password.clone(),
88            database: self.core.database.clone(),
89            query: self.core.query.clone(),
90        })
91    }
92}
93
94impl ExprRewritable for BatchPostgresQuery {}
95
96impl ExprVisitable for BatchPostgresQuery {}