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

[GitHub] [arrow-adbc] WillAyd commented on a diff in pull request #723: feat(c/driver/postgresql): Implement GetObjects for columns

WillAyd commented on code in PR #723:
URL: https://github.com/apache/arrow-adbc/pull/723#discussion_r1214515053


##########
c/driver/postgresql/connection.cc:
##########
@@ -354,22 +369,174 @@ class PqGetObjectsHelper {
       const char* table_name = row[0].data;
       const char* table_type = row[1].data;
 
-      if (depth_ > ADBC_OBJECT_DEPTH_TABLES) {
-        return ADBC_STATUS_NOT_IMPLEMENTED;
+      CHECK_NA(INTERNAL,
+               ArrowArrayAppendString(table_name_col_, ArrowCharView(table_name)),
+               error_);
+      CHECK_NA(INTERNAL,
+               ArrowArrayAppendString(table_type_col_, ArrowCharView(table_type)),
+               error_);
+      if (depth_ == ADBC_OBJECT_DEPTH_TABLES) {
+        CHECK_NA(INTERNAL, ArrowArrayAppendNull(table_columns_col_, 1), error_);
+        CHECK_NA(INTERNAL, ArrowArrayAppendNull(table_constraints_col_, 1), error_);
       } else {
+        auto table_name_s = std::string(table_name);
+        RAISE_ADBC(AppendColumns(schema_name, table_name_s));
+        RAISE_ADBC(AppendConstraints(schema_name, table_name_s));
+      }
+      CHECK_NA(INTERNAL, ArrowArrayFinishElement(schema_table_items_), error_);
+    }
+
+    CHECK_NA(INTERNAL, ArrowArrayFinishElement(db_schema_tables_col_), error_);
+    return ADBC_STATUS_OK;
+  }
+
+  AdbcStatusCode AppendColumns(std::string schema_name, std::string table_name) {
+    struct StringBuilder query = {0};
+    if (StringBuilderInit(&query, /*initial_size*/ 512)) {
+      return ADBC_STATUS_INTERNAL;
+    }
+
+    std::vector<std::string> params = {schema_name, table_name};
+    const char* stmt =
+        "SELECT attr.attname, attr.attnum, "
+        "pg_catalog.col_description(cls.oid, attr.attnum) "
+        "FROM pg_catalog.pg_attribute AS attr "
+        "INNER JOIN pg_catalog.pg_class AS cls ON attr.attrelid = cls.oid "
+        "INNER JOIN pg_catalog.pg_namespace AS nsp ON nsp.oid = cls.relnamespace "
+        "WHERE attr.attnum > 0 AND NOT attr.attisdropped "
+        "AND nsp.nspname LIKE $1 AND cls.relname LIKE $2";
+
+    if (StringBuilderAppend(&query, "%s", stmt)) {
+      StringBuilderReset(&query);
+      return ADBC_STATUS_INTERNAL;
+    }
+
+    if (column_name_ != NULL) {
+      if (StringBuilderAppend(&query, "%s", " AND attr.attname LIKE $3")) {
+        StringBuilderReset(&query);
+        return ADBC_STATUS_INTERNAL;
+      }
+
+      params.push_back(std::string(column_name_));
+    }
+
+    auto result_helper = PqResultHelper{conn_, query.buffer, params, error_};
+    StringBuilderReset(&query);
+
+    RAISE_ADBC(result_helper.Prepare());
+    RAISE_ADBC(result_helper.Execute());
+
+    for (PqResultRow row : result_helper) {
+      const char* column_name = row[0].data;
+      const char* position = row[1].data;
+
+      CHECK_NA(INTERNAL,
+               ArrowArrayAppendString(column_name_col_, ArrowCharView(column_name)),
+               error_);
+      int ival = atol(position);
+      CHECK_NA(INTERNAL,
+               ArrowArrayAppendInt(column_position_col_, static_cast<int64_t>(ival)),
+               error_);
+      if (row[2].is_null) {
+        CHECK_NA(INTERNAL, ArrowArrayAppendNull(column_remarks_col_, 1), error_);
+      } else {
+        const char* remarks = row[2].data;
         CHECK_NA(INTERNAL,
-                 ArrowArrayAppendString(table_name_col_, ArrowCharView(table_name)),
+                 ArrowArrayAppendString(column_remarks_col_, ArrowCharView(remarks)),
+                 error_);
+      }
+
+      // no xdbc_ values for now
+      for (auto i = 3; i < 19; i++) {
+        CHECK_NA(INTERNAL, ArrowArrayAppendNull(table_columns_items_->children[i], 1),
                  error_);
+      }
+
+      CHECK_NA(INTERNAL, ArrowArrayFinishElement(table_columns_items_), error_);
+    }
+
+    CHECK_NA(INTERNAL, ArrowArrayFinishElement(table_columns_col_), error_);
+    return ADBC_STATUS_OK;
+  }
+
+  AdbcStatusCode AppendConstraints(std::string schema_name, std::string table_name) {
+    struct StringBuilder query = {0};
+    if (StringBuilderInit(&query, /*initial_size*/ 512)) {
+      return ADBC_STATUS_INTERNAL;
+    }
+
+    std::vector<std::string> params = {schema_name, table_name};
+    const char* stmt =
+        "SELECT con.conname, CASE con.contype WHEN 'c' THEN 'CHECK' WHEN 'u' THEN "
+        "'UNIQUE' WHEN 'p' THEN 'PRIMARY KEY' WHEN 'f' THEN 'FOREIGN KEY' "
+        "END AS contype, ARRAY(SELECT attr.attname) AS colnames, con.confkey "
+        "FROM pg_catalog.pg_constraint AS con "
+        "CROSS JOIN UNNEST(conkey) AS conkeys "
+        "INNER JOIN pg_catalog.pg_class AS cls ON cls.oid = con.conrelid "
+        "INNER JOIN pg_catalog.pg_namespace AS nsp ON nsp.oid = cls.relnamespace "
+        "INNER JOIN pg_catalog.pg_attribute AS attr ON attr.attnum = conkeys "
+        "AND cls.oid = attr.attrelid "
+        "WHERE con.contype IN ('c', 'u', 'p', 'f') AND nsp.nspname LIKE $1 "
+        "AND cls.relname LIKE $2";
+
+    if (StringBuilderAppend(&query, "%s", stmt)) {
+      StringBuilderReset(&query);
+      return ADBC_STATUS_INTERNAL;
+    }
+
+    if (column_name_ != NULL) {
+      if (StringBuilderAppend(&query, "%s", " AND con.conname LIKE $3")) {
+        StringBuilderReset(&query);
+        return ADBC_STATUS_INTERNAL;
+      }
+
+      params.push_back(std::string(column_name_));
+    }
+
+    auto result_helper = PqResultHelper{conn_, query.buffer, params, error_};
+    StringBuilderReset(&query);
+
+    RAISE_ADBC(result_helper.Prepare());
+    RAISE_ADBC(result_helper.Execute());
+
+    for (PqResultRow row : result_helper) {
+      const char* constraint_name = row[0].data;
+      const char* constraint_type = row[1].data;
+
+      CHECK_NA(
+          INTERNAL,
+          ArrowArrayAppendString(constraint_name_col_, ArrowCharView(constraint_name)),
+          error_);
+
+      CHECK_NA(
+          INTERNAL,
+          ArrowArrayAppendString(constraint_type_col_, ArrowCharView(constraint_type)),
+          error_);
+
+      for (auto i = 0; i < row[2].len; i++) {
+        // TODO: get actual name in here
+        const char* data = row[2].data;
         CHECK_NA(INTERNAL,
-                 ArrowArrayAppendString(table_type_col_, ArrowCharView(table_type)),
+                 ArrowArrayAppendString(constraint_column_name_col_,
+                                        ArrowCharView("IMPLEMENT_ME")),

Review Comment:
   Defer to another PR. Started on it but its going to make this pretty big to implement (and add tests)



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