From 3dcbeda93e39dc691489f92cb4c95e0ad6f3aa28 Mon Sep 17 00:00:00 2001 From: Adam Curtis Date: Sun, 10 Mar 2024 14:27:53 -0400 Subject: [PATCH] make DoPutPreparedStatenentResult mandatory --- arrow-flight/examples/flight_sql_server.rs | 2 +- arrow-flight/src/sql/server.rs | 23 +++++++-------------- arrow-flight/tests/flight_sql_client_cli.rs | 17 ++++++++------- 3 files changed, 18 insertions(+), 24 deletions(-) diff --git a/arrow-flight/examples/flight_sql_server.rs b/arrow-flight/examples/flight_sql_server.rs index 9eb8c89e9521..fd7e348547c3 100644 --- a/arrow-flight/examples/flight_sql_server.rs +++ b/arrow-flight/examples/flight_sql_server.rs @@ -620,7 +620,7 @@ impl FlightSqlService for FlightSqlServiceImpl { &self, _query: CommandPreparedStatementQuery, _request: Request, - ) -> Result, Status> { + ) -> Result { Err(Status::unimplemented( "do_put_prepared_statement_query not implemented", )) diff --git a/arrow-flight/src/sql/server.rs b/arrow-flight/src/sql/server.rs index 51e1406467a6..c18024cf068a 100644 --- a/arrow-flight/src/sql/server.rs +++ b/arrow-flight/src/sql/server.rs @@ -399,15 +399,14 @@ pub trait FlightSqlService: Sync + Send + Sized + 'static { /// Bind parameters to given prepared statement. /// - /// Note that `DoPutPreparedStatementResult` contains an optional - /// opaque handle that the client should pass + /// Returns an opaque handle that the client should pass /// back to the server during subsequent requests with this /// prepared statement. async fn do_put_prepared_statement_query( &self, _query: CommandPreparedStatementQuery, _request: Request, - ) -> Result, Status> { + ) -> Result { Err(Status::unimplemented( "do_put_prepared_statement_query has no default implementation", )) @@ -715,19 +714,13 @@ where Ok(Response::new(Box::pin(output))) } Command::CommandPreparedStatementQuery(command) => { - if let Some(result) = self + let result = self .do_put_prepared_statement_query(command, request) - .await? - { - let output = futures::stream::iter(vec![Ok(PutResult { - app_metadata: result.as_any().encode_to_vec().into(), - })]); - Ok(Response::new(Box::pin(output))) - } else { - Ok(Response::new( - futures::stream::once(async { Ok(PutResult::default()) }).boxed(), - )) - } + .await?; + let output = futures::stream::iter(vec![Ok(PutResult { + app_metadata: result.as_any().encode_to_vec().into(), + })]); + Ok(Response::new(Box::pin(output))) } Command::CommandStatementSubstraitPlan(command) => { let record_count = self.do_put_substrait_plan(command, request).await?; diff --git a/arrow-flight/tests/flight_sql_client_cli.rs b/arrow-flight/tests/flight_sql_client_cli.rs index 835f73a59adc..50a4ec0d8c66 100644 --- a/arrow-flight/tests/flight_sql_client_cli.rs +++ b/arrow-flight/tests/flight_sql_client_cli.rs @@ -560,7 +560,7 @@ impl FlightSqlService for FlightSqlServiceImpl { &self, _query: CommandPreparedStatementQuery, request: Request, - ) -> Result, Status> { + ) -> Result { // just make sure decoding the parameters works let parameters = FlightRecordBatchStream::new_from_flight_data( request.into_inner().map_err(|e| e.into()), @@ -579,14 +579,15 @@ impl FlightSqlService for FlightSqlServiceImpl { ))); } } - if self.stateless_prepared_statements { - let result = DoPutPreparedStatementResult { - prepared_statement_handle: UPDATED_PREPARED_STATEMENT_HANDLE.to_string().into(), - }; - Ok(Some(result)) + let handle = if self.stateless_prepared_statements { + UPDATED_PREPARED_STATEMENT_HANDLE.to_string().into() } else { - Ok(None) - } + PREPARED_STATEMENT_HANDLE.to_string().into() + }; + let result = DoPutPreparedStatementResult { + prepared_statement_handle: Some(handle), + }; + Ok(result) } async fn do_put_prepared_statement_update(