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 2022/06/02 17:42:01 UTC

[arrow-adbc] branch main updated: Get everything building again (#3)

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 2457bfd  Get everything building again (#3)
2457bfd is described below

commit 2457bfdf729439105f6a47031adf2495b3e438ed
Author: David Li <li...@gmail.com>
AuthorDate: Thu Jun 2 13:41:56 2022 -0400

    Get everything building again (#3)
---
 .pre-commit-config.yaml                            |   2 +-
 README.md                                          |  15 ++
 adbc.h                                             |   4 +
 adbc_driver_manager/CMakeLists.txt                 |  47 ++---
 adbc_driver_manager/adbc_driver_manager_test.cc    | 194 ++++++++++++++-------
 .../AdbcDefines.cmake                              |  32 ++--
 drivers/flight_sql/CMakeLists.txt                  |  58 ++++++
 drivers/flight_sql/flight_sql.cc                   |  26 +--
 drivers/flight_sql/flight_sql_test.cc              | 130 ++++++--------
 .../sqlite}/CMakeLists.txt                         |  56 +++---
 drivers/sqlite/sqlite.cc                           |  17 +-
 drivers/sqlite/sqlite_test.cc                      | 172 +++++++-----------
 drivers/sqlite/test_sqlite_internal.cc             |  30 ----
 drivers/sqlite/test_sqlite_internal.h              |  32 ----
 drivers/test_util.h                                |  40 +++--
 drivers/util.cc                                    |   6 +-
 16 files changed, 435 insertions(+), 426 deletions(-)

diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index f8c9ab1..70bf269 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -31,4 +31,4 @@ repos:
     rev: v0.6.13
     hooks:
     - id: cmake-format
-      args: [--in-place, --config-files, .cmake-format]
+      args: [--in-place]
diff --git a/README.md b/README.md
index 4ba6cff..a268a0f 100644
--- a/README.md
+++ b/README.md
@@ -22,3 +22,18 @@
 [![License](http://img.shields.io/:license-Apache%202-blue.svg)](https://github.com/apache/arrow-adbc/blob/master/LICENSE.txt)
 
 EXPERIMENTAL. Please see the [mailing list discussion](https://lists.apache.org/thread/gnz1kz2rj3rb8rh8qz7l0mv8lvzq254w).
+
+## Building
+
+The libraries here are all **individual** CMake projects.
+
+```shell
+$ mkdir -p build/driver_manager
+$ cd build/driver_manager
+$ cmake ../../adbc_driver_manager
+$ make
+```
+
+Some of Arrow's build options are supported:
+
+- `ARROW_BUILD_TESTS`: build the unit tests
diff --git a/adbc.h b/adbc.h
index 1aa7b47..2e2df69 100644
--- a/adbc.h
+++ b/adbc.h
@@ -24,6 +24,9 @@
 extern "C" {
 #endif
 
+// Extra guard for versions of Arrow without the canonical guard
+#ifndef ARROW_FLAG_DICTIONARY_ORDERED
+
 #ifndef ARROW_C_DATA_INTERFACE
 #define ARROW_C_DATA_INTERFACE
 
@@ -108,6 +111,7 @@ struct ArrowArrayStream {
 };
 
 #endif  // ARROW_C_STREAM_INTERFACE
+#endif  // ARROW_FLAG_DICTIONARY_ORDERED
 
 #ifndef ADBC
 #define ADBC
diff --git a/adbc_driver_manager/CMakeLists.txt b/adbc_driver_manager/CMakeLists.txt
index ed41526..dfeee14 100644
--- a/adbc_driver_manager/CMakeLists.txt
+++ b/adbc_driver_manager/CMakeLists.txt
@@ -16,32 +16,35 @@
 # under the License.
 
 cmake_minimum_required(VERSION 3.10)
-
-set(ARROW_VERSION "9.0.0-SNAPSHOT")
-set(ARROW_BASE_VERSION "9.0.0")
-set(ARROW_VERSION_MAJOR "9")
-set(ARROW_VERSION_MINOR "0")
-set(ARROW_VERSION_PATCH "0")
-
-math(EXPR ARROW_SO_VERSION "${ARROW_VERSION_MAJOR} * 100 + ${ARROW_VERSION_MINOR}")
-set(ARROW_FULL_SO_VERSION "${ARROW_SO_VERSION}.${ARROW_VERSION_PATCH}.0")
-
-project(adbc_driver_manager VERSION "${ARROW_BASE_VERSION}" LANGUAGES CXX)
-
-# Common CMake utilities
-get_filename_component(REPOSITORY_ROOT ${CMAKE_SOURCE_DIR} DIRECTORY)
+get_filename_component(REPOSITORY_ROOT ".." ABSOLUTE)
+message(STATUS "${REPOSITORY_ROOT}")
 list(APPEND CMAKE_MODULE_PATH "${REPOSITORY_ROOT}/cmake_modules/")
+include(AdbcDefines)
 include(BuildUtils)
 include(DefineOptions)
 
-# Driver manager library
-add_arrow_lib(adbc_driver_manager
-  SOURCES
-  adbc_driver_manager.cc)
-include_directories(adbc_driver_manager SYSTEM
-  PRIVATE ${REPOSITORY_ROOT})
-
-# Tests
+project(adbc_driver_manager
+        VERSION "${ARROW_BASE_VERSION}"
+        LANGUAGES CXX)
+add_arrow_lib(adbc_driver_manager SOURCES adbc_driver_manager.cc)
+include_directories(SYSTEM ${REPOSITORY_ROOT})
+
+if(ARROW_BUILD_TESTS)
+  find_package(Arrow REQUIRED)
+  if(ARROW_TEST_LINKAGE STREQUAL "shared")
+    set(TEST_LINK_LIBS adbc_driver_manager_shared arrow_shared)
+  else()
+    set(TEST_LINK_LIBS adbc_driver_manager_static arrow_static)
+  endif()
+
+  add_test_case(driver_manager_test
+                PREFIX
+                adbc
+                SOURCES
+                adbc_driver_manager_test.cc
+                EXTRA_LINK_LIBS
+                ${TEST_LINK_LIBS})
+endif()
 
 validate_config()
 config_summary_message()
diff --git a/adbc_driver_manager/adbc_driver_manager_test.cc b/adbc_driver_manager/adbc_driver_manager_test.cc
index cc77958..5ba09af 100644
--- a/adbc_driver_manager/adbc_driver_manager_test.cc
+++ b/adbc_driver_manager/adbc_driver_manager_test.cc
@@ -15,89 +15,149 @@
 // specific language governing permissions and limitations
 // under the License.
 
-// Test the driver, but using the driver manager's stubs instead of
-// the function pointer table.
-
 #include <gmock/gmock.h>
 #include <gtest/gtest.h>
 
-#include "adbc/adbc.h"
-#include "adbc/adbc_driver_manager.h"
-#include "adbc/test_sqlite_internal.h"
-#include "adbc/test_util.h"
-#include "arrow/c/bridge.h"
-#include "arrow/record_batch.h"
-#include "arrow/testing/gtest_util.h"
-#include "arrow/testing/matchers.h"
+#include <arrow/c/bridge.h>
+#include <arrow/record_batch.h>
+#include <arrow/testing/matchers.h>
+
+#include "adbc.h"
+#include "adbc_driver_manager.h"
+#include "drivers/test_util.h"
+
+// Tests of the SQLite example driver, except using the driver manager
 
 namespace adbc {
 
 using arrow::PointeesEqual;
 
-TEST(AdbcDriverManager, Basics) {
-  AdbcDatabase database;
-  AdbcConnection connection;
-  AdbcError error = {};
-
-  AdbcDriver driver;
-  ASSERT_NO_FATAL_FAILURE(InitSqlite(&driver));
-
-  {
-    AdbcDatabaseOptions options;
-    std::memset(&options, 0, sizeof(options));
-    options.driver = &driver;
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              AdbcDatabaseInit(&options, &database, &error));
+class DriverManager : public ::testing::Test {
+ public:
+  void SetUp() override {
+    size_t initialized = 0;
+    ADBC_ASSERT_OK(
+        AdbcLoadDriver("Driver=libadbc_driver_sqlite.so;Entrypoint=AdbcSqliteDriverInit",
+                       ADBC_VERSION_0_0_1, &driver, &initialized));
+    ASSERT_EQ(initialized, ADBC_VERSION_0_0_1);
+
+    AdbcDatabaseOptions db_options;
+    std::memset(&db_options, 0, sizeof(db_options));
+    db_options.driver = &driver;
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcDatabaseInit(&db_options, &database, &error));
     ASSERT_NE(database.private_data, nullptr);
-  }
 
-  {
-    AdbcConnectionOptions options;
-    std::memset(&options, 0, sizeof(options));
-    options.database = &database;
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              AdbcConnectionInit(&options, &connection, &error));
+    AdbcConnectionOptions conn_options;
+    std::memset(&conn_options, 0, sizeof(conn_options));
+    conn_options.database = &database;
+    ADBC_ASSERT_OK_WITH_ERROR(error,
+                              AdbcConnectionInit(&conn_options, &connection, &error));
     ASSERT_NE(connection.private_data, nullptr);
   }
 
-  {
-    std::string query = "SELECT 1";
-    AdbcStatement statement;
-    std::memset(&statement, 0, sizeof(statement));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              AdbcStatementInit(&connection, &statement, &error));
-    ADBC_ASSERT_OK_WITH_ERROR(
-        &driver, error,
-        AdbcConnectionSqlExecute(&connection, query.c_str(), &statement, &error));
-
-    arrow::RecordBatchVector batches;
-    ArrowArrayStream stream;
-
-    ADBC_ASSERT_OK(AdbcStatementGetStream(&statement, &stream, &error));
-    ASSERT_OK_AND_ASSIGN(auto reader, arrow::ImportRecordBatchReader(&stream));
-
-    auto schema = reader->schema();
-    while (true) {
-      ASSERT_OK_AND_ASSIGN(auto batch, reader->Next());
-      if (!batch) break;
-      batches.push_back(std::move(batch));
-    }
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error, AdbcStatementRelease(&statement, &error));
-
-    arrow::AssertSchemaEqual(*schema,
-                             *arrow::schema({arrow::field("1", arrow::int64())}));
-    EXPECT_THAT(batches,
-                ::testing::UnorderedPointwise(
-                    PointeesEqual(), {
-                                         arrow::RecordBatchFromJSON(schema, "[[1]]"),
-                                     }));
+  void TearDown() override {
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionRelease(&connection, &error));
+    ASSERT_EQ(connection.private_data, nullptr);
+
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcDatabaseRelease(&database, &error));
+    ASSERT_EQ(database.private_data, nullptr);
   }
 
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error, AdbcConnectionRelease(&connection, &error));
-  ASSERT_EQ(connection.private_data, nullptr);
+ protected:
+  AdbcDriver driver;
+  AdbcDatabase database;
+  AdbcConnection connection;
+  AdbcError error = {};
+};
+
+TEST_F(DriverManager, SqlExecute) {
+  std::string query = "SELECT 1";
+  AdbcStatement statement;
+  std::memset(&statement, 0, sizeof(statement));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(
+      error, AdbcConnectionSqlExecute(&connection, query.c_str(), &statement, &error));
+
+  std::shared_ptr<arrow::Schema> schema;
+  arrow::RecordBatchVector batches;
+  ReadStatement(&statement, &schema, &batches);
+  ASSERT_SCHEMA_EQ(*schema, *arrow::schema({arrow::field("1", arrow::int64())}));
+  EXPECT_THAT(batches,
+              ::testing::UnorderedPointwise(
+                  PointeesEqual(), {
+                                       adbc::RecordBatchFromJSON(schema, "[[1]]"),
+                                   }));
+}
+
+TEST_F(DriverManager, SqlExecuteInvalid) {
+  GTEST_SKIP() << "AdbcError needs refactoring";
+
+  std::string query = "INVALID";
+  AdbcStatement statement;
+  std::memset(&statement, 0, sizeof(statement));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
+  ASSERT_NE(AdbcConnectionSqlExecute(&connection, query.c_str(), &statement, &error),
+            ADBC_STATUS_OK);
+  ADBC_ASSERT_ERROR_THAT(
+      error, ::testing::AllOf(::testing::HasSubstr("[SQLite3] sqlite3_prepare_v2:"),
+                              ::testing::HasSubstr("syntax error")));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementRelease(&statement, &error));
+}
+
+TEST_F(DriverManager, SqlPrepare) {
+  std::string query = "SELECT 1";
+  AdbcStatement statement;
+  std::memset(&statement, 0, sizeof(statement));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(
+      error, AdbcConnectionSqlPrepare(&connection, query.c_str(), &statement, &error));
+
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementExecute(&statement, &error));
+
+  std::shared_ptr<arrow::Schema> schema;
+  arrow::RecordBatchVector batches;
+  ASSERT_NO_FATAL_FAILURE(ReadStatement(&statement, &schema, &batches));
+  ASSERT_SCHEMA_EQ(*schema, *arrow::schema({arrow::field("1", arrow::int64())}));
+  EXPECT_THAT(batches,
+              ::testing::UnorderedPointwise(
+                  PointeesEqual(), {
+                                       adbc::RecordBatchFromJSON(schema, "[[1]]"),
+                                   }));
+}
 
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error, AdbcDatabaseRelease(&database, &error));
-  ASSERT_EQ(database.private_data, nullptr);
+TEST_F(DriverManager, SqlPrepareMultipleParams) {
+  auto param_schema = arrow::schema(
+      {arrow::field("1", arrow::int64()), arrow::field("2", arrow::utf8())});
+  std::string query = "SELECT ?, ?";
+  AdbcStatement statement;
+  ArrowArray export_params;
+  ArrowSchema export_schema;
+  std::memset(&statement, 0, sizeof(statement));
+
+  ASSERT_OK(ExportRecordBatch(
+      *adbc::RecordBatchFromJSON(param_schema, R"([[1, "foo"], [2, "bar"]])"),
+      &export_params));
+  ASSERT_OK(ExportSchema(*param_schema, &export_schema));
+
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(
+      error, AdbcConnectionSqlPrepare(&connection, query.c_str(), &statement, &error));
+
+  ADBC_ASSERT_OK_WITH_ERROR(
+      error, AdbcStatementBind(&statement, &export_params, &export_schema, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementExecute(&statement, &error));
+
+  std::shared_ptr<arrow::Schema> schema;
+  arrow::RecordBatchVector batches;
+  ASSERT_NO_FATAL_FAILURE(ReadStatement(&statement, &schema, &batches));
+  ASSERT_SCHEMA_EQ(*schema, *arrow::schema({arrow::field("?", arrow::int64()),
+                                            arrow::field("?", arrow::utf8())}));
+  EXPECT_THAT(batches, ::testing::UnorderedPointwise(
+                           PointeesEqual(),
+                           {
+                               adbc::RecordBatchFromJSON(schema, R"([[1, "foo"], [2,
+                        "bar"]])"),
+                           }));
 }
 
 }  // namespace adbc
diff --git a/adbc_driver_manager/CMakeLists.txt b/cmake_modules/AdbcDefines.cmake
similarity index 68%
copy from adbc_driver_manager/CMakeLists.txt
copy to cmake_modules/AdbcDefines.cmake
index ed41526..b73463f 100644
--- a/adbc_driver_manager/CMakeLists.txt
+++ b/cmake_modules/AdbcDefines.cmake
@@ -15,7 +15,10 @@
 # specific language governing permissions and limitations
 # under the License.
 
-cmake_minimum_required(VERSION 3.10)
+# Common definitions for the CMake projects in this repository.
+
+enable_language(C)
+include(DefineOptions)
 
 set(ARROW_VERSION "9.0.0-SNAPSHOT")
 set(ARROW_BASE_VERSION "9.0.0")
@@ -26,22 +29,13 @@ set(ARROW_VERSION_PATCH "0")
 math(EXPR ARROW_SO_VERSION "${ARROW_VERSION_MAJOR} * 100 + ${ARROW_VERSION_MINOR}")
 set(ARROW_FULL_SO_VERSION "${ARROW_SO_VERSION}.${ARROW_VERSION_PATCH}.0")
 
-project(adbc_driver_manager VERSION "${ARROW_BASE_VERSION}" LANGUAGES CXX)
-
-# Common CMake utilities
-get_filename_component(REPOSITORY_ROOT ${CMAKE_SOURCE_DIR} DIRECTORY)
-list(APPEND CMAKE_MODULE_PATH "${REPOSITORY_ROOT}/cmake_modules/")
-include(BuildUtils)
-include(DefineOptions)
-
-# Driver manager library
-add_arrow_lib(adbc_driver_manager
-  SOURCES
-  adbc_driver_manager.cc)
-include_directories(adbc_driver_manager SYSTEM
-  PRIVATE ${REPOSITORY_ROOT})
-
-# Tests
+if(ARROW_DEPENDENCY_SOURCE STREQUAL "CONDA")
+  message(STATUS "Adding \$CONDA_PREFIX to CMAKE_PREFIX_PATH")
+  list(APPEND CMAKE_PREFIX_PATH "$ENV{CONDA_PREFIX}")
+endif()
 
-validate_config()
-config_summary_message()
+if(ARROW_BUILD_TESTS)
+  add_custom_target(all-tests)
+  find_package(GTest)
+  set(ARROW_TEST_LINK_LIBS GTest::gtest_main GTest::gtest GTest::gmock)
+endif()
diff --git a/drivers/flight_sql/CMakeLists.txt b/drivers/flight_sql/CMakeLists.txt
new file mode 100644
index 0000000..fd1314b
--- /dev/null
+++ b/drivers/flight_sql/CMakeLists.txt
@@ -0,0 +1,58 @@
+# 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.
+
+cmake_minimum_required(VERSION 3.14)
+get_filename_component(REPOSITORY_ROOT "../.." ABSOLUTE)
+list(APPEND CMAKE_MODULE_PATH "${REPOSITORY_ROOT}/cmake_modules/")
+include(AdbcDefines)
+include(BuildUtils)
+
+project(adbc_driver_flight_sql
+        VERSION "${ARROW_BASE_VERSION}"
+        LANGUAGES CXX)
+
+find_package(Arrow REQUIRED)
+# ARROW-12175: Arrow libraries are not CMake components
+get_filename_component(ARROW_CONFIG_PATH ${Arrow_CONFIG} DIRECTORY)
+find_package(ArrowFlight REQUIRED HINTS ${ARROW_CONFIG_PATH})
+find_package(ArrowFlightSql REQUIRED HINTS ${ARROW_CONFIG_PATH})
+
+add_arrow_lib(adbc_driver_flight_sql
+              SOURCES
+              flight_sql.cc
+              ../util.cc
+              SHARED_LINK_LIBS
+              arrow_shared
+              arrow_flight_sql_shared)
+include_directories(SYSTEM ${REPOSITORY_ROOT})
+
+if(ARROW_TEST_LINKAGE STREQUAL "shared")
+  set(TEST_LINK_LIBS adbc_driver_flight_sql_shared)
+else()
+  set(TEST_LINK_LIBS adbc_driver_flight_sql_static)
+endif()
+
+add_test_case(driver_flight_sql_test
+              PREFIX
+              adbc
+              SOURCES
+              flight_sql_test.cc
+              EXTRA_LINK_LIBS
+              ${TEST_LINK_LIBS})
+
+validate_config()
+config_summary_message()
diff --git a/drivers/flight_sql/flight_sql.cc b/drivers/flight_sql/flight_sql.cc
index 03adf3f..5130d21 100644
--- a/drivers/flight_sql/flight_sql.cc
+++ b/drivers/flight_sql/flight_sql.cc
@@ -18,16 +18,16 @@
 #include <mutex>
 #include <string>
 
-#include "adbc/adbc.h"
-#include "adbc/driver/util.h"
-#include "arrow/c/bridge.h"
-#include "arrow/flight/client.h"
-#include "arrow/flight/sql/client.h"
-#include "arrow/record_batch.h"
-#include "arrow/result.h"
-#include "arrow/status.h"
-#include "arrow/util/string_builder.h"
-#include "arrow/util/string_view.h"
+#include <arrow/c/bridge.h>
+#include <arrow/flight/client.h>
+#include <arrow/flight/sql/client.h>
+#include <arrow/record_batch.h>
+#include <arrow/result.h>
+#include <arrow/status.h>
+#include <arrow/util/string_builder.h>
+#include <arrow/util/string_view.h>
+#include "adbc.h"
+#include "drivers/util.h"
 
 namespace flight = arrow::flight;
 namespace flightsql = arrow::flight::sql;
@@ -127,7 +127,7 @@ class FlightSqlStatementImpl : public arrow::RecordBatchReader {
   Status ReadNext(std::shared_ptr<arrow::RecordBatch>* batch) override {
     flight::FlightStreamChunk chunk;
     while (current_stream_ && !chunk.data) {
-      RETURN_NOT_OK(current_stream_->Next(&chunk));
+      ARROW_ASSIGN_OR_RAISE(chunk, current_stream_->Next());
       if (chunk.data) {
         *batch = chunk.data;
         break;
@@ -352,14 +352,14 @@ AdbcStatusCode AdbcDatabaseInit(const struct AdbcDatabaseOptions* options,
   }
 
   flight::Location location;
-  status = flight::Location::Parse(location_it->second, &location);
+  status = flight::Location::Parse(location_it->second).Value(&location);
   if (!status.ok()) {
     SetError(status, error);
     return ADBC_STATUS_INVALID_ARGUMENT;
   }
 
   std::unique_ptr<flight::FlightClient> flight_client;
-  status = flight::FlightClient::Connect(location, &flight_client);
+  status = flight::FlightClient::Connect(location).Value(&flight_client);
   if (!status.ok()) {
     SetError(status, error);
     return ADBC_STATUS_IO;
diff --git a/drivers/flight_sql/flight_sql_test.cc b/drivers/flight_sql/flight_sql_test.cc
index de4379e..04c8ce9 100644
--- a/drivers/flight_sql/flight_sql_test.cc
+++ b/drivers/flight_sql/flight_sql_test.cc
@@ -17,66 +17,46 @@
 
 #include <gmock/gmock.h>
 #include <gtest/gtest.h>
+#include <cstdlib>
 
-#include "adbc/adbc.h"
-#include "adbc/adbc_driver_manager.h"
-#include "adbc/test_util.h"
-#include "arrow/flight/sql/example/sqlite_server.h"
-#include "arrow/record_batch.h"
-#include "arrow/testing/gtest_util.h"
-#include "arrow/testing/matchers.h"
-#include "arrow/util/logging.h"
+#include <arrow/record_batch.h>
+#include <arrow/testing/gtest_util.h>
+#include <arrow/testing/matchers.h>
+#include <arrow/util/logging.h>
+
+#include "adbc.h"
+#include "drivers/test_util.h"
 
 namespace adbc {
 
 using arrow::PointeesEqual;
 
+static std::string kServerEnvVar = "ADBC_FLIGHT_SQL_LOCATION";
+
 class AdbcFlightSqlTest : public ::testing::Test {
  public:
   void SetUp() override {
-    {
-      arrow::flight::Location location;
-      ASSERT_OK(arrow::flight::Location::ForGrpcTcp("localhost", 0, &location));
-      arrow::flight::FlightServerOptions options(location);
-      ASSERT_OK_AND_ASSIGN(server,
-                           arrow::flight::sql::example::SQLiteFlightSqlServer::Create());
-      ASSERT_OK(server->Init(options));
-      ASSERT_GT(server->port(), 0);
-    }
-
-    {
-      size_t initialized = 0;
-      ADBC_ASSERT_OK(
-          AdbcLoadDriver("Driver=libadbc_driver_flight_sql.so;"
-                         "Entrypoint=AdbcFlightSqlDriverInit",
-                         ADBC_VERSION_0_0_1, &driver, &initialized));
-    }
-
-    {
-      AdbcDatabaseOptions options;
-      std::string target = "Location=grpc://localhost:" + std::to_string(server->port());
-      options.target = target.c_str();
-      ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                                driver.DatabaseInit(&options, &database, &error));
-    }
-
-    {
-      AdbcConnectionOptions options;
-      options.database = &database;
-      ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                                driver.ConnectionInit(&options, &connection, &error));
+    if (const char* location = std::getenv(kServerEnvVar.c_str())) {
+      AdbcDatabaseOptions db_options;
+      std::string target = "Location=";
+      target += location;
+      db_options.target = target.c_str();
+      ADBC_ASSERT_OK_WITH_ERROR(error, AdbcDatabaseInit(&db_options, &database, &error));
+      AdbcConnectionOptions conn_options;
+      conn_options.database = &database;
+      ADBC_ASSERT_OK_WITH_ERROR(error,
+                                AdbcConnectionInit(&conn_options, &connection, &error));
+    } else {
+      FAIL() << "Must provide location of Flight SQL server at " << kServerEnvVar;
     }
   }
 
   void TearDown() override {
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.ConnectionRelease(&connection, &error));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error, driver.DatabaseRelease(&database, &error));
-    ASSERT_OK(server->Shutdown());
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionRelease(&connection, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcDatabaseRelease(&database, &error));
   }
 
  protected:
-  std::shared_ptr<arrow::flight::sql::FlightSqlServerBase> server;
   AdbcDriver driver;
   AdbcDatabase database;
   AdbcConnection connection;
@@ -87,21 +67,20 @@ TEST_F(AdbcFlightSqlTest, Metadata) {
   {
     AdbcStatement statement;
     std::memset(&statement, 0, sizeof(statement));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.StatementInit(&connection, &statement, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
     ADBC_ASSERT_OK_WITH_ERROR(
-        &driver, error, driver.ConnectionGetTableTypes(&connection, &statement, &error));
+        error, AdbcConnectionGetTableTypes(&connection, &statement, &error));
 
     std::shared_ptr<arrow::Schema> schema;
     arrow::RecordBatchVector batches;
-    ReadStatement(&driver, &statement, &schema, &batches);
-    arrow::AssertSchemaEqual(
+    ReadStatement(&statement, &schema, &batches);
+    ASSERT_SCHEMA_EQ(
         *schema,
         *arrow::schema({arrow::field("table_type", arrow::utf8(), /*nullable=*/false)}));
     EXPECT_THAT(batches, ::testing::UnorderedPointwise(
                              PointeesEqual(),
                              {
-                                 arrow::RecordBatchFromJSON(schema, R"([["table"]])"),
+                                 adbc::RecordBatchFromJSON(schema, R"([["table"]])"),
                              }));
   }
 }
@@ -110,20 +89,18 @@ TEST_F(AdbcFlightSqlTest, SqlExecute) {
   std::string query = "SELECT 1";
   AdbcStatement statement;
   std::memset(&statement, 0, sizeof(statement));
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                            driver.StatementInit(&connection, &statement, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
   ADBC_ASSERT_OK_WITH_ERROR(
-      &driver, error,
-      driver.ConnectionSqlExecute(&connection, query.c_str(), &statement, &error));
+      error, AdbcConnectionSqlExecute(&connection, query.c_str(), &statement, &error));
 
   std::shared_ptr<arrow::Schema> schema;
   arrow::RecordBatchVector batches;
-  ReadStatement(&driver, &statement, &schema, &batches);
-  arrow::AssertSchemaEqual(*schema, *arrow::schema({arrow::field("1", arrow::int64())}));
+  ReadStatement(&statement, &schema, &batches);
+  ASSERT_SCHEMA_EQ(*schema, *arrow::schema({arrow::field("1", arrow::int64())}));
   EXPECT_THAT(batches,
               ::testing::UnorderedPointwise(
                   PointeesEqual(), {
-                                       arrow::RecordBatchFromJSON(schema, "[[1]]"),
+                                       adbc::RecordBatchFromJSON(schema, "[[1]]"),
                                    }));
 }
 
@@ -135,45 +112,39 @@ TEST_F(AdbcFlightSqlTest, Partitions) {
   std::string query = "SELECT 42";
   AdbcStatement statement;
   std::memset(&statement, 0, sizeof(statement));
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                            driver.StatementInit(&connection, &statement, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
   ADBC_ASSERT_OK_WITH_ERROR(
-      &driver, error,
-      driver.ConnectionSqlExecute(&connection, query.c_str(), &statement, &error));
+      error, AdbcConnectionSqlExecute(&connection, query.c_str(), &statement, &error));
 
   std::vector<std::vector<uint8_t>> descs;
 
   while (true) {
     size_t length = 0;
     ADBC_ASSERT_OK_WITH_ERROR(
-        &driver, error,
-        driver.StatementGetPartitionDescSize(&statement, &length, &error));
+        error, AdbcStatementGetPartitionDescSize(&statement, &length, &error));
     if (length == 0) break;
     descs.emplace_back(length);
     ADBC_ASSERT_OK_WITH_ERROR(
-        &driver, error,
-        driver.StatementGetPartitionDesc(&statement, descs.back().data(), &error));
+        error, AdbcStatementGetPartitionDesc(&statement, descs.back().data(), &error));
   }
 
   ASSERT_EQ(descs.size(), 1);
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error, driver.StatementRelease(&statement, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementRelease(&statement, &error));
 
   // Reconstruct the partition
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                            driver.StatementInit(&connection, &statement, &error));
-  ADBC_ASSERT_OK_WITH_ERROR(
-      &driver, error,
-      driver.ConnectionDeserializePartitionDesc(&connection, descs.back().data(),
-                                                descs.back().size(), &statement, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionDeserializePartitionDesc(
+                                       &connection, descs.back().data(),
+                                       descs.back().size(), &statement, &error));
 
   std::shared_ptr<arrow::Schema> schema;
   arrow::RecordBatchVector batches;
-  ReadStatement(&driver, &statement, &schema, &batches);
-  arrow::AssertSchemaEqual(*schema, *arrow::schema({arrow::field("42", arrow::int64())}));
+  ReadStatement(&statement, &schema, &batches);
+  ASSERT_SCHEMA_EQ(*schema, *arrow::schema({arrow::field("42", arrow::int64())}));
   EXPECT_THAT(batches,
               ::testing::UnorderedPointwise(
                   PointeesEqual(), {
-                                       arrow::RecordBatchFromJSON(schema, "[[42]]"),
+                                       adbc::RecordBatchFromJSON(schema, "[[42]]"),
                                    }));
 }
 
@@ -181,12 +152,11 @@ TEST_F(AdbcFlightSqlTest, InvalidSql) {
   std::string query = "INVALID";
   AdbcStatement statement;
   std::memset(&statement, 0, sizeof(statement));
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                            driver.StatementInit(&connection, &statement, &error));
-  ASSERT_NE(driver.ConnectionSqlExecute(&connection, query.c_str(), &statement, &error),
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
+  ASSERT_NE(AdbcConnectionSqlExecute(&connection, query.c_str(), &statement, &error),
             ADBC_STATUS_OK);
-  ADBC_ASSERT_ERROR_THAT(&driver, error, ::testing::HasSubstr("syntax error"));
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error, driver.StatementRelease(&statement, &error));
+  ADBC_ASSERT_ERROR_THAT(error, ::testing::HasSubstr("syntax error"));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementRelease(&statement, &error));
 }
 
 }  // namespace adbc
diff --git a/adbc_driver_manager/CMakeLists.txt b/drivers/sqlite/CMakeLists.txt
similarity index 51%
copy from adbc_driver_manager/CMakeLists.txt
copy to drivers/sqlite/CMakeLists.txt
index ed41526..283aef0 100644
--- a/adbc_driver_manager/CMakeLists.txt
+++ b/drivers/sqlite/CMakeLists.txt
@@ -15,33 +15,41 @@
 # specific language governing permissions and limitations
 # under the License.
 
-cmake_minimum_required(VERSION 3.10)
-
-set(ARROW_VERSION "9.0.0-SNAPSHOT")
-set(ARROW_BASE_VERSION "9.0.0")
-set(ARROW_VERSION_MAJOR "9")
-set(ARROW_VERSION_MINOR "0")
-set(ARROW_VERSION_PATCH "0")
-
-math(EXPR ARROW_SO_VERSION "${ARROW_VERSION_MAJOR} * 100 + ${ARROW_VERSION_MINOR}")
-set(ARROW_FULL_SO_VERSION "${ARROW_SO_VERSION}.${ARROW_VERSION_PATCH}.0")
-
-project(adbc_driver_manager VERSION "${ARROW_BASE_VERSION}" LANGUAGES CXX)
-
-# Common CMake utilities
-get_filename_component(REPOSITORY_ROOT ${CMAKE_SOURCE_DIR} DIRECTORY)
+cmake_minimum_required(VERSION 3.14)
+get_filename_component(REPOSITORY_ROOT "../.." ABSOLUTE)
 list(APPEND CMAKE_MODULE_PATH "${REPOSITORY_ROOT}/cmake_modules/")
+include(AdbcDefines)
 include(BuildUtils)
-include(DefineOptions)
-
-# Driver manager library
-add_arrow_lib(adbc_driver_manager
-  SOURCES
-  adbc_driver_manager.cc)
-include_directories(adbc_driver_manager SYSTEM
-  PRIVATE ${REPOSITORY_ROOT})
 
-# Tests
+project(adbc_driver_sqlite
+        VERSION "${ARROW_BASE_VERSION}"
+        LANGUAGES CXX)
+
+find_package(Arrow REQUIRED)
+find_package(SQLite3 REQUIRED)
+
+add_arrow_lib(adbc_driver_sqlite
+              SOURCES
+              sqlite.cc
+              SHARED_LINK_LIBS
+              SQLite::SQLite3
+              arrow_shared)
+include_directories(SYSTEM ${REPOSITORY_ROOT})
+include_directories(SYSTEM ${SQLite3_INCLUDE_DIRS})
+
+if(ARROW_TEST_LINKAGE STREQUAL "shared")
+  set(TEST_LINK_LIBS adbc_driver_sqlite_shared)
+else()
+  set(TEST_LINK_LIBS adbc_driver_sqlite_static)
+endif()
+
+add_test_case(driver_sqlite_test
+              PREFIX
+              adbc
+              SOURCES
+              sqlite_test.cc
+              EXTRA_LINK_LIBS
+              ${TEST_LINK_LIBS})
 
 validate_config()
 config_summary_message()
diff --git a/drivers/sqlite/sqlite.cc b/drivers/sqlite/sqlite.cc
index b6a1cbc..e94cb96 100644
--- a/drivers/sqlite/sqlite.cc
+++ b/drivers/sqlite/sqlite.cc
@@ -22,14 +22,15 @@
 #include <mutex>
 #include <string>
 
-#include "adbc/adbc.h"
-#include "adbc/driver/util.h"
-#include "arrow/builder.h"
-#include "arrow/c/bridge.h"
-#include "arrow/record_batch.h"
-#include "arrow/status.h"
-#include "arrow/util/logging.h"
-#include "arrow/util/string_builder.h"
+#include <arrow/builder.h>
+#include <arrow/c/bridge.h>
+#include <arrow/record_batch.h>
+#include <arrow/status.h>
+#include <arrow/util/logging.h>
+#include <arrow/util/string_builder.h>
+
+#include "adbc.h"
+#include "drivers/util.h"
 
 namespace {
 
diff --git a/drivers/sqlite/sqlite_test.cc b/drivers/sqlite/sqlite_test.cc
index 9f088e0..5296c8a 100644
--- a/drivers/sqlite/sqlite_test.cc
+++ b/drivers/sqlite/sqlite_test.cc
@@ -18,14 +18,14 @@
 #include <gmock/gmock.h>
 #include <gtest/gtest.h>
 
-#include "adbc/adbc.h"
-#include "adbc/adbc_driver_manager.h"
-#include "adbc/test_sqlite_internal.h"
-#include "adbc/test_util.h"
-#include "arrow/c/bridge.h"
-#include "arrow/record_batch.h"
-#include "arrow/testing/gtest_util.h"
-#include "arrow/testing/matchers.h"
+#include <arrow/c/bridge.h>
+#include <arrow/record_batch.h>
+#include <arrow/testing/matchers.h>
+
+#include "adbc.h"
+#include "drivers/test_util.h"
+
+// Tests of the SQLite example driver
 
 namespace adbc {
 
@@ -36,14 +36,10 @@ TEST(Adbc, Basics) {
   AdbcConnection connection;
   AdbcError error = {};
 
-  AdbcDriver driver;
-  ASSERT_NO_FATAL_FAILURE(InitSqlite(&driver));
-
   {
     AdbcDatabaseOptions options;
     std::memset(&options, 0, sizeof(options));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.DatabaseInit(&options, &database, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcDatabaseInit(&options, &database, &error));
     ASSERT_NE(database.private_data, nullptr);
   }
 
@@ -51,68 +47,50 @@ TEST(Adbc, Basics) {
     AdbcConnectionOptions options;
     std::memset(&options, 0, sizeof(options));
     options.database = &database;
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.ConnectionInit(&options, &connection, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionInit(&options, &connection, &error));
     ASSERT_NE(connection.private_data, nullptr);
   }
 
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                            driver.ConnectionRelease(&connection, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionRelease(&connection, &error));
   ASSERT_EQ(connection.private_data, nullptr);
 
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error, driver.DatabaseRelease(&database, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcDatabaseRelease(&database, &error));
   ASSERT_EQ(database.private_data, nullptr);
 }
 
-TEST(Adbc, Errors) {
-  AdbcDriver driver;
-  size_t initialized = 0;
-  ASSERT_NE(ADBC_STATUS_OK, AdbcLoadDriver("Driver=libadbc_driver_fake.so",
-                                           ADBC_VERSION_0_0_1, &driver, &initialized));
-}
-
 TEST(AdbcSqlite, SqlExecute) {
-  // Execute a query with the SQLite example driver.
   AdbcDatabase database;
   AdbcConnection connection;
   AdbcError error = {};
 
-  AdbcDriver driver;
-  ASSERT_NO_FATAL_FAILURE(InitSqlite(&driver));
-
   {
     AdbcDatabaseOptions options;
     std::memset(&options, 0, sizeof(options));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.DatabaseInit(&options, &database, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcDatabaseInit(&options, &database, &error));
   }
   {
     AdbcConnectionOptions options;
     std::memset(&options, 0, sizeof(options));
     options.database = &database;
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.ConnectionInit(&options, &connection, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionInit(&options, &connection, &error));
   }
 
   {
     std::string query = "SELECT 1";
     AdbcStatement statement;
     std::memset(&statement, 0, sizeof(statement));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.StatementInit(&connection, &statement, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
     ADBC_ASSERT_OK_WITH_ERROR(
-        &driver, error,
-        driver.ConnectionSqlExecute(&connection, query.c_str(), &statement, &error));
+        error, AdbcConnectionSqlExecute(&connection, query.c_str(), &statement, &error));
 
     std::shared_ptr<arrow::Schema> schema;
     arrow::RecordBatchVector batches;
-    ReadStatement(&driver, &statement, &schema, &batches);
-    arrow::AssertSchemaEqual(*schema,
-                             *arrow::schema({arrow::field("1", arrow::int64())}));
+    ReadStatement(&statement, &schema, &batches);
+    ASSERT_SCHEMA_EQ(*schema, *arrow::schema({arrow::field("1", arrow::int64())}));
     EXPECT_THAT(batches,
                 ::testing::UnorderedPointwise(
                     PointeesEqual(), {
-                                         arrow::RecordBatchFromJSON(schema, "[[1]]"),
+                                         adbc::RecordBatchFromJSON(schema, "[[1]]"),
                                      }));
   }
 
@@ -120,21 +98,17 @@ TEST(AdbcSqlite, SqlExecute) {
     std::string query = "INVALID";
     AdbcStatement statement;
     std::memset(&statement, 0, sizeof(statement));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.StatementInit(&connection, &statement, &error));
-    ASSERT_NE(driver.ConnectionSqlExecute(&connection, query.c_str(), &statement, &error),
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
+    ASSERT_NE(AdbcConnectionSqlExecute(&connection, query.c_str(), &statement, &error),
               ADBC_STATUS_OK);
     ADBC_ASSERT_ERROR_THAT(
-        &driver, error,
-        ::testing::AllOf(::testing::HasSubstr("[SQLite3] sqlite3_prepare_v2:"),
-                         ::testing::HasSubstr("syntax error")));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.StatementRelease(&statement, &error));
+        error, ::testing::AllOf(::testing::HasSubstr("[SQLite3] sqlite3_prepare_v2:"),
+                                ::testing::HasSubstr("syntax error")));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementRelease(&statement, &error));
   }
 
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                            driver.ConnectionRelease(&connection, &error));
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error, driver.DatabaseRelease(&database, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionRelease(&connection, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcDatabaseRelease(&database, &error));
 }
 
 TEST(AdbcSqlite, SqlPrepare) {
@@ -142,45 +116,36 @@ TEST(AdbcSqlite, SqlPrepare) {
   AdbcConnection connection;
   AdbcError error = {};
 
-  AdbcDriver driver;
-  ASSERT_NO_FATAL_FAILURE(InitSqlite(&driver));
-
   {
     AdbcDatabaseOptions options;
     std::memset(&options, 0, sizeof(options));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.DatabaseInit(&options, &database, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcDatabaseInit(&options, &database, &error));
   }
   {
     AdbcConnectionOptions options;
     std::memset(&options, 0, sizeof(options));
     options.database = &database;
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.ConnectionInit(&options, &connection, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionInit(&options, &connection, &error));
   }
 
   {
     std::string query = "SELECT 1";
     AdbcStatement statement;
     std::memset(&statement, 0, sizeof(statement));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.StatementInit(&connection, &statement, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
     ADBC_ASSERT_OK_WITH_ERROR(
-        &driver, error,
-        driver.ConnectionSqlPrepare(&connection, query.c_str(), &statement, &error));
+        error, AdbcConnectionSqlPrepare(&connection, query.c_str(), &statement, &error));
 
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.StatementExecute(&statement, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementExecute(&statement, &error));
 
     std::shared_ptr<arrow::Schema> schema;
     arrow::RecordBatchVector batches;
-    ASSERT_NO_FATAL_FAILURE(ReadStatement(&driver, &statement, &schema, &batches));
-    arrow::AssertSchemaEqual(*schema,
-                             *arrow::schema({arrow::field("1", arrow::int64())}));
+    ASSERT_NO_FATAL_FAILURE(ReadStatement(&statement, &schema, &batches));
+    ASSERT_SCHEMA_EQ(*schema, *arrow::schema({arrow::field("1", arrow::int64())}));
     EXPECT_THAT(batches,
                 ::testing::UnorderedPointwise(
                     PointeesEqual(), {
-                                         arrow::RecordBatchFromJSON(schema, "[[1]]"),
+                                         adbc::RecordBatchFromJSON(schema, "[[1]]"),
                                      }));
   }
 
@@ -194,110 +159,91 @@ TEST(AdbcSqlite, SqlPrepare) {
     std::memset(&statement, 0, sizeof(statement));
 
     ASSERT_OK(ExportRecordBatch(
-        *arrow::RecordBatchFromJSON(param_schema, R"([[1, "foo"], [2, "bar"]])"),
+        *adbc::RecordBatchFromJSON(param_schema, R"([[1, "foo"], [2, "bar"]])"),
         &export_params));
     ASSERT_OK(ExportSchema(*param_schema, &export_schema));
 
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.StatementInit(&connection, &statement, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection, &statement, &error));
     ADBC_ASSERT_OK_WITH_ERROR(
-        &driver, error,
-        driver.ConnectionSqlPrepare(&connection, query.c_str(), &statement, &error));
+        error, AdbcConnectionSqlPrepare(&connection, query.c_str(), &statement, &error));
 
     ADBC_ASSERT_OK_WITH_ERROR(
-        &driver, error,
-        driver.StatementBind(&statement, &export_params, &export_schema, &error));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.StatementExecute(&statement, &error));
+        error, AdbcStatementBind(&statement, &export_params, &export_schema, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementExecute(&statement, &error));
 
     std::shared_ptr<arrow::Schema> schema;
     arrow::RecordBatchVector batches;
-    ASSERT_NO_FATAL_FAILURE(ReadStatement(&driver, &statement, &schema, &batches));
-    arrow::AssertSchemaEqual(*schema, *arrow::schema({arrow::field("?", arrow::int64()),
-                                                      arrow::field("?", arrow::utf8())}));
+    ASSERT_NO_FATAL_FAILURE(ReadStatement(&statement, &schema, &batches));
+    ASSERT_SCHEMA_EQ(*schema, *arrow::schema({arrow::field("?", arrow::int64()),
+                                              arrow::field("?", arrow::utf8())}));
     EXPECT_THAT(batches,
                 ::testing::UnorderedPointwise(
                     PointeesEqual(),
                     {
-                        arrow::RecordBatchFromJSON(schema, R"([[1, "foo"], [2, "bar"]])"),
+                        adbc::RecordBatchFromJSON(schema, R"([[1, "foo"], [2, "bar"]])"),
                     }));
   }
 
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                            driver.ConnectionRelease(&connection, &error));
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error, driver.DatabaseRelease(&database, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionRelease(&connection, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcDatabaseRelease(&database, &error));
 }
 
 TEST(AdbcSqlite, MultipleConnections) {
-  // Execute a query with the SQLite example driver.
   AdbcDatabase database;
   AdbcConnection connection1, connection2;
   AdbcError error = {};
 
-  AdbcDriver driver;
-  ASSERT_NO_FATAL_FAILURE(InitSqlite(&driver));
-
   {
     AdbcDatabaseOptions options;
     std::memset(&options, 0, sizeof(options));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.DatabaseInit(&options, &database, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcDatabaseInit(&options, &database, &error));
   }
   {
     AdbcConnectionOptions options;
     std::memset(&options, 0, sizeof(options));
     options.database = &database;
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.ConnectionInit(&options, &connection1, &error));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.ConnectionInit(&options, &connection2, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionInit(&options, &connection1, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionInit(&options, &connection2, &error));
   }
 
   {
     std::string query = "CREATE TABLE foo (bar INTEGER)";
     AdbcStatement statement;
     std::memset(&statement, 0, sizeof(statement));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.StatementInit(&connection1, &statement, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection1, &statement, &error));
     ADBC_ASSERT_OK_WITH_ERROR(
-        &driver, error,
-        driver.ConnectionSqlExecute(&connection1, query.c_str(), &statement, &error));
+        error, AdbcConnectionSqlExecute(&connection1, query.c_str(), &statement, &error));
 
     std::shared_ptr<arrow::Schema> schema;
     arrow::RecordBatchVector batches;
-    ASSERT_NO_FATAL_FAILURE(ReadStatement(&driver, &statement, &schema, &batches));
-    arrow::AssertSchemaEqual(*schema, *arrow::schema({}));
+    ASSERT_NO_FATAL_FAILURE(ReadStatement(&statement, &schema, &batches));
+    ASSERT_SCHEMA_EQ(*schema, *arrow::schema({}));
     EXPECT_THAT(batches,
                 ::testing::UnorderedPointwise(
                     PointeesEqual(), std::vector<std::shared_ptr<arrow::RecordBatch>>{}));
   }
 
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                            driver.ConnectionRelease(&connection1, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionRelease(&connection1, &error));
 
   {
     std::string query = "SELECT * FROM foo";
     AdbcStatement statement;
     std::memset(&statement, 0, sizeof(statement));
-    ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                              driver.StatementInit(&connection2, &statement, &error));
+    ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementInit(&connection2, &statement, &error));
     ADBC_ASSERT_OK_WITH_ERROR(
-        &driver, error,
-        driver.ConnectionSqlExecute(&connection2, query.c_str(), &statement, &error));
+        error, AdbcConnectionSqlExecute(&connection2, query.c_str(), &statement, &error));
 
     std::shared_ptr<arrow::Schema> schema;
     arrow::RecordBatchVector batches;
-    ReadStatement(&driver, &statement, &schema, &batches);
-    arrow::AssertSchemaEqual(*schema,
-                             *arrow::schema({arrow::field("bar", arrow::null())}));
+    ReadStatement(&statement, &schema, &batches);
+    ASSERT_SCHEMA_EQ(*schema, *arrow::schema({arrow::field("bar", arrow::null())}));
     EXPECT_THAT(batches,
                 ::testing::UnorderedPointwise(
                     PointeesEqual(), std::vector<std::shared_ptr<arrow::RecordBatch>>{}));
   }
 
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error,
-                            driver.ConnectionRelease(&connection2, &error));
-  ADBC_ASSERT_OK_WITH_ERROR(&driver, error, driver.DatabaseRelease(&database, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcConnectionRelease(&connection2, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcDatabaseRelease(&database, &error));
 }
 
 }  // namespace adbc
diff --git a/drivers/sqlite/test_sqlite_internal.cc b/drivers/sqlite/test_sqlite_internal.cc
deleted file mode 100644
index 8252df5..0000000
--- a/drivers/sqlite/test_sqlite_internal.cc
+++ /dev/null
@@ -1,30 +0,0 @@
-// 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.
-
-#include "adbc/test_sqlite_internal.h"
-
-namespace adbc {
-
-void InitSqlite(AdbcDriver* driver) {
-  size_t initialized = 0;
-  ADBC_ASSERT_OK(
-      AdbcLoadDriver("Driver=libadbc_driver_sqlite.so;Entrypoint=AdbcSqliteDriverInit",
-                     ADBC_VERSION_0_0_1, driver, &initialized));
-  ASSERT_EQ(initialized, ADBC_VERSION_0_0_1);
-}
-
-}  // namespace adbc
diff --git a/drivers/sqlite/test_sqlite_internal.h b/drivers/sqlite/test_sqlite_internal.h
deleted file mode 100644
index 3adaef5..0000000
--- a/drivers/sqlite/test_sqlite_internal.h
+++ /dev/null
@@ -1,32 +0,0 @@
-// 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.
-
-#pragma once
-
-#include <gtest/gtest.h>
-
-#include "adbc/adbc.h"
-#include "adbc/adbc_driver_manager.h"
-#include "adbc/test_util.h"
-#include "arrow/util/visibility.h"
-
-namespace adbc {
-
-ARROW_EXPORT
-void InitSqlite(AdbcDriver* driver);
-
-}  // namespace adbc
diff --git a/drivers/test_util.h b/drivers/test_util.h
index adf4abb..35f887e 100644
--- a/drivers/test_util.h
+++ b/drivers/test_util.h
@@ -19,12 +19,13 @@
 
 #include <memory>
 
-#include "adbc/adbc.h"
-#include "arrow/c/bridge.h"
-#include "arrow/record_batch.h"
-#include "arrow/result.h"
-#include "arrow/testing/gtest_util.h"
-#include "arrow/util/macros.h"
+#include <arrow/c/bridge.h>
+#include <arrow/ipc/json_simple.h>
+#include <arrow/record_batch.h>
+#include <arrow/result.h>
+#include <arrow/testing/gtest_util.h>
+#include <arrow/util/macros.h>
+#include "adbc.h"
 
 namespace adbc {
 
@@ -34,31 +35,34 @@ namespace adbc {
     ASSERT_EQ(code_, ADBC_STATUS_OK); \
   } while (false)
 
-#define ADBC_ASSERT_OK_WITH_ERROR(DRIVER, ERROR, EXPR)                         \
+#define ADBC_ASSERT_OK_WITH_ERROR(ERROR, EXPR)                                 \
   do {                                                                         \
     auto code_ = (EXPR);                                                       \
     if (code_ != ADBC_STATUS_OK) {                                             \
       std::string errmsg_ = ERROR.message ? ERROR.message : "(unknown error)"; \
-      (DRIVER)->ErrorRelease(&ERROR);                                          \
+      AdbcErrorRelease(&ERROR);                                                \
       ASSERT_EQ(code_, ADBC_STATUS_OK) << errmsg_;                             \
     }                                                                          \
   } while (false)
 
-#define ADBC_ASSERT_ERROR_THAT(DRIVER, ERROR, PATTERN)                       \
+#define ADBC_ASSERT_ERROR_THAT(ERROR, PATTERN)                               \
   do {                                                                       \
     ASSERT_NE(ERROR.message, nullptr);                                       \
     std::string errmsg_ = ERROR.message ? ERROR.message : "(unknown error)"; \
-    (DRIVER)->ErrorRelease(&ERROR);                                          \
+    AdbcErrorRelease(&ERROR);                                                \
     ASSERT_THAT(errmsg_, PATTERN) << errmsg_;                                \
   } while (false)
 
-static inline void ReadStatement(AdbcDriver* driver, AdbcStatement* statement,
+#define ASSERT_SCHEMA_EQ(schema1, schema2) \
+  ASSERT_TRUE((schema1).Equals((schema2))) \
+      << "LHS: " << (schema1).ToString() << "RHS: " << (schema2).ToString()
+
+static inline void ReadStatement(AdbcStatement* statement,
                                  std::shared_ptr<arrow::Schema>* schema,
                                  arrow::RecordBatchVector* batches) {
   AdbcError error = {};
   ArrowArrayStream stream;
-  ADBC_ASSERT_OK_WITH_ERROR(driver, error,
-                            driver->StatementGetStream(statement, &stream, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementGetStream(statement, &stream, &error));
   ASSERT_OK_AND_ASSIGN(auto reader, arrow::ImportRecordBatchReader(&stream));
 
   *schema = reader->schema();
@@ -68,7 +72,15 @@ static inline void ReadStatement(AdbcDriver* driver, AdbcStatement* statement,
     if (!batch) break;
     batches->push_back(std::move(batch));
   }
-  ADBC_ASSERT_OK_WITH_ERROR(driver, error, driver->StatementRelease(statement, &error));
+  ADBC_ASSERT_OK_WITH_ERROR(error, AdbcStatementRelease(statement, &error));
+}
+
+static inline std::shared_ptr<arrow::RecordBatch> RecordBatchFromJSON(
+    const std::shared_ptr<arrow::Schema>& schema, const std::string& json) {
+  auto struct_type = arrow::struct_(schema->fields());
+  std::shared_ptr<arrow::Array> struct_array =
+      *arrow::ipc::internal::json::ArrayFromJSON(struct_type, json);
+  return *arrow::RecordBatch::FromStructArray(struct_array);
 }
 
 }  // namespace adbc
diff --git a/drivers/util.cc b/drivers/util.cc
index fe71727..dfdea3d 100644
--- a/drivers/util.cc
+++ b/drivers/util.cc
@@ -15,9 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "adbc/driver/util.h"
-#include "arrow/status.h"
-#include "arrow/util/string_view.h"
+#include "drivers/util.h"
+#include <arrow/status.h>
+#include <arrow/util/string_view.h>
 
 namespace adbc {