-
Notifications
You must be signed in to change notification settings - Fork 785
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Fix flight sql do put handling, add bind parameter support to FlightSQL cli client #4797
Changes from all commits
b5f6c83
e52373d
352555f
c82dd1c
7f06bc5
80855ec
6cfc5c4
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,6 +24,8 @@ use std::collections::HashMap; | |
use std::str::FromStr; | ||
use tonic::metadata::AsciiMetadataKey; | ||
|
||
use crate::encode::FlightDataEncoderBuilder; | ||
use crate::error::FlightError; | ||
use crate::flight_service_client::FlightServiceClient; | ||
use crate::sql::server::{CLOSE_PREPARED_STATEMENT, CREATE_PREPARED_STATEMENT}; | ||
use crate::sql::{ | ||
|
@@ -32,8 +34,8 @@ use crate::sql::{ | |
CommandGetCrossReference, CommandGetDbSchemas, CommandGetExportedKeys, | ||
CommandGetImportedKeys, CommandGetPrimaryKeys, CommandGetSqlInfo, | ||
CommandGetTableTypes, CommandGetTables, CommandGetXdbcTypeInfo, | ||
CommandPreparedStatementQuery, CommandStatementQuery, CommandStatementUpdate, | ||
DoPutUpdateResult, ProstMessageExt, SqlInfo, | ||
CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery, | ||
CommandStatementUpdate, DoPutUpdateResult, ProstMessageExt, SqlInfo, | ||
}; | ||
use crate::{ | ||
Action, FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, | ||
|
@@ -439,9 +441,12 @@ impl PreparedStatement<Channel> { | |
|
||
/// Executes the prepared statement query on the server. | ||
pub async fn execute(&mut self) -> Result<FlightInfo, ArrowError> { | ||
self.write_bind_params().await?; | ||
|
||
let cmd = CommandPreparedStatementQuery { | ||
prepared_statement_handle: self.handle.clone(), | ||
}; | ||
|
||
let result = self | ||
.flight_sql_client | ||
.get_flight_info_for_command(cmd) | ||
|
@@ -451,7 +456,9 @@ impl PreparedStatement<Channel> { | |
|
||
/// Executes the prepared statement update query on the server. | ||
pub async fn execute_update(&mut self) -> Result<i64, ArrowError> { | ||
let cmd = CommandPreparedStatementQuery { | ||
self.write_bind_params().await?; | ||
|
||
let cmd = CommandPreparedStatementUpdate { | ||
prepared_statement_handle: self.handle.clone(), | ||
}; | ||
let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec()); | ||
|
@@ -492,6 +499,36 @@ impl PreparedStatement<Channel> { | |
Ok(()) | ||
} | ||
|
||
/// Submit parameters to the server, if any have been set on this prepared statement instance | ||
async fn write_bind_params(&mut self) -> Result<(), ArrowError> { | ||
if let Some(ref params_batch) = self.parameter_binding { | ||
let cmd = CommandPreparedStatementQuery { | ||
prepared_statement_handle: self.handle.clone(), | ||
}; | ||
|
||
let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec()); | ||
let flight_stream_builder = FlightDataEncoderBuilder::new() | ||
.with_flight_descriptor(Some(descriptor)) | ||
.with_schema(params_batch.schema()); | ||
let flight_data = flight_stream_builder | ||
.build(futures::stream::iter( | ||
self.parameter_binding.clone().map(Ok), | ||
)) | ||
.try_collect::<Vec<_>>() | ||
.await | ||
.map_err(flight_error_to_arrow_error)?; | ||
|
||
self.flight_sql_client | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This appears consistent with the FlightSQL specification, it uses do_put to bind the parameter arguments. What isn't clear to me is if the result should be being used in some way. This would seem to imply some sort of server-side state which I had perhaps expected FlightSQL to not rely on There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, I think we are in agreement about it implying server-side state. FWIW FlightSQL also supports transactions which I think (maybe wrongly) would also require state. There was also some discussion happening about adding new RPC's for managing session state at some point (like a There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This seems like a fundamental flaw in FlightSQL tbh, gRPC is not a connection-oriented protocol and so the lifetime of any server state is non-deterministic... I believe @alamb plans to start a discussion to see if we can't fix this There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I filed apache/arrow#37720 and will circulate this around |
||
.do_put(stream::iter(flight_data)) | ||
.await? | ||
.try_collect::<Vec<_>>() | ||
.await | ||
.map_err(status_to_arrow_error)?; | ||
} | ||
|
||
Ok(()) | ||
} | ||
|
||
/// Close the prepared statement, so that this PreparedStatement can not used | ||
/// anymore and server can free up any resources. | ||
pub async fn close(mut self) -> Result<(), ArrowError> { | ||
|
@@ -515,6 +552,13 @@ fn status_to_arrow_error(status: tonic::Status) -> ArrowError { | |
ArrowError::IpcError(format!("{status:?}")) | ||
} | ||
|
||
fn flight_error_to_arrow_error(err: FlightError) -> ArrowError { | ||
match err { | ||
FlightError::Arrow(e) => e, | ||
e => ArrowError::ExternalError(Box::new(e)), | ||
} | ||
} | ||
|
||
// A polymorphic structure to natively represent different types of data contained in `FlightData` | ||
pub enum ArrowFlightData { | ||
RecordBatch(RecordBatch), | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I also forgot to mention, I think this was a bug in the existing implementation.
ExecuteUpdate
should be performed with aCommandPreparedStatementUpdate
command, not aCommandPreparedStatementQuery
.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I agree -
update
should useCommandPreparedStatementUpdate
https://github.com/apache/arrow/blob/15a8ac3ce4e3ac31f9f361770ad4a38c69102aa1/format/FlightSql.proto#L1769-L1780