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/05/10 18:21:15 UTC

[GitHub] [arrow-adbc] WillAyd opened a new pull request, #676: feat(c/driver/postgres): Implement GetObjectsSchema

WillAyd opened a new pull request, #676:
URL: https://github.com/apache/arrow-adbc/pull/676

   (no 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-adbc] lidavidm commented on a diff in pull request #676: feat(c/driver/postgresql): Implement GetObjectsSchema

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #676:
URL: https://github.com/apache/arrow-adbc/pull/676#discussion_r1191156831


##########
c/driver/postgresql/postgresql_test.cc:
##########
@@ -162,6 +161,53 @@ TEST_F(PostgresConnectionTest, GetInfoMetadata) {
   ASSERT_THAT(seen, ::testing::UnorderedElementsAreArray(info));
 }
 
+TEST_F(PostgresConnectionTest, GetObjectsGetCatalogs) {
+  ASSERT_THAT(AdbcConnectionNew(&connection, &error), IsOkStatus(&error));
+  ASSERT_THAT(AdbcConnectionInit(&connection, &database, &error), IsOkStatus(&error));
+
+  if (!quirks()->supports_get_objects()) {
+    GTEST_SKIP();
+  }
+
+  {

Review Comment:
   nit: why the extra nesting?



##########
c/driver/postgresql/postgresql_test.cc:
##########
@@ -162,6 +161,53 @@ TEST_F(PostgresConnectionTest, GetInfoMetadata) {
   ASSERT_THAT(seen, ::testing::UnorderedElementsAreArray(info));
 }
 
+TEST_F(PostgresConnectionTest, GetObjectsGetCatalogs) {
+  ASSERT_THAT(AdbcConnectionNew(&connection, &error), IsOkStatus(&error));
+  ASSERT_THAT(AdbcConnectionInit(&connection, &database, &error), IsOkStatus(&error));
+
+  if (!quirks()->supports_get_objects()) {
+    GTEST_SKIP();
+  }
+
+  {
+    adbc_validation::StreamReader reader;
+    ASSERT_THAT(AdbcConnectionGetObjects(&connection, ADBC_OBJECT_DEPTH_CATALOGS, nullptr,
+                                         nullptr, nullptr, nullptr, nullptr,
+                                         &reader.stream.value, &error),
+                IsOkStatus(&error));
+    ASSERT_NO_FATAL_FAILURE(reader.GetSchema());
+    ASSERT_NO_FATAL_FAILURE(reader.Next());
+    ASSERT_NE(nullptr, reader.array->release);
+    ASSERT_GT(reader.array->length, 0);
+
+    // A default psql install will contain these databases, but users
+    // can drop. Googletest does not support a warning outcome for tests (?)
+    // so these may be too strict

Review Comment:
   I think it's fine to assume default PostgreSQL



##########
c/driver/postgresql/connection.cc:
##########
@@ -145,6 +145,76 @@ AdbcStatusCode PostgresConnection::GetInfo(struct AdbcConnection* connection,
   return BatchToArrayStream(&array, &schema, out, error);
 }
 
+AdbcStatusCode PostgresConnectionGetObjectsImpl(
+    PGconn* conn, int depth, const char* catalog, const char* db_schema,
+    const char* table_name, const char** table_types, const char* column_name,
+    struct ArrowSchema* schema, struct ArrowArray* array, struct AdbcError* error) {
+  RAISE_ADBC(AdbcInitConnectionObjectsSchema(schema, error));
+
+  struct ArrowError na_error = {0};
+  CHECK_NA_DETAIL(INTERNAL, ArrowArrayInitFromSchema(array, schema, &na_error), &na_error,
+                  error);
+  CHECK_NA(INTERNAL, ArrowArrayStartAppending(array), error);
+
+  struct ArrowArray* catalog_name_col = array->children[0];
+  struct ArrowArray* catalog_db_schemas_col = array->children[1];
+
+  // TODO: support proper filters
+  if (!catalog) {
+    struct StringBuilder query = {0};
+    if (StringBuilderInit(&query, /*initial_size=*/256) != 0) return ADBC_STATUS_INTERNAL;
+
+    if (StringBuilderAppend(&query, "%s", "SELECT datname FROM pg_catalog.pg_database"))
+      return ADBC_STATUS_INTERNAL;

Review Comment:
   nit: braces? (not sure why clang-format doesn't just do that for you...)



##########
c/driver/postgresql/connection.cc:
##########
@@ -145,6 +145,76 @@ AdbcStatusCode PostgresConnection::GetInfo(struct AdbcConnection* connection,
   return BatchToArrayStream(&array, &schema, out, error);
 }
 
+AdbcStatusCode PostgresConnectionGetObjectsImpl(
+    PGconn* conn, int depth, const char* catalog, const char* db_schema,
+    const char* table_name, const char** table_types, const char* column_name,
+    struct ArrowSchema* schema, struct ArrowArray* array, struct AdbcError* error) {
+  RAISE_ADBC(AdbcInitConnectionObjectsSchema(schema, error));
+
+  struct ArrowError na_error = {0};
+  CHECK_NA_DETAIL(INTERNAL, ArrowArrayInitFromSchema(array, schema, &na_error), &na_error,
+                  error);
+  CHECK_NA(INTERNAL, ArrowArrayStartAppending(array), error);
+
+  struct ArrowArray* catalog_name_col = array->children[0];
+  struct ArrowArray* catalog_db_schemas_col = array->children[1];
+
+  // TODO: support proper filters
+  if (!catalog) {
+    struct StringBuilder query = {0};
+    if (StringBuilderInit(&query, /*initial_size=*/256) != 0) return ADBC_STATUS_INTERNAL;
+
+    if (StringBuilderAppend(&query, "%s", "SELECT datname FROM pg_catalog.pg_database"))
+      return ADBC_STATUS_INTERNAL;
+
+    PqResultHelper result_helper = PqResultHelper{conn, query.buffer};
+    StringBuilderReset(&query);
+    pg_result* result = result_helper.Execute();
+
+    ExecStatusType pq_status = PQresultStatus(result);
+    if (pq_status == PGRES_TUPLES_OK) {
+      int num_rows = PQntuples(result);
+      array->length = num_rows - 1;  // makes ArrowArrayFinishElement happy, but why?
+      for (int row = 0; row < num_rows; row++) {
+        const char* db_name = PQgetvalue(result, row, 0);
+        CHECK_NA(INTERNAL,
+                 ArrowArrayAppendString(catalog_name_col, ArrowCharView(db_name)), error);
+        if (depth == ADBC_OBJECT_DEPTH_CATALOGS) {
+          CHECK_NA(INTERNAL, ArrowArrayAppendNull(catalog_db_schemas_col, 1), error);
+        } else {
+          return ADBC_STATUS_NOT_IMPLEMENTED;
+        }
+      }
+    }
+
+    CHECK_NA(INTERNAL, ArrowArrayFinishElement(array), error);
+  }

Review Comment:
   maybe add an 'else return NOT_IMPLEMENTED' so it's clear what happened?



##########
c/driver/postgresql/postgresql_test.cc:
##########
@@ -162,6 +161,53 @@ TEST_F(PostgresConnectionTest, GetInfoMetadata) {
   ASSERT_THAT(seen, ::testing::UnorderedElementsAreArray(info));
 }
 
+TEST_F(PostgresConnectionTest, GetObjectsGetCatalogs) {
+  ASSERT_THAT(AdbcConnectionNew(&connection, &error), IsOkStatus(&error));
+  ASSERT_THAT(AdbcConnectionInit(&connection, &database, &error), IsOkStatus(&error));
+
+  if (!quirks()->supports_get_objects()) {
+    GTEST_SKIP();
+  }
+
+  {
+    adbc_validation::StreamReader reader;
+    ASSERT_THAT(AdbcConnectionGetObjects(&connection, ADBC_OBJECT_DEPTH_CATALOGS, nullptr,
+                                         nullptr, nullptr, nullptr, nullptr,
+                                         &reader.stream.value, &error),
+                IsOkStatus(&error));
+    ASSERT_NO_FATAL_FAILURE(reader.GetSchema());
+    ASSERT_NO_FATAL_FAILURE(reader.Next());
+    ASSERT_NE(nullptr, reader.array->release);
+    ASSERT_GT(reader.array->length, 0);
+
+    // A default psql install will contain these databases, but users
+    // can drop. Googletest does not support a warning outcome for tests (?)
+    // so these may be too strict
+    bool seen_postgres_db = false;
+    bool seen_template0_db = false;
+    bool seen_tempalte1_db = false;
+
+    do {
+      for (int64_t row = 0; row < reader.array->length; row++) {
+        ArrowStringView val =
+            ArrowArrayViewGetStringUnsafe(reader.array_view->children[0], row);
+        auto val_str = std::string(val.data, val.size_bytes);
+        if (val_str == "postgres")
+          seen_postgres_db = true;
+        else if (val_str == "template0")
+          seen_template0_db = true;
+        else if (val_str == "template1")
+          seen_tempalte1_db = true;

Review Comment:
   nit: braces?



##########
c/driver/postgresql/connection.cc:
##########
@@ -145,6 +145,76 @@ AdbcStatusCode PostgresConnection::GetInfo(struct AdbcConnection* connection,
   return BatchToArrayStream(&array, &schema, out, error);
 }
 
+AdbcStatusCode PostgresConnectionGetObjectsImpl(
+    PGconn* conn, int depth, const char* catalog, const char* db_schema,
+    const char* table_name, const char** table_types, const char* column_name,
+    struct ArrowSchema* schema, struct ArrowArray* array, struct AdbcError* error) {
+  RAISE_ADBC(AdbcInitConnectionObjectsSchema(schema, error));
+
+  struct ArrowError na_error = {0};
+  CHECK_NA_DETAIL(INTERNAL, ArrowArrayInitFromSchema(array, schema, &na_error), &na_error,
+                  error);
+  CHECK_NA(INTERNAL, ArrowArrayStartAppending(array), error);
+
+  struct ArrowArray* catalog_name_col = array->children[0];
+  struct ArrowArray* catalog_db_schemas_col = array->children[1];
+
+  // TODO: support proper filters
+  if (!catalog) {
+    struct StringBuilder query = {0};
+    if (StringBuilderInit(&query, /*initial_size=*/256) != 0) return ADBC_STATUS_INTERNAL;
+
+    if (StringBuilderAppend(&query, "%s", "SELECT datname FROM pg_catalog.pg_database"))
+      return ADBC_STATUS_INTERNAL;
+
+    PqResultHelper result_helper = PqResultHelper{conn, query.buffer};
+    StringBuilderReset(&query);
+    pg_result* result = result_helper.Execute();
+
+    ExecStatusType pq_status = PQresultStatus(result);
+    if (pq_status == PGRES_TUPLES_OK) {
+      int num_rows = PQntuples(result);
+      array->length = num_rows - 1;  // makes ArrowArrayFinishElement happy, but why?

Review Comment:
   Hmm, this seems suspect.



-- 
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-adbc] WillAyd commented on a diff in pull request #676: feat(c/driver/postgresql): Implement GetObjectsSchema

Posted by "WillAyd (via GitHub)" <gi...@apache.org>.
WillAyd commented on code in PR #676:
URL: https://github.com/apache/arrow-adbc/pull/676#discussion_r1191628682


##########
c/driver/postgresql/connection.cc:
##########
@@ -145,6 +145,77 @@ AdbcStatusCode PostgresConnection::GetInfo(struct AdbcConnection* connection,
   return BatchToArrayStream(&array, &schema, out, error);
 }
 
+AdbcStatusCode PostgresConnectionGetObjectsImpl(
+    PGconn* conn, int depth, const char* catalog, const char* db_schema,
+    const char* table_name, const char** table_types, const char* column_name,
+    struct ArrowSchema* schema, struct ArrowArray* array, struct AdbcError* error) {
+  RAISE_ADBC(AdbcInitConnectionObjectsSchema(schema, error));
+
+  struct ArrowError na_error = {0};
+  CHECK_NA_DETAIL(INTERNAL, ArrowArrayInitFromSchema(array, schema, &na_error), &na_error,
+                  error);
+  CHECK_NA(INTERNAL, ArrowArrayStartAppending(array), error);
+
+  struct ArrowArray* catalog_name_col = array->children[0];
+  struct ArrowArray* catalog_db_schemas_col = array->children[1];
+
+  // TODO: support proper filters
+  if (!catalog) {
+    struct StringBuilder query = {0};
+    if (StringBuilderInit(&query, /*initial_size=*/256) != 0) return ADBC_STATUS_INTERNAL;
+
+    if (StringBuilderAppend(&query, "%s", "SELECT datname FROM pg_catalog.pg_database")) {
+      return ADBC_STATUS_INTERNAL;
+    }
+
+    PqResultHelper result_helper = PqResultHelper{conn, query.buffer};
+    StringBuilderReset(&query);
+    pg_result* result = result_helper.Execute();
+
+    ExecStatusType pq_status = PQresultStatus(result);
+    if (pq_status == PGRES_TUPLES_OK) {
+      int num_rows = PQntuples(result);
+      for (int row = 0; row < num_rows; row++) {
+        const char* db_name = PQgetvalue(result, row, 0);
+        CHECK_NA(INTERNAL,
+                 ArrowArrayAppendString(catalog_name_col, ArrowCharView(db_name)), error);
+        if (depth == ADBC_OBJECT_DEPTH_CATALOGS) {
+          CHECK_NA(INTERNAL, ArrowArrayAppendNull(catalog_db_schemas_col, 1), error);
+        } else {
+          return ADBC_STATUS_NOT_IMPLEMENTED;
+        }
+        CHECK_NA(INTERNAL, ArrowArrayFinishElement(array), error);

Review Comment:
   Think I figured this out so we don't need to explicitly set the array length. This happens towards the end of the sqlite.c implementation I started with, but that implementation right now only supports one catalog. Putting this in the loop I think is the proper home



-- 
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-adbc] lidavidm merged pull request #676: feat(c/driver/postgresql): Implement GetObjectsSchema

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


-- 
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-adbc] WillAyd commented on a diff in pull request #676: feat(c/driver/postgresql): Implement GetObjectsSchema

Posted by "WillAyd (via GitHub)" <gi...@apache.org>.
WillAyd commented on code in PR #676:
URL: https://github.com/apache/arrow-adbc/pull/676#discussion_r1191614920


##########
c/driver/postgresql/postgresql_test.cc:
##########
@@ -162,6 +161,53 @@ TEST_F(PostgresConnectionTest, GetInfoMetadata) {
   ASSERT_THAT(seen, ::testing::UnorderedElementsAreArray(info));
 }
 
+TEST_F(PostgresConnectionTest, GetObjectsGetCatalogs) {
+  ASSERT_THAT(AdbcConnectionNew(&connection, &error), IsOkStatus(&error));
+  ASSERT_THAT(AdbcConnectionInit(&connection, &database, &error), IsOkStatus(&error));
+
+  if (!quirks()->supports_get_objects()) {
+    GTEST_SKIP();
+  }
+
+  {

Review Comment:
   The nesting is done in the parent test that this was copied from, but is superfluous 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-adbc] WillAyd commented on a diff in pull request #676: feat(c/driver/postgres): Implement GetObjectsSchema

Posted by "WillAyd (via GitHub)" <gi...@apache.org>.
WillAyd commented on code in PR #676:
URL: https://github.com/apache/arrow-adbc/pull/676#discussion_r1190350559


##########
c/driver/postgresql/connection.cc:
##########
@@ -145,6 +145,76 @@ AdbcStatusCode PostgresConnection::GetInfo(struct AdbcConnection* connection,
   return BatchToArrayStream(&array, &schema, out, error);
 }
 
+AdbcStatusCode PostgresConnectionGetObjectsImpl(
+    PGconn* conn, int depth, const char* catalog, const char* db_schema,
+    const char* table_name, const char** table_types, const char* column_name,
+    struct ArrowSchema* schema, struct ArrowArray* array, struct AdbcError* error) {
+  RAISE_ADBC(AdbcInitConnectionObjectsSchema(schema, error));
+
+  struct ArrowError na_error = {0};
+  CHECK_NA_DETAIL(INTERNAL, ArrowArrayInitFromSchema(array, schema, &na_error), &na_error,
+                  error);
+  CHECK_NA(INTERNAL, ArrowArrayStartAppending(array), error);
+
+  struct ArrowArray* catalog_name_col = array->children[0];
+  struct ArrowArray* catalog_db_schemas_col = array->children[1];
+
+  // TODO: support proper filters
+  if (!catalog) {
+    struct StringBuilder query = {0};
+    if (StringBuilderInit(&query, /*initial_size=*/256) != 0) return ADBC_STATUS_INTERNAL;
+
+    if (StringBuilderAppend(&query, "%s", "SELECT datname FROM pg_catalog.pg_database"))
+      return ADBC_STATUS_INTERNAL;
+
+    PqResultHelper result_helper = PqResultHelper{conn, query.buffer};
+    StringBuilderReset(&query);
+    pg_result* result = result_helper.Execute();
+
+    ExecStatusType pq_status = PQresultStatus(result);
+    if (pq_status == PGRES_TUPLES_OK) {
+      int num_rows = PQntuples(result);
+      array->length = num_rows - 1;  // makes ArrowArrayFinishElement happy, but why?

Review Comment:
   Wasn't sure about this but took inspiration from a comment I saw by @paleolimbot  https://github.com/apache/arrow-nanoarrow/issues/67#issuecomment-1324185749
   
   Had to do the `-1` to get it to work with the validation checks in `ArrowArrayFinishBuildingDefault`, but admittedly don't understand the why of 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