risingwave_frontend/handler/
fetch_cursor.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
// 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 pgwire::pg_field_descriptor::PgFieldDescriptor;
use pgwire::pg_response::{PgResponse, StatementType};
use pgwire::types::{Format, Row};
use risingwave_common::bail_not_implemented;
use risingwave_common::catalog::Schema;
use risingwave_common::types::DataType;
use risingwave_sqlparser::ast::{FetchCursorStatement, Statement};

use super::extended_handle::{PortalResult, PrepareStatement, PreparedResult};
use super::query::BoundResult;
use super::util::convert_interval_to_u64_seconds;
use super::RwPgResponse;
use crate::binder::BoundStatement;
use crate::error::Result;
use crate::handler::HandlerArgs;
use crate::{Binder, PgResponseStream, WithOptions};

pub async fn handle_fetch_cursor_execute(
    handler_args: HandlerArgs,
    portal_result: PortalResult,
) -> Result<RwPgResponse> {
    if let PortalResult {
        statement: Statement::FetchCursor { stmt },
        bound_result:
            BoundResult {
                bound: BoundStatement::FetchCursor(fetch_cursor),
                ..
            },
        result_formats,
        ..
    } = portal_result
    {
        match fetch_cursor.returning_schema {
            Some(_) => handle_fetch_cursor(handler_args, stmt, &result_formats).await,
            None => Ok(build_fetch_cursor_response(vec![], vec![])),
        }
    } else {
        bail_not_implemented!("unsupported portal {}", portal_result)
    }
}
pub async fn handle_fetch_cursor(
    handler_args: HandlerArgs,
    stmt: FetchCursorStatement,
    formats: &Vec<Format>,
) -> Result<RwPgResponse> {
    let session = handler_args.session.clone();
    let db_name = session.database();
    let (_, cursor_name) =
        Binder::resolve_schema_qualified_name(db_name, stmt.cursor_name.clone())?;

    let with_options = WithOptions::try_from(stmt.with_properties.0.as_slice())?;

    if with_options.len() > 1 {
        bail_not_implemented!("only `timeout` is supported in with options")
    }

    let timeout_seconds = with_options
        .get("timeout")
        .map(convert_interval_to_u64_seconds)
        .transpose()?;

    if with_options.len() == 1 && timeout_seconds.is_none() {
        bail_not_implemented!("only `timeout` is supported in with options")
    }

    let cursor_manager = session.get_cursor_manager();

    let (rows, pg_descs) = cursor_manager
        .get_rows_with_cursor(
            cursor_name,
            stmt.count,
            handler_args,
            formats,
            timeout_seconds,
        )
        .await?;
    Ok(build_fetch_cursor_response(rows, pg_descs))
}

fn build_fetch_cursor_response(rows: Vec<Row>, pg_descs: Vec<PgFieldDescriptor>) -> RwPgResponse {
    PgResponse::builder(StatementType::FETCH_CURSOR)
        .row_cnt_opt(Some(rows.len() as i32))
        .values(PgResponseStream::from(rows), pg_descs)
        .into()
}

pub async fn handle_parse(
    handler_args: HandlerArgs,
    statement: Statement,
    specific_param_types: Vec<Option<DataType>>,
) -> Result<PrepareStatement> {
    if let Statement::FetchCursor { stmt } = &statement {
        let session = handler_args.session.clone();
        let db_name = session.database();
        let (_, cursor_name) =
            Binder::resolve_schema_qualified_name(db_name, stmt.cursor_name.clone())?;
        let fields = session
            .get_cursor_manager()
            .get_fields_with_cursor(cursor_name.clone())
            .await?;

        let mut binder = Binder::new_with_param_types(&session, specific_param_types);
        let schema = Some(Schema::new(fields));

        let bound = binder.bind_fetch_cursor(cursor_name, stmt.count, schema)?;
        let bound_result = BoundResult {
            stmt_type: StatementType::FETCH_CURSOR,
            must_dist: false,
            bound: BoundStatement::FetchCursor(Box::new(bound)),
            param_types: binder.export_param_types()?,
            parsed_params: None,
            dependent_relations: binder.included_relations().clone(),
            dependent_udfs: binder.included_udfs().clone(),
        };
        let result = PreparedResult {
            statement,
            bound_result,
        };
        Ok(PrepareStatement::Prepared(result))
    } else {
        bail_not_implemented!("unsupported statement {:?}", statement)
    }
}