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/24 18:06:45 UTC

[arrow-adbc] branch main updated: Sketch out Python bindings (#17)

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 177d31b  Sketch out Python bindings (#17)
177d31b is described below

commit 177d31bf4fefe932f1f02775547686785ffe18ce
Author: David Li <li...@gmail.com>
AuthorDate: Fri Jun 24 14:06:40 2022 -0400

    Sketch out Python bindings (#17)
    
    These bindings are structured as a low-level module that mostly
    mirrors the ADBC API, and a TBD high-level module that will
    implement PEP 249 (except with Turbodbc-style extensions).
---
 .flake8                                            |  20 ++
 .github/workflows/cpp.yml                          |  35 ++-
 .isort.cfg                                         |  19 ++
 .pre-commit-config.yaml                            |  15 +
 adbc_driver_manager/adbc_driver_manager.cc         |   9 +-
 ci/conda_env_python.txt                            |  20 ++
 python/adbc_driver_manager/.gitignore              |  19 ++
 .../adbc_driver_manager/__init__.py                |  16 +
 .../adbc_driver_manager/_lib.pyx                   | 348 +++++++++++++++++++++
 .../adbc_driver_manager/tests/test_lowlevel.py     |  80 +++++
 python/adbc_driver_manager/requirements-dev.txt    |  19 ++
 python/adbc_driver_manager/setup.py                |  38 +++
 12 files changed, 633 insertions(+), 5 deletions(-)

diff --git a/.flake8 b/.flake8
new file mode 100644
index 0000000..cda8c86
--- /dev/null
+++ b/.flake8
@@ -0,0 +1,20 @@
+# 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.
+
+[flake8]
+max-line-length = 88
+extend-ignore = E203
diff --git a/.github/workflows/cpp.yml b/.github/workflows/cpp.yml
index 1c7458b..213effd 100644
--- a/.github/workflows/cpp.yml
+++ b/.github/workflows/cpp.yml
@@ -15,7 +15,7 @@
 # specific language governing permissions and limitations
 # under the License.
 
-name: C/C++
+name: C/C++/Python
 
 on:
   pull_request:
@@ -72,7 +72,7 @@ jobs:
       - name: Install Dependencies
         shell: bash -l {0}
         run: |
-          mamba install --file ci/conda_env_cpp.txt
+          mamba install --file ci/conda_env_cpp.txt --file ci/conda_env_python.txt
       - name: Build/Test SQLite3 Driver
         shell: bash -l {0}
         run: |
@@ -96,6 +96,26 @@ jobs:
           export LD_LIBRARY_PATH=$(pwd):../driver_sqlite
           ./adbc-driver-manager-test
           popd
+      - name: Build/Test SQLite3 Driver (No ASan)
+        shell: bash -l {0}
+        run: |
+          # Build a no-ASan version that Python can easily load
+          mkdir -p build/driver_sqlite_no_asan
+          pushd build/driver_sqlite_no_asan
+          cmake ../../drivers/sqlite -DCMAKE_BUILD_TYPE=Debug -DCMAKE_PREFIX_PATH=$CONDA_PREFIX -DADBC_BUILD_SHARED=ON -DADBC_BUILD_STATIC=OFF
+          cmake --build .
+          popd
+      - name: Build/Test Python Driver Manager
+        shell: bash -l {0}
+        run: |
+          pushd python/adbc_driver_manager
+          python setup.py build_ext --inplace
+          export PYTHONPATH=$(pwd)
+          export DYLD_LIBRARY_PATH=$(pwd)/../../build/driver_sqlite_no_asan
+          export LD_LIBRARY_PATH=$(pwd)/../../build/driver_sqlite_no_asan
+          python -m pytest -vv
+          popd
+
 
   cpp-conda-windows:
     name: "Conda/${{ matrix.os }}"
@@ -140,9 +160,16 @@ jobs:
           if %errorlevel% neq 0 then exit /b %errorlevel%
           cmake --build .
           if %errorlevel% neq 0 then exit /b %errorlevel%
-          echo ${{ github.workspace }}\build\driver_sqlite
-          dir ${{ github.workspace }}\build\driver_sqlite
           set PATH=%PATH%;${{ github.workspace }}\build\driver_sqlite
           ctest --output-on-failure --no-tests=error
           if %errorlevel% neq 0 then exit /b %errorlevel%
           cd ..\..
+      - name: Build/Test Python Driver Manager
+        shell: cmd /C call {0}
+        run: |
+          cd python\adbc_driver_manager
+          python setup.py build_ext --inplace
+          set PATH=%PATH%;${{ github.workspace }}\build\driver_sqlite
+          set PYTHONPATH=${{ github.workspace }}\python\adbc_driver_manager
+          python -m pytest -vv
+          cd ..\..
diff --git a/.isort.cfg b/.isort.cfg
new file mode 100644
index 0000000..dc371b7
--- /dev/null
+++ b/.isort.cfg
@@ -0,0 +1,19 @@
+# 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.
+
+[settings]
+profile = black
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index cf02f14..6f1c611 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -54,3 +54,18 @@ repos:
     - id: pretty-format-java
       args: [--autofix]
       types_or: [java]
+  - repo: https://github.com/psf/black
+    rev: 22.3.0
+    hooks:
+    - id: black
+      types_or: [python]
+  - repo: https://github.com/PyCQA/flake8
+    rev: 4.0.1
+    hooks:
+    - id: flake8
+      types_or: [python]
+  - repo: https://github.com/PyCQA/isort
+    rev: 5.10.1
+    hooks:
+    - id: isort
+      types_or: [python]
diff --git a/adbc_driver_manager/adbc_driver_manager.cc b/adbc_driver_manager/adbc_driver_manager.cc
index 5fc91f6..68e7223 100644
--- a/adbc_driver_manager/adbc_driver_manager.cc
+++ b/adbc_driver_manager/adbc_driver_manager.cc
@@ -87,6 +87,11 @@ AdbcStatusCode ConnectionRollback(struct AdbcConnection*, struct AdbcError* erro
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
 
+AdbcStatusCode ConnectionSetOption(struct AdbcConnection*, const char*, const char*,
+                                   struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
 AdbcStatusCode StatementBind(struct AdbcStatement*, struct ArrowArray*,
                              struct ArrowSchema*, struct AdbcError* error) {
   return ADBC_STATUS_NOT_IMPLEMENTED;
@@ -431,7 +436,7 @@ const char* AdbcStatusCodeMessage(AdbcStatusCode code) {
 #define STRINGIFY_VALUE(s) STRINGIFY(s)
 #define CASE(CONSTANT) \
   case CONSTANT:       \
-    return STRINGIFY(CONSTANT) " (" STRINGIFY_VALUE(CONSTANT) ")";
+    return #CONSTANT " (" STRINGIFY_VALUE(CONSTANT) ")";
 
   switch (code) {
     CASE(ADBC_STATUS_OK);
@@ -584,12 +589,14 @@ AdbcStatusCode AdbcLoadDriver(const char* driver_name, const char* entrypoint,
   FILL_DEFAULT(driver, ConnectionGetTableSchema);
   FILL_DEFAULT(driver, ConnectionGetTableTypes);
   FILL_DEFAULT(driver, ConnectionRollback);
+  FILL_DEFAULT(driver, ConnectionSetOption);
 
   CHECK_REQUIRED(driver, StatementNew);
   CHECK_REQUIRED(driver, StatementRelease);
   FILL_DEFAULT(driver, StatementBind);
   FILL_DEFAULT(driver, StatementExecute);
   FILL_DEFAULT(driver, StatementPrepare);
+  FILL_DEFAULT(driver, StatementSetOption);
   FILL_DEFAULT(driver, StatementSetSqlQuery);
   FILL_DEFAULT(driver, StatementSetSubstraitPlan);
 
diff --git a/ci/conda_env_python.txt b/ci/conda_env_python.txt
new file mode 100644
index 0000000..8f60ddc
--- /dev/null
+++ b/ci/conda_env_python.txt
@@ -0,0 +1,20 @@
+# 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.
+
+Cython
+pyarrow>=8.0.0
+pytest
diff --git a/python/adbc_driver_manager/.gitignore b/python/adbc_driver_manager/.gitignore
new file mode 100644
index 0000000..92bc507
--- /dev/null
+++ b/python/adbc_driver_manager/.gitignore
@@ -0,0 +1,19 @@
+# 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.
+
+adbc_driver_manager/*.c
+build/
diff --git a/python/adbc_driver_manager/adbc_driver_manager/__init__.py b/python/adbc_driver_manager/adbc_driver_manager/__init__.py
new file mode 100644
index 0000000..13a8339
--- /dev/null
+++ b/python/adbc_driver_manager/adbc_driver_manager/__init__.py
@@ -0,0 +1,16 @@
+# 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.
diff --git a/python/adbc_driver_manager/adbc_driver_manager/_lib.pyx b/python/adbc_driver_manager/adbc_driver_manager/_lib.pyx
new file mode 100644
index 0000000..614f701
--- /dev/null
+++ b/python/adbc_driver_manager/adbc_driver_manager/_lib.pyx
@@ -0,0 +1,348 @@
+# 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.
+
+# cython: language_level = 3
+
+"""Low-level ADBC API."""
+
+import typing
+
+import cython
+import pyarrow
+from libc.stdint cimport int32_t, uint8_t, uintptr_t
+from libc.string cimport memset
+
+if typing.TYPE_CHECKING:
+    from typing import Self
+
+
+cdef extern from "adbc.h":
+    # C ABI
+    cdef struct CArrowSchema"ArrowSchema":
+        pass
+    cdef struct CArrowArray"ArrowArray":
+        pass
+    cdef struct CArrowArrayStream"ArrowArrayStream":
+        pass
+
+    # ADBC
+    ctypedef uint8_t AdbcStatusCode
+    cdef AdbcStatusCode ADBC_STATUS_OK
+    cdef AdbcStatusCode ADBC_STATUS_UNKNOWN
+    cdef AdbcStatusCode ADBC_STATUS_NOT_IMPLEMENTED
+    cdef AdbcStatusCode ADBC_STATUS_NOT_FOUND
+    cdef AdbcStatusCode ADBC_STATUS_ALREADY_EXISTS
+    cdef AdbcStatusCode ADBC_STATUS_INVALID_ARGUMENT
+    cdef AdbcStatusCode ADBC_STATUS_INVALID_STATE
+    cdef AdbcStatusCode ADBC_STATUS_INVALID_DATA
+    cdef AdbcStatusCode ADBC_STATUS_INTEGRITY
+    cdef AdbcStatusCode ADBC_STATUS_INTERNAL
+    cdef AdbcStatusCode ADBC_STATUS_IO
+    cdef AdbcStatusCode ADBC_STATUS_CANCELLED
+    cdef AdbcStatusCode ADBC_STATUS_TIMEOUT
+    cdef AdbcStatusCode ADBC_STATUS_UNAUTHENTICATED
+    cdef AdbcStatusCode ADBC_STATUS_UNAUTHORIZED
+
+    ctypedef void (*CAdbcErrorRelease)(CAdbcError*)
+
+    cdef struct CAdbcError"AdbcError":
+        char* message
+        int32_t vendor_code
+        char[5] sqlstate
+        CAdbcErrorRelease release
+
+    cdef struct CAdbcDatabase"AdbcDatabase":
+        void* private_data
+
+    cdef struct CAdbcConnection"AdbcConnection":
+        void* private_data
+
+    cdef struct CAdbcStatement"AdbcStatement":
+        void* private_data
+
+    AdbcStatusCode AdbcDatabaseNew(CAdbcDatabase* database, CAdbcError* error)
+    AdbcStatusCode AdbcDatabaseSetOption(CAdbcDatabase* database, const char* key, const char* value, CAdbcError* error)
+    AdbcStatusCode AdbcDatabaseInit(CAdbcDatabase* database, CAdbcError* error)
+    AdbcStatusCode AdbcDatabaseRelease(CAdbcDatabase* database, CAdbcError* error)
+
+    AdbcStatusCode AdbcConnectionNew(CAdbcDatabase* database, CAdbcConnection* connection, CAdbcError* error)
+    AdbcStatusCode AdbcConnectionSetOption(CAdbcConnection* connection, const char* key, const char* value, CAdbcError* error)
+    AdbcStatusCode AdbcConnectionInit(CAdbcConnection* connection, CAdbcError* error)
+    AdbcStatusCode AdbcConnectionRelease(CAdbcConnection* connection, CAdbcError* error)
+
+    AdbcStatusCode AdbcStatementBind(CAdbcStatement* statement, CArrowArray*, CArrowSchema*, CAdbcError* error)
+    AdbcStatusCode AdbcStatementBindStream(CAdbcStatement* statement, CArrowArrayStream*, CAdbcError* error)
+    AdbcStatusCode AdbcStatementExecute(CAdbcStatement* statement, CAdbcError* error)
+    AdbcStatusCode AdbcStatementGetStream(CAdbcStatement* statement, CArrowArrayStream* c_stream, CAdbcError* error)
+    AdbcStatusCode AdbcStatementNew(CAdbcConnection* connection, CAdbcStatement* statement, CAdbcError* error)
+    AdbcStatusCode AdbcStatementPrepare(CAdbcStatement* statement, CAdbcError* error)
+    AdbcStatusCode AdbcStatementSetOption(CAdbcStatement* statement, const char* key, const char* value, CAdbcError* error)
+    AdbcStatusCode AdbcStatementSetSqlQuery(CAdbcStatement* statement, const char* query, CAdbcError* error)
+    AdbcStatusCode AdbcStatementRelease(CAdbcStatement* statement, CAdbcError* error)
+
+
+cdef extern from "adbc_driver_manager.h":
+    const char* AdbcStatusCodeMessage(AdbcStatusCode code)
+
+
+INGEST_OPTION_TARGET_TABLE = "adbc.ingest.target_table"
+
+
+class Error(Exception):
+    """PEP-249 compliant base exception class.
+
+    Attributes
+    ----------
+    status_code : int
+        The original ADBC status code.
+    vendor_code : int, optional
+        A vendor-specific status code if present.
+    sqlstate : str, optional
+        The SQLSTATE code if present.
+    """
+
+    def __init__(self, message, *, status_code, vendor_code=None, sqlstate=None):
+        super().__init__(message)
+        self.status_code = status_code
+        self.vendor_code = None
+        self.sqlstate = None
+
+
+class InterfaceError(Error):
+    pass
+
+
+class DatabaseError(Error):
+    pass
+
+
+class DataError(DatabaseError):
+    pass
+
+
+class OperationalError(DatabaseError):
+    pass
+
+
+class IntegrityError(DatabaseError):
+    pass
+
+
+class InternalError(DatabaseError):
+    pass
+
+
+class ProgrammingError(DatabaseError):
+    pass
+
+
+class NotSupportedError(DatabaseError):
+    pass
+
+
+cdef void check_error(AdbcStatusCode status, CAdbcError* error) except *:
+    if status == ADBC_STATUS_OK:
+        return
+
+    message = AdbcStatusCodeMessage(status).decode("utf-8")
+    vendor_code = None
+    sqlstate = None
+
+    if error != NULL:
+        if error.message != NULL:
+            message += ": "
+            message += error.message.decode("utf-8")
+        if error.vendor_code:
+            vendor_code = error.vendor_code
+        if error.sqlstate[0] != 0:
+            sqlstate = error.sqlstate.decode("ascii")
+        error.release(error)
+
+    klass = Error
+    if status in (ADBC_STATUS_INVALID_DATA,):
+        klass = DataError
+    elif status in (ADBC_STATUS_IO, ADBC_STATUS_CANCELLED, ADBC_STATUS_TIMEOUT):
+        klass = OperationalError
+    elif status in (ADBC_STATUS_INTEGRITY,):
+        klass = IntegrityError
+    elif status in (ADBC_STATUS_INTERNAL,):
+        klass = InternalError
+    elif status in (ADBC_STATUS_ALREADY_EXISTS, ADBC_STATUS_INVALID_ARGUMENT, ADBC_STATUS_INVALID_STATE, ADBC_STATUS_UNAUTHENTICATED, ADBC_STATUS_UNAUTHORIZED):
+        klass = ProgrammingError
+    elif status == ADBC_STATUS_NOT_IMPLEMENTED:
+        klass = NotSupportedError
+    raise klass(message, status_code=status)
+
+
+cdef CAdbcError empty_error():
+    cdef CAdbcError error
+    memset(&error, 0, cython.sizeof(error))
+    return error
+
+
+cdef class _AdbcHandle:
+    def __enter__(self) -> "Self":
+        return self
+
+    def __exit__(self, exc_type, exc_val, exc_tb) -> None:
+        self.close()
+
+
+cdef class AdbcDatabase(_AdbcHandle):
+    cdef:
+        CAdbcDatabase database
+
+    def __init__(self, **kwargs) -> None:
+        cdef CAdbcError c_error = empty_error()
+        cdef AdbcStatusCode status
+        cdef const char* c_key
+        cdef const char* c_value
+        memset(&self.database, 0, cython.sizeof(CAdbcDatabase))
+
+        status = AdbcDatabaseNew(&self.database, &c_error)
+        check_error(status, &c_error)
+
+        for key, value in kwargs.items():
+            key = key.encode("utf-8")
+            value = value.encode("utf-8")
+            c_key = key
+            c_value = value
+            status = AdbcDatabaseSetOption(&self.database, c_key, c_value, &c_error)
+            check_error(status, &c_error)
+
+        status = AdbcDatabaseInit(&self.database, &c_error)
+        check_error(status, &c_error)
+
+    def close(self) -> None:
+        if self.database.private_data == NULL:
+            return
+
+        cdef CAdbcError c_error = empty_error()
+        cdef AdbcStatusCode status = AdbcDatabaseRelease(&self.database, &c_error)
+        check_error(status, &c_error)
+
+
+cdef class AdbcConnection(_AdbcHandle):
+    cdef:
+        CAdbcConnection connection
+
+    def __init__(self, AdbcDatabase database, **kwargs) -> None:
+        cdef CAdbcError c_error = empty_error()
+        cdef AdbcStatusCode status
+        cdef const char* c_key
+        cdef const char* c_value
+        memset(&self.connection, 0, cython.sizeof(CAdbcConnection))
+
+        status = AdbcConnectionNew(&database.database, &self.connection, &c_error)
+        check_error(status, &c_error)
+
+        for key, value in kwargs.items():
+            key = key.encode("utf-8")
+            value = value.encode("utf-8")
+            c_key = key
+            c_value = value
+            status = AdbcConnectionSetOption(&self.connection, c_key, c_value, &c_error)
+            check_error(status, &c_error)
+
+        status = AdbcConnectionInit(&self.connection, &c_error)
+        check_error(status, &c_error)
+
+    def close(self) -> None:
+        if self.connection.private_data == NULL:
+            return
+
+        cdef CAdbcError c_error = empty_error()
+        cdef AdbcStatusCode status = AdbcConnectionRelease(&self.connection, &c_error)
+        check_error(status, &c_error)
+
+
+cdef class AdbcStatement(_AdbcHandle):
+    cdef:
+        CAdbcStatement statement
+
+    def __init__(self, AdbcConnection connection) -> None:
+        cdef CAdbcError c_error = empty_error()
+        cdef const char* c_key
+        cdef const char* c_value
+        memset(&self.statement, 0, cython.sizeof(CAdbcStatement))
+
+        status = AdbcStatementNew(&connection.connection, &self.statement, &c_error)
+        check_error(status, &c_error)
+
+    def bind(self, data) -> None:
+        """
+        Parameters
+        ----------
+        data : pyarrow.RecordBatch, pyarrow.RecordBatchReader, or pyarrow.Table
+        """
+        cdef CAdbcError c_error = empty_error()
+        cdef CArrowArray c_array
+        cdef CArrowSchema c_schema
+        cdef CArrowArrayStream c_stream
+        if isinstance(data, pyarrow.RecordBatch):
+            data._export_to_c(<uintptr_t> &c_array, <uintptr_t>&c_schema)
+            status = AdbcStatementBind(&self.statement, &c_array, &c_schema, &c_error)
+        else:
+            if isinstance(data, pyarrow.Table):
+                # Table lacks the export function
+                data = data.to_reader()
+            elif not isinstance(data, pyarrow.RecordBatchReader):
+                raise TypeError("data must be RecordBatch(Reader) or Table")
+            data._export_to_c(<uintptr_t> &c_stream)
+            status = AdbcStatementBindStream(&self.statement, &c_stream, &c_error)
+
+        check_error(status, &c_error)
+
+    def close(self) -> None:
+        if self.statement.private_data == NULL:
+            return
+
+        cdef CAdbcError c_error = empty_error()
+        cdef AdbcStatusCode status = AdbcStatementRelease(&self.statement, &c_error)
+        check_error(status, &c_error)
+
+    def execute(self) -> None:
+        cdef CAdbcError c_error = empty_error()
+        status = AdbcStatementExecute(&self.statement, &c_error)
+        check_error(status, &c_error)
+
+    def get_stream(self) -> pyarrow.RecordBatchReader:
+        cdef CAdbcError c_error = empty_error()
+        cdef CArrowArrayStream c_stream
+        status = AdbcStatementGetStream(&self.statement, &c_stream, &c_error)
+        check_error(status, &c_error)
+        return pyarrow.RecordBatchReader._import_from_c(<uintptr_t> &c_stream)
+
+    def prepare(self) -> None:
+        cdef CAdbcError c_error = empty_error()
+        status = AdbcStatementPrepare(&self.statement, &c_error)
+        check_error(status, &c_error)
+
+    def set_options(self, **kwargs) -> None:
+        cdef CAdbcError c_error = empty_error()
+        for key, value in kwargs.items():
+            key = key.encode("utf-8")
+            value = value.encode("utf-8")
+            c_key = key
+            c_value = value
+            status = AdbcStatementSetOption(&self.statement, c_key, c_value, &c_error)
+            check_error(status, &c_error)
+
+    def set_sql_query(self, query: str) -> None:
+        cdef CAdbcError c_error = empty_error()
+        status = AdbcStatementSetSqlQuery(&self.statement, query.encode("utf-8"), &c_error)
+        check_error(status, &c_error)
diff --git a/python/adbc_driver_manager/adbc_driver_manager/tests/test_lowlevel.py b/python/adbc_driver_manager/adbc_driver_manager/tests/test_lowlevel.py
new file mode 100644
index 0000000..114ccde
--- /dev/null
+++ b/python/adbc_driver_manager/adbc_driver_manager/tests/test_lowlevel.py
@@ -0,0 +1,80 @@
+# 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.
+
+import adbc_driver_manager._lib as _lib
+import pyarrow
+import pytest
+
+# TODO: make this parameterizable on different drivers?
+
+
+def test_database_init():
+    with pytest.raises(
+        _lib.ProgrammingError, match=".*Must provide 'driver' parameter.*"
+    ):
+        with _lib.AdbcDatabase():
+            pass
+
+
+@pytest.fixture
+def sqlite():
+    with _lib.AdbcDatabase(
+        driver="adbc_driver_sqlite",
+        entrypoint="AdbcSqliteDriverInit",
+    ) as db:
+        with _lib.AdbcConnection(db) as conn:
+            yield (db, conn)
+
+
+def test_query(sqlite):
+    _, conn = sqlite
+    with _lib.AdbcStatement(conn) as stmt:
+        stmt.set_sql_query("SELECT 1")
+        stmt.execute()
+        assert stmt.get_stream().read_all() == pyarrow.table([[1]], names=["1"])
+
+
+def test_prepared(sqlite):
+    _, conn = sqlite
+    with _lib.AdbcStatement(conn) as stmt:
+        stmt.set_sql_query("SELECT ?")
+        stmt.prepare()
+
+        stmt.bind(pyarrow.table([[1, 2, 3, 4]], names=["1"]))
+        stmt.execute()
+        assert stmt.get_stream().read_all() == pyarrow.table(
+            [[1, 2, 3, 4]], names=["?"]
+        )
+
+
+def test_ingest(sqlite):
+    _, conn = sqlite
+    data = pyarrow.table(
+        [
+            [1, 2, 3, 4],
+            ["a", "b", "c", "d"],
+        ],
+        names=["ints", "strs"],
+    )
+    with _lib.AdbcStatement(conn) as stmt:
+        stmt.set_options(**{_lib.INGEST_OPTION_TARGET_TABLE: "foo"})
+        stmt.bind(data)
+        stmt.execute()
+
+        stmt.set_sql_query("SELECT * FROM foo")
+        stmt.execute()
+        assert stmt.get_stream().read_all() == data
diff --git a/python/adbc_driver_manager/requirements-dev.txt b/python/adbc_driver_manager/requirements-dev.txt
new file mode 100644
index 0000000..7ede7bd
--- /dev/null
+++ b/python/adbc_driver_manager/requirements-dev.txt
@@ -0,0 +1,19 @@
+# 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.
+
+Cython
+pytest
diff --git a/python/adbc_driver_manager/setup.py b/python/adbc_driver_manager/setup.py
new file mode 100644
index 0000000..475b279
--- /dev/null
+++ b/python/adbc_driver_manager/setup.py
@@ -0,0 +1,38 @@
+#!/usr/bin/env python
+
+# 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.
+
+from Cython.Build import cythonize
+from setuptools import Extension, setup
+
+setup(
+    name="ADBC Driver Manager",
+    ext_modules=cythonize(
+        Extension(
+            name="adbc_driver_manager._lib",
+            sources=[
+                "adbc_driver_manager/_lib.pyx",
+                "../../adbc_driver_manager/adbc_driver_manager.cc",
+            ],
+            include_dirs=["../../", "../../adbc_driver_manager"],
+            # extra_compile_args=["-ggdb", "-Og"],
+        ),
+    ),
+    packages=["adbc_driver_manager"],
+    zip_safe=False,
+)