You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ko...@apache.org on 2023/04/28 04:57:38 UTC

[arrow-adbc] branch main updated: feat(glib): add gadbc_connection_get_objects() (#617)

This is an automated email from the ASF dual-hosted git repository.

kou pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-adbc.git


The following commit(s) were added to refs/heads/main by this push:
     new a175420  feat(glib): add gadbc_connection_get_objects() (#617)
a175420 is described below

commit a17542003ab5b952489db5997b9c652e82d6c926
Author: Sutou Kouhei <ko...@clear-code.com>
AuthorDate: Fri Apr 28 13:57:32 2023 +0900

    feat(glib): add gadbc_connection_get_objects() (#617)
    
    Fixes #580.
---
 glib/Gemfile                      |   2 +-
 glib/adbc-glib/connection.c       | 141 +++++++++++++++
 glib/adbc-glib/connection.h       |  30 ++++
 glib/test/test-connection.rb      | 353 ++++++++++++++++++++++++++++++++++++++
 glib/test/test-isolation-level.rb |   1 -
 5 files changed, 525 insertions(+), 2 deletions(-)

diff --git a/glib/Gemfile b/glib/Gemfile
index 4af8f0a..e8f8ba3 100644
--- a/glib/Gemfile
+++ b/glib/Gemfile
@@ -19,6 +19,6 @@
 
 source "https://rubygems.org/"
 
-gem "gobject-introspection", ">= 4.0.1"
+gem "gobject-introspection", ">= 4.0.3"
 gem "red-arrow"
 gem "test-unit"
diff --git a/glib/adbc-glib/connection.c b/glib/adbc-glib/connection.c
index 0ef77a3..bcb541c 100644
--- a/glib/adbc-glib/connection.c
+++ b/glib/adbc-glib/connection.c
@@ -321,6 +321,147 @@ gpointer gadbc_connection_get_info(GADBCConnection* connection, guint32* info_co
   }
 }
 
+/**
+ * gadbc_connection_get_objects:
+ * @connection: A #GADBCConnection.
+ * @depth: The level of nesting to display. If
+ *   @GADBC_OBJECT_DEPTH_ALL, display all levels. If
+ *   @GADBC_OBJECT_DEPTH_CATALOGS, display only catalogs
+ *   (i.e. `catalog_schemas` will be null). If
+ *   @GADBC_OBJECT_DEPTH_DB_SCHEMAS, display only catalogs and schemas
+ *   (i.e. `db_schema_tables` will be null). if
+ *   @GADBC_OBJECT_DEPTH_TABLES, display only catalogs, schemas and
+ *   tables (i.e. `table_columns` and `table_constraints` will be
+ *   null).
+ * @catalog: (nullable): Only show tables in the given catalog. If
+ *   %NULL, do not filter by catalog. If an empty string, only show
+ *   tables without a catalog. May be a search pattern (see section
+ *   documentation).
+ * @db_schema: (nullable): Only show tables in the given database
+ *   schema. If %NULL, do not filter by database schema. If an empty
+ *   string, only show tables without a database schema. May be a
+ *   search pattern (see section documentation).
+ * @table_name: (nullable): Only show tables with the given name. If
+ *   %NULL, do not filter by name. May be a search pattern (see
+ *   section documentation).
+ * @table_types: (nullable) (array zero-terminated=1): Only show
+ *   tables matching one of the given table types. If %NULL, show
+ *   tables of any type. Valid table types can be fetched from
+ *   gadbc_connection_get_table_types(). Terminate the list with a
+ *   %NULL entry.
+ * @column_name: (nullable): Only show columns with the given name. If
+ *   %NULL, do not filter by name. May be a search pattern (see section
+ *   documentation).
+ * @error: (nullable): Return location for a #GError or %NULL.
+ *
+ * Get a hierarchical view of all catalogs, database schemas, tables,
+ * and columns.
+ *
+ * The result is an Arrow dataset with the following schema:
+ *
+ * | Field Name               | Field Type              |
+ * |--------------------------|-------------------------|
+ * | catalog_name             | utf8                    |
+ * | catalog_db_schemas       | list<DB_SCHEMA_SCHEMA>  |
+ *
+ * DB_SCHEMA_SCHEMA is a Struct with fields:
+ *
+ * | Field Name               | Field Type              |
+ * |--------------------------|-------------------------|
+ * | db_schema_name           | utf8                    |
+ * | db_schema_tables         | list<TABLE_SCHEMA>      |
+ *
+ * TABLE_SCHEMA is a Struct with fields:
+ *
+ * | Field Name               | Field Type              |
+ * |--------------------------|-------------------------|
+ * | table_name               | utf8 not null           |
+ * | table_type               | utf8 not null           |
+ * | table_columns            | list<COLUMN_SCHEMA>     |
+ * | table_constraints        | list<CONSTRAINT_SCHEMA> |
+ *
+ * COLUMN_SCHEMA is a Struct with fields:
+ *
+ * | Field Name               | Field Type              | Comments |
+ * |--------------------------|-------------------------|----------|
+ * | column_name              | utf8 not null           |          |
+ * | ordinal_position         | int32                   | (1)      |
+ * | remarks                  | utf8                    | (2)      |
+ * | xdbc_data_type           | int16                   | (3)      |
+ * | xdbc_type_name           | utf8                    | (3)      |
+ * | xdbc_column_size         | int32                   | (3)      |
+ * | xdbc_decimal_digits      | int16                   | (3)      |
+ * | xdbc_num_prec_radix      | int16                   | (3)      |
+ * | xdbc_nullable            | int16                   | (3)      |
+ * | xdbc_column_def          | utf8                    | (3)      |
+ * | xdbc_sql_data_type       | int16                   | (3)      |
+ * | xdbc_datetime_sub        | int16                   | (3)      |
+ * | xdbc_char_octet_length   | int32                   | (3)      |
+ * | xdbc_is_nullable         | utf8                    | (3)      |
+ * | xdbc_scope_catalog       | utf8                    | (3)      |
+ * | xdbc_scope_schema        | utf8                    | (3)      |
+ * | xdbc_scope_table         | utf8                    | (3)      |
+ * | xdbc_is_autoincrement    | bool                    | (3)      |
+ * | xdbc_is_generatedcolumn  | bool                    | (3)      |
+ *
+ * 1. The column's ordinal position in the table (starting from 1).
+ * 2. Database-specific description of the column.
+ * 3. Optional value.  Should be null if not supported by the driver.
+ *    xdbc_ values are meant to provide JDBC/ODBC-compatible metadata
+ *    in an agnostic manner.
+ *
+ * CONSTRAINT_SCHEMA is a Struct with fields:
+ *
+ * | Field Name               | Field Type              | Comments |
+ * |--------------------------|-------------------------|----------|
+ * | constraint_name          | utf8                    |          |
+ * | constraint_type          | utf8 not null           | (1)      |
+ * | constraint_column_names  | list<utf8> not null     | (2)      |
+ * | constraint_column_usage  | list<USAGE_SCHEMA>      | (3)      |
+ *
+ * 1. One of 'CHECK', 'FOREIGN KEY', 'PRIMARY KEY', or 'UNIQUE'.
+ * 2. The columns on the current table that are constrained, in
+ *    order.
+ * 3. For FOREIGN KEY only, the referenced table and columns.
+ *
+ * USAGE_SCHEMA is a Struct with fields:
+ *
+ * | Field Name               | Field Type              |
+ * |--------------------------|-------------------------|
+ * | fk_catalog               | utf8                    |
+ * | fk_db_schema             | utf8                    |
+ * | fk_table                 | utf8 not null           |
+ * | fk_column_name           | utf8 not null           |
+ *
+ * Returns: The result set as `struct ArrowArrayStream *`. It should
+ *   be freed with the `ArrowArrayStream::release` callback then
+ *   g_free() when no longer needed.
+ *
+ * Since: 0.4.0
+ */
+gpointer gadbc_connection_get_objects(GADBCConnection* connection, GADBCObjectDepth depth,
+                                      const gchar* catalog, const gchar* db_schema,
+                                      const gchar* table_name, const gchar** table_types,
+                                      const gchar* column_name, GError** error) {
+  const gchar* context = "[adbc][connection][get-objects]";
+  struct AdbcConnection* adbc_connection =
+      gadbc_connection_get_raw(connection, context, error);
+  if (!adbc_connection) {
+    return NULL;
+  }
+  struct ArrowArrayStream* array_stream = g_new0(struct ArrowArrayStream, 1);
+  struct AdbcError adbc_error = {};
+  AdbcStatusCode status_code =
+      AdbcConnectionGetObjects(adbc_connection, depth, catalog, db_schema, table_name,
+                               table_types, column_name, array_stream, &adbc_error);
+  if (gadbc_error_check(error, status_code, &adbc_error, context)) {
+    return array_stream;
+  } else {
+    g_free(array_stream);
+    return NULL;
+  }
+}
+
 /**
  * gadbc_connection_get_table_schema:
  * @connection: A #GADBCConnection.
diff --git a/glib/adbc-glib/connection.h b/glib/adbc-glib/connection.h
index bd062c6..30df571 100644
--- a/glib/adbc-glib/connection.h
+++ b/glib/adbc-glib/connection.h
@@ -51,6 +51,31 @@ typedef enum {
   GADBC_INFO_DRIVER_ARROW_VERSION = 102,
 } GADBCInfo;
 
+/**
+ * GADBCObjectDepth:
+ * @GADBC_OBJECT_DEPTH_ALL: Return metadata on catalogs, schemas,
+ *   tables, and columns.
+ * @GADBC_OBJECT_DEPTH_CATALOGS: Return metadata on catalogs only.
+ * @GADBC_OBJECT_DEPTH_DB_SCHEMAS: Return metadata on catalogs and schemas.
+ * @GADBC_OBJECT_DEPTH_TABLES: Return metadata on catalogs, schemas,
+ *   and tables.
+ *
+ * The object depth that is used by gadbc_connection_get_objects().
+ *
+ * They are corresponding to `ADBC_OBJECT_DEPTH_*` values in `adbc.h`.
+ *
+ * Note that @ADBC_OBJECT_DEPTH_COLUMNS doesn't exist because it's
+ * same as @ADBC_OBJECT_DEPTH_ALL.
+ *
+ * Since: 0.4.0
+ */
+typedef enum {
+  GADBC_OBJECT_DEPTH_ALL = 0,
+  GADBC_OBJECT_DEPTH_CATALOGS = 1,
+  GADBC_OBJECT_DEPTH_DB_SCHEMAS = 2,
+  GADBC_OBJECT_DEPTH_TABLES = 3,
+} GADBCObjectDepth;
+
 /**
  * GADBCIsolationLevel:
  * @GADBC_ISOLATION_LEVEL_DEFAULT: Use database or driver default
@@ -144,6 +169,11 @@ GADBC_AVAILABLE_IN_0_4
 gpointer gadbc_connection_get_info(GADBCConnection* connection, guint32* info_codes,
                                    gsize n_info_codes, GError** error);
 GADBC_AVAILABLE_IN_0_4
+gpointer gadbc_connection_get_objects(GADBCConnection* connection, GADBCObjectDepth depth,
+                                      const gchar* catalog, const gchar* db_schema,
+                                      const gchar* table_name, const gchar** table_types,
+                                      const gchar* column_name, GError** error);
+GADBC_AVAILABLE_IN_0_4
 gpointer gadbc_connection_get_table_schema(GADBCConnection* connection,
                                            const gchar* catalog, const gchar* db_schema,
                                            const gchar* table_name, GError** error);
diff --git a/glib/test/test-connection.rb b/glib/test/test-connection.rb
index b959aa4..8706a02 100644
--- a/glib/test/test-connection.rb
+++ b/glib/test/test-connection.rb
@@ -98,6 +98,359 @@ class ConnectionTest < Test::Unit::TestCase
     end
   end
 
+  sub_test_case("#objects") do
+    def setup
+      super do
+        execute_sql(@connection,
+                    "CREATE TABLE data (number int, string text)",
+                    need_result: false)
+        execute_sql(@connection,
+                    "INSERT INTO data VALUES (1, 'hello')",
+                    need_result: false)
+        yield
+      end
+    end
+
+    def get_objects(*args)
+      c_abi_array_stream = @connection.get_objects(*args)
+      begin
+        reader = Arrow::RecordBatchReader.import(c_abi_array_stream)
+        yield(reader.read_all)
+      ensure
+        GLib.free(c_abi_array_stream)
+      end
+    end
+
+    def test_catalogs_all
+      get_objects(:catalogs) do |table|
+        assert_equal([["main", nil]],
+                     table.raw_records)
+      end
+    end
+
+    def test_catalogs_match
+      get_objects(:catalogs, "main") do |table|
+        assert_equal([["main", nil]],
+                     table.raw_records)
+      end
+    end
+
+    def test_catalogs_not_match
+      get_objects(:catalogs, "nonexistent") do |table|
+        assert_equal([],
+                     table.raw_records)
+      end
+    end
+
+    def test_db_schemas_all
+      get_objects(:db_schemas) do |table|
+        assert_equal([
+                       [
+                         "main",
+                         [
+                           {
+                             "db_schema_name" => nil,
+                             "db_schema_tables" => nil,
+                           },
+                         ],
+                       ],
+                     ],
+                     table.raw_records)
+      end
+    end
+
+    def test_db_schemas_match
+      get_objects(:db_schemas, nil, nil) do |table|
+        assert_equal([
+                       [
+                         "main",
+                         [
+                           {
+                             "db_schema_name" => nil,
+                             "db_schema_tables" => nil,
+                           },
+                         ],
+                       ],
+                     ],
+                     table.raw_records)
+      end
+    end
+
+    def test_db_schemas_not_match
+      get_objects(:db_schemas, nil, "nonexistent") do |table|
+        assert_equal([["main", []]],
+                     table.raw_records)
+      end
+    end
+
+    def test_tables_all
+      get_objects(:tables) do |table|
+        assert_equal([
+                       [
+                         "main",
+                         [
+                           {
+                             "db_schema_name" => nil,
+                             "db_schema_tables" => [
+                               {
+                                 "table_columns" => nil,
+                                 "table_constraints" => nil,
+                                 "table_name" => "data",
+                                 "table_type" => "table",
+                               },
+                             ],
+                           },
+                         ],
+                       ],
+                     ],
+                     table.raw_records)
+      end
+    end
+
+    def test_tables_match
+      get_objects(:tables, nil, nil, "data") do |table|
+        assert_equal([
+                       [
+                         "main",
+                         [
+                           {
+                             "db_schema_name" => nil,
+                             "db_schema_tables" => [
+                               {
+                                 "table_columns" => nil,
+                                 "table_constraints" => nil,
+                                 "table_name" => "data",
+                                 "table_type" => "table",
+                               },
+                             ],
+                           },
+                         ],
+                       ],
+                     ],
+                     table.raw_records)
+      end
+    end
+
+    def test_tables_not_match
+      get_objects(:tables, nil, nil, "nonexistent") do |table|
+        assert_equal([
+                       [
+                         "main",
+                         [
+                           {
+                             "db_schema_name" => nil,
+                             "db_schema_tables" => [],
+                           },
+                         ],
+                       ],
+                     ],
+                     table.raw_records)
+      end
+    end
+
+    def test_table_types_all
+      get_objects(:tables) do |table|
+        assert_equal([
+                       [
+                         "main",
+                         [
+                           {
+                             "db_schema_name" => nil,
+                             "db_schema_tables" => [
+                               {
+                                 "table_columns" => nil,
+                                 "table_constraints" => nil,
+                                 "table_name" => "data",
+                                 "table_type" => "table",
+                               },
+                             ],
+                           },
+                         ],
+                       ],
+                     ],
+                     table.raw_records)
+      end
+    end
+
+    def test_table_types_match
+      get_objects(:tables, nil, nil, nil, ["table", "view"]) do |table|
+        assert_equal([
+                       [
+                         "main",
+                         [
+                           {
+                             "db_schema_name" => nil,
+                             "db_schema_tables" => [
+                               {
+                                 "table_columns" => nil,
+                                 "table_constraints" => nil,
+                                 "table_name" => "data",
+                                 "table_type" => "table",
+                               },
+                             ],
+                           },
+                         ],
+                       ],
+                     ],
+                     table.raw_records)
+      end
+    end
+
+    def test_table_types_not_match
+      get_objects(:tables, nil, nil, nil, ["nonexistent"]) do |table|
+        assert_equal([
+                       [
+                         "main",
+                         [
+                           {
+                             "db_schema_name" => nil,
+                             "db_schema_tables" => [],
+                           },
+                         ],
+                       ],
+                     ],
+                     table.raw_records)
+      end
+    end
+
+    def test_column_all
+      get_objects(:all) do |table|
+        assert_equal([
+                       [
+                         "main",
+                         [
+                           {
+                             "db_schema_name" => nil,
+                             "db_schema_tables" => [
+                               {
+                                 "table_columns" => [
+                                   {
+                                     "column_name" => "number",
+                                     "ordinal_position" => 1,
+                                     "remarks" => nil,
+                                     "xdbc_char_octet_length" => nil,
+                                     "xdbc_column_def" => nil,
+                                     "xdbc_column_size" => nil,
+                                     "xdbc_data_type" => nil,
+                                     "xdbc_datetime_sub" => nil,
+                                     "xdbc_decimal_digits" => nil,
+                                     "xdbc_is_autoincrement" => nil,
+                                     "xdbc_is_generatedcolumn" => nil,
+                                     "xdbc_is_nullable" => "YES",
+                                     "xdbc_nullable" => 1,
+                                     "xdbc_num_prec_radix" => nil,
+                                     "xdbc_scope_catalog" => nil,
+                                     "xdbc_scope_schema" => nil,
+                                     "xdbc_scope_table" => nil,
+                                     "xdbc_sql_data_type" => nil,
+                                     "xdbc_type_name" => "INT",
+                                   },
+                                   {
+                                     "column_name" => "string",
+                                     "ordinal_position" => 2,
+                                     "remarks" => nil,
+                                     "xdbc_char_octet_length" => nil,
+                                     "xdbc_column_def" => nil,
+                                     "xdbc_column_size" => nil,
+                                     "xdbc_data_type" => nil,
+                                     "xdbc_datetime_sub" => nil,
+                                     "xdbc_decimal_digits" => nil,
+                                     "xdbc_is_autoincrement" => nil,
+                                     "xdbc_is_generatedcolumn" => nil,
+                                     "xdbc_is_nullable" => "YES",
+                                     "xdbc_nullable" => 1,
+                                     "xdbc_num_prec_radix" => nil,
+                                     "xdbc_scope_catalog" => nil,
+                                     "xdbc_scope_schema" => nil,
+                                     "xdbc_scope_table" => nil,
+                                     "xdbc_sql_data_type" => nil,
+                                     "xdbc_type_name" => "TEXT",
+                                   },
+                                 ],
+                                 "table_constraints" => [],
+                                 "table_name" => "data",
+                                 "table_type" => "table",
+                               },
+                             ],
+                           },
+                         ],
+                       ],
+                     ],
+                     table.raw_records)
+      end
+    end
+
+    def test_column_match
+      get_objects(:all, nil, nil, nil, nil, "number") do |table|
+        assert_equal([
+                       [
+                         "main",
+                         [
+                           {
+                             "db_schema_name" => nil,
+                             "db_schema_tables" => [
+                               {
+                                 "table_columns" => [
+                                   {
+                                     "column_name" => "number",
+                                     "ordinal_position" => 1,
+                                     "remarks" => nil,
+                                     "xdbc_char_octet_length" => nil,
+                                     "xdbc_column_def" => nil,
+                                     "xdbc_column_size" => nil,
+                                     "xdbc_data_type" => nil,
+                                     "xdbc_datetime_sub" => nil,
+                                     "xdbc_decimal_digits" => nil,
+                                     "xdbc_is_autoincrement" => nil,
+                                     "xdbc_is_generatedcolumn" => nil,
+                                     "xdbc_is_nullable" => "YES",
+                                     "xdbc_nullable" => 1,
+                                     "xdbc_num_prec_radix" => nil,
+                                     "xdbc_scope_catalog" => nil,
+                                     "xdbc_scope_schema" => nil,
+                                     "xdbc_scope_table" => nil,
+                                     "xdbc_sql_data_type" => nil,
+                                     "xdbc_type_name" => "INT",
+                                   },
+                                 ],
+                                 "table_constraints" => [],
+                                 "table_name" => "data",
+                                 "table_type" => "table",
+                               },
+                             ],
+                           },
+                         ],
+                       ],
+                     ],
+                     table.raw_records)
+      end
+    end
+
+    def test_column_not_match
+      get_objects(:all, nil, nil, nil, nil, "nonexistent") do |table|
+        assert_equal([
+                       [
+                         "main",
+                         [
+                           {
+                             "db_schema_name" => nil,
+                             "db_schema_tables" => [
+                               {
+                                 "table_columns" => [],
+                                 "table_constraints" => [],
+                                 "table_name" => "data",
+                                 "table_type" => "table",
+                               },
+                             ],
+                           },
+                         ],
+                       ],
+                     ],
+                     table.raw_records)
+      end
+    end
+  end
+
   def test_table_schema
     execute_sql(@connection,
                 "CREATE TABLE data (number int, string text)",
diff --git a/glib/test/test-isolation-level.rb b/glib/test/test-isolation-level.rb
index 973938b..8a0362b 100644
--- a/glib/test/test-isolation-level.rb
+++ b/glib/test/test-isolation-level.rb
@@ -19,7 +19,6 @@ class IsolationLevelTest < Test::Unit::TestCase
   include Helper
 
   def test_to_string
-    require_gi_bindings(4, 1, 3)
     assert_equal("adbc.connection.transaction.isolation.default",
                  ADBC::IsolationLevel.to_string(:default))
   end