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

[GitHub] [arrow-rs] alamb commented on a diff in pull request #4296: feat(flight): add helpers to handle `CommandGetCatalogs`, `CommandGetSchemas`, and `CommandGetTables` requests

alamb commented on code in PR #4296:
URL: https://github.com/apache/arrow-rs/pull/4296#discussion_r1213648736


##########
arrow-flight/src/sql/catalogs/db_schemas.rs:
##########
@@ -107,13 +136,29 @@ impl GetSchemasBuilder {
         let catalog_name = catalog_name.finish();
         let db_schema_name = db_schema_name.finish();
 
-        // the filter, if requested, getting a BooleanArray that represents the rows that passed the filter
-        let filter = db_schema_filter_pattern
-            .map(|db_schema_filter_pattern| {
-                // use like kernel to get wildcard matching
-                like_utf8_scalar(&db_schema_name, &db_schema_filter_pattern)
-            })
-            .transpose()?;
+        let mut filters = vec![];
+
+        if let Some(db_schema_filter_pattern) = db_schema_filter_pattern {
+            // use like kernel to get wildcard matching
+            filters.push(like_utf8_scalar(
+                &db_schema_name,
+                &db_schema_filter_pattern,
+            )?)
+        }
+
+        if let Some(catalog_filter_name) = catalog_filter {

Review Comment:
   it doesn't matter here, but this can be implemented more efficiently (in a follow on PR perhaps) by checking equality in `append`



##########
arrow-flight/src/sql/catalogs/tables.rs:
##########
@@ -0,0 +1,466 @@
+// 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.
+
+//! [`GetTablesBuilder`] for building responses to [`CommandGetTables`] queries.
+//!
+//! [`CommandGetTables`]: crate::sql::CommandGetTables
+
+use std::sync::Arc;
+
+use arrow_arith::boolean::{and, or};
+use arrow_array::builder::{BinaryBuilder, StringBuilder};
+use arrow_array::{ArrayRef, RecordBatch};
+use arrow_ord::comparison::eq_utf8_scalar;
+use arrow_schema::{DataType, Field, Schema, SchemaRef};
+use arrow_select::{filter::filter_record_batch, take::take};
+use arrow_string::like::like_utf8_scalar;
+use once_cell::sync::Lazy;
+
+use super::lexsort_to_indices;
+use crate::error::*;
+use crate::sql::CommandGetTables;
+use crate::{IpcMessage, IpcWriteOptions, SchemaAsIpc};
+
+/// Return the schema of the RecordBatch that will be returned from [`CommandGetTables`]
+///
+/// Note the schema differs based on the values of `include_schema
+///
+/// [`CommandGetTables`]: crate::sql::CommandGetTables
+pub fn get_tables_schema(include_schema: bool) -> SchemaRef {
+    if include_schema {
+        Arc::clone(&GET_TABLES_SCHEMA_WITH_TABLE_SCHEMA)
+    } else {
+        Arc::clone(&GET_TABLES_SCHEMA_WITHOUT_TABLE_SCHEMA)
+    }
+}
+
+/// Builds rows like this:
+///
+/// * 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.)
+pub struct GetTablesBuilder {
+    catalog_filter: Option<String>,
+    table_types_filter: Vec<String>,
+    // Optional filters to apply to schemas
+    db_schema_filter_pattern: Option<String>,
+    // Optional filters to apply to tables
+    table_name_filter_pattern: Option<String>,
+    // array builder for catalog names
+    catalog_name: StringBuilder,
+    // array builder for db schema names
+    db_schema_name: StringBuilder,
+    // array builder for tables names
+    table_name: StringBuilder,
+    // array builder for table types
+    table_type: StringBuilder,
+    // array builder for table schemas
+    table_schema: Option<BinaryBuilder>,
+}
+
+impl CommandGetTables {
+    pub fn into_builder(self) -> GetTablesBuilder {

Review Comment:
   I really like the `into_builder()` function 👍 



##########
arrow-flight/examples/flight_sql_server.rs:
##########
@@ -429,22 +429,12 @@ impl FlightSqlService for FlightSqlServiceImpl {
             })
             .collect::<HashSet<_>>();
 
-        let mut builder = GetSchemasBuilder::new(query.db_schema_filter_pattern);
-        if let Some(catalog) = query.catalog {
-            for (catalog_name, schema_name) in schemas {
-                if catalog == catalog_name {
-                    builder
-                        .append(catalog_name, schema_name)
-                        .map_err(Status::from)?;
-                }
-            }
-        } else {
-            for (catalog_name, schema_name) in schemas {
-                builder
-                    .append(catalog_name, schema_name)
-                    .map_err(Status::from)?;
-            }
-        };
+        let mut builder = query.into_builder();

Review Comment:
   ❤️ that is very nice



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