You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by li...@apache.org on 2023/05/09 17:04:08 UTC

[arrow-adbc] branch main updated: feat(c/driver/postgresql): Implement Postgres GetInfo (#658)

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

lidavidm 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 eb695e6  feat(c/driver/postgresql): Implement Postgres GetInfo (#658)
eb695e6 is described below

commit eb695e6a95e332f0212b4419d6c5bb88e36253f5
Author: William Ayd <wi...@icloud.com>
AuthorDate: Tue May 9 10:04:02 2023 -0700

    feat(c/driver/postgresql): Implement Postgres GetInfo (#658)
---
 c/driver/common/utils.c                | 83 +++++++++++++++++++++++++++++
 c/driver/common/utils.h                | 15 ++++++
 c/driver/postgresql/connection.cc      | 77 +++++++++++++++++++++++++++
 c/driver/postgresql/connection.h       |  3 ++
 c/driver/postgresql/postgresql.cc      |  5 +-
 c/driver/postgresql/postgresql_test.cc | 67 +++++++++++++++++++++++-
 c/driver/sqlite/sqlite.c               | 96 +++++-----------------------------
 c/driver/sqlite/sqlite_test.cc         | 62 ++++++++++++++++++++++
 8 files changed, 323 insertions(+), 85 deletions(-)

diff --git a/c/driver/common/utils.c b/c/driver/common/utils.c
index 26c8b9d..a8a4a1c 100644
--- a/c/driver/common/utils.c
+++ b/c/driver/common/utils.c
@@ -160,3 +160,86 @@ void StringBuilderReset(struct StringBuilder* builder) {
   }
   memset(builder, 0, sizeof(*builder));
 }
+
+AdbcStatusCode AdbcInitConnectionGetInfoSchema(const uint32_t* info_codes,
+                                               size_t info_codes_length,
+                                               struct ArrowSchema* schema,
+                                               struct ArrowArray* array,
+                                               struct AdbcError* error) {
+  // TODO: use C equivalent of UniqueSchema to avoid incomplete schema
+  // on error
+  ArrowSchemaInit(schema);
+  CHECK_NA(INTERNAL, ArrowSchemaSetTypeStruct(schema, /*num_columns=*/2), error);
+
+  CHECK_NA(INTERNAL, ArrowSchemaSetType(schema->children[0], NANOARROW_TYPE_UINT32),
+           error);
+  CHECK_NA(INTERNAL, ArrowSchemaSetName(schema->children[0], "info_name"), error);
+  schema->children[0]->flags &= ~ARROW_FLAG_NULLABLE;
+
+  struct ArrowSchema* info_value = schema->children[1];
+  CHECK_NA(INTERNAL, ArrowSchemaSetTypeUnion(info_value, NANOARROW_TYPE_DENSE_UNION, 6),
+           error);
+  CHECK_NA(INTERNAL, ArrowSchemaSetName(info_value, "info_value"), error);
+
+  CHECK_NA(INTERNAL, ArrowSchemaSetType(info_value->children[0], NANOARROW_TYPE_STRING),
+           error);
+  CHECK_NA(INTERNAL, ArrowSchemaSetName(info_value->children[0], "string_value"), error);
+  CHECK_NA(INTERNAL, ArrowSchemaSetType(info_value->children[1], NANOARROW_TYPE_BOOL),
+           error);
+  CHECK_NA(INTERNAL, ArrowSchemaSetName(info_value->children[1], "bool_value"), error);
+  CHECK_NA(INTERNAL, ArrowSchemaSetType(info_value->children[2], NANOARROW_TYPE_INT64),
+           error);
+  CHECK_NA(INTERNAL, ArrowSchemaSetName(info_value->children[2], "int64_value"), error);
+  CHECK_NA(INTERNAL, ArrowSchemaSetType(info_value->children[3], NANOARROW_TYPE_INT32),
+           error);
+  CHECK_NA(INTERNAL, ArrowSchemaSetName(info_value->children[3], "int32_bitmask"), error);
+  CHECK_NA(INTERNAL, ArrowSchemaSetType(info_value->children[4], NANOARROW_TYPE_LIST),
+           error);
+  CHECK_NA(INTERNAL, ArrowSchemaSetName(info_value->children[4], "string_list"), error);
+  CHECK_NA(INTERNAL, ArrowSchemaSetType(info_value->children[5], NANOARROW_TYPE_MAP),
+           error);
+  CHECK_NA(INTERNAL,
+           ArrowSchemaSetName(info_value->children[5], "int32_to_int32_list_map"), error);
+
+  CHECK_NA(
+      INTERNAL,
+      ArrowSchemaSetType(info_value->children[4]->children[0], NANOARROW_TYPE_STRING),
+      error);
+
+  CHECK_NA(INTERNAL,
+           ArrowSchemaSetType(info_value->children[5]->children[0]->children[0],
+                              NANOARROW_TYPE_INT32),
+           error);
+  info_value->children[5]->children[0]->children[0]->flags &= ~ARROW_FLAG_NULLABLE;
+  CHECK_NA(INTERNAL,
+           ArrowSchemaSetType(info_value->children[5]->children[0]->children[1],
+                              NANOARROW_TYPE_LIST),
+           error);
+  CHECK_NA(
+      INTERNAL,
+      ArrowSchemaSetType(info_value->children[5]->children[0]->children[1]->children[0],
+                         NANOARROW_TYPE_INT32),
+      error);
+
+  struct ArrowError na_error = {0};
+  CHECK_NA_DETAIL(INTERNAL, ArrowArrayInitFromSchema(array, schema, &na_error), &na_error,
+                  error);
+  CHECK_NA(INTERNAL, ArrowArrayStartAppending(array), error);
+
+  return ADBC_STATUS_OK;
+}  // NOLINT(whitespace/indent)
+
+AdbcStatusCode AdbcConnectionGetInfoAppendString(struct ArrowArray* array,
+                                                 uint32_t info_code,
+                                                 const char* info_value,
+                                                 struct AdbcError* error) {
+  CHECK_NA(INTERNAL, ArrowArrayAppendUInt(array->children[0], info_code), error);
+  // Append to type variant
+  struct ArrowStringView value = ArrowCharView(info_value);
+  CHECK_NA(INTERNAL, ArrowArrayAppendString(array->children[1]->children[0], value),
+           error);
+  // Append type code/offset
+  CHECK_NA(INTERNAL, ArrowArrayFinishUnionElement(array->children[1], /*type_id=*/0),
+           error);
+  return ADBC_STATUS_OK;
+}
diff --git a/c/driver/common/utils.h b/c/driver/common/utils.h
index 0223dca..21211f9 100644
--- a/c/driver/common/utils.h
+++ b/c/driver/common/utils.h
@@ -104,6 +104,21 @@ void StringBuilderReset(struct StringBuilder* builder);
     if (adbc_status_code != ADBC_STATUS_OK) return adbc_status_code; \
   } while (0)
 
+/// \defgroup adbc-connection-utils Connection Utilities
+/// Utilities for implementing connection-related functions for drivers
+///
+/// @{
+AdbcStatusCode AdbcInitConnectionGetInfoSchema(const uint32_t* info_codes,
+                                               size_t info_codes_length,
+                                               struct ArrowSchema* schema,
+                                               struct ArrowArray* array,
+                                               struct AdbcError* error);
+AdbcStatusCode AdbcConnectionGetInfoAppendString(struct ArrowArray* array,
+                                                 uint32_t info_code,
+                                                 const char* info_value,
+                                                 struct AdbcError* error);
+/// @}
+
 #ifdef __cplusplus
 }
 #endif
diff --git a/c/driver/postgresql/connection.cc b/c/driver/postgresql/connection.cc
index 0513b13..0ae2836 100644
--- a/c/driver/postgresql/connection.cc
+++ b/c/driver/postgresql/connection.cc
@@ -29,6 +29,12 @@
 #include "utils.h"
 
 namespace {
+
+static const uint32_t kSupportedInfoCodes[] = {
+    ADBC_INFO_VENDOR_NAME,    ADBC_INFO_VENDOR_VERSION,       ADBC_INFO_DRIVER_NAME,
+    ADBC_INFO_DRIVER_VERSION, ADBC_INFO_DRIVER_ARROW_VERSION,
+};
+
 class PqResultHelper {
  public:
   PqResultHelper(PGconn* conn, const char* query) : conn_(conn) {
@@ -48,6 +54,7 @@ class PqResultHelper {
   PGconn* conn_;
   std::string query_;
 };
+
 }  // namespace
 
 namespace adbcpq {
@@ -68,6 +75,76 @@ AdbcStatusCode PostgresConnection::Commit(struct AdbcError* error) {
   return ADBC_STATUS_OK;
 }
 
+AdbcStatusCode PostgresConnectionGetInfoImpl(const uint32_t* info_codes,
+                                             size_t info_codes_length,
+                                             struct ArrowSchema* schema,
+                                             struct ArrowArray* array,
+                                             struct AdbcError* error) {
+  RAISE_ADBC(AdbcInitConnectionGetInfoSchema(info_codes, info_codes_length, schema, array,
+                                             error));
+
+  for (size_t i = 0; i < info_codes_length; i++) {
+    switch (info_codes[i]) {
+      case ADBC_INFO_VENDOR_NAME:
+        RAISE_ADBC(
+            AdbcConnectionGetInfoAppendString(array, info_codes[i], "PostgreSQL", error));
+        break;
+      case ADBC_INFO_VENDOR_VERSION:
+        RAISE_ADBC(AdbcConnectionGetInfoAppendString(
+            array, info_codes[i], std::to_string(PQlibVersion()).c_str(), error));
+        break;
+      case ADBC_INFO_DRIVER_NAME:
+        RAISE_ADBC(AdbcConnectionGetInfoAppendString(array, info_codes[i],
+                                                     "ADBC PostgreSQL Driver", error));
+        break;
+      case ADBC_INFO_DRIVER_VERSION:
+        // TODO(lidavidm): fill in driver version
+        RAISE_ADBC(
+            AdbcConnectionGetInfoAppendString(array, info_codes[i], "(unknown)", error));
+        break;
+      case ADBC_INFO_DRIVER_ARROW_VERSION:
+        RAISE_ADBC(AdbcConnectionGetInfoAppendString(array, info_codes[i],
+                                                     NANOARROW_VERSION, error));
+        break;
+      default:
+        // Ignore
+        continue;
+    }
+    CHECK_NA(INTERNAL, ArrowArrayFinishElement(array), error);
+  }
+
+  struct ArrowError na_error = {0};
+  CHECK_NA_DETAIL(INTERNAL, ArrowArrayFinishBuildingDefault(array, &na_error), &na_error,
+                  error);
+
+  return ADBC_STATUS_OK;
+}
+
+AdbcStatusCode PostgresConnection::GetInfo(struct AdbcConnection* connection,
+                                           uint32_t* info_codes, size_t info_codes_length,
+                                           struct ArrowArrayStream* out,
+                                           struct AdbcError* error) {
+  // XXX: mistake in adbc.h (should have been const pointer)
+  const uint32_t* codes = info_codes;
+  if (!info_codes) {
+    codes = kSupportedInfoCodes;
+    info_codes_length = sizeof(kSupportedInfoCodes) / sizeof(kSupportedInfoCodes[0]);
+  }
+
+  struct ArrowSchema schema = {0};
+  struct ArrowArray array = {0};
+
+  AdbcStatusCode status =
+      PostgresConnectionGetInfoImpl(codes, info_codes_length, &schema, &array, error);
+  if (status != ADBC_STATUS_OK) {
+    if (schema.release) schema.release(&schema);
+    if (array.release) array.release(&array);
+    return status;
+  }
+
+  return BatchToArrayStream(&array, &schema, out, error);
+}
+
 AdbcStatusCode PostgresConnection::GetTableSchema(const char* catalog,
                                                   const char* db_schema,
                                                   const char* table_name,
diff --git a/c/driver/postgresql/connection.h b/c/driver/postgresql/connection.h
index 6f63d66..c4218e7 100644
--- a/c/driver/postgresql/connection.h
+++ b/c/driver/postgresql/connection.h
@@ -32,6 +32,9 @@ class PostgresConnection {
   PostgresConnection() : database_(nullptr), conn_(nullptr), autocommit_(true) {}
 
   AdbcStatusCode Commit(struct AdbcError* error);
+  AdbcStatusCode GetInfo(struct AdbcConnection* connection, uint32_t* info_codes,
+                         size_t info_codes_length, struct ArrowArrayStream* out,
+                         struct AdbcError* error);
   AdbcStatusCode GetTableSchema(const char* catalog, const char* db_schema,
                                 const char* table_name, struct ArrowSchema* schema,
                                 struct AdbcError* error);
diff --git a/c/driver/postgresql/postgresql.cc b/c/driver/postgresql/postgresql.cc
index 1305f19..d46e0b8 100644
--- a/c/driver/postgresql/postgresql.cc
+++ b/c/driver/postgresql/postgresql.cc
@@ -125,7 +125,10 @@ AdbcStatusCode PostgresConnectionGetInfo(struct AdbcConnection* connection,
                                          uint32_t* info_codes, size_t info_codes_length,
                                          struct ArrowArrayStream* stream,
                                          struct AdbcError* error) {
-  return ADBC_STATUS_NOT_IMPLEMENTED;
+  if (!connection->private_data) return ADBC_STATUS_INVALID_STATE;
+  auto ptr =
+      reinterpret_cast<std::shared_ptr<PostgresConnection>*>(connection->private_data);
+  return (*ptr)->GetInfo(connection, info_codes, info_codes_length, stream, error);
 }
 
 AdbcStatusCode PostgresConnectionGetObjects(
diff --git a/c/driver/postgresql/postgresql_test.cc b/c/driver/postgresql/postgresql_test.cc
index f86c820..a5613d7 100644
--- a/c/driver/postgresql/postgresql_test.cc
+++ b/c/driver/postgresql/postgresql_test.cc
@@ -86,7 +86,6 @@ class PostgresConnectionTest : public ::testing::Test,
   void SetUp() override { ASSERT_NO_FATAL_FAILURE(SetUpTest()); }
   void TearDown() override { ASSERT_NO_FATAL_FAILURE(TearDownTest()); }
 
-  void TestMetadataGetInfo() { GTEST_SKIP() << "Not yet implemented"; }
   void TestMetadataGetTableTypes() { GTEST_SKIP() << "Not yet implemented"; }
 
   void TestMetadataGetObjectsCatalogs() { GTEST_SKIP() << "Not yet implemented"; }
@@ -99,6 +98,72 @@ class PostgresConnectionTest : public ::testing::Test,
   PostgresQuirks quirks_;
 };
 
+TEST_F(PostgresConnectionTest, GetInfoMetadata) {
+  ASSERT_THAT(AdbcConnectionNew(&connection, &error), IsOkStatus(&error));
+  ASSERT_THAT(AdbcConnectionInit(&connection, &database, &error), IsOkStatus(&error));
+
+  adbc_validation::StreamReader reader;
+  std::vector<uint32_t> info = {
+      ADBC_INFO_DRIVER_NAME,
+      ADBC_INFO_DRIVER_VERSION,
+      ADBC_INFO_VENDOR_NAME,
+      ADBC_INFO_VENDOR_VERSION,
+  };
+  ASSERT_THAT(AdbcConnectionGetInfo(&connection, info.data(), info.size(),
+                                    &reader.stream.value, &error),
+              IsOkStatus(&error));
+  ASSERT_NO_FATAL_FAILURE(reader.GetSchema());
+
+  std::vector<uint32_t> seen;
+  while (true) {
+    ASSERT_NO_FATAL_FAILURE(reader.Next());
+    if (!reader.array->release) break;
+
+    for (int64_t row = 0; row < reader.array->length; row++) {
+      ASSERT_FALSE(ArrowArrayViewIsNull(reader.array_view->children[0], row));
+      const uint32_t code =
+          reader.array_view->children[0]->buffer_views[1].data.as_uint32[row];
+      seen.push_back(code);
+
+      int str_child_index = 0;
+      struct ArrowArrayView* str_child =
+          reader.array_view->children[1]->children[str_child_index];
+      switch (code) {
+        case ADBC_INFO_DRIVER_NAME: {
+          ArrowStringView val = ArrowArrayViewGetStringUnsafe(str_child, 0);
+          EXPECT_EQ("ADBC PostgreSQL Driver", std::string(val.data, val.size_bytes));
+          break;
+        }
+        case ADBC_INFO_DRIVER_VERSION: {
+          ArrowStringView val = ArrowArrayViewGetStringUnsafe(str_child, 1);
+          EXPECT_EQ("(unknown)", std::string(val.data, val.size_bytes));
+          break;
+        }
+        case ADBC_INFO_VENDOR_NAME: {
+          ArrowStringView val = ArrowArrayViewGetStringUnsafe(str_child, 2);
+          EXPECT_EQ("PostgreSQL", std::string(val.data, val.size_bytes));
+          break;
+        }
+        case ADBC_INFO_VENDOR_VERSION: {
+          ArrowStringView val = ArrowArrayViewGetStringUnsafe(str_child, 3);
+#ifdef __WIN32
+          const char* pater = "\\d\\d\\d\\d\\d\\d";
+#else
+          const char* pater = "[0-9]{6}";
+#endif
+          EXPECT_THAT(std::string(val.data, val.size_bytes),
+                      ::testing::MatchesRegex(pater));
+          break;
+        }
+        default:
+          // Ignored
+          break;
+      }
+    }
+  }
+  ASSERT_THAT(seen, ::testing::UnorderedElementsAreArray(info));
+}
+
 TEST_F(PostgresConnectionTest, MetadataGetTableSchemaInjection) {
   if (!quirks()->supports_bulk_ingest()) {
     GTEST_SKIP();
diff --git a/c/driver/sqlite/sqlite.c b/c/driver/sqlite/sqlite.c
index 43f9d98..353fb35 100644
--- a/c/driver/sqlite/sqlite.c
+++ b/c/driver/sqlite/sqlite.c
@@ -237,106 +237,35 @@ AdbcStatusCode SqliteConnectionRelease(struct AdbcConnection* connection,
   return ADBC_STATUS_OK;
 }
 
-AdbcStatusCode SqliteConnectionGetInfoAppendStringImpl(struct ArrowArray* array,
-                                                       uint32_t info_code,
-                                                       const char* info_value,
-                                                       struct AdbcError* error) {
-  CHECK_NA(INTERNAL, ArrowArrayAppendUInt(array->children[0], info_code), error);
-  // Append to type variant
-  struct ArrowStringView value = ArrowCharView(info_value);
-  CHECK_NA(INTERNAL, ArrowArrayAppendString(array->children[1]->children[0], value),
-           error);
-  // Append type code/offset
-  CHECK_NA(INTERNAL, ArrowArrayFinishUnionElement(array->children[1], /*type_id=*/0),
-           error);
-  return ADBC_STATUS_OK;
-}
-
 AdbcStatusCode SqliteConnectionGetInfoImpl(const uint32_t* info_codes,
                                            size_t info_codes_length,
                                            struct ArrowSchema* schema,
                                            struct ArrowArray* array,
                                            struct AdbcError* error) {
-  ArrowSchemaInit(schema);
-  CHECK_NA(INTERNAL, ArrowSchemaSetTypeStruct(schema, /*num_columns=*/2), error);
-
-  CHECK_NA(INTERNAL, ArrowSchemaSetType(schema->children[0], NANOARROW_TYPE_UINT32),
-           error);
-  CHECK_NA(INTERNAL, ArrowSchemaSetName(schema->children[0], "info_name"), error);
-  schema->children[0]->flags &= ~ARROW_FLAG_NULLABLE;
-
-  struct ArrowSchema* info_value = schema->children[1];
-  CHECK_NA(INTERNAL, ArrowSchemaSetTypeUnion(info_value, NANOARROW_TYPE_DENSE_UNION, 6),
-           error);
-  CHECK_NA(INTERNAL, ArrowSchemaSetName(info_value, "info_value"), error);
-
-  CHECK_NA(INTERNAL, ArrowSchemaSetType(info_value->children[0], NANOARROW_TYPE_STRING),
-           error);
-  CHECK_NA(INTERNAL, ArrowSchemaSetName(info_value->children[0], "string_value"), error);
-  CHECK_NA(INTERNAL, ArrowSchemaSetType(info_value->children[1], NANOARROW_TYPE_BOOL),
-           error);
-  CHECK_NA(INTERNAL, ArrowSchemaSetName(info_value->children[1], "bool_value"), error);
-  CHECK_NA(INTERNAL, ArrowSchemaSetType(info_value->children[2], NANOARROW_TYPE_INT64),
-           error);
-  CHECK_NA(INTERNAL, ArrowSchemaSetName(info_value->children[2], "int64_value"), error);
-  CHECK_NA(INTERNAL, ArrowSchemaSetType(info_value->children[3], NANOARROW_TYPE_INT32),
-           error);
-  CHECK_NA(INTERNAL, ArrowSchemaSetName(info_value->children[3], "int32_bitmask"), error);
-  CHECK_NA(INTERNAL, ArrowSchemaSetType(info_value->children[4], NANOARROW_TYPE_LIST),
-           error);
-  CHECK_NA(INTERNAL, ArrowSchemaSetName(info_value->children[4], "string_list"), error);
-  CHECK_NA(INTERNAL, ArrowSchemaSetType(info_value->children[5], NANOARROW_TYPE_MAP),
-           error);
-  CHECK_NA(INTERNAL,
-           ArrowSchemaSetName(info_value->children[5], "int32_to_int32_list_map"), error);
-
-  CHECK_NA(
-      INTERNAL,
-      ArrowSchemaSetType(info_value->children[4]->children[0], NANOARROW_TYPE_STRING),
-      error);
-
-  CHECK_NA(INTERNAL,
-           ArrowSchemaSetType(info_value->children[5]->children[0]->children[0],
-                              NANOARROW_TYPE_INT32),
-           error);
-  info_value->children[5]->children[0]->children[0]->flags &= ~ARROW_FLAG_NULLABLE;
-  CHECK_NA(INTERNAL,
-           ArrowSchemaSetType(info_value->children[5]->children[0]->children[1],
-                              NANOARROW_TYPE_LIST),
-           error);
-  CHECK_NA(
-      INTERNAL,
-      ArrowSchemaSetType(info_value->children[5]->children[0]->children[1]->children[0],
-                         NANOARROW_TYPE_INT32),
-      error);
-
-  struct ArrowError na_error = {0};
-  CHECK_NA_DETAIL(INTERNAL, ArrowArrayInitFromSchema(array, schema, &na_error), &na_error,
-                  error);
-  CHECK_NA(INTERNAL, ArrowArrayStartAppending(array), error);
-
+  RAISE_ADBC(AdbcInitConnectionGetInfoSchema(info_codes, info_codes_length, schema, array,
+                                             error));
   for (size_t i = 0; i < info_codes_length; i++) {
     switch (info_codes[i]) {
       case ADBC_INFO_VENDOR_NAME:
-        RAISE_ADBC(SqliteConnectionGetInfoAppendStringImpl(array, info_codes[i], "SQLite",
-                                                           error));
+        RAISE_ADBC(
+            AdbcConnectionGetInfoAppendString(array, info_codes[i], "SQLite", error));
         break;
       case ADBC_INFO_VENDOR_VERSION:
-        RAISE_ADBC(SqliteConnectionGetInfoAppendStringImpl(array, info_codes[i],
-                                                           sqlite3_libversion(), error));
+        RAISE_ADBC(AdbcConnectionGetInfoAppendString(array, info_codes[i],
+                                                     sqlite3_libversion(), error));
         break;
       case ADBC_INFO_DRIVER_NAME:
-        RAISE_ADBC(SqliteConnectionGetInfoAppendStringImpl(array, info_codes[i],
-                                                           "ADBC SQLite Driver", error));
+        RAISE_ADBC(AdbcConnectionGetInfoAppendString(array, info_codes[i],
+                                                     "ADBC SQLite Driver", error));
         break;
       case ADBC_INFO_DRIVER_VERSION:
         // TODO(lidavidm): fill in driver version
-        RAISE_ADBC(SqliteConnectionGetInfoAppendStringImpl(array, info_codes[i],
-                                                           "(unknown)", error));
+        RAISE_ADBC(
+            AdbcConnectionGetInfoAppendString(array, info_codes[i], "(unknown)", error));
         break;
       case ADBC_INFO_DRIVER_ARROW_VERSION:
-        RAISE_ADBC(SqliteConnectionGetInfoAppendStringImpl(array, info_codes[i],
-                                                           NANOARROW_VERSION, error));
+        RAISE_ADBC(AdbcConnectionGetInfoAppendString(array, info_codes[i],
+                                                     NANOARROW_VERSION, error));
         break;
       default:
         // Ignore
@@ -345,6 +274,7 @@ AdbcStatusCode SqliteConnectionGetInfoImpl(const uint32_t* info_codes,
     CHECK_NA(INTERNAL, ArrowArrayFinishElement(array), error);
   }
 
+  struct ArrowError na_error = {0};
   CHECK_NA_DETAIL(INTERNAL, ArrowArrayFinishBuildingDefault(array, &na_error), &na_error,
                   error);
 
diff --git a/c/driver/sqlite/sqlite_test.cc b/c/driver/sqlite/sqlite_test.cc
index 7e7d840..a708885 100644
--- a/c/driver/sqlite/sqlite_test.cc
+++ b/c/driver/sqlite/sqlite_test.cc
@@ -90,6 +90,68 @@ class SqliteConnectionTest : public ::testing::Test,
 };
 ADBCV_TEST_CONNECTION(SqliteConnectionTest)
 
+TEST_F(SqliteConnectionTest, GetInfoMetadata) {
+  ASSERT_THAT(AdbcConnectionNew(&connection, &error),
+              adbc_validation::IsOkStatus(&error));
+  ASSERT_THAT(AdbcConnectionInit(&connection, &database, &error),
+              adbc_validation::IsOkStatus(&error));
+
+  adbc_validation::StreamReader reader;
+  std::vector<uint32_t> info = {
+      ADBC_INFO_DRIVER_NAME,
+      ADBC_INFO_DRIVER_VERSION,
+      ADBC_INFO_VENDOR_NAME,
+      ADBC_INFO_VENDOR_VERSION,
+  };
+  ASSERT_THAT(AdbcConnectionGetInfo(&connection, info.data(), info.size(),
+                                    &reader.stream.value, &error),
+              adbc_validation::IsOkStatus(&error));
+  ASSERT_NO_FATAL_FAILURE(reader.GetSchema());
+
+  std::vector<uint32_t> seen;
+  while (true) {
+    ASSERT_NO_FATAL_FAILURE(reader.Next());
+    if (!reader.array->release) break;
+
+    for (int64_t row = 0; row < reader.array->length; row++) {
+      ASSERT_FALSE(ArrowArrayViewIsNull(reader.array_view->children[0], row));
+      const uint32_t code =
+          reader.array_view->children[0]->buffer_views[1].data.as_uint32[row];
+      seen.push_back(code);
+
+      int str_child_index = 0;
+      struct ArrowArrayView* str_child =
+          reader.array_view->children[1]->children[str_child_index];
+      switch (code) {
+        case ADBC_INFO_DRIVER_NAME: {
+          ArrowStringView val = ArrowArrayViewGetStringUnsafe(str_child, 0);
+          EXPECT_EQ("ADBC SQLite Driver", std::string(val.data, val.size_bytes));
+          break;
+        }
+        case ADBC_INFO_DRIVER_VERSION: {
+          ArrowStringView val = ArrowArrayViewGetStringUnsafe(str_child, 1);
+          EXPECT_EQ("(unknown)", std::string(val.data, val.size_bytes));
+          break;
+        }
+        case ADBC_INFO_VENDOR_NAME: {
+          ArrowStringView val = ArrowArrayViewGetStringUnsafe(str_child, 2);
+          EXPECT_EQ("SQLite", std::string(val.data, val.size_bytes));
+          break;
+        }
+        case ADBC_INFO_VENDOR_VERSION: {
+          ArrowStringView val = ArrowArrayViewGetStringUnsafe(str_child, 3);
+          EXPECT_THAT(std::string(val.data, val.size_bytes),
+                      ::testing::MatchesRegex("3\\..*"));
+        }
+        default:
+          // Ignored
+          break;
+      }
+    }
+  }
+  ASSERT_THAT(seen, ::testing::UnorderedElementsAreArray(info));
+}
+
 class SqliteStatementTest : public ::testing::Test,
                             public adbc_validation::StatementTest {
  public: