You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/03/03 02:21:35 UTC

[GitHub] [arrow-rs] wangfenjin opened a new pull request #1386: init impl flight sql

wangfenjin opened a new pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386


   This is an attempt to support flight-sql in arrow-rs. Currently I only implement the server mod, but I'd like to send out the PR so I can get the code review to make sure I'm in the right direction.
   
   example: https://github.com/wangfenjin/arrow-datafusion/pull/1
   
   TODOs:
   - [ ] Impl client
   - [ ] Make flight-sql as an optional feature?
   
   Question:
   flight-sql use protobuf Any a lot, but prost doesn't support that very well, specifically the UnpackTo/PackFrom method in cpp. I asked the question in https://github.com/fdeantoni/prost-wkt/issues/14 but got no response yet. So currently I need to use protoc-rust to generate the pb and in the [example](https://github.com/wangfenjin/arrow-datafusion/pull/1) I can do the marshal/unmarshal. Not sure if there is a better way? Or do we need to stick to prost?
   
   Address #1323 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] tustvold commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r821937337



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,585 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the table types.
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of SqlInfo results.
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the primary and foreign keys.
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the exported keys.
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the imported keys.
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the cross reference.
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    // do_put
+
+    /// Execute an update SQL statement.
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Bind parameters to given prepared statement.
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Execute an update SQL prepared statement.
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    // do_action
+
+    /// Create a prepared statement from given SQL statement.
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Close a prepared statement.
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Register a new SqlInfo result, making it available when calling GetSqlInfo.
+    async fn register_sql_info(&self, id: i32, result: &SqlInfo);
+}
+
+/// Implements the lower level interface to handle FlightSQL
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,

Review comment:
       This blanket impl really confuses me, it implements `FlightService` for all T which implement `FlightSqlService` which by definition already implement `FlightService`...
   
   

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,581 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the table types.
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of SqlInfo results.
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the primary and foreign keys.
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the exported keys.
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the imported keys.
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the cross reference.
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    // do_put
+
+    /// Execute an update SQL statement.
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Bind parameters to given prepared statement.
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Execute an update SQL prepared statement.
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    // do_action
+
+    /// Create a prepared statement from given SQL statement.
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Close a prepared statement.
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Register a new SqlInfo result, making it available when calling GetSqlInfo.
+    async fn register_sql_info(&self, id: i32, result: &SqlInfo);
+}
+
+/// Implements the lower level interface to handle FlightSQL
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,
+    >;
+    type ListFlightsStream =
+        Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + Sync + 'static>>;
+    type DoGetStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+    type DoPutStream =
+        Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + Sync + 'static>>;
+    type DoActionStream = Pin<
+        Box<
+            dyn Stream<Item = Result<super::super::Result, Status>>
+                + Send
+                + Sync
+                + 'static,
+        >,
+    >;
+    type ListActionsStream =
+        Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + Sync + 'static>>;
+    type DoExchangeStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+
+    async fn handshake(
+        &self,
+        _request: Request<Streaming<HandshakeRequest>>,
+    ) -> Result<Response<Self::HandshakeStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn list_flights(
+        &self,
+        _request: Request<Criteria>,
+    ) -> Result<Response<Self::ListFlightsStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn get_flight_info(
+        &self,
+        request: Request<FlightDescriptor>,
+    ) -> Result<Response<FlightInfo>, Status> {
+        let request = request.into_inner();
+        let any: prost_types::Any = prost::Message::decode(&*request.cmd)
+            .map_err(|_| Status::invalid_argument("Unable to parse command"))?;
+
+        if any.is::<CommandStatementQuery>() {
+            return self
+                .get_flight_info_statement(
+                    any.unpack::<CommandStatementQuery>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .get_flight_info_prepared_statement(
+                    any.unpack::<CommandPreparedStatementQuery>()
+                        .unwrap()
+                        .unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetCatalogs>() {
+            return self
+                .get_flight_info_catalogs(
+                    any.unpack::<CommandGetCatalogs>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetDbSchemas>() {
+            return self
+                .get_flight_info_schemas(
+                    any.unpack::<CommandGetDbSchemas>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetTables>() {
+            return self
+                .get_flight_info_tables(
+                    any.unpack::<CommandGetTables>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetTableTypes>() {
+            return self
+                .get_flight_info_table_types(
+                    any.unpack::<CommandGetTableTypes>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetSqlInfo>() {
+            return self
+                .get_flight_info_sql_info(
+                    any.unpack::<CommandGetSqlInfo>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetPrimaryKeys>() {
+            return self
+                .get_flight_info_primary_keys(
+                    any.unpack::<CommandGetPrimaryKeys>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetExportedKeys>() {
+            return self
+                .get_flight_info_exported_keys(
+                    any.unpack::<CommandGetExportedKeys>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetImportedKeys>() {
+            return self
+                .get_flight_info_imported_keys(
+                    any.unpack::<CommandGetImportedKeys>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetCrossReference>() {
+            return self
+                .get_flight_info_cross_reference(
+                    any.unpack::<CommandGetCrossReference>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "get_flight_info: The defined request is invalid: {:?}",
+            String::from_utf8(any.encode_to_vec()).unwrap()
+        )))
+    }
+
+    async fn get_schema(
+        &self,
+        _request: Request<FlightDescriptor>,
+    ) -> Result<Response<SchemaResult>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn do_get(
+        &self,
+        _request: Request<Ticket>,
+    ) -> Result<Response<Self::DoGetStream>, Status> {
+        let request = _request.into_inner();
+        let any: prost_types::Any = prost::Message::decode(&*request.ticket)
+            .map_err(|_| Status::invalid_argument("Unable to parse command"))?;
+
+        if any.is::<TicketStatementQuery>() {
+            return self
+                .do_get_statement(any.unpack::<TicketStatementQuery>().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .do_get_prepared_statement(
+                    any.unpack::<CommandPreparedStatementQuery>()
+                        .unwrap()
+                        .unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandGetCatalogs>() {
+            return self
+                .do_get_catalogs(any.unpack::<CommandGetCatalogs>().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetDbSchemas>() {
+            return self
+                .do_get_schemas(any.unpack::<CommandGetDbSchemas>().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetTables>() {
+            return self
+                .do_get_tables(any.unpack::<CommandGetTables>().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetTableTypes>() {
+            return self
+                .do_get_table_types(
+                    any.unpack::<CommandGetTableTypes>().unwrap().unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandGetSqlInfo>() {
+            return self
+                .do_get_sql_info(any.unpack::<CommandGetSqlInfo>().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetPrimaryKeys>() {
+            return self
+                .do_get_primary_keys(
+                    any.unpack::<CommandGetPrimaryKeys>().unwrap().unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandGetExportedKeys>() {
+            return self
+                .do_get_exported_keys(
+                    any.unpack::<CommandGetExportedKeys>().unwrap().unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandGetImportedKeys>() {
+            return self
+                .do_get_imported_keys(
+                    any.unpack::<CommandGetImportedKeys>().unwrap().unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandGetCrossReference>() {
+            return self
+                .do_get_cross_reference(
+                    any.unpack::<CommandGetCrossReference>().unwrap().unwrap(),
+                )
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "do_get: The defined request is invalid: {:?}",
+            String::from_utf8(request.ticket).unwrap()
+        )))
+    }
+
+    async fn do_put(
+        &self,
+        _request: Request<Streaming<FlightData>>,
+    ) -> Result<Response<Self::DoPutStream>, Status> {
+        let request = _request.into_inner().message().await?.unwrap();
+        let any: prost_types::Any =
+            prost::Message::decode(&*request.flight_descriptor.unwrap().cmd)
+                .map_err(|_| Status::invalid_argument("Unable to parse command"))?;
+        if any.is::<CommandStatementUpdate>() {
+            return self
+                .do_put_statement_update(
+                    any.unpack::<CommandStatementUpdate>().unwrap().unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .do_put_prepared_statement_query(
+                    any.unpack::<CommandPreparedStatementQuery>()
+                        .unwrap()
+                        .unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementUpdate>() {
+            return self
+                .do_put_prepared_statement_update(
+                    any.unpack::<CommandPreparedStatementUpdate>()
+                        .unwrap()
+                        .unwrap(),
+                )
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "do_put: The defined request is invalid: {:?}",
+            String::from_utf8(any.encode_to_vec()).unwrap()
+        )))
+    }
+
+    async fn list_actions(
+        &self,
+        _request: Request<Empty>,
+    ) -> Result<Response<Self::ListActionsStream>, Status> {
+        let create_prepared_statement_action_type = ActionType {
+            r#type: CREATE_PREPARED_STATEMENT.to_string(),
+            description: "Creates a reusable prepared statement resource on the server.\n
+                Request Message: ActionCreatePreparedStatementRequest\n
+                Response Message: ActionCreatePreparedStatementResult"
+                .into(),
+        };
+        let close_prepared_statement_action_type = ActionType {
+            r#type: CLOSE_PREPARED_STATEMENT.to_string(),
+            description: "Closes a reusable prepared statement resource on the server.\n
+                Request Message: ActionClosePreparedStatementRequest\n
+                Response Message: N/A"
+                .into(),
+        };
+        let _actions: Vec<Result<ActionType, Status>> = vec![
+            Ok(create_prepared_statement_action_type),
+            Ok(close_prepared_statement_action_type),
+        ];
+        // TODO: not sure why it's not work
+        // let output = futures::stream::iter(actions);
+        // Ok(Response::new(Box::pin(output) as Self::ListActionsStream))

Review comment:
       I'm not entirely sure I fully understand what the compiler is doing here, I don't really understand how recursive trait definitions work, but if you remove the recursive trait-definition, i.e. remove the `trait FlightSqlService: FlightService`, this compiles

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,585 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;

Review comment:
       Why is this necessary?

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,585 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the table types.
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of SqlInfo results.
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the primary and foreign keys.
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the exported keys.
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the imported keys.
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the cross reference.
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    // do_put
+
+    /// Execute an update SQL statement.
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Bind parameters to given prepared statement.
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Execute an update SQL prepared statement.
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    // do_action
+
+    /// Create a prepared statement from given SQL statement.
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Close a prepared statement.
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Register a new SqlInfo result, making it available when calling GetSqlInfo.
+    async fn register_sql_info(&self, id: i32, result: &SqlInfo);
+}
+
+/// Implements the lower level interface to handle FlightSQL
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,

Review comment:
       The `Sync` is technically not required, we recently removed it from Datafusion - https://github.com/apache/arrow-datafusion/issues/1614
   
   This would also allow you to use `futures::stream::BoxStream<'static, Result<ActionType, Status>>` which is perhaps more idiomatic

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,585 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the table types.
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of SqlInfo results.
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the primary and foreign keys.
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the exported keys.
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the imported keys.
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the cross reference.
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    // do_put
+
+    /// Execute an update SQL statement.
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Bind parameters to given prepared statement.
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Execute an update SQL prepared statement.
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    // do_action
+
+    /// Create a prepared statement from given SQL statement.
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Close a prepared statement.
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Register a new SqlInfo result, making it available when calling GetSqlInfo.
+    async fn register_sql_info(&self, id: i32, result: &SqlInfo);
+}
+
+/// Implements the lower level interface to handle FlightSQL
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,
+    >;
+    type ListFlightsStream =
+        Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + Sync + 'static>>;
+    type DoGetStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+    type DoPutStream =
+        Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + Sync + 'static>>;
+    type DoActionStream = Pin<
+        Box<
+            dyn Stream<Item = Result<super::super::Result, Status>>
+                + Send
+                + Sync
+                + 'static,
+        >,
+    >;
+    type ListActionsStream =
+        Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + Sync + 'static>>;
+    type DoExchangeStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+
+    async fn handshake(
+        &self,
+        _request: Request<Streaming<HandshakeRequest>>,
+    ) -> Result<Response<Self::HandshakeStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn list_flights(
+        &self,
+        _request: Request<Criteria>,
+    ) -> Result<Response<Self::ListFlightsStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn get_flight_info(
+        &self,
+        request: Request<FlightDescriptor>,
+    ) -> Result<Response<FlightInfo>, Status> {
+        let request = request.into_inner();
+        let any: prost_types::Any = prost::Message::decode(&*request.cmd)
+            .map_err(|_| Status::invalid_argument("Unable to parse command"))?;
+
+        if any.is::<CommandStatementQuery>() {
+            return self
+                .get_flight_info_statement(
+                    any.unpack::<CommandStatementQuery>().unwrap().unwrap(),

Review comment:
       I think the result should probably be returned as an error, e.g. `Status::invalid_argument` as above, vs panicking. It might be possible to pull this logic into a function so that `?` can be used instead of lots of manual `map_err` calls 

##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {
+    /// Check if `Any` contains a message of given type.
+    fn is<M: ProstMessageExt>(&self) -> bool;
+
+    /// Extract a message from this `Any`.
+    ///
+    /// # Returns
+    ///
+    /// * `Ok(None)` when message type mismatch
+    /// * `Err` when parse failed
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>>;
+
+    /// Pack any message into `prost_types::Any` value.
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
+}
+
+impl ProstAnyExt for prost_types::Any {
+    fn is<M: ProstMessageExt>(&self) -> bool {
+        M::type_url() == self.type_url
+    }
+
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>> {

Review comment:
       Perhaps this could return `tonic::Status` given the fact it will be predominantely used in gRPC handlers? :thinking: 

##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {

Review comment:
       Nice :ok_hand: 

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,585 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static

Review comment:
       These types seem to be recursive, something implements `FlightSqlService` which must in turn implement `FlightService` and there is then a blanket impl of `FlightService` for all `FlightSqlService`

##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {
+    /// Check if `Any` contains a message of given type.
+    fn is<M: ProstMessageExt>(&self) -> bool;
+
+    /// Extract a message from this `Any`.
+    ///
+    /// # Returns
+    ///
+    /// * `Ok(None)` when message type mismatch
+    /// * `Err` when parse failed
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>>;
+
+    /// Pack any message into `prost_types::Any` value.
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
+}
+
+impl ProstAnyExt for prost_types::Any {
+    fn is<M: ProstMessageExt>(&self) -> bool {
+        M::type_url() == self.type_url
+    }
+
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>> {
+        if !self.is::<M>() {
+            return Ok(None);
+        }
+        let m = prost::Message::decode(&*self.value).map_err(|err| {
+            ArrowError::ParseError(format!("Unable to decode Any value: {}", err))
+        })?;
+        Ok(Some(m))
+    }
+
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any> {
+        Ok(message.as_any())
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+
+    #[test]
+    fn test_type_url() {
+        assert_eq!(
+            TicketStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.TicketStatementQuery"
+        );
+        assert_eq!(
+            CommandStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.CommandStatementQuery"
+        );
+    }
+
+    #[test]
+    fn test_prost_any_pack_unpack() -> ArrowResult<()> {
+        let query = CommandStatementQuery {
+            query: "select 1".to_string(),
+        };
+        let any = prost_types::Any::pack(&query)?;
+        assert!(any.is::<CommandStatementQuery>());
+        let unpack_query: CommandStatementQuery =
+            any.unpack::<CommandStatementQuery>()?.unwrap();

Review comment:
       Is this a problem? The only way around this would be to have some macro that returns either an enumeration of the message types or some boxed trait, I'm not sure this would be an improvement?

##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {

Review comment:
       This should possibly also be crate local

##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {

Review comment:
       I think this should at least be `pub(crate)`, it feels like something best kept as an implementation detail




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r823576049



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,637 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;

Review comment:
       Same as above. But your comments helps, I checked other API again and changed some of them to return simpler Response, which also makes them identical to the cpp api design I missed previously. 😂




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb commented on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1065020222


   Merging 🚀 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r823966074



##########
File path: arrow-flight/Cargo.toml
##########
@@ -32,14 +32,23 @@ base64 = "0.13"
 tonic = "0.6"
 bytes = "1"
 prost = "0.9"
+prost-types = { version = "0.9.0", optional = true }
 prost-derive = "0.9"
 tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread"] }
+futures = { version = "0.3", default-features = false, features = ["alloc"]}
+
+[features]
+default = []
+flight-sql = ["prost-types"]

Review comment:
       ```suggestion
   flight-sql-experimental = ["prost-types"]
   ```
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822218804



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,585 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static

Review comment:
       Thanks it worked!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] codecov-commenter edited a comment on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1060951386


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1386](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8387bcf) into [master](https://codecov.io/gh/apache/arrow-rs/commit/4bcc7a678914e9271789797b833a8f731f8b224b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4bcc7a6) will **decrease** coverage by `0.50%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1386/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1386      +/-   ##
   ==========================================
   - Coverage   83.17%   82.67%   -0.51%     
   ==========================================
     Files         182      185       +3     
     Lines       53439    53764     +325     
   ==========================================
     Hits        44449    44449              
   - Misses       8990     9315     +325     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [arrow-flight/examples/flight\_sql\_server.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L2V4YW1wbGVzL2ZsaWdodF9zcWxfc2VydmVyLnJz) | `0.00% <0.00%> (ø)` | |
   | [arrow-flight/src/lib.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9saWIucnM=) | `18.54% <ø> (ø)` | |
   | [arrow-flight/src/sql/mod.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9zcWwvbW9kLnJz) | `0.00% <0.00%> (ø)` | |
   | [arrow-flight/src/sql/server.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9zcWwvc2VydmVyLnJz) | `0.00% <0.00%> (ø)` | |
   | [arrow/src/array/transform/mod.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L3RyYW5zZm9ybS9tb2QucnM=) | `86.31% <0.00%> (-0.12%)` | :arrow_down: |
   | [parquet\_derive/src/parquet\_field.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldF9kZXJpdmUvc3JjL3BhcnF1ZXRfZmllbGQucnM=) | `66.21% <0.00%> (+0.22%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [4bcc7a6...8387bcf](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] codecov-commenter edited a comment on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1060951386


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1386](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ef51ff2) into [master](https://codecov.io/gh/apache/arrow-rs/commit/4bcc7a678914e9271789797b833a8f731f8b224b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4bcc7a6) will **decrease** coverage by `0.50%`.
   > The diff coverage is `0.00%`.
   
   > :exclamation: Current head ef51ff2 differs from pull request most recent head a37e3ef. Consider uploading reports for the commit a37e3ef to get more accurate results
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1386/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1386      +/-   ##
   ==========================================
   - Coverage   83.17%   82.67%   -0.51%     
   ==========================================
     Files         182      185       +3     
     Lines       53439    53764     +325     
   ==========================================
     Hits        44449    44449              
   - Misses       8990     9315     +325     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [arrow-flight/examples/flight\_sql\_server.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L2V4YW1wbGVzL2ZsaWdodF9zcWxfc2VydmVyLnJz) | `0.00% <0.00%> (ø)` | |
   | [arrow-flight/src/lib.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9saWIucnM=) | `18.54% <ø> (ø)` | |
   | [arrow-flight/src/sql/mod.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9zcWwvbW9kLnJz) | `0.00% <0.00%> (ø)` | |
   | [arrow-flight/src/sql/server.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9zcWwvc2VydmVyLnJz) | `0.00% <0.00%> (ø)` | |
   | [arrow/src/array/transform/mod.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L3RyYW5zZm9ybS9tb2QucnM=) | `86.31% <0.00%> (-0.12%)` | :arrow_down: |
   | [parquet\_derive/src/parquet\_field.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldF9kZXJpdmUvc3JjL3BhcnF1ZXRfZmllbGQucnM=) | `65.98% <0.00%> (ø)` | |
   | [arrow/src/datatypes/field.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2RhdGF0eXBlcy9maWVsZC5ycw==) | `54.10% <0.00%> (+0.30%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [4bcc7a6...a37e3ef](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r823255873



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,637 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.

Review comment:
       updated, thanks!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r824623430



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,637 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;

Review comment:
       Yeah -- this is part of the tension I would like to explore in future PRs. Making APIs that are easier to use may be less efficient than using the low level APIs directly (aka having to create a new RecordBatch) - figuring out where to draw those lines will be important
   
   Maybe we can introduce an even higher level Flight SQL abstraction. We'll see




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1059971584


   Thank for @alamb and @tustvold for your review, I'll check all the comments and update the code later, response to some of your general comment:
   1. About the protobuf dependency, I'll check IOx 's codebase to see if I can remove the rust_protobuf
   2. As you suggested, I'll focus on server side in this PR, and leave client to the follow up. According to the cpp implementation, the client library logic might be more complex compared to server.
   3. As you may have noted, I implement an example in https://github.com/wangfenjin/arrow-datafusion/pull/1 (we need to publish a new version then I can create MR to the arrow-datafusion repo). For the integration test, **_not sure if we need to implement it in this repo_**? Because if we implement one like [this](https://github.com/apache/arrow-rs/blob/master/arrow-flight/examples/server.rs) seems useless, and if we want to implement a useful example, we'd better depends on arrow-datafusion as we need a SQL server.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r820899833



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,458 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use protobuf::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,

Review comment:
       Ha! Haven't thought about it when I'm writing the code, just follow the style in here: https://github.com/apache/arrow-rs/blob/master/arrow-flight/src/lib.rs#L51
   
   But yes, when I read the cpp implementation, they always define the struct with native fields and do the convert, even for the enum values...
   
   If we don't convert the FlightData struct, overall it should be fine and don't require too much work. But there are several drawbacks I can think of:
   1. It's kind of duplication and make the code not easy to understand. When I first read the cpp implementation, I wondering why there are duplication class definition? Are they not using any pb generator but handmade all the staff. Then I find the convert code.
   2. More code, more error prone, and more effort to maintain.
   3. The difference between cpp and rust is, we commit the generated code to repo but they don't, and the generated rust code acctually is clean and easy to understand
   4. Performance issue. In the For the struct in FlightSql, as it's small so should be fine to convert, but I'm not sure if we also want to convert FlightData, it might involve too much mem copy.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r820631649



##########
File path: arrow-flight/Cargo.toml
##########
@@ -32,14 +32,17 @@ base64 = "0.13"
 tonic = "0.6"
 bytes = "1"
 prost = "0.9"
+protobuf = "2.27.1"

Review comment:
       It is also worth pointing out that this isn't just a datafusion specific concern --  `arrow-flight` itself already uses `prost` so adding a second protobuf library seems less ideal than using the existing `prost` dependency




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin edited a comment on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin edited a comment on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1063552825


   Thanks @alamb  for your kind review, address some of your comments:
   
   1. I'll add a flight-sql feature flag for this
   2. Agree that maybe we can make the API more ergonomic, but I need to do more experiments on this (I'm trying to build a more practical flight-sql-server using this), then we will clear what we need. My suggestion is we can design the API as it is, and after we have better/ more simplified design, we can add them into the trait, and make the low level API as a default implementation in the trait, so the user still have chance to override them if they want. It's very important we leave this flexibility to the user.
   3. grpcurl may not work for our testing. As I comment in https://github.com/wangfenjin/arrow-datafusion/pull/1 this PR, I use the  [arrow-cpp-cli](https://github.com/apache/arrow/blob/master/cpp/src/arrow/flight/sql/test_app_cli.cc) to connect to this server, it helps when we don't have the client implementation, also it makes sure our implementation is compatible with the cpp. We may also need to think about maintain this compatibility in long term.
   4. For the documentation thing, I copy some useful comments from cpp implementation. For more detailed documentation like the protocols, I think it's a joint effort with the cpp community, in this repo we can focus on the rust API documentation.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] viirya commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r823043900



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,637 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the table types.
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of SqlInfo results.
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the primary and foreign keys.
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the exported keys.
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the imported keys.
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the cross reference.
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    // do_put
+
+    /// Execute an update SQL statement.
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Bind parameters to given prepared statement.
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Execute an update SQL prepared statement.
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    // do_action
+
+    /// Create a prepared statement from given SQL statement.
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Close a prepared statement.
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Register a new SqlInfo result, making it available when calling GetSqlInfo.
+    async fn register_sql_info(&self, id: i32, result: &SqlInfo);
+}
+
+/// Implements the lower level interface to handle FlightSQL
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Send,
+{
+    type HandshakeStream =
+        Pin<Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + 'static>>;
+    type ListFlightsStream =
+        Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + 'static>>;
+    type DoGetStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + 'static>>;
+    type DoPutStream =
+        Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + 'static>>;
+    type DoActionStream = Pin<
+        Box<dyn Stream<Item = Result<super::super::Result, Status>> + Send + 'static>,
+    >;
+    type ListActionsStream =
+        Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + 'static>>;
+    type DoExchangeStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + 'static>>;
+
+    async fn handshake(
+        &self,
+        _request: Request<Streaming<HandshakeRequest>>,
+    ) -> Result<Response<Self::HandshakeStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn list_flights(
+        &self,
+        _request: Request<Criteria>,
+    ) -> Result<Response<Self::ListFlightsStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn get_flight_info(
+        &self,
+        request: Request<FlightDescriptor>,
+    ) -> Result<Response<FlightInfo>, Status> {
+        let request = request.into_inner();
+        let any: prost_types::Any =
+            prost::Message::decode(&*request.cmd).map_err(decode_error_to_status)?;
+
+        if any.is::<CommandStatementQuery>() {
+            return self
+                .get_flight_info_statement(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .get_flight_info_prepared_statement(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetCatalogs>() {
+            return self
+                .get_flight_info_catalogs(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetDbSchemas>() {
+            return self
+                .get_flight_info_schemas(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetTables>() {
+            return self
+                .get_flight_info_tables(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetTableTypes>() {
+            return self
+                .get_flight_info_table_types(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetSqlInfo>() {
+            return self
+                .get_flight_info_sql_info(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetPrimaryKeys>() {
+            return self
+                .get_flight_info_primary_keys(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetExportedKeys>() {
+            return self
+                .get_flight_info_exported_keys(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetImportedKeys>() {
+            return self
+                .get_flight_info_imported_keys(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetCrossReference>() {
+            return self
+                .get_flight_info_cross_reference(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "get_flight_info: The defined request is invalid: {:?}",
+            String::from_utf8(any.encode_to_vec()).unwrap()
+        )))
+    }
+
+    async fn get_schema(
+        &self,
+        _request: Request<FlightDescriptor>,
+    ) -> Result<Response<SchemaResult>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn do_get(
+        &self,
+        _request: Request<Ticket>,
+    ) -> Result<Response<Self::DoGetStream>, Status> {
+        let request = _request.into_inner();
+        let any: prost_types::Any =
+            prost::Message::decode(&*request.ticket).map_err(decode_error_to_status)?;
+
+        if any.is::<TicketStatementQuery>() {
+            return self
+                .do_get_statement(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .do_get_prepared_statement(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetCatalogs>() {
+            return self
+                .do_get_catalogs(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetDbSchemas>() {
+            return self
+                .do_get_schemas(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetTables>() {
+            return self
+                .do_get_tables(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetTableTypes>() {
+            return self
+                .do_get_table_types(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetSqlInfo>() {
+            return self
+                .do_get_sql_info(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetPrimaryKeys>() {
+            return self
+                .do_get_primary_keys(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetExportedKeys>() {
+            return self
+                .do_get_exported_keys(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetImportedKeys>() {
+            return self
+                .do_get_imported_keys(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetCrossReference>() {
+            return self
+                .do_get_cross_reference(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "do_get: The defined request is invalid: {:?}",
+            String::from_utf8(request.ticket).unwrap()
+        )))
+    }
+
+    async fn do_put(
+        &self,
+        _request: Request<Streaming<FlightData>>,
+    ) -> Result<Response<Self::DoPutStream>, Status> {
+        let request = _request.into_inner().message().await?.unwrap();
+        let any: prost_types::Any =
+            prost::Message::decode(&*request.flight_descriptor.unwrap().cmd)
+                .map_err(decode_error_to_status)?;
+        if any.is::<CommandStatementUpdate>() {
+            return self
+                .do_put_statement_update(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .do_put_prepared_statement_query(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementUpdate>() {
+            return self
+                .do_put_prepared_statement_update(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "do_put: The defined request is invalid: {:?}",
+            String::from_utf8(any.encode_to_vec()).unwrap()
+        )))
+    }
+
+    async fn list_actions(
+        &self,
+        _request: Request<Empty>,
+    ) -> Result<Response<Self::ListActionsStream>, Status> {
+        let create_prepared_statement_action_type = ActionType {
+            r#type: CREATE_PREPARED_STATEMENT.to_string(),
+            description: "Creates a reusable prepared statement resource on the server.\n
+                Request Message: ActionCreatePreparedStatementRequest\n
+                Response Message: ActionCreatePreparedStatementResult"
+                .into(),
+        };
+        let close_prepared_statement_action_type = ActionType {
+            r#type: CLOSE_PREPARED_STATEMENT.to_string(),
+            description: "Closes a reusable prepared statement resource on the server.\n
+                Request Message: ActionClosePreparedStatementRequest\n
+                Response Message: N/A"
+                .into(),
+        };
+        let actions: Vec<Result<ActionType, Status>> = vec![
+            Ok(create_prepared_statement_action_type),
+            Ok(close_prepared_statement_action_type),
+        ];
+        let output = futures::stream::iter(actions);
+        Ok(Response::new(Box::pin(output) as Self::ListActionsStream))
+    }
+
+    async fn do_action(
+        &self,
+        _request: Request<Action>,
+    ) -> Result<Response<Self::DoActionStream>, Status> {
+        let request = _request.into_inner();
+
+        if request.r#type == CREATE_PREPARED_STATEMENT {
+            let any: prost_types::Any =
+                prost::Message::decode(&*request.body).map_err(decode_error_to_status)?;
+
+            let cmd: ActionCreatePreparedStatementRequest = any
+                .unpack()
+                .map_err(arrow_error_to_status)?
+                .ok_or_else(|| {
+                    Status::invalid_argument(
+                        "Unable to unpack ActionCreatePreparedStatementRequest.",
+                    )
+                })?;
+            return self.do_action_create_prepared_statement(cmd).await;
+        }
+        if request.r#type == CLOSE_PREPARED_STATEMENT {
+            let any: prost_types::Any =
+                prost::Message::decode(&*request.body).map_err(decode_error_to_status)?;
+
+            let cmd: ActionClosePreparedStatementRequest = any
+                .unpack()
+                .map_err(arrow_error_to_status)?
+                .ok_or_else(|| {
+                    Status::invalid_argument(
+                        "Unable to unpack CloseCreatePreparedStatementRequest.",

Review comment:
       CloseCreatePreparedStatementRequest -> ActionClosePreparedStatementRequest?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822236245



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,585 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the table types.
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of SqlInfo results.
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the primary and foreign keys.
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the exported keys.
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the imported keys.
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the cross reference.
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    // do_put
+
+    /// Execute an update SQL statement.
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Bind parameters to given prepared statement.
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Execute an update SQL prepared statement.
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    // do_action
+
+    /// Create a prepared statement from given SQL statement.
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Close a prepared statement.
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Register a new SqlInfo result, making it available when calling GetSqlInfo.
+    async fn register_sql_info(&self, id: i32, result: &SqlInfo);
+}
+
+/// Implements the lower level interface to handle FlightSQL
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,
+    >;
+    type ListFlightsStream =
+        Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + Sync + 'static>>;
+    type DoGetStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+    type DoPutStream =
+        Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + Sync + 'static>>;
+    type DoActionStream = Pin<
+        Box<
+            dyn Stream<Item = Result<super::super::Result, Status>>
+                + Send
+                + Sync
+                + 'static,
+        >,
+    >;
+    type ListActionsStream =
+        Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + Sync + 'static>>;
+    type DoExchangeStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+
+    async fn handshake(
+        &self,
+        _request: Request<Streaming<HandshakeRequest>>,
+    ) -> Result<Response<Self::HandshakeStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn list_flights(
+        &self,
+        _request: Request<Criteria>,
+    ) -> Result<Response<Self::ListFlightsStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn get_flight_info(
+        &self,
+        request: Request<FlightDescriptor>,
+    ) -> Result<Response<FlightInfo>, Status> {
+        let request = request.into_inner();
+        let any: prost_types::Any = prost::Message::decode(&*request.cmd)
+            .map_err(|_| Status::invalid_argument("Unable to parse command"))?;
+
+        if any.is::<CommandStatementQuery>() {
+            return self
+                .get_flight_info_statement(
+                    any.unpack::<CommandStatementQuery>().unwrap().unwrap(),

Review comment:
       OK!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r823100955



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,637 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;

Review comment:
       I wonder if it would be possible to change this to return something more ergonomic (though not support streaming): 
   
   ```rust
     async fn do_get_tables(
           &self,
           query: CommandGetTables,
       ) -> Result<Vec<String>, Status>;
   ```

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,637 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;

Review comment:
       What about something like 
   
   
   ```
   async fn do_get_statement(
           &self,
           ticket: TicketStatementQuery,
       ) -> Result<SendableRecordBatchStream, Status>;
   ```
   
   ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822607916



##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {
+    /// Check if `Any` contains a message of given type.
+    fn is<M: ProstMessageExt>(&self) -> bool;
+
+    /// Extract a message from this `Any`.
+    ///
+    /// # Returns
+    ///
+    /// * `Ok(None)` when message type mismatch
+    /// * `Err` when parse failed
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>>;
+
+    /// Pack any message into `prost_types::Any` value.
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
+}
+
+impl ProstAnyExt for prost_types::Any {
+    fn is<M: ProstMessageExt>(&self) -> bool {
+        M::type_url() == self.type_url
+    }
+
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>> {
+        if !self.is::<M>() {
+            return Ok(None);
+        }
+        let m = prost::Message::decode(&*self.value).map_err(|err| {
+            ArrowError::ParseError(format!("Unable to decode Any value: {}", err))
+        })?;
+        Ok(Some(m))
+    }
+
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any> {
+        Ok(message.as_any())
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+
+    #[test]
+    fn test_type_url() {
+        assert_eq!(
+            TicketStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.TicketStatementQuery"
+        );
+        assert_eq!(
+            CommandStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.CommandStatementQuery"
+        );
+    }
+
+    #[test]
+    fn test_prost_any_pack_unpack() -> ArrowResult<()> {
+        let query = CommandStatementQuery {
+            query: "select 1".to_string(),
+        };
+        let any = prost_types::Any::pack(&query)?;
+        assert!(any.is::<CommandStatementQuery>());
+        let unpack_query: CommandStatementQuery =
+            any.unpack::<CommandStatementQuery>()?.unwrap();

Review comment:
       Oh, I remembered why I make it an item, it's because if I follow your style, we can even impl ProstMessageExt for String, but String is not a prost::Message. Not sure if it's an issue. My intent is I don't want non prost::Message type to impl ProstMessageExt
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r821107879



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,458 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use protobuf::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,

Review comment:
       Let's try with the generated code then. If people decide it is too annoying to deal with, we can add some wrapper classes. 

##########
File path: arrow-flight/src/sql/arrow.flight.protocol.sql.rs
##########
@@ -0,0 +1,1169 @@
+// This file was automatically generated through the build.rs script, and should not be edited.
+
+///
+/// Represents a metadata request. Used in the command member of FlightDescriptor
+/// for the following RPC calls:
+///  - GetSchema: return the Arrow schema of the query.
+///  - GetFlightInfo: execute the metadata request.
+///
+/// The returned Arrow schema will be:
+/// <
+///  info_name: uint32 not null,
+///  value: dense_union<
+///              string_value: utf8,
+///              bool_value: bool,
+///              bigint_value: int64,
+///              int32_bitmask: int32,
+///              string_list: list<string_data: utf8>
+///              int32_to_int32_list_map: map<key: int32, value: list<$data$: int32>>
+/// >
+/// where there is one row per requested piece of metadata information.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandGetSqlInfo {
+    ///
+    /// Values are modelled after ODBC's SQLGetInfo() function. This information is intended to provide
+    /// Flight SQL clients with basic, SQL syntax and SQL functions related information.
+    /// More information types can be added in future releases.
+    /// E.g. more SQL syntax support types, scalar functions support, type conversion support etc.
+    ///
+    /// Note that the set of metadata may expand.
+    ///
+    /// Initially, Flight SQL will support the following information types:
+    /// - Server Information - Range [0-500)
+    /// - Syntax Information - Range [500-1000)
+    /// Range [0-10,000) is reserved for defaults (see SqlInfo enum for default options).
+    /// Custom options should start at 10,000.
+    ///
+    /// If omitted, then all metadata will be retrieved.
+    /// Flight SQL Servers may choose to include additional metadata above and beyond the specified set, however they must
+    /// at least return the specified set. IDs ranging from 0 to 10,000 (exclusive) are reserved for future use.
+    /// If additional metadata is included, the metadata IDs should start from 10,000.
+    #[prost(uint32, repeated, tag = "1")]
+    pub info: ::prost::alloc::vec::Vec<u32>,
+}
+///
+/// Represents a request to retrieve the list of catalogs on a Flight SQL enabled backend.
+/// The definition of a catalog depends on vendor/implementation. It is usually the database itself
+/// Used in the command member of FlightDescriptor for the following RPC calls:
+///  - GetSchema: return the Arrow schema of the query.
+///  - GetFlightInfo: execute the catalog metadata request.
+///
+/// The returned Arrow schema will be:
+/// <
+///  catalog_name: utf8 not null
+/// >
+/// The returned data should be ordered by catalog_name.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandGetCatalogs {}
+///
+/// Represents a request to retrieve the list of database schemas on a Flight SQL enabled backend.
+/// The definition of a database schema depends on vendor/implementation. It is usually a collection of tables.
+/// Used in the command member of FlightDescriptor for the following RPC calls:
+///  - GetSchema: return the Arrow schema of the query.
+///  - GetFlightInfo: execute the catalog metadata request.
+///
+/// The returned Arrow schema will be:
+/// <
+///  catalog_name: utf8,
+///  db_schema_name: utf8 not null
+/// >
+/// The returned data should be ordered by catalog_name, then db_schema_name.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandGetDbSchemas {
+    ///
+    /// Specifies the Catalog to search for the tables.
+    /// An empty string retrieves those without a catalog.
+    /// If omitted the catalog name should not be used to narrow the search.
+    #[prost(string, optional, tag = "1")]
+    pub catalog: ::core::option::Option<::prost::alloc::string::String>,
+    ///
+    /// Specifies a filter pattern for schemas to search for.
+    /// When no db_schema_filter_pattern is provided, the pattern will not be used to narrow the search.
+    /// In the pattern string, two special characters can be used to denote matching rules:
+    ///    - "%" means to match any substring with 0 or more characters.
+    ///    - "_" means to match any one character.
+    #[prost(string, optional, tag = "2")]
+    pub db_schema_filter_pattern: ::core::option::Option<::prost::alloc::string::String>,
+}
+///
+/// Represents a request to retrieve the list of tables, and optionally their schemas, on a Flight SQL enabled backend.
+/// Used in the command member of FlightDescriptor for the following RPC calls:
+///  - GetSchema: return the Arrow schema of the query.
+///  - GetFlightInfo: execute the catalog metadata request.
+///
+/// The returned Arrow schema will be:
+/// <
+///  catalog_name: utf8,
+///  db_schema_name: utf8,
+///  table_name: utf8 not null,
+///  table_type: utf8 not null,
+///  \[optional\] table_schema: bytes not null (schema of the table as described in Schema.fbs::Schema,
+///                                           it is serialized as an IPC message.)
+/// >
+/// The returned data should be ordered by catalog_name, db_schema_name, table_name, then table_type, followed by table_schema if requested.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandGetTables {
+    ///
+    /// Specifies the Catalog to search for the tables.
+    /// An empty string retrieves those without a catalog.
+    /// If omitted the catalog name should not be used to narrow the search.
+    #[prost(string, optional, tag = "1")]
+    pub catalog: ::core::option::Option<::prost::alloc::string::String>,
+    ///
+    /// Specifies a filter pattern for schemas to search for.
+    /// When no db_schema_filter_pattern is provided, all schemas matching other filters are searched.
+    /// In the pattern string, two special characters can be used to denote matching rules:
+    ///    - "%" means to match any substring with 0 or more characters.
+    ///    - "_" means to match any one character.
+    #[prost(string, optional, tag = "2")]
+    pub db_schema_filter_pattern: ::core::option::Option<::prost::alloc::string::String>,
+    ///
+    /// Specifies a filter pattern for tables to search for.
+    /// When no table_name_filter_pattern is provided, all tables matching other filters are searched.
+    /// In the pattern string, two special characters can be used to denote matching rules:
+    ///    - "%" means to match any substring with 0 or more characters.
+    ///    - "_" means to match any one character.
+    #[prost(string, optional, tag = "3")]
+    pub table_name_filter_pattern: ::core::option::Option<::prost::alloc::string::String>,
+    ///
+    /// Specifies a filter of table types which must match.
+    /// The table types depend on vendor/implementation. It is usually used to separate tables from views or system tables.
+    /// TABLE, VIEW, and SYSTEM TABLE are commonly supported.
+    #[prost(string, repeated, tag = "4")]
+    pub table_types: ::prost::alloc::vec::Vec<::prost::alloc::string::String>,
+    /// Specifies if the Arrow schema should be returned for found tables.
+    #[prost(bool, tag = "5")]
+    pub include_schema: bool,
+}
+///
+/// Represents a request to retrieve the list of table types on a Flight SQL enabled backend.
+/// The table types depend on vendor/implementation. It is usually used to separate tables from views or system tables.
+/// TABLE, VIEW, and SYSTEM TABLE are commonly supported.
+/// Used in the command member of FlightDescriptor for the following RPC calls:
+///  - GetSchema: return the Arrow schema of the query.
+///  - GetFlightInfo: execute the catalog metadata request.
+///
+/// The returned Arrow schema will be:
+/// <
+///  table_type: utf8 not null
+/// >
+/// The returned data should be ordered by table_type.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandGetTableTypes {}
+///
+/// Represents a request to retrieve the primary keys of a table on a Flight SQL enabled backend.
+/// Used in the command member of FlightDescriptor for the following RPC calls:
+///  - GetSchema: return the Arrow schema of the query.
+///  - GetFlightInfo: execute the catalog metadata request.
+///
+/// The returned Arrow schema will be:
+/// <
+///  catalog_name: utf8,
+///  db_schema_name: utf8,
+///  table_name: utf8 not null,
+///  column_name: utf8 not null,
+///  key_name: utf8,
+///  key_sequence: int not null
+/// >
+/// The returned data should be ordered by catalog_name, db_schema_name, table_name, key_name, then key_sequence.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandGetPrimaryKeys {
+    ///
+    /// Specifies the catalog to search for the table.
+    /// An empty string retrieves those without a catalog.
+    /// If omitted the catalog name should not be used to narrow the search.
+    #[prost(string, optional, tag = "1")]
+    pub catalog: ::core::option::Option<::prost::alloc::string::String>,
+    ///
+    /// Specifies the schema to search for the table.
+    /// An empty string retrieves those without a schema.
+    /// If omitted the schema name should not be used to narrow the search.
+    #[prost(string, optional, tag = "2")]
+    pub db_schema: ::core::option::Option<::prost::alloc::string::String>,
+    /// Specifies the table to get the primary keys for.
+    #[prost(string, tag = "3")]
+    pub table: ::prost::alloc::string::String,
+}
+///
+/// Represents a request to retrieve a description of the foreign key columns that reference the given table's
+/// primary key columns (the foreign keys exported by a table) of a table on a Flight SQL enabled backend.
+/// Used in the command member of FlightDescriptor for the following RPC calls:
+///  - GetSchema: return the Arrow schema of the query.
+///  - GetFlightInfo: execute the catalog metadata request.
+///
+/// The returned Arrow schema will be:
+/// <
+///  pk_catalog_name: utf8,
+///  pk_db_schema_name: utf8,
+///  pk_table_name: utf8 not null,
+///  pk_column_name: utf8 not null,
+///  fk_catalog_name: utf8,
+///  fk_db_schema_name: utf8,
+///  fk_table_name: utf8 not null,
+///  fk_column_name: utf8 not null,
+///  key_sequence: int not null,
+///  fk_key_name: utf8,
+///  pk_key_name: utf8,
+///  update_rule: uint1 not null,
+///  delete_rule: uint1 not null
+/// >
+/// The returned data should be ordered by fk_catalog_name, fk_db_schema_name, fk_table_name, fk_key_name, then key_sequence.
+/// update_rule and delete_rule returns a byte that is equivalent to actions declared on UpdateDeleteRules enum.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandGetExportedKeys {
+    ///
+    /// Specifies the catalog to search for the foreign key table.
+    /// An empty string retrieves those without a catalog.
+    /// If omitted the catalog name should not be used to narrow the search.
+    #[prost(string, optional, tag = "1")]
+    pub catalog: ::core::option::Option<::prost::alloc::string::String>,
+    ///
+    /// Specifies the schema to search for the foreign key table.
+    /// An empty string retrieves those without a schema.
+    /// If omitted the schema name should not be used to narrow the search.
+    #[prost(string, optional, tag = "2")]
+    pub db_schema: ::core::option::Option<::prost::alloc::string::String>,
+    /// Specifies the foreign key table to get the foreign keys for.
+    #[prost(string, tag = "3")]
+    pub table: ::prost::alloc::string::String,
+}
+///
+/// Represents a request to retrieve the foreign keys of a table on a Flight SQL enabled backend.
+/// Used in the command member of FlightDescriptor for the following RPC calls:
+///  - GetSchema: return the Arrow schema of the query.
+///  - GetFlightInfo: execute the catalog metadata request.
+///
+/// The returned Arrow schema will be:
+/// <
+///  pk_catalog_name: utf8,
+///  pk_db_schema_name: utf8,
+///  pk_table_name: utf8 not null,
+///  pk_column_name: utf8 not null,
+///  fk_catalog_name: utf8,
+///  fk_db_schema_name: utf8,
+///  fk_table_name: utf8 not null,
+///  fk_column_name: utf8 not null,
+///  key_sequence: int not null,
+///  fk_key_name: utf8,
+///  pk_key_name: utf8,
+///  update_rule: uint1 not null,
+///  delete_rule: uint1 not null
+/// >
+/// The returned data should be ordered by pk_catalog_name, pk_db_schema_name, pk_table_name, pk_key_name, then key_sequence.
+/// update_rule and delete_rule returns a byte that is equivalent to actions:
+///    - 0 = CASCADE
+///    - 1 = RESTRICT
+///    - 2 = SET NULL
+///    - 3 = NO ACTION
+///    - 4 = SET DEFAULT
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandGetImportedKeys {
+    ///
+    /// Specifies the catalog to search for the primary key table.
+    /// An empty string retrieves those without a catalog.
+    /// If omitted the catalog name should not be used to narrow the search.
+    #[prost(string, optional, tag = "1")]
+    pub catalog: ::core::option::Option<::prost::alloc::string::String>,
+    ///
+    /// Specifies the schema to search for the primary key table.
+    /// An empty string retrieves those without a schema.
+    /// If omitted the schema name should not be used to narrow the search.
+    #[prost(string, optional, tag = "2")]
+    pub db_schema: ::core::option::Option<::prost::alloc::string::String>,
+    /// Specifies the primary key table to get the foreign keys for.
+    #[prost(string, tag = "3")]
+    pub table: ::prost::alloc::string::String,
+}
+///
+/// Represents a request to retrieve a description of the foreign key columns in the given foreign key table that
+/// reference the primary key or the columns representing a unique constraint of the parent table (could be the same
+/// or a different table) on a Flight SQL enabled backend.
+/// Used in the command member of FlightDescriptor for the following RPC calls:
+///  - GetSchema: return the Arrow schema of the query.
+///  - GetFlightInfo: execute the catalog metadata request.
+///
+/// The returned Arrow schema will be:
+/// <
+///  pk_catalog_name: utf8,
+///  pk_db_schema_name: utf8,
+///  pk_table_name: utf8 not null,
+///  pk_column_name: utf8 not null,
+///  fk_catalog_name: utf8,
+///  fk_db_schema_name: utf8,
+///  fk_table_name: utf8 not null,
+///  fk_column_name: utf8 not null,
+///  key_sequence: int not null,
+///  fk_key_name: utf8,
+///  pk_key_name: utf8,
+///  update_rule: uint1 not null,
+///  delete_rule: uint1 not null
+/// >
+/// The returned data should be ordered by pk_catalog_name, pk_db_schema_name, pk_table_name, pk_key_name, then key_sequence.
+/// update_rule and delete_rule returns a byte that is equivalent to actions:
+///    - 0 = CASCADE
+///    - 1 = RESTRICT
+///    - 2 = SET NULL
+///    - 3 = NO ACTION
+///    - 4 = SET DEFAULT
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandGetCrossReference {
+    ///*
+    /// The catalog name where the parent table is.
+    /// An empty string retrieves those without a catalog.
+    /// If omitted the catalog name should not be used to narrow the search.
+    #[prost(string, optional, tag = "1")]
+    pub pk_catalog: ::core::option::Option<::prost::alloc::string::String>,
+    ///*
+    /// The Schema name where the parent table is.
+    /// An empty string retrieves those without a schema.
+    /// If omitted the schema name should not be used to narrow the search.
+    #[prost(string, optional, tag = "2")]
+    pub pk_db_schema: ::core::option::Option<::prost::alloc::string::String>,
+    ///*
+    /// The parent table name. It cannot be null.
+    #[prost(string, tag = "3")]
+    pub pk_table: ::prost::alloc::string::String,
+    ///*
+    /// The catalog name where the foreign table is.
+    /// An empty string retrieves those without a catalog.
+    /// If omitted the catalog name should not be used to narrow the search.
+    #[prost(string, optional, tag = "4")]
+    pub fk_catalog: ::core::option::Option<::prost::alloc::string::String>,
+    ///*
+    /// The schema name where the foreign table is.
+    /// An empty string retrieves those without a schema.
+    /// If omitted the schema name should not be used to narrow the search.
+    #[prost(string, optional, tag = "5")]
+    pub fk_db_schema: ::core::option::Option<::prost::alloc::string::String>,
+    ///*
+    /// The foreign table name. It cannot be null.
+    #[prost(string, tag = "6")]
+    pub fk_table: ::prost::alloc::string::String,
+}
+// SQL Execution Action Messages
+
+///
+/// Request message for the "CreatePreparedStatement" action on a Flight SQL enabled backend.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct ActionCreatePreparedStatementRequest {
+    /// The valid SQL string to create a prepared statement for.
+    #[prost(string, tag = "1")]
+    pub query: ::prost::alloc::string::String,
+}
+///
+/// Wrap the result of a "GetPreparedStatement" action.
+///
+/// The resultant PreparedStatement can be closed either:
+/// - Manually, through the "ClosePreparedStatement" action;
+/// - Automatically, by a server timeout.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct ActionCreatePreparedStatementResult {
+    /// Opaque handle for the prepared statement on the server.
+    #[prost(bytes = "vec", tag = "1")]
+    pub prepared_statement_handle: ::prost::alloc::vec::Vec<u8>,
+    /// If a result set generating query was provided, dataset_schema contains the
+    /// schema of the dataset as described in Schema.fbs::Schema, it is serialized as an IPC message.
+    #[prost(bytes = "vec", tag = "2")]
+    pub dataset_schema: ::prost::alloc::vec::Vec<u8>,
+    /// If the query provided contained parameters, parameter_schema contains the
+    /// schema of the expected parameters as described in Schema.fbs::Schema, it is serialized as an IPC message.
+    #[prost(bytes = "vec", tag = "3")]
+    pub parameter_schema: ::prost::alloc::vec::Vec<u8>,
+}
+///
+/// Request message for the "ClosePreparedStatement" action on a Flight SQL enabled backend.
+/// Closes server resources associated with the prepared statement handle.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct ActionClosePreparedStatementRequest {
+    /// Opaque handle for the prepared statement on the server.
+    #[prost(bytes = "vec", tag = "1")]
+    pub prepared_statement_handle: ::prost::alloc::vec::Vec<u8>,
+}
+// SQL Execution Messages.
+
+///
+/// Represents a SQL query. Used in the command member of FlightDescriptor
+/// for the following RPC calls:
+///  - GetSchema: return the Arrow schema of the query.
+///  - GetFlightInfo: execute the query.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandStatementQuery {
+    /// The SQL syntax.
+    #[prost(string, tag = "1")]
+    pub query: ::prost::alloc::string::String,
+}
+///*
+/// Represents a ticket resulting from GetFlightInfo with a CommandStatementQuery.
+/// This should be used only once and treated as an opaque value, that is, clients should not attempt to parse this.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct TicketStatementQuery {
+    /// Unique identifier for the instance of the statement to execute.
+    #[prost(bytes = "vec", tag = "1")]
+    pub statement_handle: ::prost::alloc::vec::Vec<u8>,
+}
+///
+/// Represents an instance of executing a prepared statement. Used in the command member of FlightDescriptor for
+/// the following RPC calls:
+///  - DoPut: bind parameter values. All of the bound parameter sets will be executed as a single atomic execution.
+///  - GetFlightInfo: execute the prepared statement instance.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandPreparedStatementQuery {
+    /// Opaque handle for the prepared statement on the server.
+    #[prost(bytes = "vec", tag = "1")]
+    pub prepared_statement_handle: ::prost::alloc::vec::Vec<u8>,
+}
+///
+/// Represents a SQL update query. Used in the command member of FlightDescriptor
+/// for the the RPC call DoPut to cause the server to execute the included SQL update.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandStatementUpdate {
+    /// The SQL syntax.
+    #[prost(string, tag = "1")]
+    pub query: ::prost::alloc::string::String,
+}
+///
+/// Represents a SQL update query. Used in the command member of FlightDescriptor
+/// for the the RPC call DoPut to cause the server to execute the included
+/// prepared statement handle as an update.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct CommandPreparedStatementUpdate {
+    /// Opaque handle for the prepared statement on the server.
+    #[prost(bytes = "vec", tag = "1")]
+    pub prepared_statement_handle: ::prost::alloc::vec::Vec<u8>,
+}
+///
+/// Returned from the RPC call DoPut when a CommandStatementUpdate
+/// CommandPreparedStatementUpdate was in the request, containing
+/// results from the update.
+#[derive(Clone, PartialEq, ::prost::Message)]
+pub struct DoPutUpdateResult {
+    /// The number of records updated. A return value of -1 represents
+    /// an unknown updated record count.
+    #[prost(int64, tag = "1")]
+    pub record_count: i64,
+}
+/// Options for CommandGetSqlInfo.

Review comment:
       it is awesome that all these comments (from the protobuf) got kept




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822221312



##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {
+    /// Check if `Any` contains a message of given type.
+    fn is<M: ProstMessageExt>(&self) -> bool;
+
+    /// Extract a message from this `Any`.
+    ///
+    /// # Returns
+    ///
+    /// * `Ok(None)` when message type mismatch
+    /// * `Err` when parse failed
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>>;
+
+    /// Pack any message into `prost_types::Any` value.
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
+}
+
+impl ProstAnyExt for prost_types::Any {
+    fn is<M: ProstMessageExt>(&self) -> bool {
+        M::type_url() == self.type_url
+    }
+
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>> {

Review comment:
       Most public API in this lib will return ArrowResult, I think it's better to keep it consistent




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] codecov-commenter edited a comment on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1060951386


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1386](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dc3bcaf) into [master](https://codecov.io/gh/apache/arrow-rs/commit/a2e629d4eeb8b9a80ff1fd00c8d1a69736e87b19?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a2e629d) will **decrease** coverage by `0.28%`.
   > The diff coverage is `8.92%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1386/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1386      +/-   ##
   ==========================================
   - Coverage   83.03%   82.74%   -0.29%     
   ==========================================
     Files         181      184       +3     
     Lines       52956    53607     +651     
   ==========================================
   + Hits        43972    44358     +386     
   - Misses       8984     9249     +265     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [arrow-flight/src/lib.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9saWIucnM=) | `18.54% <ø> (ø)` | |
   | [arrow-flight/src/sql/server.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9zcWwvc2VydmVyLnJz) | `0.00% <0.00%> (ø)` | |
   | [arrow-flight/src/sql/mod.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9zcWwvbW9kLnJz) | `92.59% <92.59%> (ø)` | |
   | [parquet/src/file/statistics.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvZmlsZS9zdGF0aXN0aWNzLnJz) | `91.73% <0.00%> (-2.07%)` | :arrow_down: |
   | [parquet/src/util/cursor.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvdXRpbC9jdXJzb3IucnM=) | `77.31% <0.00%> (-1.69%)` | :arrow_down: |
   | [parquet/src/schema/types.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvc2NoZW1hL3R5cGVzLnJz) | `85.64% <0.00%> (-1.51%)` | :arrow_down: |
   | [arrow/src/array/array\_dictionary.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L2FycmF5X2RpY3Rpb25hcnkucnM=) | `91.12% <0.00%> (-0.47%)` | :arrow_down: |
   | [arrow/src/array/data.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L2RhdGEucnM=) | `83.15% <0.00%> (-0.44%)` | :arrow_down: |
   | [arrow/src/ipc/writer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2lwYy93cml0ZXIucnM=) | `83.14% <0.00%> (-0.32%)` | :arrow_down: |
   | [arrow/src/datatypes/field.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2RhdGF0eXBlcy9maWVsZC5ycw==) | `53.79% <0.00%> (-0.31%)` | :arrow_down: |
   | ... and [27 more](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [a2e629d...dc3bcaf](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb commented on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1064304973


   > I'll add a flight-sql feature flag for this
   
   Thank you
   
   > Agree that maybe we can make the API more ergonomic, but I need to do more experiments on this (I'm trying to build a more practical flight-sql-server using this), then we will clear what we need. My suggestion is we can design the API as it is, and after we have better/ more simplified design, we can add them into the trait, and make the low level API as a default implementation in the trait, so the user still have chance to override them if they want. It's very important we leave this flexibility to the user.
   
   100% agree
   
   > grpcurl may not work for our testing. As I comment in https://github.com/wangfenjin/arrow-datafusion/pull/1 this PR, I use the [arrow-cpp-cli](https://github.com/apache/arrow/blob/master/cpp/src/arrow/flight/sql/test_app_cli.cc) to connect to this server, it helps when we don't have the client implementation, also it makes sure our implementation is compatible with the cpp. We may also need to think about maintain this compatibility in long term.
   
   > For the documentation thing, I copy some useful comments from cpp implementation. For more detailed documentation like the protocols, I think it's a joint effort with the cpp community, in this repo we can focus on the rust API documentation.
   
   Thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb commented on pull request #1386: init impl flight sql

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1058348646


   Thank you @wangfenjin  -- I will try and give this a review over the next few days


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822219873



##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {

Review comment:
       We may need to use it in service implementation like here https://github.com/wangfenjin/arrow-datafusion/pull/1/files#diff-d942c264020a5d47b87deaca1b1064b53f3819a8f90764fad8fa3c2b9ccf6225R81




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822221312



##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {
+    /// Check if `Any` contains a message of given type.
+    fn is<M: ProstMessageExt>(&self) -> bool;
+
+    /// Extract a message from this `Any`.
+    ///
+    /// # Returns
+    ///
+    /// * `Ok(None)` when message type mismatch
+    /// * `Err` when parse failed
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>>;
+
+    /// Pack any message into `prost_types::Any` value.
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
+}
+
+impl ProstAnyExt for prost_types::Any {
+    fn is<M: ProstMessageExt>(&self) -> bool {
+        M::type_url() == self.type_url
+    }
+
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>> {

Review comment:
       Most publish API in this lib will return ArrowResult, I think it's better to keep it consistent




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822499799



##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {
+    /// Check if `Any` contains a message of given type.
+    fn is<M: ProstMessageExt>(&self) -> bool;
+
+    /// Extract a message from this `Any`.
+    ///
+    /// # Returns
+    ///
+    /// * `Ok(None)` when message type mismatch
+    /// * `Err` when parse failed
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>>;
+
+    /// Pack any message into `prost_types::Any` value.
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
+}
+
+impl ProstAnyExt for prost_types::Any {
+    fn is<M: ProstMessageExt>(&self) -> bool {
+        M::type_url() == self.type_url
+    }
+
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>> {
+        if !self.is::<M>() {
+            return Ok(None);
+        }
+        let m = prost::Message::decode(&*self.value).map_err(|err| {
+            ArrowError::ParseError(format!("Unable to decode Any value: {}", err))
+        })?;
+        Ok(Some(m))
+    }
+
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any> {
+        Ok(message.as_any())
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+
+    #[test]
+    fn test_type_url() {
+        assert_eq!(
+            TicketStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.TicketStatementQuery"
+        );
+        assert_eq!(
+            CommandStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.CommandStatementQuery"
+        );
+    }
+
+    #[test]
+    fn test_prost_any_pack_unpack() -> ArrowResult<()> {
+        let query = CommandStatementQuery {
+            query: "select 1".to_string(),
+        };
+        let any = prost_types::Any::pack(&query)?;
+        assert!(any.is::<CommandStatementQuery>());
+        let unpack_query: CommandStatementQuery =
+            any.unpack::<CommandStatementQuery>()?.unwrap();

Review comment:
       Because when I use [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/7a55256b3348f098846feb53dae71afb33fede41/protobuf/src/well_known_types_util/any.rs#L86), it don't require us to annotate the type, I'm not sure which part is wrong in my implementation. Get ride of the type annotation can make the code more compact.
   
   Do you have example how should I fix this? I have no idea about the way you mentioned 😂




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] tustvold commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822460994



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,585 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the table types.
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of SqlInfo results.
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the primary and foreign keys.
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the exported keys.
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the imported keys.
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the cross reference.
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    // do_put
+
+    /// Execute an update SQL statement.
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Bind parameters to given prepared statement.
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Execute an update SQL prepared statement.
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    // do_action
+
+    /// Create a prepared statement from given SQL statement.
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Close a prepared statement.
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Register a new SqlInfo result, making it available when calling GetSqlInfo.
+    async fn register_sql_info(&self, id: i32, result: &SqlInfo);
+}
+
+/// Implements the lower level interface to handle FlightSQL
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,

Review comment:
       A Sync object can have member functions that return non-Sync objects? There is no real reason for a stream to be Sync as it can only be consumed through a mutable reference




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822608896



##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {
+    /// Check if `Any` contains a message of given type.
+    fn is<M: ProstMessageExt>(&self) -> bool;
+
+    /// Extract a message from this `Any`.
+    ///
+    /// # Returns
+    ///
+    /// * `Ok(None)` when message type mismatch
+    /// * `Err` when parse failed
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>>;
+
+    /// Pack any message into `prost_types::Any` value.
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
+}
+
+impl ProstAnyExt for prost_types::Any {
+    fn is<M: ProstMessageExt>(&self) -> bool {
+        M::type_url() == self.type_url
+    }
+
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>> {
+        if !self.is::<M>() {
+            return Ok(None);
+        }
+        let m = prost::Message::decode(&*self.value).map_err(|err| {
+            ArrowError::ParseError(format!("Unable to decode Any value: {}", err))
+        })?;
+        Ok(Some(m))
+    }
+
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any> {
+        Ok(message.as_any())
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+
+    #[test]
+    fn test_type_url() {
+        assert_eq!(
+            TicketStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.TicketStatementQuery"
+        );
+        assert_eq!(
+            CommandStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.CommandStatementQuery"
+        );
+    }
+
+    #[test]
+    fn test_prost_any_pack_unpack() -> ArrowResult<()> {
+        let query = CommandStatementQuery {
+            query: "select 1".to_string(),
+        };
+        let any = prost_types::Any::pack(&query)?;
+        assert!(any.is::<CommandStatementQuery>());
+        let unpack_query: CommandStatementQuery =
+            any.unpack::<CommandStatementQuery>()?.unwrap();

Review comment:
       But anyway I think I'll follow your style, to make the function call more compact




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] tustvold commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r820236601



##########
File path: arrow-flight/Cargo.toml
##########
@@ -32,14 +32,17 @@ base64 = "0.13"
 tonic = "0.6"
 bytes = "1"
 prost = "0.9"
+protobuf = "2.27.1"

Review comment:
       As Data fusion uses prost and tonic already, it seems unfortunate to bring in a other protobuf library to decode the any payloads

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,458 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use protobuf::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, TicketStatementQuery,
+};
+
+use lazy_static::lazy_static;
+lazy_static! {
+    static ref CREATE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "CreatePreparedStatement".into(),
+        description: "Creates a reusable prepared statement resource on the server.\n
+                Request Message: ActionCreatePreparedStatementRequest\n
+                Response Message: ActionCreatePreparedStatementResult"
+            .into(),
+    };
+    static ref CLOSE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "ClosePreparedStatement".into(),
+        description: "Closes a reusable prepared statement resource on the server.\n
+                Request Message: ActionClosePreparedStatementRequest\n
+                Response Message: N/A"
+            .into(),
+    };
+}
+
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    type FlightService: FlightService;
+
+    // get_flight_info
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    // do_get
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    // do_put
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    // do_action
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+}
+
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,
+    >;
+    type ListFlightsStream =
+        Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + Sync + 'static>>;
+    type DoGetStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+    type DoPutStream =
+        Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + Sync + 'static>>;
+    type DoActionStream = Pin<
+        Box<
+            dyn Stream<Item = Result<super::super::Result, Status>>
+                + Send
+                + Sync
+                + 'static,
+        >,
+    >;
+    type ListActionsStream =
+        Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + Sync + 'static>>;
+    type DoExchangeStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+
+    async fn handshake(
+        &self,
+        _request: Request<Streaming<HandshakeRequest>>,
+    ) -> Result<Response<Self::HandshakeStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn list_flights(
+        &self,
+        _request: Request<Criteria>,
+    ) -> Result<Response<Self::ListFlightsStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn get_flight_info(
+        &self,
+        request: Request<FlightDescriptor>,
+    ) -> Result<Response<FlightInfo>, Status> {
+        let request = request.into_inner();
+        let any: protobuf::well_known_types::Any =
+            protobuf::Message::parse_from_bytes(&request.cmd)

Review comment:
       I'm not familiar with this different protobuf library, but at least with prost I'm not aware of anything better than matching on type URL... 

##########
File path: arrow-flight/build.rs
##########
@@ -49,6 +49,40 @@ fn main() -> Result<(), Box<dyn std::error::Error>> {
         file.write_all(buffer.as_bytes())?;
     }
 
+    // The current working directory can vary depending on how the project is being
+    // built or released so we build an absolute path to the proto file
+    let path = Path::new("../format/FlightSql.proto");

Review comment:
       I think is bringing in a second protobuf library which seems not ideal... I think it would be better to use prost here




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb commented on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1061142897


   Thanks @wangfenjin  -- I will try and review this carefully tomorrow


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb commented on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1060608164


   > [As you may have noted, I implement an example in https://github.com/wangfenjin/arrow-datafusion/pull/1 (we need to publish a new version then I can create MR to the arrow-datafusion repo). For the integration test, not sure if we need to implement it in [this](https://github.com/apache/arrow-rs/blob/master/arrow-flight/examples/server.rs) repo? Because if we implement one like this seems useless, and if we want to implement a useful example, we'd better depends on arrow-datafusion as we need a SQL server.](https://github.com/apache/arrow-rs/pull/1386#issuecomment-1059971584)
   
   I actually think both types of examples are useful, but for different purposes:
   1. Example just in arrow-flight (no actual SQL implementation) such as [this](https://github.com/apache/arrow-rs/blob/master/arrow-flight/examples/server.rs): helps users of arrow-flight who will not be using datafusion something to start with that compiles so they can plug in their own implementation that without having to cut out datafusion specific stuff
   
   2. Example in datafusion such as [this](https://github.com/wangfenjin/arrow-datafusion/pull/1): Shows a real end to end use of flight sql and how one system connects it together
   
   The example in arrow can be done as a follow on PR (maybe someone else will do it) -- I'll plan to file tickets for follow on work after your initial PR
   
   Thanks again @wangfenjin 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] tustvold commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r820237002



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,458 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use protobuf::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, TicketStatementQuery,
+};
+
+use lazy_static::lazy_static;
+lazy_static! {
+    static ref CREATE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "CreatePreparedStatement".into(),
+        description: "Creates a reusable prepared statement resource on the server.\n
+                Request Message: ActionCreatePreparedStatementRequest\n
+                Response Message: ActionCreatePreparedStatementResult"
+            .into(),
+    };
+    static ref CLOSE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "ClosePreparedStatement".into(),
+        description: "Closes a reusable prepared statement resource on the server.\n
+                Request Message: ActionClosePreparedStatementRequest\n
+                Response Message: N/A"
+            .into(),
+    };
+}
+
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    type FlightService: FlightService;
+
+    // get_flight_info
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    // do_get
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    // do_put
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    // do_action
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+}
+
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,
+    >;
+    type ListFlightsStream =
+        Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + Sync + 'static>>;
+    type DoGetStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+    type DoPutStream =
+        Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + Sync + 'static>>;
+    type DoActionStream = Pin<
+        Box<
+            dyn Stream<Item = Result<super::super::Result, Status>>
+                + Send
+                + Sync
+                + 'static,
+        >,
+    >;
+    type ListActionsStream =
+        Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + Sync + 'static>>;
+    type DoExchangeStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+
+    async fn handshake(
+        &self,
+        _request: Request<Streaming<HandshakeRequest>>,
+    ) -> Result<Response<Self::HandshakeStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn list_flights(
+        &self,
+        _request: Request<Criteria>,
+    ) -> Result<Response<Self::ListFlightsStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn get_flight_info(
+        &self,
+        request: Request<FlightDescriptor>,
+    ) -> Result<Response<FlightInfo>, Status> {
+        let request = request.into_inner();
+        let any: protobuf::well_known_types::Any =
+            protobuf::Message::parse_from_bytes(&request.cmd)

Review comment:
       I'm not familiar with this different protobuf library, but at least with prost I'm not aware of anything better than matching on type URL which is effectively what this seems to be doing...




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] tustvold commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822613926



##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {
+    /// Check if `Any` contains a message of given type.
+    fn is<M: ProstMessageExt>(&self) -> bool;
+
+    /// Extract a message from this `Any`.
+    ///
+    /// # Returns
+    ///
+    /// * `Ok(None)` when message type mismatch
+    /// * `Err` when parse failed
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>>;
+
+    /// Pack any message into `prost_types::Any` value.
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
+}
+
+impl ProstAnyExt for prost_types::Any {
+    fn is<M: ProstMessageExt>(&self) -> bool {
+        M::type_url() == self.type_url
+    }
+
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>> {
+        if !self.is::<M>() {
+            return Ok(None);
+        }
+        let m = prost::Message::decode(&*self.value).map_err(|err| {
+            ArrowError::ParseError(format!("Unable to decode Any value: {}", err))
+        })?;
+        Ok(Some(m))
+    }
+
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any> {
+        Ok(message.as_any())
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+
+    #[test]
+    fn test_type_url() {
+        assert_eq!(
+            TicketStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.TicketStatementQuery"
+        );
+        assert_eq!(
+            CommandStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.CommandStatementQuery"
+        );
+    }
+
+    #[test]
+    fn test_prost_any_pack_unpack() -> ArrowResult<()> {
+        let query = CommandStatementQuery {
+            query: "select 1".to_string(),
+        };
+        let any = prost_types::Any::pack(&query)?;
+        assert!(any.is::<CommandStatementQuery>());
+        let unpack_query: CommandStatementQuery =
+            any.unpack::<CommandStatementQuery>()?.unwrap();

Review comment:
       I think having an extension type not inherit from the base trait it is extending, and instead expose this as an associated type is a little odd. So sounds good to me 👍




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] GavinRay97 commented on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
GavinRay97 commented on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1060026352


   This is awesome! Thank you =)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] codecov-commenter edited a comment on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1060951386


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1386](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5f8919f) into [master](https://codecov.io/gh/apache/arrow-rs/commit/4bcc7a678914e9271789797b833a8f731f8b224b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4bcc7a6) will **decrease** coverage by `0.48%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1386/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1386      +/-   ##
   ==========================================
   - Coverage   83.17%   82.69%   -0.49%     
   ==========================================
     Files         182      185       +3     
     Lines       53439    53752     +313     
   ==========================================
     Hits        44449    44449              
   - Misses       8990     9303     +313     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [arrow-flight/examples/flight\_sql\_server.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L2V4YW1wbGVzL2ZsaWdodF9zcWxfc2VydmVyLnJz) | `0.00% <0.00%> (ø)` | |
   | [arrow-flight/src/lib.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9saWIucnM=) | `18.54% <ø> (ø)` | |
   | [arrow-flight/src/sql/mod.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9zcWwvbW9kLnJz) | `0.00% <0.00%> (ø)` | |
   | [arrow-flight/src/sql/server.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9zcWwvc2VydmVyLnJz) | `0.00% <0.00%> (ø)` | |
   | [arrow/src/datatypes/datatype.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2RhdGF0eXBlcy9kYXRhdHlwZS5ycw==) | `66.40% <0.00%> (-0.40%)` | :arrow_down: |
   | [arrow/src/array/transform/mod.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L3RyYW5zZm9ybS9tb2QucnM=) | `86.31% <0.00%> (-0.12%)` | :arrow_down: |
   | [parquet\_derive/src/parquet\_field.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldF9kZXJpdmUvc3JjL3BhcnF1ZXRfZmllbGQucnM=) | `66.21% <0.00%> (+0.22%)` | :arrow_up: |
   | [arrow/src/datatypes/field.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2RhdGF0eXBlcy9maWVsZC5ycw==) | `54.10% <0.00%> (+0.30%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [4bcc7a6...5f8919f](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] codecov-commenter edited a comment on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1060951386


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1386](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (49d5916) into [master](https://codecov.io/gh/apache/arrow-rs/commit/a2e629d4eeb8b9a80ff1fd00c8d1a69736e87b19?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a2e629d) will **decrease** coverage by `0.25%`.
   > The diff coverage is `67.57%`.
   
   > :exclamation: Current head 49d5916 differs from pull request most recent head b384d83. Consider uploading reports for the commit b384d83 to get more accurate results
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1386/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1386      +/-   ##
   ==========================================
   - Coverage   83.03%   82.77%   -0.26%     
   ==========================================
     Files         181      184       +3     
     Lines       52956    53674     +718     
   ==========================================
   + Hits        43972    44431     +459     
   - Misses       8984     9243     +259     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [arrow-flight/src/lib.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9saWIucnM=) | `18.54% <ø> (ø)` | |
   | [arrow-flight/src/sql/server.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9zcWwvc2VydmVyLnJz) | `0.00% <0.00%> (ø)` | |
   | [arrow/src/array/array\_dictionary.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L2FycmF5X2RpY3Rpb25hcnkucnM=) | `91.12% <0.00%> (-0.47%)` | :arrow_down: |
   | [arrow/src/array/mod.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L21vZC5ycw==) | `100.00% <ø> (ø)` | |
   | [arrow/src/compute/kernels/window.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2NvbXB1dGUva2VybmVscy93aW5kb3cucnM=) | `100.00% <ø> (ø)` | |
   | [arrow/src/csv/reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2Nzdi9yZWFkZXIucnM=) | `89.89% <ø> (+1.64%)` | :arrow_up: |
   | [arrow/src/ipc/writer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2lwYy93cml0ZXIucnM=) | `83.14% <ø> (-0.32%)` | :arrow_down: |
   | [...-testing/src/bin/flight-test-integration-client.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aW50ZWdyYXRpb24tdGVzdGluZy9zcmMvYmluL2ZsaWdodC10ZXN0LWludGVncmF0aW9uLWNsaWVudC5ycw==) | `0.00% <ø> (ø)` | |
   | [...ng/src/flight\_client\_scenarios/integration\_test.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aW50ZWdyYXRpb24tdGVzdGluZy9zcmMvZmxpZ2h0X2NsaWVudF9zY2VuYXJpb3MvaW50ZWdyYXRpb25fdGVzdC5ycw==) | `0.00% <0.00%> (ø)` | |
   | [parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyLnJz) | `78.27% <0.00%> (ø)` | |
   | ... and [42 more](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [a2e629d...b384d83](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1063552825


   Thanks @alamb  for your kind review, address some of your comments:
   
   1. I'll add a flight-sql feature flag for this
   2. Agree that maybe we can make the API more ergonomic, but I need to do more experiments on this (I'm trying to build a more practical flight-sql-server using this), then we will clear what we need. My suggestion is we can design the API as it is, and we have better/ more simplified design, we can add them into the trait, and make the low level API as a default implementation in the trait, so the user still have chance to override them if they want. It's very important we leave this flexibility to the user.
   3. grpcurl may not work for our testing. As I comment in https://github.com/wangfenjin/arrow-datafusion/pull/1 this PR, I use the  [arrow-cpp-cli](https://github.com/apache/arrow/blob/master/cpp/src/arrow/flight/sql/test_app_cli.cc) to connect to this server, it helps when we don't have the client implementation, also it makes sure our implementation is compatible with the cpp. We may also need to think about maintain this compatibility in long term.
   4. For the documentation thing, I copy some useful comments from cpp implementation. For more detailed documentation like the protocols, I think it's a joint effort with the cpp community, in this repo we can focus on the rust API documentation.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r821727089



##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {
+    /// Check if `Any` contains a message of given type.
+    fn is<M: ProstMessageExt>(&self) -> bool;
+
+    /// Extract a message from this `Any`.
+    ///
+    /// # Returns
+    ///
+    /// * `Ok(None)` when message type mismatch
+    /// * `Err` when parse failed
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>>;
+
+    /// Pack any message into `prost_types::Any` value.
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
+}
+
+impl ProstAnyExt for prost_types::Any {
+    fn is<M: ProstMessageExt>(&self) -> bool {
+        M::type_url() == self.type_url
+    }
+
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>> {
+        if !self.is::<M>() {
+            return Ok(None);
+        }
+        let m = prost::Message::decode(&*self.value).map_err(|err| {
+            ArrowError::ParseError(format!("Unable to decode Any value: {}", err))
+        })?;
+        Ok(Some(m))
+    }
+
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any> {
+        Ok(message.as_any())
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+
+    #[test]
+    fn test_type_url() {
+        assert_eq!(
+            TicketStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.TicketStatementQuery"
+        );
+        assert_eq!(
+            CommandStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.CommandStatementQuery"
+        );
+    }
+
+    #[test]
+    fn test_prost_any_pack_unpack() -> ArrowResult<()> {
+        let query = CommandStatementQuery {
+            query: "select 1".to_string(),
+        };
+        let any = prost_types::Any::pack(&query)?;
+        assert!(any.is::<CommandStatementQuery>());
+        let unpack_query: CommandStatementQuery =
+            any.unpack::<CommandStatementQuery>()?.unwrap();

Review comment:
       @alamb Could you help on this this? I always need to add the type annotation for the unpack() method, otherwise it will report error:
   
   ```
   error[E0284]: type annotations needed: cannot satisfy `<_ as sql::ProstMessageExt>::Item == sql::gen::CommandStatementQuery`
      --> arrow-flight/src/sql/mod.rs:179:17
       |
   179 |             any.unpack()?.unwrap();
       |                 ^^^^^^ cannot satisfy `<_ as sql::ProstMessageExt>::Item == sql::gen::CommandStatementQuery`
   
   For more information about this error, try `rustc --explain E0284`.
   ```
   Not sure how to make it more convenient.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb commented on a change in pull request #1386: init impl flight sql

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r820233556



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,458 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use protobuf::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, TicketStatementQuery,
+};
+
+use lazy_static::lazy_static;
+lazy_static! {
+    static ref CREATE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "CreatePreparedStatement".into(),
+        description: "Creates a reusable prepared statement resource on the server.\n
+                Request Message: ActionCreatePreparedStatementRequest\n
+                Response Message: ActionCreatePreparedStatementResult"
+            .into(),
+    };
+    static ref CLOSE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "ClosePreparedStatement".into(),
+        description: "Closes a reusable prepared statement resource on the server.\n
+                Request Message: ActionClosePreparedStatementRequest\n
+                Response Message: N/A"
+            .into(),
+    };
+}
+
+#[tonic::async_trait]
+pub trait FlightSqlService:

Review comment:
       👍 
   
   I wonder if we an add some comments here explaining what this does and how to use it?
   
   Eventually it would be really cool to add an doc example, but we can do that as a follow on task I think

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,458 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use protobuf::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,

Review comment:
       Something we have gone back and forth with in IOx and elsewhere is "should we wrap the protobuf generated structs with Rust types". I wonder if you have an opinion on this matter. 
   
   So in this case it would be something like
   
   ```rust
   struct ActionClosePreparedStatementRequest { 
   native fields
   }
   ```
   
   And then a 
   
   ```rust
   impl TryFrom<proto::gen::ActionClosePreparedStatementRequest> for ActionClosePreparedStatementRequest {
     // do the conversion here
   }
   ```
   
   The upside of a wrapping / translation layer is to insulate the user from the generated code / vagaries of protobuf handling (e.g. that everything is `Option`).
   
   The downside is that it is more code and the translation layer needs to be maintained. 
   
   Thoughts?

##########
File path: arrow-flight/build.rs
##########
@@ -49,6 +49,40 @@ fn main() -> Result<(), Box<dyn std::error::Error>> {
         file.write_all(buffer.as_bytes())?;
     }
 
+    // The current working directory can vary depending on how the project is being
+    // built or released so we build an absolute path to the proto file
+    let path = Path::new("../format/FlightSql.proto");

Review comment:
       cc @tustvold  do you have any thoughts on this approach to compiling protobuf? I know you have spent somewhat more time on the topic than I have
   
   I don't understand the distinction between `prost` and using `protoc_rust` directly

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,458 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use protobuf::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, TicketStatementQuery,
+};
+
+use lazy_static::lazy_static;
+lazy_static! {
+    static ref CREATE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "CreatePreparedStatement".into(),
+        description: "Creates a reusable prepared statement resource on the server.\n
+                Request Message: ActionCreatePreparedStatementRequest\n
+                Response Message: ActionCreatePreparedStatementResult"
+            .into(),
+    };
+    static ref CLOSE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "ClosePreparedStatement".into(),
+        description: "Closes a reusable prepared statement resource on the server.\n
+                Request Message: ActionClosePreparedStatementRequest\n
+                Response Message: N/A"
+            .into(),
+    };
+}
+
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    type FlightService: FlightService;
+
+    // get_flight_info
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    // do_get
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    // do_put
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    // do_action
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+}
+
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,
+    >;
+    type ListFlightsStream =
+        Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + Sync + 'static>>;
+    type DoGetStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+    type DoPutStream =
+        Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + Sync + 'static>>;
+    type DoActionStream = Pin<
+        Box<
+            dyn Stream<Item = Result<super::super::Result, Status>>
+                + Send
+                + Sync
+                + 'static,
+        >,
+    >;
+    type ListActionsStream =
+        Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + Sync + 'static>>;
+    type DoExchangeStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+
+    async fn handshake(
+        &self,
+        _request: Request<Streaming<HandshakeRequest>>,
+    ) -> Result<Response<Self::HandshakeStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn list_flights(
+        &self,
+        _request: Request<Criteria>,
+    ) -> Result<Response<Self::ListFlightsStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn get_flight_info(
+        &self,
+        request: Request<FlightDescriptor>,
+    ) -> Result<Response<FlightInfo>, Status> {
+        let request = request.into_inner();
+        let any: protobuf::well_known_types::Any =
+            protobuf::Message::parse_from_bytes(&request.cmd)

Review comment:
       @tustvold  any suggestions for `Any` here?

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,458 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use protobuf::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, TicketStatementQuery,
+};
+
+use lazy_static::lazy_static;
+lazy_static! {
+    static ref CREATE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "CreatePreparedStatement".into(),
+        description: "Creates a reusable prepared statement resource on the server.\n
+                Request Message: ActionCreatePreparedStatementRequest\n
+                Response Message: ActionCreatePreparedStatementResult"
+            .into(),
+    };
+    static ref CLOSE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "ClosePreparedStatement".into(),
+        description: "Closes a reusable prepared statement resource on the server.\n
+                Request Message: ActionClosePreparedStatementRequest\n
+                Response Message: N/A"
+            .into(),
+    };
+}
+
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    type FlightService: FlightService;
+
+    // get_flight_info
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    // do_get
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    // do_put
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    // do_action
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+}
+
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,
+    >;
+    type ListFlightsStream =
+        Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + Sync + 'static>>;
+    type DoGetStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+    type DoPutStream =
+        Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + Sync + 'static>>;
+    type DoActionStream = Pin<
+        Box<
+            dyn Stream<Item = Result<super::super::Result, Status>>
+                + Send
+                + Sync
+                + 'static,
+        >,
+    >;
+    type ListActionsStream =
+        Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + Sync + 'static>>;
+    type DoExchangeStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+
+    async fn handshake(
+        &self,
+        _request: Request<Streaming<HandshakeRequest>>,
+    ) -> Result<Response<Self::HandshakeStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn list_flights(
+        &self,
+        _request: Request<Criteria>,
+    ) -> Result<Response<Self::ListFlightsStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn get_flight_info(
+        &self,
+        request: Request<FlightDescriptor>,
+    ) -> Result<Response<FlightInfo>, Status> {
+        let request = request.into_inner();
+        let any: protobuf::well_known_types::Any =

Review comment:
       I am a big fan of handling of Any / protobuf translation at this level so that by the time an `impl FlightSqlServer` is invoked all of the protobuf handling has been done

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,458 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use protobuf::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, TicketStatementQuery,
+};
+
+use lazy_static::lazy_static;
+lazy_static! {
+    static ref CREATE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "CreatePreparedStatement".into(),
+        description: "Creates a reusable prepared statement resource on the server.\n
+                Request Message: ActionCreatePreparedStatementRequest\n
+                Response Message: ActionCreatePreparedStatementResult"
+            .into(),
+    };
+    static ref CLOSE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "ClosePreparedStatement".into(),
+        description: "Closes a reusable prepared statement resource on the server.\n
+                Request Message: ActionClosePreparedStatementRequest\n
+                Response Message: N/A"
+            .into(),
+    };
+}
+
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    type FlightService: FlightService;
+
+    // get_flight_info
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    // do_get
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    // do_put
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    // do_action
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+}
+
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T

Review comment:
       ```suggestion
   /// Implements the lower level interface to handle FlightSQL
   impl<T: 'static> FlightService for T
   ```

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,458 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use protobuf::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, TicketStatementQuery,
+};
+
+use lazy_static::lazy_static;
+lazy_static! {
+    static ref CREATE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "CreatePreparedStatement".into(),
+        description: "Creates a reusable prepared statement resource on the server.\n
+                Request Message: ActionCreatePreparedStatementRequest\n
+                Response Message: ActionCreatePreparedStatementResult"
+            .into(),
+    };
+    static ref CLOSE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "ClosePreparedStatement".into(),
+        description: "Closes a reusable prepared statement resource on the server.\n
+                Request Message: ActionClosePreparedStatementRequest\n
+                Response Message: N/A"
+            .into(),
+    };
+}
+
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    type FlightService: FlightService;
+
+    // get_flight_info
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    // do_get
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    // do_put
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    // do_action
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+}
+
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,
+    >;
+    type ListFlightsStream =
+        Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + Sync + 'static>>;
+    type DoGetStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+    type DoPutStream =
+        Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + Sync + 'static>>;
+    type DoActionStream = Pin<
+        Box<
+            dyn Stream<Item = Result<super::super::Result, Status>>
+                + Send
+                + Sync
+                + 'static,
+        >,
+    >;
+    type ListActionsStream =
+        Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + Sync + 'static>>;
+    type DoExchangeStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+
+    async fn handshake(
+        &self,
+        _request: Request<Streaming<HandshakeRequest>>,
+    ) -> Result<Response<Self::HandshakeStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn list_flights(
+        &self,
+        _request: Request<Criteria>,
+    ) -> Result<Response<Self::ListFlightsStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn get_flight_info(
+        &self,
+        request: Request<FlightDescriptor>,
+    ) -> Result<Response<FlightInfo>, Status> {
+        let request = request.into_inner();
+        let any: protobuf::well_known_types::Any =
+            protobuf::Message::parse_from_bytes(&request.cmd)
+                .map_err(|_| Status::invalid_argument("Unable to parse command"))?;
+
+        if any.is::<CommandStatementQuery>() {
+            return self
+                .get_flight_info_statement(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .get_flight_info_prepared_statement(
+                    any.unpack().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetCatalogs>() {
+            return self
+                .get_flight_info_catalogs(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetDbSchemas>() {
+            return self
+                .get_flight_info_schemas(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetTables>() {
+            return self
+                .get_flight_info_tables(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetTableTypes>() {
+            return self
+                .get_flight_info_table_types(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetSqlInfo>() {
+            return self
+                .get_flight_info_sql_info(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetPrimaryKeys>() {
+            return self
+                .get_flight_info_primary_keys(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetExportedKeys>() {
+            return self
+                .get_flight_info_exported_keys(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetImportedKeys>() {
+            return self
+                .get_flight_info_imported_keys(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetCrossReference>() {
+            return self
+                .get_flight_info_cross_reference(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "get_flight_info: The defined request is invalid: {:?}",
+            String::from_utf8(any.write_to_bytes().unwrap()).unwrap()
+        )))
+    }
+
+    async fn get_schema(
+        &self,
+        _request: Request<FlightDescriptor>,
+    ) -> Result<Response<SchemaResult>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn do_get(
+        &self,
+        _request: Request<Ticket>,
+    ) -> Result<Response<Self::DoGetStream>, Status> {
+        let request = _request.into_inner();
+        let any: protobuf::well_known_types::Any =
+            protobuf::Message::parse_from_bytes(&request.ticket)
+                .map_err(|_| Status::invalid_argument("Unable to parse ticket."))?;
+
+        if any.is::<TicketStatementQuery>() {
+            return self.do_get_statement(any.unpack().unwrap().unwrap()).await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .do_get_prepared_statement(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetCatalogs>() {
+            return self.do_get_catalogs(any.unpack().unwrap().unwrap()).await;
+        }
+        if any.is::<CommandGetDbSchemas>() {
+            return self.do_get_schemas(any.unpack().unwrap().unwrap()).await;
+        }
+        if any.is::<CommandGetTables>() {
+            return self.do_get_tables(any.unpack().unwrap().unwrap()).await;
+        }
+        if any.is::<CommandGetTableTypes>() {
+            return self
+                .do_get_table_types(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetSqlInfo>() {
+            return self.do_get_sql_info(any.unpack().unwrap().unwrap()).await;
+        }
+        if any.is::<CommandGetPrimaryKeys>() {
+            return self
+                .do_get_primary_keys(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetExportedKeys>() {
+            return self
+                .do_get_exported_keys(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetImportedKeys>() {
+            return self
+                .do_get_imported_keys(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetCrossReference>() {
+            return self
+                .do_get_cross_reference(any.unpack().unwrap().unwrap())
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "do_get: The defined request is invalid: {:?}",
+            String::from_utf8(request.ticket).unwrap()
+        )))
+    }
+
+    async fn do_put(
+        &self,
+        _request: Request<Streaming<FlightData>>,
+    ) -> Result<Response<Self::DoPutStream>, Status> {
+        let request = _request.into_inner().message().await?.unwrap();
+        let any: protobuf::well_known_types::Any =
+            protobuf::Message::parse_from_bytes(&request.flight_descriptor.unwrap().cmd)
+                .map_err(|_| Status::invalid_argument("Unable to parse command."))?;
+
+        if any.is::<CommandStatementUpdate>() {
+            return self
+                .do_put_statement_update(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .do_put_prepared_statement_query(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandPreparedStatementUpdate>() {
+            return self
+                .do_put_prepared_statement_update(any.unpack().unwrap().unwrap())
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "do_put: The defined request is invalid: {:?}",
+            String::from_utf8(any.write_to_bytes().unwrap()).unwrap()
+        )))
+    }
+
+    async fn list_actions(
+        &self,
+        _request: Request<Empty>,
+    ) -> Result<Response<Self::ListActionsStream>, Status> {
+        let _actions = [
+            CREATE_PREPARED_STATEMENT_ACTION_TYPE.clone(),

Review comment:
       FWIW since these things are `clone()`d anyways, I don't think there is much value in a lazy static for them rather than a generator function like 
   
   ```rust
   fn create_prepared_statement_action_type() -> ActionType {
   ...
   }
   
   I am just trying to keep the dependency list down in arrow. 

##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,458 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use protobuf::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, TicketStatementQuery,
+};
+
+use lazy_static::lazy_static;
+lazy_static! {
+    static ref CREATE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "CreatePreparedStatement".into(),
+        description: "Creates a reusable prepared statement resource on the server.\n
+                Request Message: ActionCreatePreparedStatementRequest\n
+                Response Message: ActionCreatePreparedStatementResult"
+            .into(),
+    };
+    static ref CLOSE_PREPARED_STATEMENT_ACTION_TYPE: ActionType = ActionType {
+        r#type: "ClosePreparedStatement".into(),
+        description: "Closes a reusable prepared statement resource on the server.\n
+                Request Message: ActionClosePreparedStatementRequest\n
+                Response Message: N/A"
+            .into(),
+    };
+}
+
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    type FlightService: FlightService;
+
+    // get_flight_info
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+    // do_get
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+    // do_put
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+    // do_action
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+}
+
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,
+    >;
+    type ListFlightsStream =
+        Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + Sync + 'static>>;
+    type DoGetStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+    type DoPutStream =
+        Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + Sync + 'static>>;
+    type DoActionStream = Pin<
+        Box<
+            dyn Stream<Item = Result<super::super::Result, Status>>
+                + Send
+                + Sync
+                + 'static,
+        >,
+    >;
+    type ListActionsStream =
+        Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + Sync + 'static>>;
+    type DoExchangeStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+
+    async fn handshake(
+        &self,
+        _request: Request<Streaming<HandshakeRequest>>,
+    ) -> Result<Response<Self::HandshakeStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn list_flights(
+        &self,
+        _request: Request<Criteria>,
+    ) -> Result<Response<Self::ListFlightsStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn get_flight_info(
+        &self,
+        request: Request<FlightDescriptor>,
+    ) -> Result<Response<FlightInfo>, Status> {
+        let request = request.into_inner();
+        let any: protobuf::well_known_types::Any =
+            protobuf::Message::parse_from_bytes(&request.cmd)
+                .map_err(|_| Status::invalid_argument("Unable to parse command"))?;
+
+        if any.is::<CommandStatementQuery>() {
+            return self
+                .get_flight_info_statement(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .get_flight_info_prepared_statement(
+                    any.unpack().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetCatalogs>() {
+            return self
+                .get_flight_info_catalogs(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetDbSchemas>() {
+            return self
+                .get_flight_info_schemas(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetTables>() {
+            return self
+                .get_flight_info_tables(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetTableTypes>() {
+            return self
+                .get_flight_info_table_types(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetSqlInfo>() {
+            return self
+                .get_flight_info_sql_info(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetPrimaryKeys>() {
+            return self
+                .get_flight_info_primary_keys(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetExportedKeys>() {
+            return self
+                .get_flight_info_exported_keys(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetImportedKeys>() {
+            return self
+                .get_flight_info_imported_keys(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+        if any.is::<CommandGetCrossReference>() {
+            return self
+                .get_flight_info_cross_reference(any.unpack().unwrap().unwrap(), request)
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "get_flight_info: The defined request is invalid: {:?}",
+            String::from_utf8(any.write_to_bytes().unwrap()).unwrap()
+        )))
+    }
+
+    async fn get_schema(
+        &self,
+        _request: Request<FlightDescriptor>,
+    ) -> Result<Response<SchemaResult>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn do_get(
+        &self,
+        _request: Request<Ticket>,
+    ) -> Result<Response<Self::DoGetStream>, Status> {
+        let request = _request.into_inner();
+        let any: protobuf::well_known_types::Any =
+            protobuf::Message::parse_from_bytes(&request.ticket)
+                .map_err(|_| Status::invalid_argument("Unable to parse ticket."))?;
+
+        if any.is::<TicketStatementQuery>() {
+            return self.do_get_statement(any.unpack().unwrap().unwrap()).await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .do_get_prepared_statement(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetCatalogs>() {
+            return self.do_get_catalogs(any.unpack().unwrap().unwrap()).await;
+        }
+        if any.is::<CommandGetDbSchemas>() {
+            return self.do_get_schemas(any.unpack().unwrap().unwrap()).await;
+        }
+        if any.is::<CommandGetTables>() {
+            return self.do_get_tables(any.unpack().unwrap().unwrap()).await;
+        }
+        if any.is::<CommandGetTableTypes>() {
+            return self
+                .do_get_table_types(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetSqlInfo>() {
+            return self.do_get_sql_info(any.unpack().unwrap().unwrap()).await;
+        }
+        if any.is::<CommandGetPrimaryKeys>() {
+            return self
+                .do_get_primary_keys(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetExportedKeys>() {
+            return self
+                .do_get_exported_keys(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetImportedKeys>() {
+            return self
+                .do_get_imported_keys(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetCrossReference>() {
+            return self
+                .do_get_cross_reference(any.unpack().unwrap().unwrap())
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "do_get: The defined request is invalid: {:?}",
+            String::from_utf8(request.ticket).unwrap()
+        )))
+    }
+
+    async fn do_put(
+        &self,
+        _request: Request<Streaming<FlightData>>,
+    ) -> Result<Response<Self::DoPutStream>, Status> {
+        let request = _request.into_inner().message().await?.unwrap();
+        let any: protobuf::well_known_types::Any =
+            protobuf::Message::parse_from_bytes(&request.flight_descriptor.unwrap().cmd)
+                .map_err(|_| Status::invalid_argument("Unable to parse command."))?;
+
+        if any.is::<CommandStatementUpdate>() {
+            return self
+                .do_put_statement_update(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .do_put_prepared_statement_query(any.unpack().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandPreparedStatementUpdate>() {
+            return self
+                .do_put_prepared_statement_update(any.unpack().unwrap().unwrap())
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "do_put: The defined request is invalid: {:?}",
+            String::from_utf8(any.write_to_bytes().unwrap()).unwrap()
+        )))
+    }
+
+    async fn list_actions(
+        &self,
+        _request: Request<Empty>,
+    ) -> Result<Response<Self::ListActionsStream>, Status> {
+        let _actions = [
+            CREATE_PREPARED_STATEMENT_ACTION_TYPE.clone(),
+            CLOSE_PREPARED_STATEMENT_ACTION_TYPE.clone(),
+        ];
+        // TODO: return

Review comment:
       not sure what the TODO means here




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] viirya commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r823039040



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,637 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.

Review comment:
       This comment looks incorrect?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb commented on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1063360683


   cc @nevi-me  @e-dard and @seddonm1  given your comments on https://github.com/apache/arrow-rs/issues/1323


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] tustvold commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r823164915



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,637 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;

Review comment:
       Isn't SendableRecordBatchStream a DataFusion thing?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r823572998



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,637 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;

Review comment:
       I checked the logic again, we can do that but I think not necessary. If we want it be easy to construct a FlightData, we may provide from utility methods like we did in [here](https://github.com/wangfenjin/arrow-rs/blob/flight-sql/arrow-flight/src/lib.rs#L246).
   
   Some system already return FlightData or Arrow Schema/RecordBatch, if we force to return Vec of String, user might need to do the converting and then we convert it again.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r823572998



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,637 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;

Review comment:
       I checked the logic again, we can do that but I think not necessary. If we want it be easy to construct a FlightData, we may provide some utility methods like we did in [here](https://github.com/wangfenjin/arrow-rs/blob/flight-sql/arrow-flight/src/lib.rs#L246).
   
   Some system already return FlightData or Arrow Schema/RecordBatch, if we force to return Vec of String, user might need to do the converting and then we convert it again.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r820915995



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,581 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the table types.
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of SqlInfo results.
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the primary and foreign keys.
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the exported keys.
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the imported keys.
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the cross reference.
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    // do_put
+
+    /// Execute an update SQL statement.
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Bind parameters to given prepared statement.
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Execute an update SQL prepared statement.
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    // do_action
+
+    /// Create a prepared statement from given SQL statement.
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Close a prepared statement.
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Register a new SqlInfo result, making it available when calling GetSqlInfo.
+    async fn register_sql_info(&self, id: i32, result: &SqlInfo);
+}
+
+/// Implements the lower level interface to handle FlightSQL
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,
+    >;
+    type ListFlightsStream =
+        Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + Sync + 'static>>;
+    type DoGetStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+    type DoPutStream =
+        Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + Sync + 'static>>;
+    type DoActionStream = Pin<
+        Box<
+            dyn Stream<Item = Result<super::super::Result, Status>>
+                + Send
+                + Sync
+                + 'static,
+        >,
+    >;
+    type ListActionsStream =
+        Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + Sync + 'static>>;
+    type DoExchangeStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
+
+    async fn handshake(
+        &self,
+        _request: Request<Streaming<HandshakeRequest>>,
+    ) -> Result<Response<Self::HandshakeStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn list_flights(
+        &self,
+        _request: Request<Criteria>,
+    ) -> Result<Response<Self::ListFlightsStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn get_flight_info(
+        &self,
+        request: Request<FlightDescriptor>,
+    ) -> Result<Response<FlightInfo>, Status> {
+        let request = request.into_inner();
+        let any: prost_types::Any = prost::Message::decode(&*request.cmd)
+            .map_err(|_| Status::invalid_argument("Unable to parse command"))?;
+
+        if any.is::<CommandStatementQuery>() {
+            return self
+                .get_flight_info_statement(
+                    any.unpack::<CommandStatementQuery>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .get_flight_info_prepared_statement(
+                    any.unpack::<CommandPreparedStatementQuery>()
+                        .unwrap()
+                        .unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetCatalogs>() {
+            return self
+                .get_flight_info_catalogs(
+                    any.unpack::<CommandGetCatalogs>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetDbSchemas>() {
+            return self
+                .get_flight_info_schemas(
+                    any.unpack::<CommandGetDbSchemas>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetTables>() {
+            return self
+                .get_flight_info_tables(
+                    any.unpack::<CommandGetTables>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetTableTypes>() {
+            return self
+                .get_flight_info_table_types(
+                    any.unpack::<CommandGetTableTypes>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetSqlInfo>() {
+            return self
+                .get_flight_info_sql_info(
+                    any.unpack::<CommandGetSqlInfo>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetPrimaryKeys>() {
+            return self
+                .get_flight_info_primary_keys(
+                    any.unpack::<CommandGetPrimaryKeys>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetExportedKeys>() {
+            return self
+                .get_flight_info_exported_keys(
+                    any.unpack::<CommandGetExportedKeys>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetImportedKeys>() {
+            return self
+                .get_flight_info_imported_keys(
+                    any.unpack::<CommandGetImportedKeys>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetCrossReference>() {
+            return self
+                .get_flight_info_cross_reference(
+                    any.unpack::<CommandGetCrossReference>().unwrap().unwrap(),
+                    request,
+                )
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "get_flight_info: The defined request is invalid: {:?}",
+            String::from_utf8(any.encode_to_vec()).unwrap()
+        )))
+    }
+
+    async fn get_schema(
+        &self,
+        _request: Request<FlightDescriptor>,
+    ) -> Result<Response<SchemaResult>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn do_get(
+        &self,
+        _request: Request<Ticket>,
+    ) -> Result<Response<Self::DoGetStream>, Status> {
+        let request = _request.into_inner();
+        let any: prost_types::Any = prost::Message::decode(&*request.ticket)
+            .map_err(|_| Status::invalid_argument("Unable to parse command"))?;
+
+        if any.is::<TicketStatementQuery>() {
+            return self
+                .do_get_statement(any.unpack::<TicketStatementQuery>().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .do_get_prepared_statement(
+                    any.unpack::<CommandPreparedStatementQuery>()
+                        .unwrap()
+                        .unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandGetCatalogs>() {
+            return self
+                .do_get_catalogs(any.unpack::<CommandGetCatalogs>().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetDbSchemas>() {
+            return self
+                .do_get_schemas(any.unpack::<CommandGetDbSchemas>().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetTables>() {
+            return self
+                .do_get_tables(any.unpack::<CommandGetTables>().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetTableTypes>() {
+            return self
+                .do_get_table_types(
+                    any.unpack::<CommandGetTableTypes>().unwrap().unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandGetSqlInfo>() {
+            return self
+                .do_get_sql_info(any.unpack::<CommandGetSqlInfo>().unwrap().unwrap())
+                .await;
+        }
+        if any.is::<CommandGetPrimaryKeys>() {
+            return self
+                .do_get_primary_keys(
+                    any.unpack::<CommandGetPrimaryKeys>().unwrap().unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandGetExportedKeys>() {
+            return self
+                .do_get_exported_keys(
+                    any.unpack::<CommandGetExportedKeys>().unwrap().unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandGetImportedKeys>() {
+            return self
+                .do_get_imported_keys(
+                    any.unpack::<CommandGetImportedKeys>().unwrap().unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandGetCrossReference>() {
+            return self
+                .do_get_cross_reference(
+                    any.unpack::<CommandGetCrossReference>().unwrap().unwrap(),
+                )
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "do_get: The defined request is invalid: {:?}",
+            String::from_utf8(request.ticket).unwrap()
+        )))
+    }
+
+    async fn do_put(
+        &self,
+        _request: Request<Streaming<FlightData>>,
+    ) -> Result<Response<Self::DoPutStream>, Status> {
+        let request = _request.into_inner().message().await?.unwrap();
+        let any: prost_types::Any =
+            prost::Message::decode(&*request.flight_descriptor.unwrap().cmd)
+                .map_err(|_| Status::invalid_argument("Unable to parse command"))?;
+        if any.is::<CommandStatementUpdate>() {
+            return self
+                .do_put_statement_update(
+                    any.unpack::<CommandStatementUpdate>().unwrap().unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .do_put_prepared_statement_query(
+                    any.unpack::<CommandPreparedStatementQuery>()
+                        .unwrap()
+                        .unwrap(),
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementUpdate>() {
+            return self
+                .do_put_prepared_statement_update(
+                    any.unpack::<CommandPreparedStatementUpdate>()
+                        .unwrap()
+                        .unwrap(),
+                )
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "do_put: The defined request is invalid: {:?}",
+            String::from_utf8(any.encode_to_vec()).unwrap()
+        )))
+    }
+
+    async fn list_actions(
+        &self,
+        _request: Request<Empty>,
+    ) -> Result<Response<Self::ListActionsStream>, Status> {
+        let create_prepared_statement_action_type = ActionType {
+            r#type: CREATE_PREPARED_STATEMENT.to_string(),
+            description: "Creates a reusable prepared statement resource on the server.\n
+                Request Message: ActionCreatePreparedStatementRequest\n
+                Response Message: ActionCreatePreparedStatementResult"
+                .into(),
+        };
+        let close_prepared_statement_action_type = ActionType {
+            r#type: CLOSE_PREPARED_STATEMENT.to_string(),
+            description: "Closes a reusable prepared statement resource on the server.\n
+                Request Message: ActionClosePreparedStatementRequest\n
+                Response Message: N/A"
+                .into(),
+        };
+        let _actions: Vec<Result<ActionType, Status>> = vec![
+            Ok(create_prepared_statement_action_type),
+            Ok(close_prepared_statement_action_type),
+        ];
+        // TODO: not sure why it's not work
+        // let output = futures::stream::iter(actions);
+        // Ok(Response::new(Box::pin(output) as Self::ListActionsStream))

Review comment:
       This line will report error:
   
       `non-primitive cast: `Pin<Box<futures::stream::Iter<std::vec::IntoIter<std::result::Result<ActionType, Status>>>>>` as `<T as FlightService>::ListActionsStream``
   
   But I adapt it from here: https://github.com/wangfenjin/arrow-datafusion/pull/1/files#diff-d942c264020a5d47b87deaca1b1064b53f3819a8f90764fad8fa3c2b9ccf6225R215




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822219968



##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {

Review comment:
       Same as ProstMessageExt




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822491549



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,585 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the table types.
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of SqlInfo results.
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the primary and foreign keys.
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the exported keys.
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the imported keys.
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the cross reference.
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    // do_put
+
+    /// Execute an update SQL statement.
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Bind parameters to given prepared statement.
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Execute an update SQL prepared statement.
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    // do_action
+
+    /// Create a prepared statement from given SQL statement.
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Close a prepared statement.
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Register a new SqlInfo result, making it available when calling GetSqlInfo.
+    async fn register_sql_info(&self, id: i32, result: &SqlInfo);
+}
+
+/// Implements the lower level interface to handle FlightSQL
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,

Review comment:
       Understood! Thanks! Never think about this before




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] alamb merged pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
alamb merged pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r820899833



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,458 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use protobuf::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,

Review comment:
       Ha! Haven't thought about it when I'm writing the code, just follow the style in here: https://github.com/apache/arrow-rs/blob/master/arrow-flight/src/lib.rs#L51
   
   But yes, when I read the cpp implementation, they always define the struct with native fields and do the convert, even for the enum values...
   
   If we don't convert the FlightData struct, overall it should be fine and don't require too much work. But there are several drawbacks I can think of:
   1. It's kind of duplication and make the code not easy to understand. When I first read the cpp implementation, I'm wondering why there are duplication class definition? Are they not using any pb generator but handmade all the staff. Then I find the convert code.
   2. More code, more error prone, and more effort to maintain.
   3. The difference between cpp and rust is, we commit the generated code to repo but they don't, and the generated rust code acctually is clean and easy to understand
   4. Performance issue. For the struct in FlightSql, as it's small so should be fine to convert, but I'm not sure if we also want to convert FlightData, it might involve too much mem copy.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] codecov-commenter commented on pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#issuecomment-1060951386


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1386](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (84eebaf) into [master](https://codecov.io/gh/apache/arrow-rs/commit/a2e629d4eeb8b9a80ff1fd00c8d1a69736e87b19?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a2e629d) will **decrease** coverage by `0.23%`.
   > The diff coverage is `2.71%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1386/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1386      +/-   ##
   ==========================================
   - Coverage   83.03%   82.80%   -0.24%     
   ==========================================
     Files         181      184       +3     
     Lines       52956    53542     +586     
   ==========================================
   + Hits        43972    44333     +361     
   - Misses       8984     9209     +225     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [arrow-flight/src/lib.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9saWIucnM=) | `18.54% <ø> (ø)` | |
   | [arrow-flight/src/sql/server.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9zcWwvc2VydmVyLnJz) | `0.00% <0.00%> (ø)` | |
   | [arrow-flight/src/sql/mod.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3ctZmxpZ2h0L3NyYy9zcWwvbW9kLnJz) | `30.00% <30.00%> (ø)` | |
   | [parquet/src/file/statistics.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvZmlsZS9zdGF0aXN0aWNzLnJz) | `91.73% <0.00%> (-2.07%)` | :arrow_down: |
   | [parquet/src/util/cursor.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvdXRpbC9jdXJzb3IucnM=) | `77.31% <0.00%> (-1.69%)` | :arrow_down: |
   | [parquet/src/schema/types.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvc2NoZW1hL3R5cGVzLnJz) | `85.64% <0.00%> (-1.51%)` | :arrow_down: |
   | [arrow/src/array/array\_dictionary.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L2FycmF5X2RpY3Rpb25hcnkucnM=) | `91.12% <0.00%> (-0.47%)` | :arrow_down: |
   | [arrow/src/array/data.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L2RhdGEucnM=) | `83.15% <0.00%> (-0.44%)` | :arrow_down: |
   | [arrow/src/ipc/writer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2lwYy93cml0ZXIucnM=) | `83.14% <0.00%> (-0.32%)` | :arrow_down: |
   | [parquet\_derive/src/parquet\_field.rs](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldF9kZXJpdmUvc3JjL3BhcnF1ZXRfZmllbGQucnM=) | `65.98% <0.00%> (-0.23%)` | :arrow_down: |
   | ... and [25 more](https://codecov.io/gh/apache/arrow-rs/pull/1386/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [a2e629d...84eebaf](https://codecov.io/gh/apache/arrow-rs/pull/1386?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] nevi-me commented on a change in pull request #1386: init impl flight sql

Posted by GitBox <gi...@apache.org>.
nevi-me commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r818356379



##########
File path: arrow-flight/Cargo.toml
##########
@@ -32,14 +32,17 @@ base64 = "0.13"
 tonic = "0.6"
 bytes = "1"
 prost = "0.9"
+protobuf = "2.27.1"

Review comment:
       We use prost as the protobuf implementation, adding protobuf results in 2 protobuf crates




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822238447



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,585 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + FlightService + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the table types.
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of SqlInfo results.
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the primary and foreign keys.
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the exported keys.
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the imported keys.
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the cross reference.
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    // do_put
+
+    /// Execute an update SQL statement.
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Bind parameters to given prepared statement.
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Execute an update SQL prepared statement.
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    // do_action
+
+    /// Create a prepared statement from given SQL statement.
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Close a prepared statement.
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Register a new SqlInfo result, making it available when calling GetSqlInfo.
+    async fn register_sql_info(&self, id: i32, result: &SqlInfo);
+}
+
+/// Implements the lower level interface to handle FlightSQL
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Sync + std::marker::Send,
+{
+    type HandshakeStream = Pin<
+        Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,

Review comment:
       They are not related? In here I try to implement FlightService which requires Sync `pub trait FlightService: Send + Sync + 'static `




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822596950



##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {
+    /// Check if `Any` contains a message of given type.
+    fn is<M: ProstMessageExt>(&self) -> bool;
+
+    /// Extract a message from this `Any`.
+    ///
+    /// # Returns
+    ///
+    /// * `Ok(None)` when message type mismatch
+    /// * `Err` when parse failed
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>>;
+
+    /// Pack any message into `prost_types::Any` value.
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
+}
+
+impl ProstAnyExt for prost_types::Any {
+    fn is<M: ProstMessageExt>(&self) -> bool {
+        M::type_url() == self.type_url
+    }
+
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>> {
+        if !self.is::<M>() {
+            return Ok(None);
+        }
+        let m = prost::Message::decode(&*self.value).map_err(|err| {
+            ArrowError::ParseError(format!("Unable to decode Any value: {}", err))
+        })?;
+        Ok(Some(m))
+    }
+
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any> {
+        Ok(message.as_any())
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+
+    #[test]
+    fn test_type_url() {
+        assert_eq!(
+            TicketStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.TicketStatementQuery"
+        );
+        assert_eq!(
+            CommandStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.CommandStatementQuery"
+        );
+    }
+
+    #[test]
+    fn test_prost_any_pack_unpack() -> ArrowResult<()> {
+        let query = CommandStatementQuery {
+            query: "select 1".to_string(),
+        };
+        let any = prost_types::Any::pack(&query)?;
+        assert!(any.is::<CommandStatementQuery>());
+        let unpack_query: CommandStatementQuery =
+            any.unpack::<CommandStatementQuery>()?.unwrap();

Review comment:
       Thank you so much!!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822499799



##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {
+    /// Check if `Any` contains a message of given type.
+    fn is<M: ProstMessageExt>(&self) -> bool;
+
+    /// Extract a message from this `Any`.
+    ///
+    /// # Returns
+    ///
+    /// * `Ok(None)` when message type mismatch
+    /// * `Err` when parse failed
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>>;
+
+    /// Pack any message into `prost_types::Any` value.
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
+}
+
+impl ProstAnyExt for prost_types::Any {
+    fn is<M: ProstMessageExt>(&self) -> bool {
+        M::type_url() == self.type_url
+    }
+
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>> {
+        if !self.is::<M>() {
+            return Ok(None);
+        }
+        let m = prost::Message::decode(&*self.value).map_err(|err| {
+            ArrowError::ParseError(format!("Unable to decode Any value: {}", err))
+        })?;
+        Ok(Some(m))
+    }
+
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any> {
+        Ok(message.as_any())
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+
+    #[test]
+    fn test_type_url() {
+        assert_eq!(
+            TicketStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.TicketStatementQuery"
+        );
+        assert_eq!(
+            CommandStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.CommandStatementQuery"
+        );
+    }
+
+    #[test]
+    fn test_prost_any_pack_unpack() -> ArrowResult<()> {
+        let query = CommandStatementQuery {
+            query: "select 1".to_string(),
+        };
+        let any = prost_types::Any::pack(&query)?;
+        assert!(any.is::<CommandStatementQuery>());
+        let unpack_query: CommandStatementQuery =
+            any.unpack::<CommandStatementQuery>()?.unwrap();

Review comment:
       Because when I use [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/7a55256b3348f098846feb53dae71afb33fede41/protobuf/src/well_known_types_util/any.rs#L86), it don't require us to annotate the type, I'm not sure why part is wrong in my implementation. Get ride of the type annotation can make the code more compact.
   
   Do you have example how should I fix this? I have no idea about the way you mentioned 😂




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] tustvold commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r822584923



##########
File path: arrow-flight/src/sql/mod.rs
##########
@@ -0,0 +1,183 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow::error::{ArrowError, Result as ArrowResult};
+use prost::Message;
+
+mod gen {
+    #![allow(clippy::all)]
+    include!("arrow.flight.protocol.sql.rs");
+}
+
+pub use gen::ActionClosePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementRequest;
+pub use gen::ActionCreatePreparedStatementResult;
+pub use gen::CommandGetCatalogs;
+pub use gen::CommandGetCrossReference;
+pub use gen::CommandGetDbSchemas;
+pub use gen::CommandGetExportedKeys;
+pub use gen::CommandGetImportedKeys;
+pub use gen::CommandGetPrimaryKeys;
+pub use gen::CommandGetSqlInfo;
+pub use gen::CommandGetTableTypes;
+pub use gen::CommandGetTables;
+pub use gen::CommandPreparedStatementQuery;
+pub use gen::CommandPreparedStatementUpdate;
+pub use gen::CommandStatementQuery;
+pub use gen::CommandStatementUpdate;
+pub use gen::DoPutUpdateResult;
+pub use gen::SqlInfo;
+pub use gen::SqlNullOrdering;
+pub use gen::SqlOuterJoinsSupportLevel;
+pub use gen::SqlSupportedCaseSensitivity;
+pub use gen::SqlSupportedElementActions;
+pub use gen::SqlSupportedGroupBy;
+pub use gen::SqlSupportedPositionedCommands;
+pub use gen::SqlSupportedResultSetConcurrency;
+pub use gen::SqlSupportedResultSetType;
+pub use gen::SqlSupportedSubqueries;
+pub use gen::SqlSupportedTransactions;
+pub use gen::SqlSupportedUnions;
+pub use gen::SqlSupportsConvert;
+pub use gen::SqlTransactionIsolationLevel;
+pub use gen::SupportedSqlGrammar;
+pub use gen::TicketStatementQuery;
+pub use gen::UpdateDeleteRules;
+
+pub mod server;
+
+/// ProstMessageExt are useful utility methods for prost::Message types
+pub trait ProstMessageExt {
+    /// Item is the return value of prost_type::Any::unpack()
+    type Item: prost::Message + Default;
+
+    /// type_url for this Message
+    fn type_url() -> &'static str;
+
+    /// Convert this Message to prost_types::Any
+    fn as_any(&self) -> prost_types::Any;
+}
+
+macro_rules! prost_message_ext {
+    ($($name:ty,)*) => {
+        $(
+            impl ProstMessageExt for $name {
+                type Item = $name;
+                fn type_url() -> &'static str {
+                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
+                }
+
+                fn as_any(&self) -> prost_types::Any {
+                    prost_types::Any {
+                        type_url: <$name>::type_url().to_string(),
+                        value: self.encode_to_vec(),
+                    }
+                }
+            }
+        )*
+    };
+}
+
+// Implement ProstMessageExt for all structs defined in FlightSql.proto
+prost_message_ext!(
+    ActionClosePreparedStatementRequest,
+    ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult,
+    CommandGetCatalogs,
+    CommandGetCrossReference,
+    CommandGetDbSchemas,
+    CommandGetExportedKeys,
+    CommandGetImportedKeys,
+    CommandGetPrimaryKeys,
+    CommandGetSqlInfo,
+    CommandGetTableTypes,
+    CommandGetTables,
+    CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate,
+    CommandStatementQuery,
+    CommandStatementUpdate,
+    DoPutUpdateResult,
+    TicketStatementQuery,
+);
+
+/// ProstAnyExt are useful utility methods for prost_types::Any
+/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
+pub trait ProstAnyExt {
+    /// Check if `Any` contains a message of given type.
+    fn is<M: ProstMessageExt>(&self) -> bool;
+
+    /// Extract a message from this `Any`.
+    ///
+    /// # Returns
+    ///
+    /// * `Ok(None)` when message type mismatch
+    /// * `Err` when parse failed
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>>;
+
+    /// Pack any message into `prost_types::Any` value.
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
+}
+
+impl ProstAnyExt for prost_types::Any {
+    fn is<M: ProstMessageExt>(&self) -> bool {
+        M::type_url() == self.type_url
+    }
+
+    fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M::Item>> {
+        if !self.is::<M>() {
+            return Ok(None);
+        }
+        let m = prost::Message::decode(&*self.value).map_err(|err| {
+            ArrowError::ParseError(format!("Unable to decode Any value: {}", err))
+        })?;
+        Ok(Some(m))
+    }
+
+    fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any> {
+        Ok(message.as_any())
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+
+    #[test]
+    fn test_type_url() {
+        assert_eq!(
+            TicketStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.TicketStatementQuery"
+        );
+        assert_eq!(
+            CommandStatementQuery::type_url(),
+            "type.googleapis.com/arrow.flight.protocol.sql.CommandStatementQuery"
+        );
+    }
+
+    #[test]
+    fn test_prost_any_pack_unpack() -> ArrowResult<()> {
+        let query = CommandStatementQuery {
+            query: "select 1".to_string(),
+        };
+        let any = prost_types::Any::pack(&query)?;
+        assert!(any.is::<CommandStatementQuery>());
+        let unpack_query: CommandStatementQuery =
+            any.unpack::<CommandStatementQuery>()?.unwrap();

Review comment:
       Ah I see, the problem is the indirection of `ProstMessageExt::Item` if you change `ProstMessageExt` to
   ```
   pub trait ProstMessageExt: prost::Message + Default {
       /// type_url for this Message
       fn type_url() -> &'static str;
   
       /// Convert this Message to prost_types::Any
       fn as_any(&self) -> prost_types::Any;
   }
   ```
   And `ProstAnyExt` to
   
   ```
   pub trait ProstAnyExt {
       /// Check if `Any` contains a message of given type.
       fn is<M: ProstMessageExt>(&self) -> bool;
   
       /// Extract a message from this `Any`.
       ///
       /// # Returns
       ///
       /// * `Ok(None)` when message type mismatch
       /// * `Err` when parse failed
       fn unpack<M: ProstMessageExt>(&self) -> ArrowResult<Option<M>>;
   
       /// Pack any message into `prost_types::Any` value.
       fn pack<M: ProstMessageExt>(message: &M) -> ArrowResult<prost_types::Any>;
   }
   ```
   
   The code works. The problem is as currently written there is no one-to-one relationship between the return type `M::Item` and the generic parameter `M`, and so type inference cannot work




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-rs] wangfenjin commented on a change in pull request #1386: Implement basic FlightSQL Server

Posted by GitBox <gi...@apache.org>.
wangfenjin commented on a change in pull request #1386:
URL: https://github.com/apache/arrow-rs/pull/1386#discussion_r823255317



##########
File path: arrow-flight/src/sql/server.rs
##########
@@ -0,0 +1,637 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 std::pin::Pin;
+
+use futures::Stream;
+use prost::Message;
+use tonic::{Request, Response, Status, Streaming};
+
+use super::{
+    super::{
+        flight_service_server::FlightService, Action, ActionType, Criteria, Empty,
+        FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
+        PutResult, SchemaResult, Ticket,
+    },
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas,
+    CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys,
+    CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery,
+    CommandStatementUpdate, ProstAnyExt, SqlInfo, TicketStatementQuery,
+};
+
+static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
+static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
+
+/// Implements FlightSqlService to handle the flight sql protocol
+#[tonic::async_trait]
+pub trait FlightSqlService:
+    std::marker::Sync + std::marker::Send + std::marker::Sized + 'static
+{
+    /// When impl FlightSqlService, you can always set FlightService to Self
+    type FlightService: FlightService;
+
+    /// Get a FlightInfo for executing a SQL query.
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn get_flight_info_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing catalogs.
+    async fn get_flight_info_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing schemas.
+    async fn get_flight_info_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for listing tables.
+    async fn get_flight_info_tables(
+        &self,
+        query: CommandGetTables,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about the table types.
+    async fn get_flight_info_table_types(
+        &self,
+        query: CommandGetTableTypes,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo for retrieving other information (See SqlInfo).
+    async fn get_flight_info_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about primary and foreign keys.
+    async fn get_flight_info_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about exported keys.
+    async fn get_flight_info_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about imported keys.
+    async fn get_flight_info_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    /// Get a FlightInfo to extract information about cross reference.
+    async fn get_flight_info_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status>;
+
+    // do_get
+
+    /// Get a FlightInfo for executing an already created prepared statement.
+    async fn do_get_statement(
+        &self,
+        ticket: TicketStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the prepared statement query results.
+    async fn do_get_prepared_statement(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of catalogs.
+    async fn do_get_catalogs(
+        &self,
+        query: CommandGetCatalogs,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of schemas.
+    async fn do_get_schemas(
+        &self,
+        query: CommandGetDbSchemas,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of tables.
+    async fn do_get_tables(
+        &self,
+        query: CommandGetTables,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the table types.
+    async fn do_get_table_types(
+        &self,
+        query: CommandGetTableTypes,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the list of SqlInfo results.
+    async fn do_get_sql_info(
+        &self,
+        query: CommandGetSqlInfo,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the primary and foreign keys.
+    async fn do_get_primary_keys(
+        &self,
+        query: CommandGetPrimaryKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the exported keys.
+    async fn do_get_exported_keys(
+        &self,
+        query: CommandGetExportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the imported keys.
+    async fn do_get_imported_keys(
+        &self,
+        query: CommandGetImportedKeys,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    /// Get a FlightDataStream containing the data related to the cross reference.
+    async fn do_get_cross_reference(
+        &self,
+        query: CommandGetCrossReference,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status>;
+
+    // do_put
+
+    /// Execute an update SQL statement.
+    async fn do_put_statement_update(
+        &self,
+        ticket: CommandStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Bind parameters to given prepared statement.
+    async fn do_put_prepared_statement_query(
+        &self,
+        query: CommandPreparedStatementQuery,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    /// Execute an update SQL prepared statement.
+    async fn do_put_prepared_statement_update(
+        &self,
+        query: CommandPreparedStatementUpdate,
+    ) -> Result<Response<<Self as FlightService>::DoPutStream>, Status>;
+
+    // do_action
+
+    /// Create a prepared statement from given SQL statement.
+    async fn do_action_create_prepared_statement(
+        &self,
+        query: ActionCreatePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Close a prepared statement.
+    async fn do_action_close_prepared_statement(
+        &self,
+        query: ActionClosePreparedStatementRequest,
+    ) -> Result<Response<<Self as FlightService>::DoActionStream>, Status>;
+
+    /// Register a new SqlInfo result, making it available when calling GetSqlInfo.
+    async fn register_sql_info(&self, id: i32, result: &SqlInfo);
+}
+
+/// Implements the lower level interface to handle FlightSQL
+#[tonic::async_trait]
+impl<T: 'static> FlightService for T
+where
+    T: FlightSqlService + std::marker::Send,
+{
+    type HandshakeStream =
+        Pin<Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + 'static>>;
+    type ListFlightsStream =
+        Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + 'static>>;
+    type DoGetStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + 'static>>;
+    type DoPutStream =
+        Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + 'static>>;
+    type DoActionStream = Pin<
+        Box<dyn Stream<Item = Result<super::super::Result, Status>> + Send + 'static>,
+    >;
+    type ListActionsStream =
+        Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + 'static>>;
+    type DoExchangeStream =
+        Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + 'static>>;
+
+    async fn handshake(
+        &self,
+        _request: Request<Streaming<HandshakeRequest>>,
+    ) -> Result<Response<Self::HandshakeStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn list_flights(
+        &self,
+        _request: Request<Criteria>,
+    ) -> Result<Response<Self::ListFlightsStream>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn get_flight_info(
+        &self,
+        request: Request<FlightDescriptor>,
+    ) -> Result<Response<FlightInfo>, Status> {
+        let request = request.into_inner();
+        let any: prost_types::Any =
+            prost::Message::decode(&*request.cmd).map_err(decode_error_to_status)?;
+
+        if any.is::<CommandStatementQuery>() {
+            return self
+                .get_flight_info_statement(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .get_flight_info_prepared_statement(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetCatalogs>() {
+            return self
+                .get_flight_info_catalogs(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetDbSchemas>() {
+            return self
+                .get_flight_info_schemas(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetTables>() {
+            return self
+                .get_flight_info_tables(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetTableTypes>() {
+            return self
+                .get_flight_info_table_types(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetSqlInfo>() {
+            return self
+                .get_flight_info_sql_info(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetPrimaryKeys>() {
+            return self
+                .get_flight_info_primary_keys(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetExportedKeys>() {
+            return self
+                .get_flight_info_exported_keys(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetImportedKeys>() {
+            return self
+                .get_flight_info_imported_keys(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+        if any.is::<CommandGetCrossReference>() {
+            return self
+                .get_flight_info_cross_reference(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                    request,
+                )
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "get_flight_info: The defined request is invalid: {:?}",
+            String::from_utf8(any.encode_to_vec()).unwrap()
+        )))
+    }
+
+    async fn get_schema(
+        &self,
+        _request: Request<FlightDescriptor>,
+    ) -> Result<Response<SchemaResult>, Status> {
+        Err(Status::unimplemented("Not yet implemented"))
+    }
+
+    async fn do_get(
+        &self,
+        _request: Request<Ticket>,
+    ) -> Result<Response<Self::DoGetStream>, Status> {
+        let request = _request.into_inner();
+        let any: prost_types::Any =
+            prost::Message::decode(&*request.ticket).map_err(decode_error_to_status)?;
+
+        if any.is::<TicketStatementQuery>() {
+            return self
+                .do_get_statement(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .do_get_prepared_statement(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetCatalogs>() {
+            return self
+                .do_get_catalogs(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetDbSchemas>() {
+            return self
+                .do_get_schemas(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetTables>() {
+            return self
+                .do_get_tables(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetTableTypes>() {
+            return self
+                .do_get_table_types(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetSqlInfo>() {
+            return self
+                .do_get_sql_info(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetPrimaryKeys>() {
+            return self
+                .do_get_primary_keys(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetExportedKeys>() {
+            return self
+                .do_get_exported_keys(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetImportedKeys>() {
+            return self
+                .do_get_imported_keys(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandGetCrossReference>() {
+            return self
+                .do_get_cross_reference(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "do_get: The defined request is invalid: {:?}",
+            String::from_utf8(request.ticket).unwrap()
+        )))
+    }
+
+    async fn do_put(
+        &self,
+        _request: Request<Streaming<FlightData>>,
+    ) -> Result<Response<Self::DoPutStream>, Status> {
+        let request = _request.into_inner().message().await?.unwrap();
+        let any: prost_types::Any =
+            prost::Message::decode(&*request.flight_descriptor.unwrap().cmd)
+                .map_err(decode_error_to_status)?;
+        if any.is::<CommandStatementUpdate>() {
+            return self
+                .do_put_statement_update(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementQuery>() {
+            return self
+                .do_put_prepared_statement_query(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+        if any.is::<CommandPreparedStatementUpdate>() {
+            return self
+                .do_put_prepared_statement_update(
+                    any.unpack()
+                        .map_err(arrow_error_to_status)?
+                        .expect("unreachable"),
+                )
+                .await;
+        }
+
+        Err(Status::unimplemented(format!(
+            "do_put: The defined request is invalid: {:?}",
+            String::from_utf8(any.encode_to_vec()).unwrap()
+        )))
+    }
+
+    async fn list_actions(
+        &self,
+        _request: Request<Empty>,
+    ) -> Result<Response<Self::ListActionsStream>, Status> {
+        let create_prepared_statement_action_type = ActionType {
+            r#type: CREATE_PREPARED_STATEMENT.to_string(),
+            description: "Creates a reusable prepared statement resource on the server.\n
+                Request Message: ActionCreatePreparedStatementRequest\n
+                Response Message: ActionCreatePreparedStatementResult"
+                .into(),
+        };
+        let close_prepared_statement_action_type = ActionType {
+            r#type: CLOSE_PREPARED_STATEMENT.to_string(),
+            description: "Closes a reusable prepared statement resource on the server.\n
+                Request Message: ActionClosePreparedStatementRequest\n
+                Response Message: N/A"
+                .into(),
+        };
+        let actions: Vec<Result<ActionType, Status>> = vec![
+            Ok(create_prepared_statement_action_type),
+            Ok(close_prepared_statement_action_type),
+        ];
+        let output = futures::stream::iter(actions);
+        Ok(Response::new(Box::pin(output) as Self::ListActionsStream))
+    }
+
+    async fn do_action(
+        &self,
+        _request: Request<Action>,
+    ) -> Result<Response<Self::DoActionStream>, Status> {
+        let request = _request.into_inner();
+
+        if request.r#type == CREATE_PREPARED_STATEMENT {
+            let any: prost_types::Any =
+                prost::Message::decode(&*request.body).map_err(decode_error_to_status)?;
+
+            let cmd: ActionCreatePreparedStatementRequest = any
+                .unpack()
+                .map_err(arrow_error_to_status)?
+                .ok_or_else(|| {
+                    Status::invalid_argument(
+                        "Unable to unpack ActionCreatePreparedStatementRequest.",
+                    )
+                })?;
+            return self.do_action_create_prepared_statement(cmd).await;
+        }
+        if request.r#type == CLOSE_PREPARED_STATEMENT {
+            let any: prost_types::Any =
+                prost::Message::decode(&*request.body).map_err(decode_error_to_status)?;
+
+            let cmd: ActionClosePreparedStatementRequest = any
+                .unpack()
+                .map_err(arrow_error_to_status)?
+                .ok_or_else(|| {
+                    Status::invalid_argument(
+                        "Unable to unpack CloseCreatePreparedStatementRequest.",

Review comment:
       thanks!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org