You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "roeap (via GitHub)" <gi...@apache.org> on 2023/06/03 12:37:16 UTC

[GitHub] [arrow-rs] roeap opened a new pull request, #4359: feat(flight): add xdbc type info helpers

roeap opened a new pull request, #4359:
URL: https://github.com/apache/arrow-rs/pull/4359

   Keeping it as a draft to get some feedback - currently some docs and tests are still missing.
   
   # Which issue does this PR close?
   
   Closes #4257.
   
   # Rationale for this change
    
   see #4257.
   
   # What changes are included in this PR?
   
   Again, this is inspired by the iOx implementation, with some updates, that hopefully (๐Ÿคž) improve the ergonomics. We introduce three new structs:
   
   - `XdbcTypeInfo` - a typed struct that holds the xdbc info corresponding to expected schema.
   - `XdbcTypeInfoListBuilder` - a builder for collecting type infos and building a conformant `RecordBatch`.
   - `XdbcTypeInfoList` - a helper type wrapping a `RecordBatch` used for handling `CommandGetXdbcTypeInfo` requests.
   
   The assumption is, that this kind of information should not change at runtime, and thus an instance of `XdbcTypeInfoList` would usually be constructed statically.
   
   @alamb, @avantgardnerio - If we want to go forward with this API, `SqlInfoList` should be made to look similar. Based on your feedback I could include that in this PR or a follow-up. 
   
   # Are there any user-facing changes?
   
   Users can leverage new types to handle `CommandGetXdbcTypeInfo` requests.
   


-- 
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 diff in pull request #4359: feat(flight): add xdbc type info helpers

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359#discussion_r1218581209


##########
arrow-flight/src/sql/metadata/xdbc_info.rs:
##########
@@ -76,15 +75,13 @@ impl From<CommandGetXdbcTypeInfo> for Option<i32> {
 /// [`CommandGetXdbcTypeInfo`] are metadata requests used by a Flight SQL
 /// server to communicate supported capabilities to Flight SQL clients.
 ///
-/// Servers constuct a [`XdbcTypeInfoList`] via the [`XdbcTypeInfoListBuilder`],
-/// and build responses using the [`encode`] method.
-///
-/// [`encode`]: XdbcTypeInfoList::encode
-pub struct XdbcTypeInfoList {
+/// Servers constuct - usually static - [`XdbcTypeInfoData`] via the [XdbcTypeInfoDataBuilder`],

Review Comment:
   ๐Ÿ‘ 



##########
arrow-flight/src/sql/metadata/xdbc_info.rs:
##########
@@ -0,0 +1,441 @@
+// 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.
+
+//! Helpers for [`CommandGetXdbcTypeInfo`] metadata requests.
+//!
+//! - [`XdbcTypeInfo`] - a typed struct that holds the xdbc info corresponding to expected schema.
+//! - [`XdbcTypeInfoListBuilder`] - a builder for collecting type infos
+//!   and building a conformant `RecordBatch`.
+//! - [`XdbcTypeInfoList`] - a helper type wrapping a `RecordBatch`
+//!   used for handling [`CommandGetXdbcTypeInfo`] requests.
+//!
+use std::sync::Arc;
+
+use arrow_array::builder::{BooleanBuilder, Int32Builder, ListBuilder, StringBuilder};
+use arrow_array::cast::downcast_array;
+use arrow_array::{ArrayRef, Int32Array, ListArray, RecordBatch};
+use arrow_ord::comparison::eq_scalar;
+use arrow_schema::{DataType, Field, Schema, SchemaRef};
+use arrow_select::filter::filter_record_batch;
+use arrow_select::take::take;
+use once_cell::sync::Lazy;
+
+use super::lexsort_to_indices;
+use crate::error::*;
+use crate::sql::{
+    CommandGetXdbcTypeInfo, Nullable, Searchable, XdbcDataType, XdbcDatetimeSubcode,
+};
+
+/// Data structure representing type information for xdbc types.
+#[derive(Debug, Clone, Default)]
+pub struct XdbcTypeInfo {
+    pub type_name: String,
+    pub data_type: XdbcDataType,
+    pub column_size: Option<i32>,
+    pub literal_prefix: Option<String>,
+    pub literal_suffix: Option<String>,
+    pub create_params: Option<Vec<String>>,
+    pub nullable: Nullable,
+    pub case_sensitive: bool,
+    pub searchable: Searchable,
+    pub unsigned_attribute: Option<bool>,
+    pub fixed_prec_scale: bool,
+    pub auto_increment: Option<bool>,
+    pub local_type_name: Option<String>,
+    pub minimum_scale: Option<i32>,
+    pub maximum_scale: Option<i32>,
+    pub sql_data_type: XdbcDataType,
+    pub datetime_subcode: Option<XdbcDatetimeSubcode>,
+    pub num_prec_radix: Option<i32>,
+    pub interval_precision: Option<i32>,
+}
+
+impl From<CommandGetXdbcTypeInfo> for Option<i32> {

Review Comment:
   I wonder if we still need this `From` impl ๐Ÿค” 



##########
arrow-flight/src/sql/metadata/xdbc_info.rs:
##########
@@ -0,0 +1,441 @@
+// 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.
+
+//! Helpers for [`CommandGetXdbcTypeInfo`] metadata requests.
+//!
+//! - [`XdbcTypeInfo`] - a typed struct that holds the xdbc info corresponding to expected schema.
+//! - [`XdbcTypeInfoListBuilder`] - a builder for collecting type infos
+//!   and building a conformant `RecordBatch`.
+//! - [`XdbcTypeInfoList`] - a helper type wrapping a `RecordBatch`
+//!   used for handling [`CommandGetXdbcTypeInfo`] requests.
+//!
+use std::sync::Arc;
+
+use arrow_array::builder::{BooleanBuilder, Int32Builder, ListBuilder, StringBuilder};
+use arrow_array::cast::downcast_array;
+use arrow_array::{ArrayRef, Int32Array, ListArray, RecordBatch};
+use arrow_ord::comparison::eq_scalar;
+use arrow_schema::{DataType, Field, Schema, SchemaRef};
+use arrow_select::filter::filter_record_batch;
+use arrow_select::take::take;
+use once_cell::sync::Lazy;
+
+use super::lexsort_to_indices;
+use crate::error::*;
+use crate::sql::{
+    CommandGetXdbcTypeInfo, Nullable, Searchable, XdbcDataType, XdbcDatetimeSubcode,
+};
+
+/// Data structure representing type information for xdbc types.
+#[derive(Debug, Clone, Default)]
+pub struct XdbcTypeInfo {
+    pub type_name: String,
+    pub data_type: XdbcDataType,
+    pub column_size: Option<i32>,
+    pub literal_prefix: Option<String>,
+    pub literal_suffix: Option<String>,
+    pub create_params: Option<Vec<String>>,
+    pub nullable: Nullable,
+    pub case_sensitive: bool,
+    pub searchable: Searchable,
+    pub unsigned_attribute: Option<bool>,
+    pub fixed_prec_scale: bool,
+    pub auto_increment: Option<bool>,
+    pub local_type_name: Option<String>,
+    pub minimum_scale: Option<i32>,
+    pub maximum_scale: Option<i32>,
+    pub sql_data_type: XdbcDataType,
+    pub datetime_subcode: Option<XdbcDatetimeSubcode>,
+    pub num_prec_radix: Option<i32>,
+    pub interval_precision: Option<i32>,
+}
+
+impl From<CommandGetXdbcTypeInfo> for Option<i32> {
+    fn from(value: CommandGetXdbcTypeInfo) -> Self {
+        value.data_type
+    }
+}
+
+/// Helper to create [`CommandGetXdbcTypeInfo`] responses.
+///
+/// [`CommandGetXdbcTypeInfo`] are metadata requests used by a Flight SQL
+/// server to communicate supported capabilities to Flight SQL clients.
+///
+/// Servers constuct - usually static - [`XdbcTypeInfoData`] via the [XdbcTypeInfoDataBuilder`],
+/// and build responses by passing the [`GetXdbcTypeInfoBuilder`].
+pub struct XdbcTypeInfoData {
+    batch: RecordBatch,
+}
+
+impl XdbcTypeInfoData {
+    /// Return the raw (not encoded) RecordBatch that will be returned
+    /// from [`CommandGetXdbcTypeInfo`]
+    pub fn record_batch(&self, data_type: impl Into<Option<i32>>) -> Result<RecordBatch> {
+        if let Some(dt) = data_type.into() {
+            let arr: Int32Array = downcast_array(self.batch.column(1).as_ref());
+            let filter = eq_scalar(&arr, dt)?;
+            Ok(filter_record_batch(&self.batch, &filter)?)
+        } else {
+            Ok(self.batch.clone())
+        }
+    }
+
+    /// Return the schema of the RecordBatch that will be returned
+    /// from [`CommandGetXdbcTypeInfo`]
+    pub fn schema(&self) -> SchemaRef {
+        self.batch.schema()
+    }
+}
+
+pub struct XdbcTypeInfoDataBuilder {
+    infos: Vec<XdbcTypeInfo>,
+}
+
+impl Default for XdbcTypeInfoDataBuilder {
+    fn default() -> Self {
+        Self::new()
+    }
+}
+
+/// A builder for [`XdbcTypeInfoList`] which is used to create [`CommandGetXdbcTypeInfo`].responses.

Review Comment:
   ```suggestion
   /// A builder for [`XdbcTypeInfoList`] which is used to create [`CommandGetXdbcTypeInfo`] responses.
   ```



##########
arrow-flight/examples/flight_sql_server.rs:
##########
@@ -367,12 +395,20 @@ impl FlightSqlService for FlightSqlServiceImpl {
 
     async fn get_flight_info_xdbc_type_info(
         &self,
-        _query: CommandGetXdbcTypeInfo,
-        _request: Request<FlightDescriptor>,
+        query: CommandGetXdbcTypeInfo,
+        request: Request<FlightDescriptor>,
     ) -> Result<Response<FlightInfo>, Status> {
-        Err(Status::unimplemented(
-            "get_flight_info_xdbc_type_info not implemented",
-        ))
+        let flight_descriptor = request.into_inner();
+        let ticket = Ticket::new(query.encode_to_vec());
+        let endpoint = FlightEndpoint::new().with_ticket(ticket);
+
+        let flight_info = FlightInfo::new()
+            .try_with_schema(INSTANCE_XDBC_INFO.schema().as_ref())

Review Comment:
   This probably needs to be updated to just use `query.into_builder()`. 
   



-- 
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] avantgardnerio commented on pull request #4359: feat(flight): add xdbc type info helpers

Posted by "avantgardnerio (via GitHub)" <gi...@apache.org>.
avantgardnerio commented on PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359#issuecomment-1578353506

   > Looks good to me -- do you plan any more changes @roeap or are we ready to merge?
   > 
   > @avantgardnerio let us know if you want additional time to review this as well
   
   Thanks @alamb ... my bandwidth is pretty constrained right now. I trust your opinion on this.


-- 
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 #4359: feat(flight): add xdbc type info helpers

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb merged PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359


-- 
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] roeap commented on a diff in pull request #4359: feat(flight): add xdbc type info helpers

Posted by "roeap (via GitHub)" <gi...@apache.org>.
roeap commented on code in PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359#discussion_r1218479700


##########
arrow-flight/examples/flight_sql_server.rs:
##########
@@ -544,12 +580,11 @@ impl FlightSqlService for FlightSqlServiceImpl {
 
     async fn do_get_xdbc_type_info(
         &self,
-        _query: CommandGetXdbcTypeInfo,
+        query: CommandGetXdbcTypeInfo,
         _request: Request<Ticket>,
     ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
-        Err(Status::unimplemented(
-            "do_get_xdbc_type_info not implemented",
-        ))
+        let stream = INSTANCE_XDBC_INFO.encode(query).map_err(Status::from);

Review Comment:
   > I may have missed it, but this doesn't seem to ever construct a XdbcTypeInfoListBuilder
   
   The builder is used to statically create the instance of `XdbcTypeInfoList`, which then directly contains the sorted etc record batch. Since the data never changes at runtime, I thought encoding it should also be done only once? Then again, I do not really have a good feeling for how expensive then encoding is and if it is worthwhile to save that effort at query time. 
   
   > What do you think about making this mirror the other builders, so the code would look like
   
   That was my initial instinct too, somehow I felt though that xdbc and sql info are different a bit nature as discussed above. If we want to construct the record batch at runtime, I think we should do that though...
   
   > Maybe we can write some sort of generic function (either on FlightDataEncoderBuilder or as a free function) that takes a impl Into<SchemaAndBatch>
   
   Hmm, I have no too strong feelings about this, in the end, the `stream` function only "saves" a very few lines of code, so maybe either do it everywhere (includig catalogs) or never? In that case I would go for never, and just return the batch, since I expect the `FlightDataEncoderBuilder ` will be used very frequently throughout the codebase, so users will be quite familiar with it.



-- 
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 diff in pull request #4359: feat(flight): add xdbc type info helpers

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359#discussion_r1218502420


##########
arrow-flight/examples/flight_sql_server.rs:
##########
@@ -544,12 +580,11 @@ impl FlightSqlService for FlightSqlServiceImpl {
 
     async fn do_get_xdbc_type_info(
         &self,
-        _query: CommandGetXdbcTypeInfo,
+        query: CommandGetXdbcTypeInfo,
         _request: Request<Ticket>,
     ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
-        Err(Status::unimplemented(
-            "do_get_xdbc_type_info not implemented",
-        ))
+        let stream = INSTANCE_XDBC_INFO.encode(query).map_err(Status::from);

Review Comment:
   > The builder is used to statically create the instance of XdbcTypeInfoList, which then directly contains the sorted etc record batch. Since the data never changes at runtime, I thought encoding it should also be done only once? Then again, I do not really have a good feeling for how expensive then encoding is and if it is worthwhile to save that effort at query time.
   
   I thought about this a bunch too. Since this is a metadata query that gets run over and over again I do think it is worth optimizing the performance for that common case. However, given the Builders for the other metadata endpoints are builders for handling the parameters (e.g. filters) having an XDBC builder do something different may be confusing.
   
   Maybe we can use the pattern in this PR but change the names a bit? For example
   
   ```rust
           // create a builder with pre-defined Xdbc data:
           let mut builder = query.into_builder(&INSTANCE_XBDC_DATA);
           let schema = builder.schema();
           let batch = builder.build();
           let stream = FlightDataEncoderBuilder::new()
               .with_schema(schema)
               .build(futures::stream::once(async { batch }))
               .map_err(Status::from);
           Ok(Response::new(Box::pin(stream)))
   ```
   
   And then pre-define `INSTANCE_XDBC_Data` with another builder? Something like 
   
   ```rust
   static INSTANCE_XDBC_INFO: Lazy<XdbcTypeInfoData> = Lazy::new(|| {
       let mut builder = XdbcTypeInfoDataBuilder::new();
       builder.append(XdbcTypeInfo {
           type_name: "INTEGER".into(),
           data_type: XdbcDataType::XdbcInteger,
           column_size: Some(32),
           literal_prefix: None,
           literal_suffix: None,
           create_params: None,
           nullable: Nullable::NullabilityNullable,
           case_sensitive: false,
           searchable: Searchable::Full,
           unsigned_attribute: Some(false),
           fixed_prec_scale: false,
           auto_increment: Some(false),
           local_type_name: Some("INTEGER".into()),
           minimum_scale: None,
           maximum_scale: None,
           sql_data_type: XdbcDataType::XdbcInteger,
           datetime_subcode: None,
           num_prec_radix: Some(2),
           interval_precision: None,
       });
       builder.build().unwrap()
   });
   ```
   
   ๐Ÿค” 
   
   



-- 
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] roeap commented on pull request #4359: feat(flight): add xdbc type info helpers

Posted by "roeap (via GitHub)" <gi...@apache.org>.
roeap commented on PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359#issuecomment-1578362432

   @alamb - we're good to go. I'll follow up with the SQL infos soon, and do a thorough pass once we have consistent APIs ..


-- 
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 #4359: feat(flight): add xdbc type info helpers

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359#issuecomment-1577463741

   cc @appletreeisyellow  if you have some thoughts on this API (that @roeap  is porting upstream to arrow-rs โค๏ธ ).


-- 
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] roeap commented on a diff in pull request #4359: feat(flight): add xdbc type info helpers

Posted by "roeap (via GitHub)" <gi...@apache.org>.
roeap commented on code in PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359#discussion_r1218479700


##########
arrow-flight/examples/flight_sql_server.rs:
##########
@@ -544,12 +580,11 @@ impl FlightSqlService for FlightSqlServiceImpl {
 
     async fn do_get_xdbc_type_info(
         &self,
-        _query: CommandGetXdbcTypeInfo,
+        query: CommandGetXdbcTypeInfo,
         _request: Request<Ticket>,
     ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
-        Err(Status::unimplemented(
-            "do_get_xdbc_type_info not implemented",
-        ))
+        let stream = INSTANCE_XDBC_INFO.encode(query).map_err(Status::from);

Review Comment:
   > I may have missed it, but this doesn't seem to ever construct a XdbcTypeInfoListBuilder
   
   The builder is used to statically create the instance of `XdbcTypeInfoList`, which then directly contains the sorted etc record batch. Since the data never changes at runtime, I thought encoding it should also be done only once? Then again, I do not really have a good feeling for how expensive then encoding is and if it is worthwhile to save that effort at query time. 



-- 
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] roeap commented on a diff in pull request #4359: feat(flight): add xdbc type info helpers

Posted by "roeap (via GitHub)" <gi...@apache.org>.
roeap commented on code in PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359#discussion_r1218591968


##########
arrow-flight/src/sql/metadata/xdbc_info.rs:
##########
@@ -0,0 +1,441 @@
+// 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.
+
+//! Helpers for [`CommandGetXdbcTypeInfo`] metadata requests.
+//!
+//! - [`XdbcTypeInfo`] - a typed struct that holds the xdbc info corresponding to expected schema.
+//! - [`XdbcTypeInfoListBuilder`] - a builder for collecting type infos
+//!   and building a conformant `RecordBatch`.
+//! - [`XdbcTypeInfoList`] - a helper type wrapping a `RecordBatch`
+//!   used for handling [`CommandGetXdbcTypeInfo`] requests.
+//!
+use std::sync::Arc;
+
+use arrow_array::builder::{BooleanBuilder, Int32Builder, ListBuilder, StringBuilder};
+use arrow_array::cast::downcast_array;
+use arrow_array::{ArrayRef, Int32Array, ListArray, RecordBatch};
+use arrow_ord::comparison::eq_scalar;
+use arrow_schema::{DataType, Field, Schema, SchemaRef};
+use arrow_select::filter::filter_record_batch;
+use arrow_select::take::take;
+use once_cell::sync::Lazy;
+
+use super::lexsort_to_indices;
+use crate::error::*;
+use crate::sql::{
+    CommandGetXdbcTypeInfo, Nullable, Searchable, XdbcDataType, XdbcDatetimeSubcode,
+};
+
+/// Data structure representing type information for xdbc types.
+#[derive(Debug, Clone, Default)]
+pub struct XdbcTypeInfo {
+    pub type_name: String,
+    pub data_type: XdbcDataType,
+    pub column_size: Option<i32>,
+    pub literal_prefix: Option<String>,
+    pub literal_suffix: Option<String>,
+    pub create_params: Option<Vec<String>>,
+    pub nullable: Nullable,
+    pub case_sensitive: bool,
+    pub searchable: Searchable,
+    pub unsigned_attribute: Option<bool>,
+    pub fixed_prec_scale: bool,
+    pub auto_increment: Option<bool>,
+    pub local_type_name: Option<String>,
+    pub minimum_scale: Option<i32>,
+    pub maximum_scale: Option<i32>,
+    pub sql_data_type: XdbcDataType,
+    pub datetime_subcode: Option<XdbcDatetimeSubcode>,
+    pub num_prec_radix: Option<i32>,
+    pub interval_precision: Option<i32>,
+}
+
+impl From<CommandGetXdbcTypeInfo> for Option<i32> {

Review Comment:
   we do not :)



-- 
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 diff in pull request #4359: feat(flight): add xdbc type info helpers

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359#discussion_r1218599270


##########
arrow-flight/src/sql/metadata/xdbc_info.rs:
##########
@@ -0,0 +1,441 @@
+// 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.
+
+//! Helpers for [`CommandGetXdbcTypeInfo`] metadata requests.
+//!
+//! - [`XdbcTypeInfo`] - a typed struct that holds the xdbc info corresponding to expected schema.
+//! - [`XdbcTypeInfoListBuilder`] - a builder for collecting type infos
+//!   and building a conformant `RecordBatch`.
+//! - [`XdbcTypeInfoList`] - a helper type wrapping a `RecordBatch`
+//!   used for handling [`CommandGetXdbcTypeInfo`] requests.
+//!
+use std::sync::Arc;
+
+use arrow_array::builder::{BooleanBuilder, Int32Builder, ListBuilder, StringBuilder};
+use arrow_array::cast::downcast_array;
+use arrow_array::{ArrayRef, Int32Array, ListArray, RecordBatch};
+use arrow_ord::comparison::eq_scalar;
+use arrow_schema::{DataType, Field, Schema, SchemaRef};
+use arrow_select::filter::filter_record_batch;
+use arrow_select::take::take;
+use once_cell::sync::Lazy;
+
+use super::lexsort_to_indices;
+use crate::error::*;
+use crate::sql::{
+    CommandGetXdbcTypeInfo, Nullable, Searchable, XdbcDataType, XdbcDatetimeSubcode,
+};
+
+/// Data structure representing type information for xdbc types.
+#[derive(Debug, Clone, Default)]
+pub struct XdbcTypeInfo {
+    pub type_name: String,
+    pub data_type: XdbcDataType,
+    pub column_size: Option<i32>,
+    pub literal_prefix: Option<String>,
+    pub literal_suffix: Option<String>,
+    pub create_params: Option<Vec<String>>,
+    pub nullable: Nullable,
+    pub case_sensitive: bool,
+    pub searchable: Searchable,
+    pub unsigned_attribute: Option<bool>,
+    pub fixed_prec_scale: bool,
+    pub auto_increment: Option<bool>,
+    pub local_type_name: Option<String>,
+    pub minimum_scale: Option<i32>,
+    pub maximum_scale: Option<i32>,
+    pub sql_data_type: XdbcDataType,
+    pub datetime_subcode: Option<XdbcDatetimeSubcode>,
+    pub num_prec_radix: Option<i32>,
+    pub interval_precision: Option<i32>,
+}
+
+impl From<CommandGetXdbcTypeInfo> for Option<i32> {

Review Comment:
   ๐Ÿงน 



-- 
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 diff in pull request #4359: feat(flight): add xdbc type info helpers

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359#discussion_r1218469383


##########
arrow-flight/examples/flight_sql_server.rs:
##########
@@ -544,12 +580,11 @@ impl FlightSqlService for FlightSqlServiceImpl {
 
     async fn do_get_xdbc_type_info(
         &self,
-        _query: CommandGetXdbcTypeInfo,
+        query: CommandGetXdbcTypeInfo,
         _request: Request<Ticket>,
     ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
-        Err(Status::unimplemented(
-            "do_get_xdbc_type_info not implemented",
-        ))
+        let stream = INSTANCE_XDBC_INFO.encode(query).map_err(Status::from);

Review Comment:
   I may have missed it, but this doesn't seem to ever construct a `XdbcTypeInfoListBuilder` (instead it calls `encode` directly)
   
   What do you think about making this mirror the other builders, so the code would look like
   
   ```rust
           let mut builder = query.into_builder(&INSTANCE_XBDC_INFO);
           let schema = builder.schema();
           let batch = builder.build();
           let stream = FlightDataEncoderBuilder::new()
               .with_schema(schema)
               .build(futures::stream::once(async { batch }))
               .map_err(Status::from);
           Ok(Response::new(Box::pin(stream)))
    ```
   
   An alternate idea for construction ๐Ÿค” 
   
   ```rust
           let mut builder = query.into_builder()
           builder.append_list(&INSTANCE_XBDC_INFO);?
   ```
   
   As you say, I agree we should also apply the same pattern (whatever it is) to `SqlInfoList`. I recommend we do so as part of another PR after we have worked out the pattern for `CommandGetXdbcTypeInfo`
   
   
   Finally, in terms of avoiding the boiler plate construction of 
   
   ```
           let batch = builder.build();
           let stream = FlightDataEncoderBuilder::new()
               .with_schema(schema)
               .build(futures::stream::once(async { batch }))
               .map_err(Status::from);
           Ok(Response::new(Box::pin(stream)))
   ```
   
   Maybe we can write some sort of generic function (either on `FlightDataEncoderBuilder` or as a free function) that takes a `impl Into<SchemaAndBatch>`
   
   where `SchemaAndBatch` looks something like
   
   ```rust
   pub trait SchemaAndBatch {
      /// produce the schema and result of encoding
     fn schema_and_batch(self) -> (SchemaRef, Result<RecordBatch>);
   }
   
   impl SchemaAndBatch for XdbcTypeInfoListBuilder {
   ...
   }
   ```
   
   Though maybe that is overly complicated ๐Ÿค” 



-- 
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] roeap commented on a diff in pull request #4359: feat(flight): add xdbc type info helpers

Posted by "roeap (via GitHub)" <gi...@apache.org>.
roeap commented on code in PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359#discussion_r1218520767


##########
arrow-flight/examples/flight_sql_server.rs:
##########
@@ -544,12 +580,11 @@ impl FlightSqlService for FlightSqlServiceImpl {
 
     async fn do_get_xdbc_type_info(
         &self,
-        _query: CommandGetXdbcTypeInfo,
+        query: CommandGetXdbcTypeInfo,
         _request: Request<Ticket>,
     ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
-        Err(Status::unimplemented(
-            "do_get_xdbc_type_info not implemented",
-        ))
+        let stream = INSTANCE_XDBC_INFO.encode(query).map_err(Status::from);

Review Comment:
   > Maybe we can use the pattern in this PR but change the names a bit? 
   
   Makes a lot of sense to me, I'll make the changes.



-- 
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] roeap commented on pull request #4359: feat(flight): add xdbc type info helpers

Posted by "roeap (via GitHub)" <gi...@apache.org>.
roeap commented on PR #4359:
URL: https://github.com/apache/arrow-rs/pull/4359#issuecomment-1582999743

   Thanks @appletreeisyellow - lot's of great inspiration in iOx ๐Ÿ™‚.


-- 
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