risingwave_batch/executor/
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
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
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
// 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 anyhow::Context;
use futures_async_stream::try_stream;
use futures_util::stream::StreamExt;
use risingwave_common::catalog::{Field, Schema};
use risingwave_common::row::OwnedRow;
use risingwave_common::types::{DataType, Datum, Decimal, ScalarImpl};
use risingwave_common::util::chunk_coalesce::DataChunkBuilder;
use risingwave_pb::batch_plan::plan_node::NodeBody;
use thiserror_ext::AsReport;
use tokio_postgres;

use crate::error::BatchError;
use crate::executor::{BoxedExecutor, BoxedExecutorBuilder, DataChunk, Executor, ExecutorBuilder};
use crate::task::BatchTaskContext;

/// `PostgresQuery` executor. Runs a query against a Postgres database.
pub struct PostgresQueryExecutor {
    schema: Schema,
    host: String,
    port: String,
    username: String,
    password: String,
    database: String,
    query: String,
    identity: String,
    chunk_size: usize,
}

impl Executor for PostgresQueryExecutor {
    fn schema(&self) -> &risingwave_common::catalog::Schema {
        &self.schema
    }

    fn identity(&self) -> &str {
        &self.identity
    }

    fn execute(self: Box<Self>) -> super::BoxedDataChunkStream {
        self.do_execute().boxed()
    }
}

pub fn postgres_row_to_owned_row(
    row: tokio_postgres::Row,
    schema: &Schema,
) -> Result<OwnedRow, BatchError> {
    let mut datums = vec![];
    for i in 0..schema.fields.len() {
        let rw_field = &schema.fields[i];
        let name = rw_field.name.as_str();
        let datum = postgres_cell_to_scalar_impl(&row, &rw_field.data_type, i, name)?;
        datums.push(datum);
    }
    Ok(OwnedRow::new(datums))
}

// TODO(kwannoel): Support more types, see postgres connector's ScalarAdapter.
fn postgres_cell_to_scalar_impl(
    row: &tokio_postgres::Row,
    data_type: &DataType,
    i: usize,
    name: &str,
) -> Result<Datum, BatchError> {
    let datum = match data_type {
        DataType::Boolean
        | DataType::Int16
        | DataType::Int32
        | DataType::Int64
        | DataType::Float32
        | DataType::Float64
        | DataType::Date
        | DataType::Time
        | DataType::Timestamp
        | DataType::Timestamptz
        | DataType::Jsonb
        | DataType::Interval
        | DataType::Varchar
        | DataType::Bytea => {
            // ScalarAdapter is also fine. But ScalarImpl is more efficient
            row.try_get::<_, Option<ScalarImpl>>(i)?
        }
        DataType::Decimal => {
            // Decimal is more efficient than PgNumeric in ScalarAdapter
            let val = row.try_get::<_, Option<Decimal>>(i)?;
            val.map(ScalarImpl::from)
        }
        _ => {
            tracing::warn!(name, ?data_type, "unsupported data type, set to null");
            None
        }
    };
    Ok(datum)
}

impl PostgresQueryExecutor {
    pub fn new(
        schema: Schema,
        host: String,
        port: String,
        username: String,
        password: String,
        database: String,
        query: String,
        identity: String,
        chunk_size: usize,
    ) -> Self {
        Self {
            schema,
            host,
            port,
            username,
            password,
            database,
            query,
            identity,
            chunk_size,
        }
    }

    #[try_stream(ok = DataChunk, error = BatchError)]
    async fn do_execute(self: Box<Self>) {
        tracing::debug!("postgres_query_executor: started");
        let conn_str = format!(
            "host={} port={} user={} password={} dbname={}",
            self.host, self.port, self.username, self.password, self.database
        );
        let (client, conn) = tokio_postgres::connect(&conn_str, tokio_postgres::NoTls).await?;

        tokio::spawn(async move {
            if let Err(e) = conn.await {
                tracing::error!(
                    "postgres_query_executor: connection error: {:?}",
                    e.as_report()
                );
            }
        });

        let params: &[&str] = &[];
        let row_stream = client
            .query_raw(&self.query, params)
            .await
            .context("postgres_query received error from remote server")?;
        let mut builder = DataChunkBuilder::new(self.schema.data_types(), self.chunk_size);
        tracing::debug!("postgres_query_executor: query executed, start deserializing rows");
        // deserialize the rows
        #[for_await]
        for row in row_stream {
            let row = row?;
            let owned_row = postgres_row_to_owned_row(row, &self.schema)?;
            if let Some(chunk) = builder.append_one_row(owned_row) {
                yield chunk;
            }
        }
        if let Some(chunk) = builder.consume_all() {
            yield chunk;
        }
        return Ok(());
    }
}

pub struct PostgresQueryExecutorBuilder {}

#[async_trait::async_trait]
impl BoxedExecutorBuilder for PostgresQueryExecutorBuilder {
    async fn new_boxed_executor<C: BatchTaskContext>(
        source: &ExecutorBuilder<'_, C>,
        _inputs: Vec<BoxedExecutor>,
    ) -> crate::error::Result<BoxedExecutor> {
        let postgres_query_node = try_match_expand!(
            source.plan_node().get_node_body().unwrap(),
            NodeBody::PostgresQuery
        )?;

        Ok(Box::new(PostgresQueryExecutor::new(
            Schema::from_iter(postgres_query_node.columns.iter().map(Field::from)),
            postgres_query_node.hostname.clone(),
            postgres_query_node.port.clone(),
            postgres_query_node.username.clone(),
            postgres_query_node.password.clone(),
            postgres_query_node.database.clone(),
            postgres_query_node.query.clone(),
            source.plan_node().get_identity().clone(),
            source.context.get_config().developer.chunk_size,
        )))
    }
}