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/06/14 15:22:06 UTC

[arrow-adbc] branch main updated: test(c): add integration test with DuckDB (#779)

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 80572fe8 test(c): add integration test with DuckDB (#779)
80572fe8 is described below

commit 80572fe8eb6fe7831980496e770e9bc3014b2e08
Author: David Li <li...@gmail.com>
AuthorDate: Wed Jun 14 11:22:01 2023 -0400

    test(c): add integration test with DuckDB (#779)
    
    Fixes #595.
---
 .github/workflows/integration.yml   |  49 ++++++++++++++++++
 c/CMakeLists.txt                    |   8 +++
 c/cmake_modules/DefineOptions.cmake |   5 +-
 c/integration/duckdb/CMakeLists.txt |  75 +++++++++++++++++++++++++++
 c/integration/duckdb/duckdb_test.cc | 100 ++++++++++++++++++++++++++++++++++++
 c/validation/adbc_validation.cc     |  37 ++++++++++---
 c/validation/adbc_validation.h      |   3 ++
 ci/scripts/cpp_build.sh             |   5 +-
 ci/scripts/cpp_test.sh              |   4 ++
 9 files changed, 276 insertions(+), 10 deletions(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 2f39eee3..3a06373b 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -49,6 +49,55 @@ env:
   CACHE_NUMBER: "0"
 
 jobs:
+  duckdb:
+    name: "DuckDB Integration Tests"
+    runs-on: ubuntu-latest
+    steps:
+      - uses: actions/checkout@v3
+        with:
+          fetch-depth: 0
+          persist-credentials: false
+      - name: Get Date
+        id: get-date
+        shell: bash
+        run: |
+          echo "today=$(/bin/date -u '+%Y%m%d')" >> $GITHUB_OUTPUT
+      - name: Cache Conda
+        uses: actions/cache/restore@v3
+        with:
+          path: ~/conda_pkgs_dir
+          key: conda-${{ runner.os }}-${{ steps.get-date.outputs.today }}-${{ env.CACHE_NUMBER }}-${{ hashFiles('ci/**') }}
+      - uses: conda-incubator/setup-miniconda@v2
+        with:
+          miniforge-variant: Mambaforge
+          miniforge-version: latest
+          use-only-tar-bz2: false
+          use-mamba: true
+      - name: Install Dependencies
+        shell: bash -l {0}
+        run: |
+          mamba install -c conda-forge \
+            --file ci/conda_env_cpp.txt
+      - name: Build DuckDB Integration Tests
+        shell: bash -l {0}
+        env:
+          BUILD_ALL: "0"
+          BUILD_DRIVER_MANAGER: "1"
+          BUILD_INTEGRATION_DUCKDB: "1"
+          ADBC_USE_ASAN: "OFF"
+          ADBC_USE_UBSAN: "OFF"
+          # Make seems to have issues with the nested DuckDB build
+          ADBC_CMAKE_ARGS: "-GNinja"
+        run: |
+          ./ci/scripts/cpp_build.sh "$(pwd)" "$(pwd)/build"
+      - name: Run DuckDB Integration Tests
+        shell: bash -l {0}
+        env:
+          BUILD_ALL: "0"
+          BUILD_INTEGRATION_DUCKDB: "1"
+        run: |
+          ./ci/scripts/cpp_test.sh "$(pwd)/build"
+
   flightsql:
     name: "FlightSQL Integration Tests (Dremio and SQLite)"
     runs-on: ubuntu-latest
diff --git a/c/CMakeLists.txt b/c/CMakeLists.txt
index ac594b03..4115da7d 100644
--- a/c/CMakeLists.txt
+++ b/c/CMakeLists.txt
@@ -33,6 +33,10 @@ if(ADBC_BUILD_TESTS)
   add_subdirectory(validation)
 endif()
 
+if(ADBC_INTEGRATION_DUCKDB)
+  set(ADBC_DRIVER_MANAGER ON)
+endif()
+
 if(ADBC_DRIVER_FLIGHTSQL)
   add_subdirectory(driver/flightsql)
 endif()
@@ -53,5 +57,9 @@ if(ADBC_DRIVER_SNOWFLAKE)
   add_subdirectory(driver/snowflake)
 endif()
 
+if(ADBC_INTEGRATION_DUCKDB)
+  add_subdirectory(integration/duckdb)
+endif()
+
 validate_config()
 config_summary_message()
diff --git a/c/cmake_modules/DefineOptions.cmake b/c/cmake_modules/DefineOptions.cmake
index f0156831..42b8f4f5 100644
--- a/c/cmake_modules/DefineOptions.cmake
+++ b/c/cmake_modules/DefineOptions.cmake
@@ -233,6 +233,8 @@ if("${CMAKE_SOURCE_DIR}" STREQUAL "${CMAKE_CURRENT_SOURCE_DIR}")
   define_option(ADBC_DRIVER_POSTGRESQL "Build the PostgreSQL driver" OFF)
   define_option(ADBC_DRIVER_SQLITE "Build the SQLite driver" OFF)
   define_option(ADBC_DRIVER_SNOWFLAKE "Build the Snowflake driver" OFF)
+
+  define_option(ADBC_INTEGRATION_DUCKDB "Build the test suite for DuckDB" OFF)
 endif()
 
 macro(validate_config)
@@ -255,10 +257,11 @@ endmacro()
 
 macro(config_summary_message)
   message(STATUS "---------------------------------------------------------------------")
-  message(STATUS "Arrow version:                                 ${ADBC_VERSION}")
+  message(STATUS "ADBC version: ${ADBC_VERSION}")
   message(STATUS)
   message(STATUS "Build configuration summary:")
 
+  message(STATUS "  CMake version: ${CMAKE_VERSION}")
   message(STATUS "  Generator: ${CMAKE_GENERATOR}")
   message(STATUS "  Build type: ${CMAKE_BUILD_TYPE}")
   message(STATUS "  Source directory: ${CMAKE_CURRENT_SOURCE_DIR}")
diff --git a/c/integration/duckdb/CMakeLists.txt b/c/integration/duckdb/CMakeLists.txt
new file mode 100644
index 00000000..52fb9d0f
--- /dev/null
+++ b/c/integration/duckdb/CMakeLists.txt
@@ -0,0 +1,75 @@
+# 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(FetchContent)
+
+if(ADBC_BUILD_TESTS)
+  fetchcontent_declare(duckdb
+                       GIT_REPOSITORY https://github.com/duckdb/duckdb.git
+                       GIT_TAG e8e4cea5ec9d1a84c1f516d0f0674f8785a3e786 # v0.8.0
+                       GIT_PROGRESS TRUE
+                       USES_TERMINAL_DOWNLOAD TRUE)
+  set(BUILD_JEMALLOC_EXTENSION
+      OFF
+      CACHE INTERNAL "Disable")
+  set(BUILD_NODE
+      OFF
+      CACHE INTERNAL "Disable")
+  set(BUILD_PYTHON
+      OFF
+      CACHE INTERNAL "Disable")
+  set(BUILD_R
+      OFF
+      CACHE INTERNAL "Disable")
+  set(BUILD_SHELL
+      OFF
+      CACHE INTERNAL "Disable")
+  set(BUILD_UNITTESTS
+      OFF
+      CACHE INTERNAL "Disable")
+  set(ENABLE_SANITIZER
+      OFF
+      CACHE INTERNAL "Disable ASAN")
+  set(ENABLE_UBSAN
+      OFF
+      CACHE INTERNAL "Disable UBSAN")
+  # Force cmake to honor our options here in the subproject
+  cmake_policy(SET CMP0077 NEW)
+  fetchcontent_makeavailable(duckdb)
+
+  include_directories(SYSTEM ${REPOSITORY_ROOT})
+  include_directories(SYSTEM ${REPOSITORY_ROOT}/c/)
+  include_directories(PRIVATE ${REPOSITORY_ROOT}/c/driver)
+  include_directories(SYSTEM ${REPOSITORY_ROOT}/c/vendor)
+
+  add_test_case(integration_duckdb_test
+                PREFIX
+                adbc
+                EXTRA_LABELS
+                integration-duckdb
+                SOURCES
+                duckdb_test.cc
+                EXTRA_LINK_LIBS
+                adbc_driver_common
+                adbc_driver_manager_static
+                adbc_validation
+                duckdb
+                nanoarrow)
+  add_dependencies(adbc-integration-duckdb-test duckdb)
+  target_compile_features(adbc-integration-duckdb-test PRIVATE cxx_std_17)
+  adbc_configure_target(adbc-integration-duckdb-test)
+endif()
diff --git a/c/integration/duckdb/duckdb_test.cc b/c/integration/duckdb/duckdb_test.cc
new file mode 100644
index 00000000..b2a5c524
--- /dev/null
+++ b/c/integration/duckdb/duckdb_test.cc
@@ -0,0 +1,100 @@
+// 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 <cstdlib>
+
+#include <adbc.h>
+#include <driver_manager/adbc_driver_manager.h>
+#include <duckdb/common/adbc/adbc-init.hpp>
+
+#include "validation/adbc_validation.h"
+#include "validation/adbc_validation_util.h"
+
+// Convert between our definitions and DuckDB's
+AdbcStatusCode DuckDbDriverInitFunc(int version, void* driver, struct AdbcError* error) {
+  return duckdb_adbc_init(static_cast<size_t>(version),
+                          reinterpret_cast<duckdb_adbc::AdbcDriver*>(driver),
+                          reinterpret_cast<duckdb_adbc::AdbcError*>(error));
+}
+
+class DuckDbQuirks : public adbc_validation::DriverQuirks {
+ public:
+  AdbcStatusCode SetupDatabase(struct AdbcDatabase* database,
+                               struct AdbcError* error) const override {
+    if (auto status =
+            AdbcDriverManagerDatabaseSetInitFunc(database, DuckDbDriverInitFunc, error);
+        status != ADBC_STATUS_OK) {
+      return status;
+    }
+
+    return ADBC_STATUS_OK;
+  }
+
+  std::string BindParameter(int index) const override { return "?"; }
+
+  bool supports_bulk_ingest() const override { return false; }
+  bool supports_concurrent_statements() const override { return true; }
+  bool supports_dynamic_parameter_binding() const override { return false; }
+  bool supports_get_sql_info() const override { return false; }
+  bool supports_get_objects() const override { return false; }
+  bool supports_rows_affected() const override { return false; }
+  bool supports_transactions() const override { return false; }
+};
+
+class DuckDbDatabaseTest : public ::testing::Test, public adbc_validation::DatabaseTest {
+ public:
+  const adbc_validation::DriverQuirks* quirks() const override { return &quirks_; }
+  void SetUp() override { ASSERT_NO_FATAL_FAILURE(SetUpTest()); }
+  void TearDown() override { ASSERT_NO_FATAL_FAILURE(TearDownTest()); }
+
+ protected:
+  DuckDbQuirks quirks_;
+};
+ADBCV_TEST_DATABASE(DuckDbDatabaseTest)
+
+class DuckDbConnectionTest : public ::testing::Test,
+                             public adbc_validation::ConnectionTest {
+ public:
+  const adbc_validation::DriverQuirks* quirks() const override { return &quirks_; }
+  void SetUp() override { ASSERT_NO_FATAL_FAILURE(SetUpTest()); }
+  void TearDown() override { ASSERT_NO_FATAL_FAILURE(TearDownTest()); }
+
+  void TestAutocommitDefault() { GTEST_SKIP(); }
+  void TestMetadataGetTableSchema() { GTEST_SKIP(); }
+  void TestMetadataGetTableTypes() { GTEST_SKIP(); }
+
+ protected:
+  DuckDbQuirks quirks_;
+};
+ADBCV_TEST_CONNECTION(DuckDbConnectionTest)
+
+class DuckDbStatementTest : public ::testing::Test,
+                            public adbc_validation::StatementTest {
+ public:
+  const adbc_validation::DriverQuirks* quirks() const override { return &quirks_; }
+  void SetUp() override { ASSERT_NO_FATAL_FAILURE(SetUpTest()); }
+  void TearDown() override { ASSERT_NO_FATAL_FAILURE(TearDownTest()); }
+
+  // DuckDB doesn't guard against this
+  void TestNewInit() { GTEST_SKIP(); }
+  // Accepts Prepare() without any query
+  void TestSqlPrepareErrorNoQuery() { GTEST_SKIP(); }
+
+ protected:
+  DuckDbQuirks quirks_;
+};
+ADBCV_TEST_STATEMENT(DuckDbStatementTest)
diff --git a/c/validation/adbc_validation.cc b/c/validation/adbc_validation.cc
index 2d88347c..d73b5564 100644
--- a/c/validation/adbc_validation.cc
+++ b/c/validation/adbc_validation.cc
@@ -1460,8 +1460,13 @@ void StatementTest::TestSqlPrepareSelectNoParams() {
   ASSERT_THAT(AdbcStatementExecuteQuery(&statement, &reader.stream.value,
                                         &reader.rows_affected, &error),
               IsOkStatus(&error));
-  ASSERT_THAT(reader.rows_affected,
-              ::testing::AnyOf(::testing::Eq(1), ::testing::Eq(-1)));
+  if (quirks()->supports_rows_affected()) {
+    ASSERT_THAT(reader.rows_affected,
+                ::testing::AnyOf(::testing::Eq(1), ::testing::Eq(-1)));
+  } else {
+    ASSERT_THAT(reader.rows_affected,
+                ::testing::Not(::testing::AnyOf(::testing::Eq(1), ::testing::Eq(-1))));
+  }
 
   ASSERT_NO_FATAL_FAILURE(reader.GetSchema());
   ASSERT_EQ(1, reader.schema->n_children);
@@ -1789,8 +1794,13 @@ void StatementTest::TestSqlQueryInts() {
     ASSERT_THAT(AdbcStatementExecuteQuery(&statement, &reader.stream.value,
                                           &reader.rows_affected, &error),
                 IsOkStatus(&error));
-    ASSERT_THAT(reader.rows_affected,
-                ::testing::AnyOf(::testing::Eq(1), ::testing::Eq(-1)));
+    if (quirks()->supports_rows_affected()) {
+      ASSERT_THAT(reader.rows_affected,
+                  ::testing::AnyOf(::testing::Eq(1), ::testing::Eq(-1)));
+    } else {
+      ASSERT_THAT(reader.rows_affected,
+                  ::testing::Not(::testing::AnyOf(::testing::Eq(1), ::testing::Eq(-1))));
+    }
 
     ASSERT_NO_FATAL_FAILURE(reader.GetSchema());
     ASSERT_EQ(1, reader.schema->n_children);
@@ -1830,8 +1840,13 @@ void StatementTest::TestSqlQueryFloats() {
     ASSERT_THAT(AdbcStatementExecuteQuery(&statement, &reader.stream.value,
                                           &reader.rows_affected, &error),
                 IsOkStatus(&error));
-    ASSERT_THAT(reader.rows_affected,
-                ::testing::AnyOf(::testing::Eq(1), ::testing::Eq(-1)));
+    if (quirks()->supports_rows_affected()) {
+      ASSERT_THAT(reader.rows_affected,
+                  ::testing::AnyOf(::testing::Eq(1), ::testing::Eq(-1)));
+    } else {
+      ASSERT_THAT(reader.rows_affected,
+                  ::testing::Not(::testing::AnyOf(::testing::Eq(1), ::testing::Eq(-1))));
+    }
 
     ASSERT_NO_FATAL_FAILURE(reader.GetSchema());
     ASSERT_EQ(1, reader.schema->n_children);
@@ -1873,8 +1888,13 @@ void StatementTest::TestSqlQueryStrings() {
     ASSERT_THAT(AdbcStatementExecuteQuery(&statement, &reader.stream.value,
                                           &reader.rows_affected, &error),
                 IsOkStatus(&error));
-    ASSERT_THAT(reader.rows_affected,
-                ::testing::AnyOf(::testing::Eq(1), ::testing::Eq(-1)));
+    if (quirks()->supports_rows_affected()) {
+      ASSERT_THAT(reader.rows_affected,
+                  ::testing::AnyOf(::testing::Eq(1), ::testing::Eq(-1)));
+    } else {
+      ASSERT_THAT(reader.rows_affected,
+                  ::testing::Not(::testing::AnyOf(::testing::Eq(1), ::testing::Eq(-1))));
+    }
 
     ASSERT_NO_FATAL_FAILURE(reader.GetSchema());
     ASSERT_EQ(1, reader.schema->n_children);
@@ -1887,6 +1907,7 @@ void StatementTest::TestSqlQueryStrings() {
     ASSERT_FALSE(ArrowArrayViewIsNull(&reader.array_view.value, 0));
     ASSERT_FALSE(ArrowArrayViewIsNull(reader.array_view->children[0], 0));
     switch (reader.fields[0].type) {
+      case NANOARROW_TYPE_LARGE_STRING:
       case NANOARROW_TYPE_STRING: {
         ASSERT_NO_FATAL_FAILURE(
             CompareArray<std::string>(reader.array_view->children[0], {"SaShiSuSeSo"}));
diff --git a/c/validation/adbc_validation.h b/c/validation/adbc_validation.h
index aa49fd40..ffff035e 100644
--- a/c/validation/adbc_validation.h
+++ b/c/validation/adbc_validation.h
@@ -103,6 +103,9 @@ class DriverQuirks {
   /// \brief Whether dynamic parameter bindings are supported for prepare
   virtual bool supports_dynamic_parameter_binding() const { return true; }
 
+  /// \brief Whether ExecuteQuery sets rows_affected appropriately
+  virtual bool supports_rows_affected() const { return true; }
+
   /// \brief Default catalog to use for tests
   virtual std::string catalog() const { return ""; }
 
diff --git a/ci/scripts/cpp_build.sh b/ci/scripts/cpp_build.sh
index 0c9e496e..f7a37f6a 100755
--- a/ci/scripts/cpp_build.sh
+++ b/ci/scripts/cpp_build.sh
@@ -24,9 +24,11 @@ set -e
 : ${BUILD_DRIVER_SQLITE:=${BUILD_ALL}}
 : ${BUILD_DRIVER_FLIGHTSQL:=${BUILD_ALL}}
 : ${BUILD_DRIVER_SNOWFLAKE:=${BUILD_ALL}}
+# Must be explicitly enabled
+: ${BUILD_INTEGRATION_DUCKDB:=0}
 
 : ${ADBC_BUILD_SHARED:=ON}
-: ${ADBC_BUILD_STATIC:=OFF}
+: ${ADBC_BUILD_STATIC:=${BUILD_INTEGRATION_DUCKDB}}
 : ${ADBC_BUILD_TESTS:=ON}
 : ${ADBC_USE_ASAN:=ON}
 : ${ADBC_USE_UBSAN:=ON}
@@ -56,6 +58,7 @@ build_subproject() {
           -DADBC_DRIVER_SQLITE="${BUILD_DRIVER_SQLITE}" \
           -DADBC_DRIVER_FLIGHTSQL="${BUILD_DRIVER_FLIGHTSQL}" \
           -DADBC_DRIVER_SNOWFLAKE="${BUILD_DRIVER_SNOWFLAKE}" \
+          -DADBC_INTEGRATION_DUCKDB="${BUILD_INTEGRATION_DUCKDB}" \
           -DADBC_BUILD_STATIC="${ADBC_BUILD_STATIC}" \
           -DADBC_BUILD_TESTS="${ADBC_BUILD_TESTS}" \
           -DADBC_USE_ASAN="${ADBC_USE_ASAN}" \
diff --git a/ci/scripts/cpp_test.sh b/ci/scripts/cpp_test.sh
index 003ba467..e4f7977f 100755
--- a/ci/scripts/cpp_test.sh
+++ b/ci/scripts/cpp_test.sh
@@ -24,6 +24,7 @@ set -e
 : ${BUILD_DRIVER_SQLITE:=${BUILD_ALL}}
 : ${BUILD_DRIVER_FLIGHTSQL:=${BUILD_ALL}}
 : ${BUILD_DRIVER_SNOWFLAKE:=${BUILD_ALL}}
+: ${BUILD_INTEGRATION_DUCKDB:=${BUILD_ALL}}
 
 test_project() {
     local -r build_dir="${1}"
@@ -46,6 +47,9 @@ test_project() {
     if [[ "${BUILD_DRIVER_SNOWFLAKE}" -gt 0 ]]; then
        labels="${labels}|driver-snowflake"
     fi
+    if [[ "${BUILD_INTEGRATION_DUCKDB}" -gt 0 ]]; then
+       labels="${labels}|integration-duckdb"
+    fi
 
     ctest \
         --output-on-failure \