-
Notifications
You must be signed in to change notification settings - Fork 796
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 4 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,17 @@ 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, | ||
FlightError::NotYetImplemented(s) => ArrowError::NotYetImplemented(s), | ||
FlightError::Tonic(status) => status_to_arrow_error(status), | ||
FlightError::ProtocolError(e) => ArrowError::IpcError(e), | ||
FlightError::DecodeError(s) => ArrowError::IpcError(s), | ||
FlightError::ExternalError(e) => ArrowError::ExternalError(e), | ||
} | ||
suremarc marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
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 am not sure why we return 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 agree this should probably be FlightError |
||
|
||
// A polymorphic structure to natively represent different types of data contained in `FlightData` | ||
pub enum ArrowFlightData { | ||
RecordBatch(RecordBatch), | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,7 +19,7 @@ | |
|
||
use std::pin::Pin; | ||
|
||
use futures::Stream; | ||
use futures::{stream::Peekable, Stream}; | ||
use prost::Message; | ||
use tonic::{Request, Response, Status, Streaming}; | ||
|
||
|
@@ -366,7 +366,7 @@ pub trait FlightSqlService: Sync + Send + Sized + 'static { | |
/// Implementors may override to handle additional calls to do_put() | ||
async fn do_put_fallback( | ||
&self, | ||
_request: Request<Streaming<FlightData>>, | ||
_request: Request<Peekable<Streaming<FlightData>>>, | ||
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. Another option might be to pass the first ticket request as a separate argument. I don't feel strongly either way 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. It's a tough decision for me. I prefer using 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 server is the scaffolding to help people build flightsql servers -- they can always use the raw FlightService if they prefer. Thus I think the change in API is less critical and given the requirements it seems inevitable. I think the only thing we should try and improve in this PR is improving the documentation to explain why peekable is used somehow (to lower the cognative burden on people trying to use this). One potential option to document this is rather than using For example: /// A wrapper around `Streaming` that allows inspection of the first message.
/// This is needed because sometimes the first request in the stream will contain
/// a [`FlightDescriptor`] in addition to potentially any data and the dispatch logic
/// must inspect this information.
///
/// # example:
/// <show an example here of calling `into_inner()` to get the original data back
struct PeekableStreaming {
inner: Peekable<Streaming<FlightData>>
}
impl PeekableStreaming {
/// return the inner stream
pub fn into_inner(self) -> Streaming<FlightData> { self.inner.into_inner() }
...
} We could also potentially use something like Thus I think this design is the best of several less than ideal solutions. To proceed perhaps we can add some documentation on the 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 added a new type, |
||
message: Any, | ||
) -> Result<Response<<Self as FlightService>::DoPutStream>, Status> { | ||
Err(Status::unimplemented(format!( | ||
|
@@ -379,7 +379,7 @@ pub trait FlightSqlService: Sync + Send + Sized + 'static { | |
async fn do_put_statement_update( | ||
&self, | ||
_ticket: CommandStatementUpdate, | ||
_request: Request<Streaming<FlightData>>, | ||
_request: Request<Peekable<Streaming<FlightData>>>, | ||
) -> Result<i64, Status> { | ||
Err(Status::unimplemented( | ||
"do_put_statement_update has no default implementation", | ||
|
@@ -390,7 +390,7 @@ pub trait FlightSqlService: Sync + Send + Sized + 'static { | |
async fn do_put_prepared_statement_query( | ||
&self, | ||
_query: CommandPreparedStatementQuery, | ||
_request: Request<Streaming<FlightData>>, | ||
_request: Request<Peekable<Streaming<FlightData>>>, | ||
) -> Result<Response<<Self as FlightService>::DoPutStream>, Status> { | ||
Err(Status::unimplemented( | ||
"do_put_prepared_statement_query has no default implementation", | ||
|
@@ -401,7 +401,7 @@ pub trait FlightSqlService: Sync + Send + Sized + 'static { | |
async fn do_put_prepared_statement_update( | ||
&self, | ||
_query: CommandPreparedStatementUpdate, | ||
_request: Request<Streaming<FlightData>>, | ||
_request: Request<Peekable<Streaming<FlightData>>>, | ||
) -> Result<i64, Status> { | ||
Err(Status::unimplemented( | ||
"do_put_prepared_statement_update has no default implementation", | ||
|
@@ -412,7 +412,7 @@ pub trait FlightSqlService: Sync + Send + Sized + 'static { | |
async fn do_put_substrait_plan( | ||
&self, | ||
_query: CommandStatementSubstraitPlan, | ||
_request: Request<Streaming<FlightData>>, | ||
_request: Request<Peekable<Streaming<FlightData>>>, | ||
) -> Result<i64, Status> { | ||
Err(Status::unimplemented( | ||
"do_put_substrait_plan has no default implementation", | ||
|
@@ -688,9 +688,17 @@ where | |
|
||
async fn do_put( | ||
&self, | ||
mut request: Request<Streaming<FlightData>>, | ||
request: Request<Streaming<FlightData>>, | ||
) -> Result<Response<Self::DoPutStream>, Status> { | ||
let cmd = request.get_mut().message().await?.unwrap(); | ||
// See issue #4658: https://github.com/apache/arrow-rs/issues/4658 | ||
// To dispatch to the correct `do_put` method, we cannot discard the first message, | ||
// as it may contain the Arrow schema, which the `do_put` handler may need. | ||
// To allow the first message to be reused by the `do_put` handler, | ||
// we wrap this stream in a `Peekable` one, which allows us to peek at | ||
// the first message without discarding it. | ||
let mut request = request.map(futures::StreamExt::peekable); | ||
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. So if I am following correctly, the issue is do_put accepts a FlightData stream, but the first request in the stream will contain a FlightDescriptor in addition to potentially any data. I continue to be utterly baffled by the design of Flight 😅 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. Yes, that is my understanding as well. Prior to this change, decoding a flight stream inside one of the |
||
let cmd = Pin::new(request.get_mut()).peek().await.unwrap().clone()?; | ||
|
||
let message = Any::decode(&*cmd.flight_descriptor.unwrap().cmd) | ||
.map_err(decode_error_to_status)?; | ||
match Command::try_from(message).map_err(arrow_error_to_status)? { | ||
|
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