diff --git a/Cargo.lock b/Cargo.lock index fc193157f44..07cbd4244c5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10337,6 +10337,28 @@ dependencies = [ "vortex-session", ] +[[package]] +name = "vortex-clickhouse" +version = "0.1.0" +dependencies = [ + "async-fs", + "bitvec", + "futures", + "glob", + "itertools 0.14.0", + "num-traits", + "object_store", + "parking_lot", + "paste", + "smol", + "tempfile", + "tracing", + "url", + "vortex", + "vortex-utils", + "walkdir", +] + [[package]] name = "vortex-compute" version = "0.1.0" diff --git a/Cargo.toml b/Cargo.toml index 732c61aea52..101e18a4800 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -24,6 +24,7 @@ members = [ "vortex-bench", "vortex-datafusion", "vortex-duckdb", + "vortex-clickhouse", "vortex-cuda", "vortex-cuda/cub", "vortex-cuda/macros", @@ -280,6 +281,7 @@ vortex-zstd = { version = "0.1.0", path = "./encodings/zstd", default-features = # No version constraints for unpublished crates. vortex-bench = { path = "./vortex-bench", default-features = false } +vortex-clickhouse = { path = "./vortex-clickhouse" } vortex-cuda = { path = "./vortex-cuda", default-features = false } vortex-cuda-macros = { path = "./vortex-cuda/macros" } vortex-duckdb = { path = "./vortex-duckdb", default-features = false } diff --git a/vortex-clickhouse/Cargo.toml b/vortex-clickhouse/Cargo.toml new file mode 100644 index 00000000000..f1a2f019d3b --- /dev/null +++ b/vortex-clickhouse/Cargo.toml @@ -0,0 +1,49 @@ +[package] +name = "vortex-clickhouse" +version = { workspace = true } +description = "Vortex ClickHouse extension" +homepage = { workspace = true } +repository = { workspace = true } +authors = { workspace = true } +license = { workspace = true } +publish = false +keywords = { workspace = true } +include = { workspace = true } +edition = { workspace = true } +rust-version = { workspace = true } +categories = { workspace = true } +readme = { workspace = true } + +# The `links` key enables DEP_CLICKHOUSE_* environment variables in dependent build scripts. +# See: https://doc.rust-lang.org/cargo/reference/build-scripts.html#the-links-manifest-key +links = "clickhouse" + +[lib] +name = "vortex_clickhouse" +path = "src/lib.rs" +crate-type = ["staticlib", "cdylib", "rlib"] + +[dependencies] +async-fs = { workspace = true } +bitvec = { workspace = true } +futures = { workspace = true } +glob = { workspace = true } +itertools = { workspace = true } +num-traits = { workspace = true } +object_store = { workspace = true, features = ["aws", "gcp", "azure", "http"] } +parking_lot = { workspace = true } +paste = { workspace = true } +smol = { workspace = true } +tracing = { workspace = true } +url = { workspace = true } +vortex = { workspace = true, features = ["files", "tokio", "object_store"] } +vortex-utils = { workspace = true, features = ["dashmap"] } + +[dev-dependencies] +tempfile = { workspace = true } + +[lints] +workspace = true + +[build-dependencies] +walkdir = { workspace = true } diff --git a/vortex-clickhouse/README.md b/vortex-clickhouse/README.md new file mode 100644 index 00000000000..eb0c5c08ce3 --- /dev/null +++ b/vortex-clickhouse/README.md @@ -0,0 +1,65 @@ +# Vortex ClickHouse + +ClickHouse format plugin for reading and writing [Vortex](https://github.com/spiraldb/vortex) files. Implemented as a Rust static library linked into ClickHouse via C++ FFI wrappers. + +## Features + +- Read Vortex files: `SELECT * FROM file('data.vortex', 'Vortex')` +- Write Vortex files: `INSERT ... TO 'output.vortex' FORMAT Vortex` +- Automatic schema inference +- Predicate & projection pushdown + +## Prerequisites + +- **Ninja**: `brew install ninja` (macOS) | `apt-get install ninja-build` (Ubuntu) +- **CMake 3.20+**: `brew install cmake` (macOS) | `apt-get install cmake` (Ubuntu) +- **Rust 1.89+** +- **C++17 compatible compiler**: GCC or Clang + +## Build Modes + +### Default (Release) + +```bash +cargo build -p vortex-clickhouse +``` + +### Debug Build + +Opt into ClickHouse debug build: `VX_CLICKHOUSE_DEBUG=1`. + +```bash +VX_CLICKHOUSE_DEBUG=1 cargo build -p vortex-clickhouse +``` + +## Environment Variables + +| Variable | Effect | +| ---------------------- | ------------------------------------------------------------- | +| `VX_CLICKHOUSE_DEBUG` | Build ClickHouse in debug mode | +| `CLICKHOUSE_VERSION` | ClickHouse version to build against (default: latest release) | +| `CLICKHOUSE_SOURCE_DIR`| Path to ClickHouse source directory | + +## Running Tests + +```bash +# Default release build +cargo test -p vortex-clickhouse + +# Debug build +VX_CLICKHOUSE_DEBUG=1 cargo test -p vortex-clickhouse +``` + +## Usage + +```sql +-- Read from a Vortex file +SELECT * FROM file('data.vortex', 'Vortex'); + +-- Read with predicate pushdown +SELECT * FROM file('data.vortex', 'Vortex') WHERE id > 100; + +-- Write query results to Vortex +INSERT INTO FUNCTION file('output.vortex', 'Vortex') +SELECT * FROM my_table; +``` diff --git a/vortex-clickhouse/build.rs b/vortex-clickhouse/build.rs new file mode 100644 index 00000000000..6989bf6c8c8 --- /dev/null +++ b/vortex-clickhouse/build.rs @@ -0,0 +1,71 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +#![allow(clippy::unwrap_used)] +#![allow(clippy::expect_used)] +#![allow(clippy::panic)] + +//! Build script for vortex-clickhouse. +//! +//! This script: +//! 1. Downloads ClickHouse source code (for headers) - optional +//! 2. Generates Rust bindings from C++ headers (bindgen) - when headers are available +//! 3. Creates placeholder cpp.rs for now + +use std::env; +use std::fs; +use std::path::PathBuf; + +fn main() { + println!("cargo:rerun-if-env-changed=CLICKHOUSE_VERSION"); + println!("cargo:rerun-if-env-changed=CLICKHOUSE_SOURCE_DIR"); + + let crate_dir = PathBuf::from(env::var("CARGO_MANIFEST_DIR").unwrap()); + + // For now, we'll create a minimal cpp.rs since we don't have ClickHouse headers yet + // The full implementation will require ClickHouse headers + let cpp_rs_content = r#"// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! FFI bindings for ClickHouse. +//! +//! This module is auto-generated by bindgen when ClickHouse headers are available. +//! For now, it contains placeholder definitions. + +#![allow(dead_code)] +#![allow(non_camel_case_types)] +#![allow(non_upper_case_globals)] +#![allow(non_snake_case)] + +// Placeholder types - will be replaced by bindgen-generated code +// when ClickHouse headers are available. + +/// Result status for operations. +#[repr(C)] +#[derive(Debug, Copy, Clone, PartialEq, Eq)] +pub enum vortex_ch_state { + Success = 0, + Error = 1, +} +"#; + + let cpp_rs_path = crate_dir.join("src/cpp.rs"); + let existing = fs::read_to_string(&cpp_rs_path).unwrap_or_default(); + if existing != cpp_rs_content { + fs::write(&cpp_rs_path, cpp_rs_content).expect("Failed to write cpp.rs"); + } + + // Watch C/C++ source files for changes. + for entry in walkdir::WalkDir::new(crate_dir.join("cpp")) + .into_iter() + .flatten() + { + if entry + .path() + .extension() + .is_some_and(|ext| ext == "cpp" || ext == "h" || ext == "hpp") + { + println!("cargo:rerun-if-changed={}", entry.path().display()); + } + } +} diff --git a/vortex-clickhouse/cbindgen.toml b/vortex-clickhouse/cbindgen.toml new file mode 100644 index 00000000000..749d14367da --- /dev/null +++ b/vortex-clickhouse/cbindgen.toml @@ -0,0 +1,27 @@ +# cbindgen configuration for generating C header from Rust code. +# This exports Rust functions for C++ to call. + +language = "C" +include_guard = "VORTEX_CLICKHOUSE_H" +autogen_warning = "/* Warning: this file is autogenerated by cbindgen. Do not edit manually. */" + +# Include all exported Rust types +[export] +prefix = "vortex_" +include = [] +exclude = [] + +[parse] +parse_deps = false +clean = false + +[parse.expand] +crates = ["vortex-clickhouse"] + +[fn] +# Use C ABI for all exported functions +args = "vertical" + +[struct] +derive_eq = true +derive_neq = true diff --git a/vortex-clickhouse/cpp/CMakeLists.txt b/vortex-clickhouse/cpp/CMakeLists.txt new file mode 100644 index 00000000000..f1ad25ab44e --- /dev/null +++ b/vortex-clickhouse/cpp/CMakeLists.txt @@ -0,0 +1,26 @@ +cmake_minimum_required(VERSION 3.20) + +# This CMakeLists.txt is used when building the C++ components +# as part of ClickHouse's build system. + +# Source files +set(VORTEX_CLICKHOUSE_SOURCES + VortexBlockInputFormat.cpp + VortexBlockOutputFormat.cpp + registerFormats.cpp +) + +# Headers +set(VORTEX_CLICKHOUSE_HEADERS + include/clickhouse_vx.h + include/clickhouse_vx/common.h + include/clickhouse_vx/format.h + include/clickhouse_vx/column.h +) + +# Include directories +include_directories(include) + +# Note: When integrated into ClickHouse build system, this will be built +# as part of the main ClickHouse library. The vortex-clickhouse Rust crate +# will be linked as a static library. diff --git a/vortex-clickhouse/cpp/include/clickhouse_vx.h b/vortex-clickhouse/cpp/include/clickhouse_vx.h new file mode 100644 index 00000000000..2b6a747b8ae --- /dev/null +++ b/vortex-clickhouse/cpp/include/clickhouse_vx.h @@ -0,0 +1,53 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +#pragma once + +// Main header file for vortex-clickhouse C++ components. +// +// This header provides the C API for integrating Vortex with ClickHouse. +// Include this header in your ClickHouse plugin code. +// +// The API is organized into several components: +// +// - common.h: Basic types and handles +// - scanner.h: Reading Vortex files +// - exporter.h: Extracting data from Vortex arrays +// - writer.h: Writing Vortex files +// - column.h: Column type utilities +// - format.h: Format constants +// +// Example usage (reading): +// +// #include +// +// // Open a Vortex file +// VortexScanner* scanner = vortex_scanner_new("/path/to/data.vortex"); +// if (!scanner) { return handleError(); } +// +// // Get schema +// size_t num_cols = vortex_scanner_num_columns(scanner); +// for (size_t i = 0; i < num_cols; i++) { +// const char* name = vortex_scanner_column_name(scanner, i); +// const char* type = vortex_scanner_column_type(scanner, i); +// // ... configure ClickHouse columns ... +// } +// +// // Read data +// while (vortex_scanner_has_more(scanner)) { +// VortexExporterHandle* batch = vortex_scanner_read_batch(scanner); +// while (vortex_exporter_has_more(batch)) { +// // Export to ClickHouse columns +// vortex_exporter_export(batch, buffer, max_rows); +// } +// vortex_exporter_free(batch); +// } +// +// vortex_scanner_free(scanner); + +#include "clickhouse_vx/common.h" +#include "clickhouse_vx/format.h" +#include "clickhouse_vx/column.h" +#include "clickhouse_vx/scanner.h" +#include "clickhouse_vx/exporter.h" +#include "clickhouse_vx/writer.h" diff --git a/vortex-clickhouse/cpp/include/clickhouse_vx/column.h b/vortex-clickhouse/cpp/include/clickhouse_vx/column.h new file mode 100644 index 00000000000..88fa22cb7ec --- /dev/null +++ b/vortex-clickhouse/cpp/include/clickhouse_vx/column.h @@ -0,0 +1,67 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +#pragma once + +#include "common.h" + +// Column conversion utilities for ClickHouse integration. +// +// These functions provide type conversion between ClickHouse column types +// and Vortex array types. + +#ifdef __cplusplus +extern "C" { +#endif + +// ============================================================================= +// Type Information +// ============================================================================= + +/// Get the size in bytes for a ClickHouse type. +/// +/// @param clickhouse_type ClickHouse type string (e.g., "Int64", "Float64"). +/// @return Size in bytes for the type, or 0 for variable-length types or errors. +size_t vortex_clickhouse_type_size(const char* clickhouse_type); + +/// Check if a ClickHouse type is fixed-width. +/// +/// @param clickhouse_type ClickHouse type string. +/// @return 1 if fixed-width, 0 if variable-length or error. +int32_t vortex_clickhouse_type_is_fixed(const char* clickhouse_type); + +/// Check if a ClickHouse type is nullable. +/// +/// @param clickhouse_type ClickHouse type string (e.g., "Nullable(Int64)"). +/// @return 1 if nullable, 0 if not nullable. +int32_t vortex_clickhouse_type_is_nullable(const char* clickhouse_type); + +/// Get the inner type of a Nullable type. +/// +/// @param clickhouse_type ClickHouse type string (e.g., "Nullable(Int64)"). +/// @param inner_type Buffer to write the inner type string. +/// @param buffer_size Size of the buffer. +/// @return Length of the inner type string, or negative on error. +int32_t vortex_clickhouse_type_unwrap_nullable( + const char* clickhouse_type, + char* inner_type, + size_t buffer_size +); + +// ============================================================================= +// Column Data Helpers +// ============================================================================= + +/// Calculate the required buffer size for exporting a column. +/// +/// For fixed-width types, this returns num_rows * type_size. +/// For variable-length types, use vortex_scanner_column_data_size() instead. +/// +/// @param clickhouse_type ClickHouse type string. +/// @param num_rows Number of rows. +/// @return Required buffer size in bytes, or 0 on error. +size_t vortex_column_buffer_size(const char* clickhouse_type, size_t num_rows); + +#ifdef __cplusplus +} +#endif diff --git a/vortex-clickhouse/cpp/include/clickhouse_vx/common.h b/vortex-clickhouse/cpp/include/clickhouse_vx/common.h new file mode 100644 index 00000000000..1867be6d01d --- /dev/null +++ b/vortex-clickhouse/cpp/include/clickhouse_vx/common.h @@ -0,0 +1,70 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +#pragma once + +#include +#include + +// Common types and definitions for vortex-clickhouse. + +#ifdef __cplusplus +extern "C" { +#endif + +/// Result status for operations. +typedef enum { + VORTEX_CH_SUCCESS = 0, + VORTEX_CH_ERROR = 1, +} vortex_ch_status; + +/// Opaque handle to Vortex scanner. +/// Created by `vortex_scanner_new`, freed by `vortex_scanner_free`. +/// +/// Thread safety: NOT thread-safe. All calls on a given handle must be +/// serialized by the caller. See scanner.h for details. +typedef struct VortexScanner VortexScanner; + +/// Opaque handle to Vortex writer. +/// Created by vortex_writer_new(), freed by vortex_writer_free(). +typedef struct VortexWriter VortexWriter; + +/// Opaque handle to a column exporter. +/// Created by vortex_scanner_read_batch(), freed by vortex_exporter_free(). +typedef struct VortexExporterHandle VortexExporterHandle; + +// ============================================================================= +// Error Handling API +// ============================================================================= + +/// Get the last error message. +/// +/// Returns a null-terminated C string with the last error message, +/// or NULL if no error was set. The returned string must be freed +/// by calling `vortex_free_string()`. +/// +/// @return Error message string, or NULL if no error. Caller must free. +char* vortex_get_last_error(void); + +/// Check if there is a pending error. +/// +/// @return 1 if an error is set, 0 otherwise. +int32_t vortex_has_error(void); + +/// Clear the last error. +/// +/// Call this before starting a new operation if you want to ensure +/// no stale error messages are present. +void vortex_clear_error(void); + +/// Free a string returned by vortex FFI functions. +/// +/// This function must be called to free strings returned by functions like +/// `vortex_get_last_error()`, `vortex_scanner_column_name()`, etc. +/// +/// @param ptr String pointer to free. NULL is safely ignored. +void vortex_free_string(char* ptr); + +#ifdef __cplusplus +} +#endif diff --git a/vortex-clickhouse/cpp/include/clickhouse_vx/exporter.h b/vortex-clickhouse/cpp/include/clickhouse_vx/exporter.h new file mode 100644 index 00000000000..a486da13332 --- /dev/null +++ b/vortex-clickhouse/cpp/include/clickhouse_vx/exporter.h @@ -0,0 +1,225 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +#pragma once + +#include "common.h" + +// Exporter API for extracting data from Vortex arrays. +// +// Exporters are returned by vortex_scanner_read_batch() and provide a way to +// copy data from Vortex arrays into ClickHouse column buffers. +// +// Usage: +// VortexExporterHandle* batch = vortex_scanner_read_batch(scanner); +// while (vortex_exporter_has_more(batch)) { +// int64_t rows = vortex_exporter_export(batch, buffer, buffer_size); +// if (rows < 0) { /* handle error */ } +// // Process exported rows... +// } +// vortex_exporter_free(batch); + +#ifdef __cplusplus +extern "C" { +#endif + +// ============================================================================= +// Exporter Lifecycle +// ============================================================================= + +/// Free an exporter handle. +/// +/// @param handle Exporter handle to free. NULL is safely ignored. +void vortex_exporter_free(VortexExporterHandle* handle); + +// ============================================================================= +// Data Export +// ============================================================================= + +/// Check if the exporter has more data to export. +/// +/// @param handle Exporter handle. +/// @return 1 if more data available, 0 if no more data or handle is NULL. +int32_t vortex_exporter_has_more(const VortexExporterHandle* handle); + +/// Get the total number of rows in the exporter. +/// +/// @param handle Exporter handle. +/// @return Total number of rows, or 0 if handle is NULL. +size_t vortex_exporter_len(const VortexExporterHandle* handle); + +/// Export data to a buffer. +/// +/// This function copies data from the Vortex array into the provided buffer. +/// The buffer must be pre-allocated by the caller with sufficient size. +/// +/// For primitive types (Int8, Int16, Int32, Int64, UInt8, UInt16, UInt32, UInt64, +/// Float32, Float64), the buffer should be an array of the corresponding C type. +/// +/// For example, for Int64 columns: +/// int64_t* buffer = (int64_t*)malloc(sizeof(int64_t) * max_rows); +/// int64_t rows = vortex_exporter_export(handle, buffer, sizeof(int64_t) * max_rows, max_rows); +/// +/// @param handle Exporter handle. +/// @param buffer Pointer to pre-allocated buffer. +/// @param buffer_size_bytes Size of the buffer in bytes. +/// @param max_rows Maximum number of rows to export. +/// @return Number of rows actually exported, or negative on error: +/// -1: handle or buffer is NULL +/// -2: export failed +int64_t vortex_exporter_export( + VortexExporterHandle* handle, + void* buffer, + size_t buffer_size_bytes, + size_t max_rows +); + +// ============================================================================= +// Struct Exporter (for multi-column data) +// ============================================================================= + +/// Get the number of fields in a struct exporter. +/// +/// This function is only valid for struct-typed arrays. +/// +/// @param handle Exporter handle. +/// @return Number of fields, or 0 if not a struct or handle is NULL. +size_t vortex_exporter_num_fields(const VortexExporterHandle* handle); + +/// Get a field exporter from a struct exporter. +/// +/// This function returns an exporter for a specific field of a struct. +/// The returned exporter is owned by the caller and must be freed. +/// +/// @param handle Struct exporter handle. +/// @param index Field index (0-based). +/// @return Field exporter handle, or NULL on error. +VortexExporterHandle* vortex_exporter_get_field( + VortexExporterHandle* handle, + size_t index +); + +// ============================================================================= +// String/VarBinView Exporter +// ============================================================================= + +/// Export string data. +/// +/// This function exports variable-length string data. For each row, it writes: +/// - The string length to lengths[i] +/// - The string data starting at data + offsets[i] +/// +/// @param handle Exporter handle. +/// @param data Buffer for string data (concatenated). +/// @param lengths Buffer for string lengths. +/// @param offsets Buffer for string offsets in data buffer. +/// @param max_rows Maximum number of rows to export. +/// @return Number of rows actually exported, or negative on error. +int64_t vortex_exporter_export_strings( + VortexExporterHandle* handle, + char* data, + uint32_t* lengths, + uint64_t* offsets, + size_t max_rows +); + +/// Get the total size of string data for the remaining rows. +/// +/// This function is useful for pre-allocating buffers before calling +/// vortex_exporter_export_strings. It returns the total number of bytes +/// needed for all string data and the number of remaining rows. +/// +/// @param handle Exporter handle (must be a string exporter). +/// @param total_bytes Output parameter for total bytes needed. +/// @param num_rows Output parameter for number of remaining rows. +/// @return 0 on success, negative on error: +/// -1: handle or output pointers are NULL +/// -2: not a string exporter or other error +int32_t vortex_exporter_string_data_size( + const VortexExporterHandle* handle, + size_t* total_bytes, + size_t* num_rows +); + +// ============================================================================= +// Nullable Data Support +// ============================================================================= + +/// Export validity (null) bitmap. +/// +/// This function exports the validity bitmap for nullable columns. +/// Each bit indicates whether the corresponding row is valid (1) or null (0). +/// +/// The bitmap is stored in little-endian byte order, with the first row +/// corresponding to the least significant bit of the first byte. +/// +/// @param handle Exporter handle. +/// @param validity_bitmap Buffer for validity bitmap. +/// Size must be at least (max_rows + 7) / 8 bytes. +/// @param max_rows Maximum number of rows. +/// @return Number of rows, or negative on error. +int64_t vortex_exporter_export_validity( + VortexExporterHandle* handle, + uint8_t* validity_bitmap, + size_t max_rows +); + +/// Check if the column is nullable. +/// +/// @param handle Exporter handle. +/// @return 1 if nullable, 0 if non-nullable or handle is NULL. +int32_t vortex_exporter_is_nullable(const VortexExporterHandle* handle); + +// ============================================================================= +// List/Array Exporter +// ============================================================================= + +/// Check if the exporter is a list (array) exporter. +/// +/// List exporters are used for ClickHouse Array columns. +/// +/// @param handle Exporter handle. +/// @return 1 if it's a list exporter, 0 otherwise. +int32_t vortex_exporter_is_list(const VortexExporterHandle* handle); + +/// Export list offsets. +/// +/// For list arrays (ClickHouse Array columns), this exports the offsets +/// that indicate where each array element starts in the flattened elements. +/// +/// The offsets array will have num_rows + 1 elements written, where: +/// - offsets[i] is the start index of array i in the flattened elements +/// - offsets[num_rows] is the total number of elements +/// +/// @param handle Exporter handle (must be a list exporter). +/// @param offsets Buffer for offsets (must have space for max_rows + 1 uint64_t values). +/// @param max_rows Maximum number of rows (arrays) to export. +/// @return Number of rows (arrays) exported, or negative on error. +int64_t vortex_exporter_export_list_offsets( + VortexExporterHandle* handle, + uint64_t* offsets, + size_t max_rows +); + +/// Get the element exporter from a list exporter. +/// +/// This returns an exporter for the flattened elements of all arrays. +/// Use this exporter to export the actual element data after exporting offsets. +/// +/// @param handle List exporter handle. +/// @return Element exporter handle, or NULL on error. Caller must free. +VortexExporterHandle* vortex_exporter_get_list_elements( + VortexExporterHandle* handle +); + +/// Get the total number of elements in all arrays (for a list exporter). +/// +/// This is useful for pre-allocating the element buffer. +/// +/// @param handle List exporter handle. +/// @return Total number of elements, or 0 if not a list exporter or on error. +size_t vortex_exporter_list_total_elements(const VortexExporterHandle* handle); + +#ifdef __cplusplus +} +#endif diff --git a/vortex-clickhouse/cpp/include/clickhouse_vx/format.h b/vortex-clickhouse/cpp/include/clickhouse_vx/format.h new file mode 100644 index 00000000000..b27a9e24457 --- /dev/null +++ b/vortex-clickhouse/cpp/include/clickhouse_vx/format.h @@ -0,0 +1,28 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +#pragma once + +#include "common.h" + +// Format-related constants for ClickHouse integration. + +#ifdef __cplusplus +extern "C" { +#endif + +/// Format name constant used in ClickHouse registration. +extern const char* VORTEX_FORMAT_NAME; + +/// Default file extension for Vortex files. +extern const char* VORTEX_FILE_EXTENSION; + +/// Magic bytes at the start of a Vortex file. +extern const uint8_t VORTEX_MAGIC_BYTES[4]; + +/// Current Vortex format version. +extern const uint32_t VORTEX_FORMAT_VERSION; + +#ifdef __cplusplus +} +#endif diff --git a/vortex-clickhouse/cpp/include/clickhouse_vx/scanner.h b/vortex-clickhouse/cpp/include/clickhouse_vx/scanner.h new file mode 100644 index 00000000000..8c1893019c6 --- /dev/null +++ b/vortex-clickhouse/cpp/include/clickhouse_vx/scanner.h @@ -0,0 +1,173 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +#pragma once + +#include "common.h" + +// Scanner API for reading Vortex files from ClickHouse. +// +// Thread safety: +// A VortexScanner instance is NOT thread-safe. The caller must serialize all +// calls to a given scanner handle. Do not call any scanner function from one +// thread while another thread is using the same handle. Each thread should +// either create its own scanner via `vortex_scanner_new`, or the caller must +// use external synchronization (e.g., a mutex) to protect shared access. +// The error reporting functions (`vortex_get_last_error`, `vortex_has_error`, +// `vortex_clear_error`) use thread-local storage and are safe to call from +// any thread. +// +// Usage: +// VortexScanner* scanner = vortex_scanner_new("/path/to/file.vortex"); +// if (scanner == NULL) { /* handle error */ } +// +// // Get schema information +// size_t num_cols = vortex_scanner_num_columns(scanner); +// for (size_t i = 0; i < num_cols; i++) { +// const char* name = vortex_scanner_column_name(scanner, i); +// const char* type = vortex_scanner_column_type(scanner, i); +// } +// +// // Set projection if needed +// size_t indices[] = {0, 2}; +// vortex_scanner_set_projection(scanner, indices, 2); +// +// // Read batches +// while (vortex_scanner_has_more(scanner)) { +// VortexExporterHandle* batch = vortex_scanner_read_batch(scanner); +// // Export data from batch... +// vortex_exporter_free(batch); +// } +// +// vortex_scanner_free(scanner); + +#ifdef __cplusplus +extern "C" { +#endif + +// ============================================================================= +// Scanner Creation and Destruction +// ============================================================================= + +/// Create a new Vortex scanner for the given path. +/// +/// The path can be: +/// - A local file path: "/path/to/file.vortex" +/// - A local glob pattern: "/path/to/*.vortex" +/// - A remote URL: "s3://bucket/path/to/file.vortex" +/// - A remote glob: "s3://bucket/path/to/*.vortex" +/// +/// @param path Null-terminated C string with the file path or glob pattern. +/// @return Scanner handle, or NULL on error. +VortexScanner* vortex_scanner_new(const char* path); + +/// Free a Vortex scanner. +/// +/// @param scanner Scanner handle to free. NULL is safely ignored. +void vortex_scanner_free(VortexScanner* scanner); + +// ============================================================================= +// Schema Introspection +// ============================================================================= + +/// Get the number of columns in the schema. +/// +/// @param scanner Scanner handle. +/// @return Number of columns, or 0 if scanner is NULL. +size_t vortex_scanner_num_columns(const VortexScanner* scanner); + +/// Get a column name by index. +/// +/// @param scanner Scanner handle. +/// @param index Column index (0-based). +/// @return Null-terminated column name, or NULL if index is out of bounds. +/// The returned string is owned by the caller and must be freed +/// with vortex_free_string(). +char* vortex_scanner_column_name(const VortexScanner* scanner, size_t index); + +/// Get the ClickHouse type string for a column. +/// +/// @param scanner Scanner handle. +/// @param index Column index (0-based). +/// @return Null-terminated ClickHouse type string (e.g., "Int64", "String"), +/// or NULL if index is out of bounds. +/// The returned string is owned by the caller and must be freed +/// with vortex_free_string(). +char* vortex_scanner_column_type(const VortexScanner* scanner, size_t index); + +// ============================================================================= +// Scan Configuration +// ============================================================================= + +/// Set the columns to project (by index). +/// +/// @param scanner Scanner handle. +/// @param indices Array of column indices to project. +/// @param num_indices Number of indices in the array. +/// @return 0 on success, negative error code on failure: +/// -1: scanner is NULL +/// -2: indices is NULL but num_indices > 0 +/// -3: invalid column index +int32_t vortex_scanner_set_projection( + VortexScanner* scanner, + const size_t* indices, + size_t num_indices +); + +/// Set the batch size for reading. +/// +/// @param scanner Scanner handle. +/// @param batch_size Number of rows per batch. Minimum is 1. +void vortex_scanner_set_batch_size(VortexScanner* scanner, size_t batch_size); + +// ============================================================================= +// Data Reading +// ============================================================================= + +/// Check if there are more batches to read. +/// +/// @param scanner Scanner handle. +/// @return 1 if more data available, 0 if no more data or scanner is NULL. +int32_t vortex_scanner_has_more(const VortexScanner* scanner); + +/// Read the next batch of data. +/// +/// @param scanner Scanner handle. +/// @return Exporter handle for the batch, or NULL if no more data or on error. +/// The returned handle must be freed with vortex_exporter_free(). +VortexExporterHandle* vortex_scanner_read_batch(VortexScanner* scanner); + +// ============================================================================= +// Progress Tracking +// ============================================================================= + +/// Get the number of files to scan. +/// +/// @param scanner Scanner handle. +/// @return Number of files, or 0 if scanner is NULL. +size_t vortex_scanner_num_files(const VortexScanner* scanner); + +/// Get the current file index being scanned. +/// +/// @param scanner Scanner handle. +/// @return Current file index (0-based), or 0 if scanner is NULL. +size_t vortex_scanner_current_file_index(const VortexScanner* scanner); + +/// Get the total number of rows read so far. +/// +/// @param scanner Scanner handle. +/// @return Total rows read, or 0 if scanner is NULL. +uint64_t vortex_scanner_total_rows_read(const VortexScanner* scanner); + +/// Get the total row count across all files. +/// +/// This function reads metadata from all files to compute the total row count. +/// Note: This may be slow for large file sets as it opens each file's metadata. +/// +/// @param scanner Scanner handle. +/// @return Total row count, or 0 on error. Call vortex_get_last_error() for details. +uint64_t vortex_scanner_total_row_count(const VortexScanner* scanner); + +#ifdef __cplusplus +} +#endif diff --git a/vortex-clickhouse/cpp/include/clickhouse_vx/writer.h b/vortex-clickhouse/cpp/include/clickhouse_vx/writer.h new file mode 100644 index 00000000000..8d0d9bc2736 --- /dev/null +++ b/vortex-clickhouse/cpp/include/clickhouse_vx/writer.h @@ -0,0 +1,409 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +#pragma once + +#include "common.h" + +// Writer API for creating Vortex files from ClickHouse. +// +// Note: The writer API is still under development and may change. +// +// Usage (planned): +// VortexWriter* writer = vortex_writer_new("/path/to/output.vortex"); +// if (writer == NULL) { /* handle error */ } +// +// // Set schema +// vortex_writer_set_schema(writer, schema); +// +// // Write data +// vortex_writer_write_batch(writer, columns, num_columns, num_rows); +// +// // Finalize +// vortex_writer_finalize(writer); +// vortex_writer_free(writer); + +#ifdef __cplusplus +extern "C" { +#endif + +// ============================================================================= +// Writer Creation and Destruction +// ============================================================================= + +/// Create a new Vortex writer for the given output path. +/// +/// @param path Null-terminated C string with the output file path. +/// @return Writer handle, or NULL on error. +VortexWriter* vortex_writer_new(const char* path); + +/// Free a Vortex writer. +/// +/// @param writer Writer handle to free. NULL is safely ignored. +void vortex_writer_free(VortexWriter* writer); + +// ============================================================================= +// Schema Configuration +// ============================================================================= + +/// Add a column to the writer's schema. +/// +/// @param writer Writer handle. +/// @param name Column name (null-terminated). +/// @param clickhouse_type ClickHouse type string (e.g., "Int64", "String"). +/// @param nullable Whether the column is nullable. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_add_column( + VortexWriter* writer, + const char* name, + const char* clickhouse_type, + int32_t nullable +); + +// ============================================================================= +// Data Writing +// ============================================================================= + +/// Write a batch of data (simplified API for primitive-only columns). +/// +/// For columns that include string types, use the batch API: +/// vortex_writer_begin_batch(), vortex_writer_write_column_*(), vortex_writer_end_batch() +/// +/// @param writer Writer handle. +/// @param data Array of pointers to column data. +/// @param num_columns Number of columns. +/// @param num_rows Number of rows in this batch. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_write_batch( + VortexWriter* writer, + const void* const* data, + size_t num_columns, + size_t num_rows +); + +// ============================================================================= +// Batch Writing API (supports mixed primitive and string columns) +// ============================================================================= + +/// Begin writing a new batch with the given number of rows. +/// +/// After calling this, use vortex_writer_write_column() or +/// vortex_writer_write_string_column() to write each column's data, +/// then call vortex_writer_end_batch() to commit the batch. +/// +/// @param writer Writer handle. +/// @param num_rows Number of rows in this batch. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_begin_batch( + VortexWriter* writer, + size_t num_rows +); + +/// Write a primitive column by index. +/// +/// Must be called between vortex_writer_begin_batch() and vortex_writer_end_batch(). +/// +/// @param writer Writer handle. +/// @param column_index Column index (0-based). +/// @param data Pointer to column data (array of the appropriate primitive type). +/// @param num_rows Number of rows (must match begin_batch). +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_write_column( + VortexWriter* writer, + size_t column_index, + const void* data, + size_t num_rows +); + +/// Write a nullable primitive column by index with null map. +/// +/// The null_map uses ClickHouse's convention: one byte per row where: +/// - 0 = valid (not null) +/// - 1 = null +/// +/// Must be called between vortex_writer_begin_batch() and vortex_writer_end_batch(). +/// +/// @param writer Writer handle. +/// @param column_index Column index (0-based). +/// @param data Pointer to column data (array of the appropriate primitive type). +/// @param null_map Pointer to null map (array of num_rows bytes), or NULL for all-valid. +/// @param num_rows Number of rows (must match begin_batch). +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_write_column_nullable( + VortexWriter* writer, + size_t column_index, + const void* data, + const uint8_t* null_map, + size_t num_rows +); + +/// Write a string column by index. +/// +/// The strings are provided as concatenated data with offsets. +/// The offsets array must have num_rows + 1 elements, where: +/// - offsets[i] is the start offset of string i +/// - offsets[num_rows] is the total data length +/// +/// Must be called between vortex_writer_begin_batch() and vortex_writer_end_batch(). +/// +/// @param writer Writer handle. +/// @param column_index Column index (0-based). +/// @param data Pointer to concatenated string data. +/// @param offsets Pointer to offsets array (num_rows + 1 elements). +/// @param num_rows Number of rows (must match begin_batch). +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_write_string_column( + VortexWriter* writer, + size_t column_index, + const uint8_t* data, + const uint64_t* offsets, + size_t num_rows +); + +/// Write a nullable string column by index with null map. +/// +/// The strings are provided as concatenated data with offsets. +/// The null_map uses ClickHouse's convention: one byte per row where: +/// - 0 = valid (not null) +/// - 1 = null +/// +/// Must be called between vortex_writer_begin_batch() and vortex_writer_end_batch(). +/// +/// @param writer Writer handle. +/// @param column_index Column index (0-based). +/// @param data Pointer to concatenated string data. +/// @param offsets Pointer to offsets array (num_rows + 1 elements). +/// @param null_map Pointer to null map (array of num_rows bytes), or NULL for all-valid. +/// @param num_rows Number of rows (must match begin_batch). +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_write_string_column_nullable( + VortexWriter* writer, + size_t column_index, + const uint8_t* data, + const uint64_t* offsets, + const uint8_t* null_map, + size_t num_rows +); + +/// End the current batch and commit it. +/// +/// All columns must be written before calling this function. +/// +/// @param writer Writer handle. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_end_batch(VortexWriter* writer); + +// ============================================================================= +// List (Array) Column Writing +// ============================================================================= + +/// Write the offsets for a List (Array) column. +/// +/// After writing offsets, use vortex_writer_list_write_element_column() or +/// vortex_writer_list_write_element_string_column() to write the nested elements. +/// Finally, call vortex_writer_list_end() to commit the list column. +/// +/// @param writer Writer handle. +/// @param column_index Column index (0-based) of the list column. +/// @param offsets Array of num_rows + 1 uint64_t offsets (start offset of each list element). +/// @param null_map Pointer to null map (one byte per row, 0=valid, 1=null), or NULL for all-valid. +/// @param num_rows Number of rows (must match begin_batch). +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_list_write_offsets( + VortexWriter* writer, + size_t column_index, + const uint64_t* offsets, + const uint8_t* null_map, + size_t num_rows +); + +/// Write primitive element data for a list column. +/// +/// Must be called after vortex_writer_list_write_offsets(). +/// The element count must equal offsets[num_rows] - offsets[0]. +/// +/// @param writer Writer handle. +/// @param column_index Column index of the list column. +/// @param data Pointer to element data. +/// @param num_elements Number of elements. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_list_write_element_column( + VortexWriter* writer, + size_t column_index, + const void* data, + size_t num_elements +); + +/// Write string element data for a list column. +/// +/// Must be called after vortex_writer_list_write_offsets(). +/// +/// @param writer Writer handle. +/// @param column_index Column index of the list column. +/// @param data Pointer to concatenated string data. +/// @param offsets Pointer to string offsets array (num_elements + 1). +/// @param num_elements Number of string elements. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_list_write_element_string_column( + VortexWriter* writer, + size_t column_index, + const uint8_t* data, + const uint64_t* offsets, + size_t num_elements +); + +/// Write nullable primitive element data for a list column. +/// +/// Must be called after vortex_writer_list_write_offsets(). +/// The null_map uses ClickHouse's convention: one byte per element where +/// 0 = valid (not null), 1 = null. +/// +/// @param writer Writer handle. +/// @param column_index Column index of the list column. +/// @param data Pointer to element data. +/// @param null_map Pointer to null map (one byte per element), or NULL for all-valid. +/// @param num_elements Number of elements. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_list_write_element_column_nullable( + VortexWriter* writer, + size_t column_index, + const void* data, + const uint8_t* null_map, + size_t num_elements +); + +/// Write nullable string element data for a list column. +/// +/// Must be called after vortex_writer_list_write_offsets(). +/// The null_map uses ClickHouse's convention: one byte per element where +/// 0 = valid (not null), 1 = null. +/// +/// @param writer Writer handle. +/// @param column_index Column index of the list column. +/// @param data Pointer to concatenated string data. +/// @param offsets Pointer to string offsets array (num_elements + 1). +/// @param null_map Pointer to null map (one byte per element), or NULL for all-valid. +/// @param num_elements Number of string elements. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_list_write_element_string_column_nullable( + VortexWriter* writer, + size_t column_index, + const uint8_t* data, + const uint64_t* offsets, + const uint8_t* null_map, + size_t num_elements +); + +/// Finalize the list column data. +/// +/// Must be called after element data is written. +/// +/// @param writer Writer handle. +/// @param column_index Column index of the list column. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_list_end( + VortexWriter* writer, + size_t column_index +); + +// ============================================================================= +// Struct (Tuple) Column Writing +// ============================================================================= + +/// Begin writing a struct (Tuple) column. +/// +/// After calling this, use vortex_writer_struct_write_field() or +/// vortex_writer_struct_write_field_string() to write each field. +/// Then call vortex_writer_struct_end() to commit the struct column. +/// +/// @param writer Writer handle. +/// @param column_index Column index of the struct column. +/// @param null_map Pointer to null map (one byte per row, 0=valid, 1=null), or NULL for all-valid. +/// @param num_rows Number of rows. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_struct_begin( + VortexWriter* writer, + size_t column_index, + const uint8_t* null_map, + size_t num_rows +); + +/// Write a primitive field of a struct column. +/// +/// @param writer Writer handle. +/// @param column_index Column index of the struct column. +/// @param field_index Field index within the struct (0-based). +/// @param data Pointer to field data. +/// @param null_map Pointer to null map for this field, or NULL for all-valid. +/// @param num_rows Number of rows. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_struct_write_field( + VortexWriter* writer, + size_t column_index, + size_t field_index, + const void* data, + const uint8_t* null_map, + size_t num_rows +); + +/// Write a string field of a struct column. +/// +/// @param writer Writer handle. +/// @param column_index Column index of the struct column. +/// @param field_index Field index within the struct (0-based). +/// @param data Pointer to concatenated string data. +/// @param offsets Pointer to string offsets array (num_rows + 1). +/// @param null_map Pointer to null map for this field, or NULL for all-valid. +/// @param num_rows Number of rows. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_struct_write_field_string( + VortexWriter* writer, + size_t column_index, + size_t field_index, + const uint8_t* data, + const uint64_t* offsets, + const uint8_t* null_map, + size_t num_rows +); + +/// Finalize the struct column data. +/// +/// Must be called after all fields are written. +/// +/// @param writer Writer handle. +/// @param column_index Column index of the struct column. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_struct_end( + VortexWriter* writer, + size_t column_index +); + +// ============================================================================= +// Writer Finalization +// ============================================================================= + +/// Finalize the writer and flush all data. +/// +/// @param writer Writer handle. +/// @return 0 on success, negative error code on failure. +int32_t vortex_writer_finalize(VortexWriter* writer); + +// ============================================================================= +// Writer Information +// ============================================================================= + +/// Get the number of columns in the writer's schema. +/// +/// @param writer Writer handle. +/// @return Number of columns, or 0 if writer is NULL. +size_t vortex_writer_num_columns(const VortexWriter* writer); + +/// Get the total number of rows written. +/// +/// @param writer Writer handle. +/// @return Total rows written, or 0 if writer is NULL. +size_t vortex_writer_total_rows(const VortexWriter* writer); + +#ifdef __cplusplus +} +#endif diff --git a/vortex-clickhouse/src/clickhouse/mod.rs b/vortex-clickhouse/src/clickhouse/mod.rs new file mode 100644 index 00000000000..68b00e2d9ee --- /dev/null +++ b/vortex-clickhouse/src/clickhouse/mod.rs @@ -0,0 +1,6 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! ClickHouse-specific wrappers and utilities. +//! +//! This module provides safe Rust wrappers around ClickHouse's C/C++ APIs. diff --git a/vortex-clickhouse/src/convert/column.rs b/vortex-clickhouse/src/convert/column.rs new file mode 100644 index 00000000000..47017ff4b87 --- /dev/null +++ b/vortex-clickhouse/src/convert/column.rs @@ -0,0 +1,595 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Column data conversion between Vortex Array and ClickHouse columns. +//! +//! This module handles the actual data conversion between Vortex's Array types +//! and ClickHouse's column data structures. It provides functions for both +//! reading (Vortex -> ClickHouse) and writing (ClickHouse -> Vortex) directions. + +use std::ffi::c_void; + +use vortex::array::arrays::{BoolArray, PrimitiveArray, StructArray, VarBinViewArray}; +use vortex::array::validity::Validity; +use vortex::array::{Array, ArrayRef, IntoArray, ToCanonical}; +use vortex::buffer::Buffer; +use vortex::dtype::{DType, FieldNames, PType}; +use vortex::error::{VortexResult, vortex_bail, vortex_err}; + +use super::dtype::clickhouse_type_to_vortex; +use crate::exporter::{ColumnExporter, PrimitiveExporter}; + +/// Convert a ClickHouse column to a Vortex Array. +/// +/// This is used when writing data from ClickHouse to Vortex format. +/// The column data is read through the FFI boundary and converted to +/// the appropriate Vortex array type. +/// +/// # Arguments +/// * `column_ptr` - Pointer to the ClickHouse column data +/// * `num_rows` - Number of rows to read +/// * `ch_type` - ClickHouse type string (e.g., "Int32", "String", "Array(UInt64)") +/// +/// # Returns +/// A Vortex ArrayRef containing the converted data. +pub fn clickhouse_column_to_vortex( + column_ptr: *const c_void, + num_rows: usize, + ch_type: &str, +) -> VortexResult { + if column_ptr.is_null() { + vortex_bail!("column_ptr is null"); + } + + if num_rows == 0 { + // Return empty array of the appropriate type + let dtype = clickhouse_type_to_vortex(ch_type)?; + return create_empty_array(&dtype); + } + + let dtype = clickhouse_type_to_vortex(ch_type)?; + + match &dtype { + DType::Primitive(ptype, nullability) => { + convert_primitive_column(column_ptr, num_rows, *ptype, *nullability) + } + DType::Utf8(nullability) => convert_string_column(column_ptr, num_rows, *nullability), + DType::Bool(nullability) => convert_bool_column(column_ptr, num_rows, *nullability), + _ => { + vortex_bail!( + "Unsupported ClickHouse type for column conversion: {}", + ch_type + ) + } + } +} + +/// Convert a Vortex Array to ClickHouse column data. +/// +/// This is used when reading Vortex data into ClickHouse. +/// The Vortex array is converted and written to the provided column buffer. +/// +/// # Arguments +/// * `array` - The Vortex array to convert +/// * `column_ptr` - Pointer to the ClickHouse column buffer +/// +/// # Returns +/// Ok(()) on success, or an error if conversion fails. +pub fn vortex_to_clickhouse_column(array: &ArrayRef, column_ptr: *mut c_void) -> VortexResult<()> { + if column_ptr.is_null() { + vortex_bail!("column_ptr is null"); + } + + match array.dtype() { + DType::Primitive(_, _) => { + let mut exporter = PrimitiveExporter::new(array.clone())?; + let buffer_size_bytes = exporter.element_size_bytes() * array.len(); + exporter.export(column_ptr, buffer_size_bytes, array.len())?; + Ok(()) + } + DType::Utf8(_) | DType::Binary(_) => { + // String export requires callback-based approach + // This is handled differently in the C++ layer + vortex_bail!("String columns should use export_with_callback") + } + _ => { + vortex_bail!( + "Unsupported Vortex type for column conversion: {:?}", + array.dtype() + ) + } + } +} + +/// Create an empty array of the given dtype. +fn create_empty_array(dtype: &DType) -> VortexResult { + match dtype { + DType::Primitive(ptype, nullability) => { + let validity = if nullability.is_nullable() { + Validity::AllValid + } else { + Validity::NonNullable + }; + match ptype { + PType::I8 => { + Ok(PrimitiveArray::new(Buffer::::from(vec![]), validity).into_array()) + } + PType::I16 => { + Ok(PrimitiveArray::new(Buffer::::from(vec![]), validity).into_array()) + } + PType::I32 => { + Ok(PrimitiveArray::new(Buffer::::from(vec![]), validity).into_array()) + } + PType::I64 => { + Ok(PrimitiveArray::new(Buffer::::from(vec![]), validity).into_array()) + } + PType::U8 => { + Ok(PrimitiveArray::new(Buffer::::from(vec![]), validity).into_array()) + } + PType::U16 => { + Ok(PrimitiveArray::new(Buffer::::from(vec![]), validity).into_array()) + } + PType::U32 => { + Ok(PrimitiveArray::new(Buffer::::from(vec![]), validity).into_array()) + } + PType::U64 => { + Ok(PrimitiveArray::new(Buffer::::from(vec![]), validity).into_array()) + } + PType::F32 => { + Ok(PrimitiveArray::new(Buffer::::from(vec![]), validity).into_array()) + } + PType::F64 => { + Ok(PrimitiveArray::new(Buffer::::from(vec![]), validity).into_array()) + } + PType::F16 => vortex_bail!("F16 type not supported"), + } + } + DType::Utf8(_) => Ok(VarBinViewArray::from_iter_str(Vec::<&str>::new()).into_array()), + DType::Bool(_) => Ok(BoolArray::from_iter(Vec::::new()).into_array()), + _ => vortex_bail!("Unsupported dtype for empty array: {:?}", dtype), + } +} + +/// Convert a primitive column from ClickHouse to Vortex. +fn convert_primitive_column( + column_ptr: *const c_void, + num_rows: usize, + ptype: PType, + nullability: vortex::dtype::Nullability, +) -> VortexResult { + let validity = if nullability.is_nullable() { + // Note: For nullable columns, ClickHouse's null bitmap is handled at the C++ FFI layer + // via vx_ch_array_builder_append_batch_nullable_* functions + Validity::AllValid + } else { + Validity::NonNullable + }; + + macro_rules! convert_primitive { + ($rust_ty:ty) => {{ + let src = column_ptr as *const $rust_ty; + let mut data = Vec::with_capacity(num_rows); + unsafe { + data.set_len(num_rows); + std::ptr::copy_nonoverlapping(src, data.as_mut_ptr(), num_rows); + } + let buffer: Buffer<$rust_ty> = data.into(); + Ok(PrimitiveArray::new(buffer, validity).into_array()) + }}; + } + + match ptype { + PType::I8 => convert_primitive!(i8), + PType::I16 => convert_primitive!(i16), + PType::I32 => convert_primitive!(i32), + PType::I64 => convert_primitive!(i64), + PType::U8 => convert_primitive!(u8), + PType::U16 => convert_primitive!(u16), + PType::U32 => convert_primitive!(u32), + PType::U64 => convert_primitive!(u64), + PType::F32 => convert_primitive!(f32), + PType::F64 => convert_primitive!(f64), + PType::F16 => vortex_bail!("F16 type not supported"), + } +} + +/// Convert a string column from ClickHouse to Vortex. +/// +/// This expects the column data to be laid out as: +/// - An array of (offset, length) pairs followed by the actual string data +/// +/// For simplicity, we use a callback-based approach where ClickHouse provides +/// individual strings. +fn convert_string_column( + _column_ptr: *const c_void, + _num_rows: usize, + _nullability: vortex::dtype::Nullability, +) -> VortexResult { + // String conversion requires special handling due to ClickHouse's string layout + // This is typically done through the callback-based approach in the C++ layer + vortex_bail!("String column conversion requires callback-based approach") +} + +/// Convert a bool column from ClickHouse to Vortex. +fn convert_bool_column( + column_ptr: *const c_void, + num_rows: usize, + _nullability: vortex::dtype::Nullability, +) -> VortexResult { + // ClickHouse stores bools as UInt8 (0 or 1) + let src = column_ptr as *const u8; + let mut data = Vec::with_capacity(num_rows); + + for i in 0..num_rows { + let val = unsafe { *src.add(i) }; + data.push(val != 0); + } + + Ok(BoolArray::from_iter(data).into_array()) +} + +/// Builder for constructing Vortex arrays from ClickHouse data incrementally. +/// +/// This is useful when receiving data row by row or in batches from ClickHouse. +pub struct VortexColumnBuilder { + dtype: DType, + inner: ColumnBuilderInner, +} + +enum ColumnBuilderInner { + Bool(Vec>), + I8(Vec, Vec), + I16(Vec, Vec), + I32(Vec, Vec), + I64(Vec, Vec), + U8(Vec, Vec), + U16(Vec, Vec), + U32(Vec, Vec), + U64(Vec, Vec), + F32(Vec, Vec), + F64(Vec, Vec), + String(Vec>), +} + +impl VortexColumnBuilder { + /// Create a new column builder for the given ClickHouse type. + pub fn new(ch_type: &str, capacity: usize) -> VortexResult { + let dtype = clickhouse_type_to_vortex(ch_type)?; + + let inner = match &dtype { + DType::Bool(_) => ColumnBuilderInner::Bool(Vec::with_capacity(capacity)), + DType::Primitive(ptype, _) => match ptype { + PType::I8 => ColumnBuilderInner::I8( + Vec::with_capacity(capacity), + Vec::with_capacity(capacity), + ), + PType::I16 => ColumnBuilderInner::I16( + Vec::with_capacity(capacity), + Vec::with_capacity(capacity), + ), + PType::I32 => ColumnBuilderInner::I32( + Vec::with_capacity(capacity), + Vec::with_capacity(capacity), + ), + PType::I64 => ColumnBuilderInner::I64( + Vec::with_capacity(capacity), + Vec::with_capacity(capacity), + ), + PType::U8 => ColumnBuilderInner::U8( + Vec::with_capacity(capacity), + Vec::with_capacity(capacity), + ), + PType::U16 => ColumnBuilderInner::U16( + Vec::with_capacity(capacity), + Vec::with_capacity(capacity), + ), + PType::U32 => ColumnBuilderInner::U32( + Vec::with_capacity(capacity), + Vec::with_capacity(capacity), + ), + PType::U64 => ColumnBuilderInner::U64( + Vec::with_capacity(capacity), + Vec::with_capacity(capacity), + ), + PType::F32 => ColumnBuilderInner::F32( + Vec::with_capacity(capacity), + Vec::with_capacity(capacity), + ), + PType::F64 => ColumnBuilderInner::F64( + Vec::with_capacity(capacity), + Vec::with_capacity(capacity), + ), + PType::F16 => vortex_bail!("F16 type not supported"), + }, + DType::Utf8(_) => ColumnBuilderInner::String(Vec::with_capacity(capacity)), + _ => vortex_bail!("Unsupported type for column builder: {:?}", dtype), + }; + + Ok(Self { dtype, inner }) + } + + /// Append a null value. + pub fn append_null(&mut self) { + match &mut self.inner { + ColumnBuilderInner::Bool(v) => v.push(None), + ColumnBuilderInner::I8(values, validity) => { + values.push(0); + validity.push(false); + } + ColumnBuilderInner::I16(values, validity) => { + values.push(0); + validity.push(false); + } + ColumnBuilderInner::I32(values, validity) => { + values.push(0); + validity.push(false); + } + ColumnBuilderInner::I64(values, validity) => { + values.push(0); + validity.push(false); + } + ColumnBuilderInner::U8(values, validity) => { + values.push(0); + validity.push(false); + } + ColumnBuilderInner::U16(values, validity) => { + values.push(0); + validity.push(false); + } + ColumnBuilderInner::U32(values, validity) => { + values.push(0); + validity.push(false); + } + ColumnBuilderInner::U64(values, validity) => { + values.push(0); + validity.push(false); + } + ColumnBuilderInner::F32(values, validity) => { + values.push(0.0); + validity.push(false); + } + ColumnBuilderInner::F64(values, validity) => { + values.push(0.0); + validity.push(false); + } + ColumnBuilderInner::String(v) => v.push(None), + } + } + + /// Append an i8 value. + pub fn append_i8(&mut self, value: i8) { + if let ColumnBuilderInner::I8(values, validity) = &mut self.inner { + values.push(value); + validity.push(true); + } + } + + /// Append an i16 value. + pub fn append_i16(&mut self, value: i16) { + if let ColumnBuilderInner::I16(values, validity) = &mut self.inner { + values.push(value); + validity.push(true); + } + } + + /// Append an i32 value. + pub fn append_i32(&mut self, value: i32) { + if let ColumnBuilderInner::I32(values, validity) = &mut self.inner { + values.push(value); + validity.push(true); + } + } + + /// Append an i64 value. + pub fn append_i64(&mut self, value: i64) { + if let ColumnBuilderInner::I64(values, validity) = &mut self.inner { + values.push(value); + validity.push(true); + } + } + + /// Append a u8 value. + pub fn append_u8(&mut self, value: u8) { + if let ColumnBuilderInner::U8(values, validity) = &mut self.inner { + values.push(value); + validity.push(true); + } + } + + /// Append a u16 value. + pub fn append_u16(&mut self, value: u16) { + if let ColumnBuilderInner::U16(values, validity) = &mut self.inner { + values.push(value); + validity.push(true); + } + } + + /// Append a u32 value. + pub fn append_u32(&mut self, value: u32) { + if let ColumnBuilderInner::U32(values, validity) = &mut self.inner { + values.push(value); + validity.push(true); + } + } + + /// Append a u64 value. + pub fn append_u64(&mut self, value: u64) { + if let ColumnBuilderInner::U64(values, validity) = &mut self.inner { + values.push(value); + validity.push(true); + } + } + + /// Append an f32 value. + pub fn append_f32(&mut self, value: f32) { + if let ColumnBuilderInner::F32(values, validity) = &mut self.inner { + values.push(value); + validity.push(true); + } + } + + /// Append an f64 value. + pub fn append_f64(&mut self, value: f64) { + if let ColumnBuilderInner::F64(values, validity) = &mut self.inner { + values.push(value); + validity.push(true); + } + } + + /// Append a string value. + pub fn append_string(&mut self, value: &str) { + if let ColumnBuilderInner::String(v) = &mut self.inner { + v.push(Some(value.to_string())); + } + } + + /// Finish building and return the Vortex array. + pub fn finish(self) -> VortexResult { + let is_nullable = self.dtype.is_nullable(); + + match self.inner { + ColumnBuilderInner::Bool(values) => Ok(BoolArray::from_iter(values).into_array()), + ColumnBuilderInner::I8(values, validity) => { + let validity = make_validity(is_nullable, &validity); + let buffer: Buffer = values.into(); + Ok(PrimitiveArray::new(buffer, validity).into_array()) + } + ColumnBuilderInner::I16(values, validity) => { + let validity = make_validity(is_nullable, &validity); + let buffer: Buffer = values.into(); + Ok(PrimitiveArray::new(buffer, validity).into_array()) + } + ColumnBuilderInner::I32(values, validity) => { + let validity = make_validity(is_nullable, &validity); + let buffer: Buffer = values.into(); + Ok(PrimitiveArray::new(buffer, validity).into_array()) + } + ColumnBuilderInner::I64(values, validity) => { + let validity = make_validity(is_nullable, &validity); + let buffer: Buffer = values.into(); + Ok(PrimitiveArray::new(buffer, validity).into_array()) + } + ColumnBuilderInner::U8(values, validity) => { + let validity = make_validity(is_nullable, &validity); + let buffer: Buffer = values.into(); + Ok(PrimitiveArray::new(buffer, validity).into_array()) + } + ColumnBuilderInner::U16(values, validity) => { + let validity = make_validity(is_nullable, &validity); + let buffer: Buffer = values.into(); + Ok(PrimitiveArray::new(buffer, validity).into_array()) + } + ColumnBuilderInner::U32(values, validity) => { + let validity = make_validity(is_nullable, &validity); + let buffer: Buffer = values.into(); + Ok(PrimitiveArray::new(buffer, validity).into_array()) + } + ColumnBuilderInner::U64(values, validity) => { + let validity = make_validity(is_nullable, &validity); + let buffer: Buffer = values.into(); + Ok(PrimitiveArray::new(buffer, validity).into_array()) + } + ColumnBuilderInner::F32(values, validity) => { + let validity = make_validity(is_nullable, &validity); + let buffer: Buffer = values.into(); + Ok(PrimitiveArray::new(buffer, validity).into_array()) + } + ColumnBuilderInner::F64(values, validity) => { + let validity = make_validity(is_nullable, &validity); + let buffer: Buffer = values.into(); + Ok(PrimitiveArray::new(buffer, validity).into_array()) + } + ColumnBuilderInner::String(values) => { + Ok(VarBinViewArray::from_iter_nullable_str(values).into_array()) + } + } + } +} + +fn make_validity(is_nullable: bool, validity: &[bool]) -> Validity { + if is_nullable && validity.iter().any(|&v| !v) { + Validity::from_iter(validity.iter().copied()) + } else { + Validity::NonNullable + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_primitive_column_conversion() { + // Create test data + let data: Vec = vec![1, 2, 3, 4, 5]; + + // Convert from ClickHouse to Vortex + let array = convert_primitive_column( + data.as_ptr() as *const c_void, + data.len(), + PType::I32, + vortex::dtype::Nullability::NonNullable, + ) + .expect("Conversion failed"); + + assert_eq!(array.len(), 5); + + // Verify values + let primitive = array.to_primitive(); + let values = primitive.as_slice::(); + assert_eq!(values, &[1, 2, 3, 4, 5]); + } + + #[test] + fn test_column_builder() { + let mut builder = VortexColumnBuilder::new("Int64", 5).unwrap(); + + builder.append_i64(10); + builder.append_i64(20); + builder.append_null(); + builder.append_i64(40); + builder.append_i64(50); + + let array = builder.finish().unwrap(); + assert_eq!(array.len(), 5); + } + + #[test] + fn test_string_builder() { + let mut builder = VortexColumnBuilder::new("String", 3).unwrap(); + + builder.append_string("hello"); + builder.append_null(); + builder.append_string("world"); + + let array = builder.finish().unwrap(); + assert_eq!(array.len(), 3); + + // Verify via scalar + let scalar = array.scalar_at(0).unwrap(); + assert!(!scalar.is_null()); + + let scalar = array.scalar_at(1).unwrap(); + assert!(scalar.is_null()); + } + + #[test] + fn test_bool_column_conversion() { + let data: Vec = vec![1, 0, 1, 1, 0]; + + let array = convert_bool_column( + data.as_ptr() as *const c_void, + data.len(), + vortex::dtype::Nullability::NonNullable, + ) + .expect("Conversion failed"); + + assert_eq!(array.len(), 5); + + // Verify values via scalar + for (i, expected) in [true, false, true, true, false].iter().enumerate() { + let scalar = array.scalar_at(i).unwrap(); + assert_eq!(scalar.as_bool().value().unwrap(), *expected); + } + } +} diff --git a/vortex-clickhouse/src/convert/dtype.rs b/vortex-clickhouse/src/convert/dtype.rs new file mode 100644 index 00000000000..4fda1c70c56 --- /dev/null +++ b/vortex-clickhouse/src/convert/dtype.rs @@ -0,0 +1,1338 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Type mapping between Vortex DType and ClickHouse types. +//! +//! # ClickHouse Type System +//! +//! ClickHouse has a rich type system that includes: +//! - Numeric types: Int8-Int256, UInt8-UInt256, Float32, Float64 +//! - String types: String, FixedString(N) +//! - Date/Time: Date, Date32, DateTime, DateTime64 +//! - Compound: Array(T), Tuple(T1, T2, ...), Map(K, V), Nested +//! - Special: Nullable(T), LowCardinality(T), Enum +//! +//! # Mapping Strategy +//! +//! We map Vortex types to the most appropriate ClickHouse types: +//! - Primitive types map directly to corresponding ClickHouse numeric types +//! - Vortex `Utf8` maps to ClickHouse `String` +//! - Vortex `List` maps to ClickHouse `Array` +//! - Vortex `Struct` maps to ClickHouse `Tuple` +//! - Temporal extensions map to DateTime64 with appropriate precision + +use std::sync::Arc; + +use vortex::dtype::Nullability::{NonNullable, Nullable}; +use vortex::dtype::{DType, DecimalDType, FieldName, FieldNames, PType, StructFields}; +use vortex::error::{VortexResult, vortex_bail}; + +use crate::ext_types::UUID; +use crate::ext_types::{BigInt, BigIntType}; +use crate::ext_types::{ + ClickHouseDate, ClickHouseDateTime, ClickHouseEnum, ClickHouseFixedString, + ClickHouseLowCardinality, DateTimeMetadata, LowCardinalityMetadata, +}; +use crate::ext_types::{Geo, GeoType}; +use crate::ext_types::{IPAddress, IPAddressType}; + +/// Convert a ClickHouse type string to Vortex DType. +/// +/// # Arguments +/// * `ch_type` - ClickHouse type string (e.g., "Int32", "String", "Array(UInt64)") +/// +/// # Returns +/// The corresponding Vortex DType. +pub fn clickhouse_type_to_vortex(ch_type: &str) -> VortexResult { + let ch_type = ch_type.trim(); + + // Handle Nullable wrapper + if let Some(inner) = ch_type + .strip_prefix("Nullable(") + .and_then(|s| s.strip_suffix(')')) + { + let inner_dtype = clickhouse_type_to_vortex(inner)?; + return Ok(inner_dtype.with_nullability(Nullable)); + } + + // Handle LowCardinality wrapper — preserve as extension type + if let Some(inner) = ch_type + .strip_prefix("LowCardinality(") + .and_then(|s| s.strip_suffix(')')) + { + let inner_dtype = clickhouse_type_to_vortex(inner)?; + let storage = match &inner_dtype { + DType::Extension(ext) => ext.storage_dtype().clone(), + other => other.clone(), + }; + return Ok(ClickHouseLowCardinality::dtype( + inner.trim().to_string(), + storage, + inner_dtype.nullability(), + )); + } + + match ch_type { + // Boolean + "Bool" => Ok(DType::Bool(NonNullable)), + + // Signed integers + "Int8" => Ok(DType::Primitive(PType::I8, NonNullable)), + "Int16" => Ok(DType::Primitive(PType::I16, NonNullable)), + "Int32" => Ok(DType::Primitive(PType::I32, NonNullable)), + "Int64" => Ok(DType::Primitive(PType::I64, NonNullable)), + // Large signed integers - use Extension types for disambiguation + "Int128" => Ok(BigInt::dtype(BigIntType::Int128, NonNullable)), + "Int256" => Ok(BigInt::dtype(BigIntType::Int256, NonNullable)), + + // Unsigned integers + "UInt8" => Ok(DType::Primitive(PType::U8, NonNullable)), + "UInt16" => Ok(DType::Primitive(PType::U16, NonNullable)), + "UInt32" => Ok(DType::Primitive(PType::U32, NonNullable)), + "UInt64" => Ok(DType::Primitive(PType::U64, NonNullable)), + // Large unsigned integers - use Extension types for disambiguation + "UInt128" => Ok(BigInt::dtype(BigIntType::UInt128, NonNullable)), + "UInt256" => Ok(BigInt::dtype(BigIntType::UInt256, NonNullable)), + + // Floating point + "Float32" => Ok(DType::Primitive(PType::F32, NonNullable)), + "Float64" => Ok(DType::Primitive(PType::F64, NonNullable)), + + // IP Address types + // IPv4 is stored as UInt32 (4 bytes), same as Parquet/Arrow + "IPv4" => Ok(DType::Primitive(PType::U32, NonNullable)), + // IPv6 is stored as 16-byte fixed binary, using Extension type for disambiguation + "IPv6" => Ok(IPAddress::dtype(IPAddressType::IPv6, NonNullable)), + + // UUID type - using Extension type for disambiguation from Int128 + "UUID" => Ok(UUID::dtype(NonNullable)), + + // String types + "String" => Ok(DType::Utf8(NonNullable)), + + // Date/Time types - use extension types to preserve Date semantics + "Date" => { + // Date is days since 1970-01-01, stored as UInt16 + Ok(ClickHouseDate::dtype(false, NonNullable)) + } + "Date32" => { + // Date32 is days since 1970-01-01, stored as Int32 + Ok(ClickHouseDate::dtype(true, NonNullable)) + } + + // GEO types - stored as WKB-encoded String in Vortex using Extension type. + // The C++ side converts GEO columns to/from WKB binary strings. + // The Geo Extension type preserves the type name so the C++ read side + // can reconstruct the GEO column. + "Point" => Ok(Geo::dtype(GeoType::Point, NonNullable)), + "LineString" => Ok(Geo::dtype(GeoType::LineString, NonNullable)), + "Ring" => Ok(Geo::dtype(GeoType::Ring, NonNullable)), + "Polygon" => Ok(Geo::dtype(GeoType::Polygon, NonNullable)), + "MultiLineString" => Ok(Geo::dtype(GeoType::MultiLineString, NonNullable)), + "MultiPolygon" => Ok(Geo::dtype(GeoType::MultiPolygon, NonNullable)), + + // Handle complex types + _ => parse_complex_clickhouse_type(ch_type), + } +} + +/// Parse an Enum8 or Enum16 type string (e.g., `Enum8('a' = 1, 'b' = 2)`). +/// +/// In native mode the enum names are discarded and the underlying integer type is returned. +fn parse_enum_type(ch_type: &str) -> VortexResult { + if ch_type.starts_with("Enum8(") { + Ok(DType::Primitive(PType::I8, NonNullable)) + } else if ch_type.starts_with("Enum16(") { + Ok(DType::Primitive(PType::I16, NonNullable)) + } else { + vortex_bail!("Not an Enum type: {}", ch_type) + } +} + +/// Parse a `Map(K, V)` type string into `List(Struct([key K, value V]))`. +/// +/// This follows the standard Arrow/Parquet convention for representing Maps. +fn parse_map_type(inner: &str) -> VortexResult { + let parts = split_balanced_commas(inner); + if parts.len() != 2 { + vortex_bail!( + "Map type expects exactly 2 type arguments, got {}", + parts.len() + ); + } + let key_dtype = clickhouse_type_to_vortex(parts[0].trim())?; + let value_dtype = clickhouse_type_to_vortex(parts[1].trim())?; + + // Build Struct([key K, value V]) + let field_names = FieldNames::from(vec![FieldName::from("key"), FieldName::from("value")]); + let entry_struct = DType::Struct( + StructFields::new(field_names, vec![key_dtype, value_dtype]), + NonNullable, + ); + // Wrap in List + Ok(DType::List(Arc::new(entry_struct), NonNullable)) +} + +/// Parse complex ClickHouse types like Array, Tuple, DateTime64, Decimal, Enum, Map, etc. +fn parse_complex_clickhouse_type(ch_type: &str) -> VortexResult { + // Array(T) + if let Some(inner) = ch_type + .strip_prefix("Array(") + .and_then(|s| s.strip_suffix(')')) + { + let element_dtype = clickhouse_type_to_vortex(inner)?; + return Ok(DType::List(Arc::new(element_dtype), NonNullable)); + } + + // Map(K, V) → List(Struct([key K, value V])) + if let Some(inner) = ch_type + .strip_prefix("Map(") + .and_then(|s| s.strip_suffix(')')) + { + return parse_map_type(inner); + } + + // FixedString(N) + if ch_type.starts_with("FixedString(") { + // Treat FixedString as String for now + return Ok(DType::Utf8(NonNullable)); + } + + // DateTime (seconds precision) - use extension type to preserve DateTime semantics + if ch_type == "DateTime" || ch_type.starts_with("DateTime(") { + let timezone = if ch_type.starts_with("DateTime('") { + ch_type + .strip_prefix("DateTime('") + .and_then(|s| s.strip_suffix("')")) + .map(|s| s.to_string()) + } else { + None + }; + let metadata = DateTimeMetadata { + precision: 0, + timezone, + }; + return Ok(ClickHouseDateTime::dtype(metadata, NonNullable)); + } + + // DateTime64(precision, [timezone]) - use extension type to preserve DateTime64 semantics + if ch_type.starts_with("DateTime64(") { + let inner = ch_type + .strip_prefix("DateTime64(") + .and_then(|s| s.strip_suffix(')')) + .unwrap_or("3"); + let parts: Vec<&str> = inner.splitn(2, ',').collect(); + let precision: u8 = parts[0].trim().parse().unwrap_or(3); + let timezone = if parts.len() > 1 { + let tz = parts[1].trim(); + tz.strip_prefix('\'') + .and_then(|s| s.strip_suffix('\'')) + .map(|s| s.to_string()) + } else { + None + }; + let metadata = DateTimeMetadata { + precision, + timezone, + }; + return Ok(ClickHouseDateTime::dtype(metadata, NonNullable)); + } + + // Decimal(P, S) or Decimal32/64/128/256 + if ch_type.starts_with("Decimal") { + return parse_decimal_type(ch_type); + } + + // Enum8/Enum16 → Primitive(I8/I16) in native mode + if ch_type.starts_with("Enum8(") || ch_type.starts_with("Enum16(") { + return parse_enum_type(ch_type); + } + + // Tuple(T1, T2, ...) + if let Some(inner) = ch_type + .strip_prefix("Tuple(") + .and_then(|s| s.strip_suffix(')')) + { + let fields = parse_tuple_fields(inner)?; + return Ok(DType::Struct(fields, NonNullable)); + } + + vortex_bail!("Unsupported ClickHouse type: {}", ch_type) +} + +/// Split a string by commas, respecting nested parentheses. +/// +/// Only splits at commas where the parenthesis depth is zero. +/// For example, `"a Array(Int32), b String"` splits into `["a Array(Int32)", " b String"]`. +fn split_balanced_commas(s: &str) -> Vec<&str> { + let mut result = Vec::new(); + let mut depth = 0usize; + let mut start = 0; + + for (i, ch) in s.char_indices() { + match ch { + '(' => depth += 1, + ')' => depth = depth.saturating_sub(1), + ',' if depth == 0 => { + result.push(&s[start..i]); + start = i + 1; + } + _ => {} + } + } + result.push(&s[start..]); + result +} + +/// Parse tuple field definitions. +fn parse_tuple_fields(fields_str: &str) -> VortexResult { + let mut dtypes = Vec::new(); + let mut names = Vec::new(); + + for (i, field) in split_balanced_commas(fields_str).into_iter().enumerate() { + let field = field.trim(); + // Check if field has name: "name Type" or just "Type" + if let Some((name, type_str)) = field.split_once(' ') { + names.push(FieldName::from(name.trim())); + dtypes.push(clickhouse_type_to_vortex(type_str.trim())?); + } else { + names.push(FieldName::from(format!("_{}", i))); + dtypes.push(clickhouse_type_to_vortex(field)?); + } + } + + let field_names = FieldNames::from(names); + Ok(StructFields::new(field_names, dtypes)) +} + +/// Parse ClickHouse Decimal types into Vortex Decimal DType. +/// +/// ClickHouse supports these Decimal types: +/// - Decimal(P, S) - Generic decimal with precision P and scale S +/// - Decimal32(S) - Decimal with precision 9 and scale S (stored as Int32) +/// - Decimal64(S) - Decimal with precision 18 and scale S (stored as Int64) +/// - Decimal128(S) - Decimal with precision 38 and scale S (stored as Int128) +/// - Decimal256(S) - Decimal with precision 76 and scale S (stored as Int256) +fn parse_decimal_type(ch_type: &str) -> VortexResult { + // Decimal(P, S) + if let Some(inner) = ch_type + .strip_prefix("Decimal(") + .and_then(|s| s.strip_suffix(')')) + { + let parts: Vec<&str> = inner.split(',').map(|s| s.trim()).collect(); + if parts.len() != 2 { + vortex_bail!( + "Invalid Decimal type '{}': expected Decimal(precision, scale)", + ch_type + ); + } + let precision: u8 = parts[0].parse().map_err(|_| { + vortex::error::vortex_err!( + "Invalid precision '{}' in Decimal type '{}'", + parts[0], + ch_type + ) + })?; + let scale: i8 = parts[1].parse().map_err(|_| { + vortex::error::vortex_err!("Invalid scale '{}' in Decimal type '{}'", parts[1], ch_type) + })?; + let decimal_dtype = DecimalDType::try_new(precision, scale)?; + return Ok(DType::Decimal(decimal_dtype, NonNullable)); + } + + // Decimal32(S) - precision 9 + if let Some(inner) = ch_type + .strip_prefix("Decimal32(") + .and_then(|s| s.strip_suffix(')')) + { + let scale: i8 = inner.trim().parse().map_err(|_| { + vortex::error::vortex_err!("Invalid scale '{}' in Decimal32 type '{}'", inner, ch_type) + })?; + let decimal_dtype = DecimalDType::try_new(9, scale)?; + return Ok(DType::Decimal(decimal_dtype, NonNullable)); + } + + // Decimal64(S) - precision 18 + if let Some(inner) = ch_type + .strip_prefix("Decimal64(") + .and_then(|s| s.strip_suffix(')')) + { + let scale: i8 = inner.trim().parse().map_err(|_| { + vortex::error::vortex_err!("Invalid scale '{}' in Decimal64 type '{}'", inner, ch_type) + })?; + let decimal_dtype = DecimalDType::try_new(18, scale)?; + return Ok(DType::Decimal(decimal_dtype, NonNullable)); + } + + // Decimal128(S) - precision 38 + if let Some(inner) = ch_type + .strip_prefix("Decimal128(") + .and_then(|s| s.strip_suffix(')')) + { + let scale: i8 = inner.trim().parse().map_err(|_| { + vortex::error::vortex_err!("Invalid scale '{}' in Decimal128 type '{}'", inner, ch_type) + })?; + let decimal_dtype = DecimalDType::try_new(38, scale)?; + return Ok(DType::Decimal(decimal_dtype, NonNullable)); + } + + // Decimal256(S) - precision 76 + if let Some(inner) = ch_type + .strip_prefix("Decimal256(") + .and_then(|s| s.strip_suffix(')')) + { + let scale: i8 = inner.trim().parse().map_err(|_| { + vortex::error::vortex_err!("Invalid scale '{}' in Decimal256 type '{}'", inner, ch_type) + })?; + let decimal_dtype = DecimalDType::try_new(76, scale)?; + return Ok(DType::Decimal(decimal_dtype, NonNullable)); + } + + vortex_bail!("Unsupported Decimal type: {}", ch_type) +} + +/// Convert a Vortex DType to ClickHouse type string. +/// +/// # Arguments +/// * `dtype` - The Vortex DType to convert +/// +/// # Returns +/// The corresponding ClickHouse type string. +pub fn vortex_to_clickhouse_type(dtype: &DType) -> VortexResult { + let base_type = match dtype { + DType::Null => return Ok("Nothing".to_string()), + DType::Bool(_) => "Bool".to_string(), + DType::Primitive(ptype, _) => ptype_to_clickhouse(*ptype), + DType::Utf8(_) => "String".to_string(), + DType::Binary(_) => "String".to_string(), // ClickHouse uses String for binary + DType::Struct(fields, _) => { + let mut field_strs = Vec::new(); + for (name, dtype) in fields.names().iter().zip(fields.fields()) { + let ch_type = vortex_to_clickhouse_type(&dtype)?; + field_strs.push(format!("{} {}", name, ch_type)); + } + format!("Tuple({})", field_strs.join(", ")) + } + DType::List(elem, _) => { + let elem_type = vortex_to_clickhouse_type(elem)?; + format!("Array({})", elem_type) + } + DType::FixedSizeList(elem, size, _) => { + // Check if this is a big integer type (FixedSizeList) + if matches!(elem.as_ref(), DType::Primitive(PType::U8, _)) { + match *size { + 16 => "Int128".to_string(), // Default to signed for 128-bit + 32 => "Int256".to_string(), // Default to signed for 256-bit + _ => { + let elem_type = vortex_to_clickhouse_type(elem)?; + format!("Array({})", elem_type) + } + } + } else { + let elem_type = vortex_to_clickhouse_type(elem)?; + format!("Array({})", elem_type) // ClickHouse doesn't have FixedSizeArray + } + } + DType::Extension(_ext) => { + // Check for BigInt extension type + if let Some(bigint_type) = BigInt::try_get_type(dtype) { + bigint_type.clickhouse_type_name().to_string() + } + // Check for Geo extension type + else if let Some(geo_type) = Geo::try_get_type(dtype) { + geo_type.clickhouse_type_name().to_string() + } + // Check for UUID extension type + else if UUID::is_uuid(dtype) { + UUID::clickhouse_type_name().to_string() + } + // Check for IPAddress extension type + else if let Some(ip_type) = IPAddress::try_get_type(dtype) { + ip_type.clickhouse_type_name().to_string() + } + // Check for ClickHouse Enum extension type + else if let Some(metadata) = ClickHouseEnum::try_get_metadata(dtype) { + ClickHouseEnum::to_clickhouse_type(&metadata) + } + // Check for ClickHouse DateTime extension type + else if let Some(metadata) = ClickHouseDateTime::try_get_metadata(dtype) { + ClickHouseDateTime::to_clickhouse_type(&metadata) + } + // Check for ClickHouse Date extension type + else if let Some(metadata) = ClickHouseDate::try_get_metadata(dtype) { + ClickHouseDate::to_clickhouse_type(&metadata) + } + // Check for ClickHouse LowCardinality extension type + else if let Some(metadata) = ClickHouseLowCardinality::try_get_metadata(dtype) { + ClickHouseLowCardinality::to_clickhouse_type(&metadata) + } + // Check for ClickHouse FixedString extension type + else if let Some(metadata) = ClickHouseFixedString::try_get_metadata(dtype) { + ClickHouseFixedString::to_clickhouse_type(&metadata) + } else { + // For other extension types, return String as a fallback + "String".to_string() + } + } + DType::Decimal(decimal_dtype, _) => { + // Map Vortex Decimal to ClickHouse Decimal + format!( + "Decimal({}, {})", + decimal_dtype.precision(), + decimal_dtype.scale() + ) + } + }; + + // Wrap in Nullable if needed + if dtype.is_nullable() && !matches!(dtype, DType::Null) { + Ok(format!("Nullable({})", base_type)) + } else { + Ok(base_type) + } +} + +/// Convert Vortex PType to ClickHouse type string. +fn ptype_to_clickhouse(ptype: PType) -> String { + match ptype { + PType::I8 => "Int8", + PType::I16 => "Int16", + PType::I32 => "Int32", + PType::I64 => "Int64", + PType::U8 => "UInt8", + PType::U16 => "UInt16", + PType::U32 => "UInt32", + PType::U64 => "UInt64", + PType::F16 => "Float32", // ClickHouse doesn't have Float16, upcast + PType::F32 => "Float32", + PType::F64 => "Float64", + } + .to_string() +} + +#[cfg(test)] +mod tests { + use vortex::dtype::Nullability::NonNullable; + + use super::*; + use crate::ext_types::{IPAddressType, UUID}; + + // ========================================================================== + // ClickHouse -> Vortex conversion tests + // ========================================================================== + + #[test] + fn test_primitive_type_conversion() { + // ClickHouse -> Vortex (bare types are NonNullable) + assert!(matches!( + clickhouse_type_to_vortex("Int32").unwrap(), + DType::Primitive(PType::I32, NonNullable) + )); + assert!(matches!( + clickhouse_type_to_vortex("UInt64").unwrap(), + DType::Primitive(PType::U64, NonNullable) + )); + assert!(matches!( + clickhouse_type_to_vortex("Float64").unwrap(), + DType::Primitive(PType::F64, NonNullable) + )); + + // Vortex -> ClickHouse + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::I32, NonNullable)).unwrap(), + "Int32" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::I32, Nullable)).unwrap(), + "Nullable(Int32)" + ); + } + + #[test] + fn test_all_signed_integers() { + // ClickHouse -> Vortex (bare types are NonNullable) + assert!(matches!( + clickhouse_type_to_vortex("Int8").unwrap(), + DType::Primitive(PType::I8, NonNullable) + )); + assert!(matches!( + clickhouse_type_to_vortex("Int16").unwrap(), + DType::Primitive(PType::I16, NonNullable) + )); + assert!(matches!( + clickhouse_type_to_vortex("Int32").unwrap(), + DType::Primitive(PType::I32, NonNullable) + )); + assert!(matches!( + clickhouse_type_to_vortex("Int64").unwrap(), + DType::Primitive(PType::I64, NonNullable) + )); + + // Vortex -> ClickHouse + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::I8, NonNullable)).unwrap(), + "Int8" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::I16, NonNullable)).unwrap(), + "Int16" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::I32, NonNullable)).unwrap(), + "Int32" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::I64, NonNullable)).unwrap(), + "Int64" + ); + } + + #[test] + fn test_all_unsigned_integers() { + // ClickHouse -> Vortex (bare types are NonNullable) + assert!(matches!( + clickhouse_type_to_vortex("UInt8").unwrap(), + DType::Primitive(PType::U8, NonNullable) + )); + assert!(matches!( + clickhouse_type_to_vortex("UInt16").unwrap(), + DType::Primitive(PType::U16, NonNullable) + )); + assert!(matches!( + clickhouse_type_to_vortex("UInt32").unwrap(), + DType::Primitive(PType::U32, NonNullable) + )); + assert!(matches!( + clickhouse_type_to_vortex("UInt64").unwrap(), + DType::Primitive(PType::U64, NonNullable) + )); + + // Vortex -> ClickHouse + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::U8, NonNullable)).unwrap(), + "UInt8" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::U16, NonNullable)).unwrap(), + "UInt16" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::U32, NonNullable)).unwrap(), + "UInt32" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::U64, NonNullable)).unwrap(), + "UInt64" + ); + } + + #[test] + fn test_floating_point_types() { + // ClickHouse -> Vortex (bare types are NonNullable) + assert!(matches!( + clickhouse_type_to_vortex("Float32").unwrap(), + DType::Primitive(PType::F32, NonNullable) + )); + assert!(matches!( + clickhouse_type_to_vortex("Float64").unwrap(), + DType::Primitive(PType::F64, NonNullable) + )); + + // Vortex -> ClickHouse + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::F32, NonNullable)).unwrap(), + "Float32" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::F64, NonNullable)).unwrap(), + "Float64" + ); + // F16 should upcast to Float32 + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::F16, NonNullable)).unwrap(), + "Float32" + ); + } + + #[test] + fn test_boolean_type() { + // ClickHouse -> Vortex (bare type is NonNullable) + assert!(matches!( + clickhouse_type_to_vortex("Bool").unwrap(), + DType::Bool(NonNullable) + )); + + // Vortex -> ClickHouse + assert_eq!( + vortex_to_clickhouse_type(&DType::Bool(NonNullable)).unwrap(), + "Bool" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Bool(Nullable)).unwrap(), + "Nullable(Bool)" + ); + } + + #[test] + fn test_string_type_conversion() { + assert!(matches!( + clickhouse_type_to_vortex("String").unwrap(), + DType::Utf8(NonNullable) + )); + + assert_eq!( + vortex_to_clickhouse_type(&DType::Utf8(NonNullable)).unwrap(), + "String" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Utf8(Nullable)).unwrap(), + "Nullable(String)" + ); + } + + #[test] + fn test_fixed_string_type() { + // FixedString(N) should map to Utf8 NonNullable (bare type) + assert!(matches!( + clickhouse_type_to_vortex("FixedString(10)").unwrap(), + DType::Utf8(NonNullable) + )); + assert!(matches!( + clickhouse_type_to_vortex("FixedString(256)").unwrap(), + DType::Utf8(NonNullable) + )); + } + + #[test] + fn test_binary_type() { + // Vortex Binary -> ClickHouse String + assert_eq!( + vortex_to_clickhouse_type(&DType::Binary(NonNullable)).unwrap(), + "String" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Binary(Nullable)).unwrap(), + "Nullable(String)" + ); + } + + // ========================================================================== + // Date/Time type tests + // ========================================================================== + + #[test] + fn test_date_types() { + // Date -> Extension(clickhouse.date) + let dtype = clickhouse_type_to_vortex("Date").unwrap(); + assert!(matches!(dtype, DType::Extension(_))); + let metadata = ClickHouseDate::try_get_metadata(&dtype).unwrap(); + assert!(!metadata.is_date32); + + // Date32 -> Extension(clickhouse.date) + let dtype = clickhouse_type_to_vortex("Date32").unwrap(); + assert!(matches!(dtype, DType::Extension(_))); + let metadata = ClickHouseDate::try_get_metadata(&dtype).unwrap(); + assert!(metadata.is_date32); + } + + #[test] + fn test_datetime_types() { + // DateTime -> Extension(clickhouse.datetime) + let dtype = clickhouse_type_to_vortex("DateTime").unwrap(); + assert!(matches!(dtype, DType::Extension(_))); + let metadata = ClickHouseDateTime::try_get_metadata(&dtype).unwrap(); + assert_eq!(metadata.precision, 0); + assert_eq!(metadata.timezone, None); + + // DateTime with timezone + let dtype = clickhouse_type_to_vortex("DateTime('UTC')").unwrap(); + assert!(matches!(dtype, DType::Extension(_))); + let metadata = ClickHouseDateTime::try_get_metadata(&dtype).unwrap(); + assert_eq!(metadata.precision, 0); + assert_eq!(metadata.timezone, Some("UTC".to_string())); + + let dtype = clickhouse_type_to_vortex("DateTime('Asia/Shanghai')").unwrap(); + let metadata = ClickHouseDateTime::try_get_metadata(&dtype).unwrap(); + assert_eq!(metadata.timezone, Some("Asia/Shanghai".to_string())); + } + + #[test] + fn test_datetime64_types() { + // DateTime64(3) -> Extension(clickhouse.datetime) + let dtype = clickhouse_type_to_vortex("DateTime64(3)").unwrap(); + assert!(matches!(dtype, DType::Extension(_))); + let metadata = ClickHouseDateTime::try_get_metadata(&dtype).unwrap(); + assert_eq!(metadata.precision, 3); + assert_eq!(metadata.timezone, None); + + let dtype = clickhouse_type_to_vortex("DateTime64(6, 'UTC')").unwrap(); + let metadata = ClickHouseDateTime::try_get_metadata(&dtype).unwrap(); + assert_eq!(metadata.precision, 6); + assert_eq!(metadata.timezone, Some("UTC".to_string())); + + let dtype = clickhouse_type_to_vortex("DateTime64(9, 'America/New_York')").unwrap(); + let metadata = ClickHouseDateTime::try_get_metadata(&dtype).unwrap(); + assert_eq!(metadata.precision, 9); + assert_eq!(metadata.timezone, Some("America/New_York".to_string())); + } + + // ========================================================================== + // Complex type tests + // ========================================================================== + + #[test] + fn test_array_type_conversion() { + let dtype = clickhouse_type_to_vortex("Array(Int32)").unwrap(); + assert!(matches!(dtype, DType::List(_, NonNullable))); + + let list_dtype = DType::List( + Arc::new(DType::Primitive(PType::I32, NonNullable)), + NonNullable, + ); + assert_eq!( + vortex_to_clickhouse_type(&list_dtype).unwrap(), + "Array(Int32)" + ); + } + + #[test] + fn test_array_of_various_types() { + // Array of String + let dtype = clickhouse_type_to_vortex("Array(String)").unwrap(); + if let DType::List(elem, _) = dtype { + assert!(matches!(elem.as_ref(), DType::Utf8(NonNullable))); + } else { + panic!("Expected List type"); + } + + // Array of Float64 + let dtype = clickhouse_type_to_vortex("Array(Float64)").unwrap(); + if let DType::List(elem, _) = dtype { + assert!(matches!( + elem.as_ref(), + DType::Primitive(PType::F64, NonNullable) + )); + } else { + panic!("Expected List type"); + } + + // Array of Bool + let dtype = clickhouse_type_to_vortex("Array(Bool)").unwrap(); + if let DType::List(elem, _) = dtype { + assert!(matches!(elem.as_ref(), DType::Bool(NonNullable))); + } else { + panic!("Expected List type"); + } + } + + #[test] + fn test_nested_array() { + // Array(Array(Int32)) + let dtype = clickhouse_type_to_vortex("Array(Array(Int32))").unwrap(); + if let DType::List(outer_elem, _) = dtype { + if let DType::List(inner_elem, _) = outer_elem.as_ref() { + assert!(matches!( + inner_elem.as_ref(), + DType::Primitive(PType::I32, NonNullable) + )); + } else { + panic!("Expected nested List type"); + } + } else { + panic!("Expected List type"); + } + } + + #[test] + fn test_tuple_unnamed_fields() { + // Tuple(Int32, String) + let dtype = clickhouse_type_to_vortex("Tuple(Int32, String)").unwrap(); + if let DType::Struct(fields, _) = dtype { + assert_eq!(fields.nfields(), 2); + // Unnamed fields should get default names + assert_eq!(fields.field_name(0).unwrap().as_ref(), "_0"); + assert_eq!(fields.field_name(1).unwrap().as_ref(), "_1"); + assert!(matches!( + fields.field_by_index(0).unwrap(), + DType::Primitive(PType::I32, NonNullable) + )); + assert!(matches!( + fields.field_by_index(1).unwrap(), + DType::Utf8(NonNullable) + )); + } else { + panic!("Expected Struct type"); + } + } + + #[test] + fn test_tuple_named_fields() { + // Tuple(id Int32, name String) + let dtype = clickhouse_type_to_vortex("Tuple(id Int32, name String)").unwrap(); + if let DType::Struct(fields, _) = dtype { + assert_eq!(fields.nfields(), 2); + assert_eq!(fields.field_name(0).unwrap().as_ref(), "id"); + assert_eq!(fields.field_name(1).unwrap().as_ref(), "name"); + assert!(matches!( + fields.field_by_index(0).unwrap(), + DType::Primitive(PType::I32, NonNullable) + )); + assert!(matches!( + fields.field_by_index(1).unwrap(), + DType::Utf8(NonNullable) + )); + } else { + panic!("Expected Struct type"); + } + } + + #[test] + fn test_struct_to_tuple_conversion() { + // Build a Vortex struct + let names = FieldNames::from(vec![ + FieldName::from("col_a"), + FieldName::from("col_b"), + FieldName::from("col_c"), + ]); + let dtypes = vec![ + DType::Primitive(PType::I64, NonNullable), + DType::Utf8(NonNullable), + DType::Bool(NonNullable), + ]; + let struct_dtype = DType::Struct(StructFields::new(names, dtypes), NonNullable); + + let ch_type = vortex_to_clickhouse_type(&struct_dtype).unwrap(); + assert_eq!(ch_type, "Tuple(col_a Int64, col_b String, col_c Bool)"); + } + + #[test] + fn test_fixed_size_list() { + // FixedSizeList should map to Array + let fsl_dtype = DType::FixedSizeList( + Arc::new(DType::Primitive(PType::F32, NonNullable)), + 4, + NonNullable, + ); + assert_eq!( + vortex_to_clickhouse_type(&fsl_dtype).unwrap(), + "Array(Float32)" + ); + } + + // ========================================================================== + // Nullable wrapper tests + // ========================================================================== + + #[test] + fn test_nullable_wrapper() { + let dtype = clickhouse_type_to_vortex("Nullable(Int32)").unwrap(); + assert!(dtype.is_nullable()); + assert!(matches!(dtype, DType::Primitive(PType::I32, Nullable))); + } + + #[test] + fn test_nullable_string() { + let dtype = clickhouse_type_to_vortex("Nullable(String)").unwrap(); + assert!(dtype.is_nullable()); + assert!(matches!(dtype, DType::Utf8(Nullable))); + } + + #[test] + fn test_nullable_float() { + let dtype = clickhouse_type_to_vortex("Nullable(Float64)").unwrap(); + assert!(dtype.is_nullable()); + assert!(matches!(dtype, DType::Primitive(PType::F64, Nullable))); + } + + #[test] + fn test_low_cardinality_wrapper() { + // LowCardinality should be preserved as Extension type + let dtype = clickhouse_type_to_vortex("LowCardinality(String)").unwrap(); + assert!(matches!(dtype, DType::Extension(_))); + let metadata = ClickHouseLowCardinality::try_get_metadata(&dtype).unwrap(); + assert_eq!(metadata.inner_type, "String"); + + // Roundtrip + let ch_type = vortex_to_clickhouse_type(&dtype).unwrap(); + assert_eq!(ch_type, "LowCardinality(String)"); + + // LowCardinality with Nullable + let dtype = clickhouse_type_to_vortex("LowCardinality(Nullable(String))").unwrap(); + assert!(matches!(dtype, DType::Extension(_))); + let metadata = ClickHouseLowCardinality::try_get_metadata(&dtype).unwrap(); + assert_eq!(metadata.inner_type, "Nullable(String)"); + } + + // ========================================================================== + // Special type tests + // ========================================================================== + + #[test] + fn test_uuid_type() { + // UUID -> Extension(clickhouse.uuid) + let dtype = clickhouse_type_to_vortex("UUID").unwrap(); + assert!(matches!(dtype, DType::Extension(_))); + assert!(UUID::is_uuid(&dtype)); + + // Roundtrip + let ch_type = vortex_to_clickhouse_type(&dtype).unwrap(); + assert_eq!(ch_type, "UUID"); + + // Nullable UUID + let dtype = clickhouse_type_to_vortex("Nullable(UUID)").unwrap(); + assert!(matches!(dtype, DType::Extension(_))); + assert!(UUID::is_uuid(&dtype)); + } + + #[test] + fn test_decimal_types() { + // Decimal(P, S) should map to Vortex Decimal + let dtype = clickhouse_type_to_vortex("Decimal(10, 2)").unwrap(); + assert!(matches!(dtype, DType::Decimal(_, NonNullable))); + if let DType::Decimal(decimal_dtype, _) = dtype { + assert_eq!(decimal_dtype.precision(), 10); + assert_eq!(decimal_dtype.scale(), 2); + } + + // Decimal32(S) - precision 9 + let dtype = clickhouse_type_to_vortex("Decimal32(4)").unwrap(); + assert!(matches!(dtype, DType::Decimal(_, NonNullable))); + if let DType::Decimal(decimal_dtype, _) = dtype { + assert_eq!(decimal_dtype.precision(), 9); + assert_eq!(decimal_dtype.scale(), 4); + } + + // Decimal64(S) - precision 18 + let dtype = clickhouse_type_to_vortex("Decimal64(8)").unwrap(); + assert!(matches!(dtype, DType::Decimal(_, NonNullable))); + if let DType::Decimal(decimal_dtype, _) = dtype { + assert_eq!(decimal_dtype.precision(), 18); + assert_eq!(decimal_dtype.scale(), 8); + } + + // Decimal128(S) - precision 38 + let dtype = clickhouse_type_to_vortex("Decimal128(18)").unwrap(); + assert!(matches!(dtype, DType::Decimal(_, NonNullable))); + if let DType::Decimal(decimal_dtype, _) = dtype { + assert_eq!(decimal_dtype.precision(), 38); + assert_eq!(decimal_dtype.scale(), 18); + } + + // Decimal256(S) - precision 76 + let dtype = clickhouse_type_to_vortex("Decimal256(30)").unwrap(); + assert!(matches!(dtype, DType::Decimal(_, NonNullable))); + if let DType::Decimal(decimal_dtype, _) = dtype { + assert_eq!(decimal_dtype.precision(), 76); + assert_eq!(decimal_dtype.scale(), 30); + } + } + + #[test] + fn test_decimal_roundtrip() { + // Test Vortex Decimal -> ClickHouse Decimal -> Vortex Decimal + use crate::convert::dtype::DecimalDType; + + let decimal_dtype = DecimalDType::new(10, 2); + let vortex_dtype = DType::Decimal(decimal_dtype, NonNullable); + let ch_type = vortex_to_clickhouse_type(&vortex_dtype).unwrap(); + assert_eq!(ch_type, "Decimal(10, 2)"); + + let roundtrip = clickhouse_type_to_vortex(&ch_type).unwrap(); + if let DType::Decimal(rt_decimal, _) = roundtrip { + assert_eq!(rt_decimal.precision(), 10); + assert_eq!(rt_decimal.scale(), 2); + } else { + panic!("Expected Decimal type"); + } + } + + #[test] + fn test_nullable_decimal() { + let dtype = clickhouse_type_to_vortex("Nullable(Decimal(10, 2))").unwrap(); + assert!(dtype.is_nullable()); + assert!(matches!(dtype, DType::Decimal(_, Nullable))); + } + + #[test] + fn test_null_type() { + // Vortex Null -> ClickHouse Nothing + assert_eq!(vortex_to_clickhouse_type(&DType::Null).unwrap(), "Nothing"); + } + + // ========================================================================== + // Edge cases and error handling tests + // ========================================================================== + + #[test] + fn test_whitespace_handling() { + // Type strings with extra whitespace + assert!(matches!( + clickhouse_type_to_vortex(" Int32 ").unwrap(), + DType::Primitive(PType::I32, NonNullable) + )); + assert!(matches!( + clickhouse_type_to_vortex("Nullable( Int32 )").unwrap(), + DType::Primitive(PType::I32, Nullable) + )); + } + + #[test] + fn test_unsupported_type_error() { + // Unknown type should return error + let result = clickhouse_type_to_vortex("SomeUnknownType(1, 2)"); + assert!(result.is_err()); + } + + #[test] + fn test_enum8_type() { + // Enum8 should map to Primitive(I8) + let dtype = clickhouse_type_to_vortex("Enum8('a' = 1, 'b' = 2)").unwrap(); + assert!(matches!(dtype, DType::Primitive(PType::I8, NonNullable))); + } + + #[test] + fn test_enum16_type() { + // Enum16 should map to Primitive(I16) + let dtype = clickhouse_type_to_vortex("Enum16('x' = 100, 'y' = 200)").unwrap(); + assert!(matches!(dtype, DType::Primitive(PType::I16, NonNullable))); + } + + #[test] + fn test_map_type() { + // Map(String, Int32) should map to List(Struct([key String, value Int32])) + let dtype = clickhouse_type_to_vortex("Map(String, Int32)").unwrap(); + if let DType::List(elem, _) = &dtype { + if let DType::Struct(fields, _) = elem.as_ref() { + assert_eq!(fields.nfields(), 2); + assert_eq!(fields.field_name(0).unwrap().as_ref(), "key"); + assert_eq!(fields.field_name(1).unwrap().as_ref(), "value"); + assert!(matches!(fields.field_by_index(0).unwrap(), DType::Utf8(..))); + assert!(matches!( + fields.field_by_index(1).unwrap(), + DType::Primitive(PType::I32, ..) + )); + } else { + panic!("Expected Struct element type"); + } + } else { + panic!("Expected List type for Map"); + } + } + + #[test] + fn test_map_nested_type() { + // Map(String, Array(Int32)) + let dtype = clickhouse_type_to_vortex("Map(String, Array(Int32))").unwrap(); + if let DType::List(elem, _) = &dtype { + if let DType::Struct(fields, _) = elem.as_ref() { + assert_eq!(fields.nfields(), 2); + assert!(matches!(fields.field_by_index(1).unwrap(), DType::List(..))); + } else { + panic!("Expected Struct element type"); + } + } else { + panic!("Expected List type for Map"); + } + } + + #[test] + fn test_nullable_enum8() { + let dtype = clickhouse_type_to_vortex("Nullable(Enum8('a' = 1))").unwrap(); + assert!(dtype.is_nullable()); + assert!(matches!(dtype, DType::Primitive(PType::I8, Nullable))); + } + + #[test] + fn test_lowcardinality_enum() { + // LowCardinality(Enum8(...)) should preserve LowCardinality wrapper + let dtype = clickhouse_type_to_vortex("LowCardinality(Enum8('a' = 1, 'b' = 2))").unwrap(); + assert!(matches!(dtype, DType::Extension(_))); + let metadata = ClickHouseLowCardinality::try_get_metadata(&dtype).unwrap(); + assert_eq!(metadata.inner_type, "Enum8('a' = 1, 'b' = 2)"); + } + + #[test] + fn test_nested_tuple_parsing() { + // Tuple(a Array(Int32), b String) + let dtype = clickhouse_type_to_vortex("Tuple(a Array(Int32), b String)").unwrap(); + if let DType::Struct(fields, _) = dtype { + assert_eq!(fields.nfields(), 2); + assert_eq!(fields.field_name(0).unwrap().as_ref(), "a"); + assert_eq!(fields.field_name(1).unwrap().as_ref(), "b"); + assert!(matches!(fields.field_by_index(0).unwrap(), DType::List(..))); + assert!(matches!(fields.field_by_index(1).unwrap(), DType::Utf8(..))); + } else { + panic!("Expected Struct type for nested Tuple"); + } + } + + #[test] + fn test_deeply_nested_tuple() { + // Tuple(x Tuple(a Int32, b Float64), y Array(String)) + let dtype = + clickhouse_type_to_vortex("Tuple(x Tuple(a Int32, b Float64), y Array(String))") + .unwrap(); + if let DType::Struct(fields, _) = dtype { + assert_eq!(fields.nfields(), 2); + assert_eq!(fields.field_name(0).unwrap().as_ref(), "x"); + assert_eq!(fields.field_name(1).unwrap().as_ref(), "y"); + assert!(matches!( + fields.field_by_index(0).unwrap(), + DType::Struct(..) + )); + assert!(matches!(fields.field_by_index(1).unwrap(), DType::List(..))); + } else { + panic!("Expected Struct type"); + } + } + + #[test] + fn test_split_balanced_commas() { + let result = split_balanced_commas("a Array(Int32), b String"); + assert_eq!(result, vec!["a Array(Int32)", " b String"]); + + let result = split_balanced_commas("x Tuple(a Int32, b Float64), y Array(String)"); + assert_eq!( + result, + vec!["x Tuple(a Int32, b Float64)", " y Array(String)"] + ); + + let result = split_balanced_commas("Int32, String"); + assert_eq!(result, vec!["Int32", " String"]); + + let result = split_balanced_commas("single"); + assert_eq!(result, vec!["single"]); + } + + #[test] + fn test_ip_type_conversion() { + // IPv4 should map to Primitive(U32) + let result = clickhouse_type_to_vortex("IPv4").unwrap(); + assert!(matches!(result, DType::Primitive(PType::U32, _))); + + // IPv6 should map to Extension(clickhouse.ip) + let result = clickhouse_type_to_vortex("IPv6").unwrap(); + assert!(matches!(result, DType::Extension(_))); + assert_eq!(IPAddress::try_get_type(&result), Some(IPAddressType::IPv6)); + + // Roundtrip + let ch_type = vortex_to_clickhouse_type(&result).unwrap(); + assert_eq!(ch_type, "IPv6"); + + // Nullable IPv4 + let result = clickhouse_type_to_vortex("Nullable(IPv4)").unwrap(); + assert!(matches!(result, DType::Primitive(PType::U32, Nullable))); + + // Nullable IPv6 + let result = clickhouse_type_to_vortex("Nullable(IPv6)").unwrap(); + assert!(matches!(result, DType::Extension(_))); + assert_eq!(IPAddress::try_get_type(&result), Some(IPAddressType::IPv6)); + } + + // ========================================================================== + // Roundtrip tests + // ========================================================================== + + #[test] + fn test_primitive_roundtrip() { + // Non-nullable types + for ptype in [ + PType::I8, + PType::I16, + PType::I32, + PType::I64, + PType::U8, + PType::U16, + PType::U32, + PType::U64, + PType::F32, + PType::F64, + ] { + let vortex_dtype = DType::Primitive(ptype, NonNullable); + let ch_type = vortex_to_clickhouse_type(&vortex_dtype).unwrap(); + let roundtrip = clickhouse_type_to_vortex(&ch_type).unwrap(); + // Roundtrip preserves nullability: NonNullable CH type -> NonNullable Vortex type + assert!(matches!(roundtrip, DType::Primitive(..))); + } + } + + #[test] + fn test_list_roundtrip() { + let vortex_dtype = DType::List( + Arc::new(DType::Primitive(PType::I32, NonNullable)), + NonNullable, + ); + let ch_type = vortex_to_clickhouse_type(&vortex_dtype).unwrap(); + assert_eq!(ch_type, "Array(Int32)"); + + let roundtrip = clickhouse_type_to_vortex(&ch_type).unwrap(); + assert!(matches!(roundtrip, DType::List(..))); + } + + #[test] + fn test_complex_nested_struct() { + // Create a complex nested struct: Tuple(id Int64, data Tuple(x Float64, y Float64), tags Array(String)) + let inner_names = FieldNames::from(vec![FieldName::from("x"), FieldName::from("y")]); + let inner_dtypes = vec![ + DType::Primitive(PType::F64, NonNullable), + DType::Primitive(PType::F64, NonNullable), + ]; + let inner_struct = DType::Struct(StructFields::new(inner_names, inner_dtypes), NonNullable); + + let outer_names = FieldNames::from(vec![ + FieldName::from("id"), + FieldName::from("data"), + FieldName::from("tags"), + ]); + let outer_dtypes = vec![ + DType::Primitive(PType::I64, NonNullable), + inner_struct, + DType::List(Arc::new(DType::Utf8(NonNullable)), NonNullable), + ]; + let outer_struct = DType::Struct(StructFields::new(outer_names, outer_dtypes), NonNullable); + + let ch_type = vortex_to_clickhouse_type(&outer_struct).unwrap(); + assert_eq!( + ch_type, + "Tuple(id Int64, data Tuple(x Float64, y Float64), tags Array(String))" + ); + } + + #[test] + fn test_uuid_roundtrip() { + let ch_type = "UUID"; + let dtype = clickhouse_type_to_vortex(ch_type).unwrap(); + let back = vortex_to_clickhouse_type(&dtype).unwrap(); + assert_eq!(back, "UUID"); + } + + #[test] + fn test_ipv6_roundtrip() { + let ch_type = "IPv6"; + let dtype = clickhouse_type_to_vortex(ch_type).unwrap(); + let back = vortex_to_clickhouse_type(&dtype).unwrap(); + assert_eq!(back, "IPv6"); + } + + #[test] + fn test_bigint_roundtrip() { + for (ch_type, expected_back) in [ + ("Int128", "Int128"), + ("UInt128", "UInt128"), + ("Int256", "Int256"), + ("UInt256", "UInt256"), + ] { + let dtype = clickhouse_type_to_vortex(ch_type).unwrap(); + let back = vortex_to_clickhouse_type(&dtype).unwrap(); + assert_eq!(back, expected_back, "Roundtrip failed for {}", ch_type); + } + } + + #[test] + fn test_lowcardinality_roundtrip() { + let ch_type = "LowCardinality(String)"; + let dtype = clickhouse_type_to_vortex(ch_type).unwrap(); + let back = vortex_to_clickhouse_type(&dtype).unwrap(); + assert_eq!(back, "LowCardinality(String)"); + } +} diff --git a/vortex-clickhouse/src/convert/mod.rs b/vortex-clickhouse/src/convert/mod.rs new file mode 100644 index 00000000000..fcb572e04fe --- /dev/null +++ b/vortex-clickhouse/src/convert/mod.rs @@ -0,0 +1,12 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Type conversion between Vortex and ClickHouse. +//! +//! This module provides bidirectional type mapping between Vortex's `DType` and +//! ClickHouse's type system. + +pub mod column; +pub mod dtype; +pub mod scalar; +pub mod table_filter; diff --git a/vortex-clickhouse/src/convert/scalar.rs b/vortex-clickhouse/src/convert/scalar.rs new file mode 100644 index 00000000000..9577e865d38 --- /dev/null +++ b/vortex-clickhouse/src/convert/scalar.rs @@ -0,0 +1,17 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Scalar value conversion between Vortex and ClickHouse. + +use vortex::error::{VortexResult, vortex_bail}; +use vortex::scalar::Scalar; + +/// Convert a ClickHouse value to a Vortex Scalar. +pub fn clickhouse_value_to_vortex(_value_ptr: *const std::ffi::c_void) -> VortexResult { + vortex_bail!("ClickHouse value to Vortex scalar conversion not yet implemented") +} + +/// Convert a Vortex Scalar to a ClickHouse value. +pub fn vortex_to_clickhouse_value(_scalar: &Scalar) -> VortexResult<*mut std::ffi::c_void> { + vortex_bail!("Vortex scalar to ClickHouse value conversion not yet implemented") +} diff --git a/vortex-clickhouse/src/convert/table_filter.rs b/vortex-clickhouse/src/convert/table_filter.rs new file mode 100644 index 00000000000..b49072279e3 --- /dev/null +++ b/vortex-clickhouse/src/convert/table_filter.rs @@ -0,0 +1,28 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Table filter conversion for predicate pushdown. +//! +//! This module converts ClickHouse's filter expressions to Vortex's +//! expression format for predicate pushdown support. + +use vortex::error::{VortexResult, vortex_bail}; + +/// Convert a ClickHouse table filter to Vortex expression. +/// +/// ClickHouse passes filters through the `IInputFormat::setQueryInfo` method. +/// We extract the filter predicates and convert them to Vortex expressions +/// that can be pushed down to the Vortex file reader. +pub fn clickhouse_filter_to_vortex(_filter_ptr: *const std::ffi::c_void) -> VortexResult<()> { + vortex_bail!("ClickHouse filter to Vortex expression conversion not yet implemented") +} + +/// Check if a filter can be pushed down to ClickHouse. +/// +/// Not all filters can be converted to ClickHouse filters. +/// This function checks if a given filter is supported. +pub fn can_pushdown_to_clickhouse() -> bool { + // TODO: Implement pushdown capability check + // Check if the expression uses only supported operators and types. + false +} diff --git a/vortex-clickhouse/src/copy.rs b/vortex-clickhouse/src/copy.rs new file mode 100644 index 00000000000..a1a35d9dc3f --- /dev/null +++ b/vortex-clickhouse/src/copy.rs @@ -0,0 +1,2715 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Vortex file writing for ClickHouse. +//! +//! This module implements the write path for Vortex files in ClickHouse. +//! It provides the core logic for the `VortexBlockOutputFormat` C++ class. +//! +//! # Architecture +//! +//! The writer uses a streaming architecture to avoid accumulating all data in memory: +//! +//! 1. On creation, a background write task is spawned that consumes arrays from a channel +//! 2. Each batch is sent through the channel and written incrementally +//! 3. On finalize, the channel is closed and we wait for the write task to complete +//! +//! This ensures memory usage is bounded by the channel buffer size (currently 32 batches). +//! +//! # FFI Interface +//! +//! The following C functions are exported for ClickHouse to use: +//! +//! - `vortex_writer_new` - Create a new writer +//! - `vortex_writer_free` - Free a writer +//! - `vortex_writer_add_column` - Add a column to the schema +//! - `vortex_writer_write_batch` - Write a batch of data +//! - `vortex_writer_write_string_column` - Write a string column +//! - `vortex_writer_finalize` - Finalize and flush the file + +use std::ffi::{CStr, c_char, c_void}; +use std::ptr; +use std::sync::Arc; + +use futures::SinkExt; +use futures::StreamExt; +use futures::channel::mpsc; +use futures::channel::mpsc::Sender; +use vortex::array::arrays::{ + DecimalArray, FixedSizeListArray, ListViewArray, PrimitiveArray, StructArray, VarBinViewArray, +}; +use vortex::array::builders::{ArrayBuilder, VarBinViewBuilder}; +use vortex::array::stream::ArrayStreamAdapter; +use vortex::array::validity::Validity; +use vortex::array::{ArrayRef, IntoArray}; +use vortex::buffer::Buffer; +use vortex::dtype::{ + DType, DecimalDType, DecimalType, FieldNames, Nullability, PType, StructFields, +}; +use vortex::error::{VortexResult, vortex_bail, vortex_err}; +use vortex::file::{WriteOptionsSessionExt, WriteSummary}; +use vortex::io::runtime::{BlockingRuntime, Task}; +use vortex::io::session::RuntimeSessionExt; + +use crate::convert::dtype::clickhouse_type_to_vortex; +use crate::error::{clear_last_error, set_last_error}; +use crate::{RUNTIME, SESSION}; + +/// Channel buffer size for streaming writes. +/// This limits memory usage to approximately this many batches in flight. +const CHANNEL_BUFFER_SIZE: usize = 32; + +/// Column definition for schema building. +#[derive(Clone)] +struct ColumnDef { + /// Column name + name: String, + /// Vortex dtype + dtype: DType, +} + +/// String data for a column, provided from ClickHouse side. +#[derive(Clone)] +pub struct StringColumnData { + /// Raw string data (concatenated) + pub data: Vec, + /// Offsets into data for each string (length = num_rows + 1) + pub offsets: Vec, +} + +/// State of the streaming writer +enum WriterState { + /// Schema is being built, no data written yet + Building, + /// Writer is active, data can be written + Active { + /// Channel to send arrays to the background writer + sender: Sender>, + /// Handle to the background write task + writer_task: Task>, + }, + /// Writer has been finalized + Finalized, +} + +/// Pending state for a list column being written. +struct PendingList { + /// The list offsets (num_rows + 1 elements) + offsets: Vec, + /// Validity for the list itself + validity: Validity, + /// Number of rows + num_rows: usize, + /// The element array (set when elements are written) + elements: Option, +} + +/// Pending state for a struct column being written. +struct PendingStruct { + /// Validity for the struct itself + validity: Validity, + /// Number of rows + num_rows: usize, + /// The field arrays, indexed by field_index + fields: Vec>, + /// Number of fields expected + num_fields: usize, +} + +/// Vortex file writer that implements the write logic. +/// +/// Uses a streaming architecture to avoid accumulating all data in memory. +/// Data is written incrementally through a channel to a background task. +pub struct VortexWriter { + /// Output file path. + output_path: String, + /// Column definitions (schema). + columns: Vec, + /// Writer state machine + state: WriterState, + /// Total rows written. + total_rows: usize, + /// Pending column arrays for the current batch. + pending_columns: Vec>, + /// Expected number of rows for current pending batch. + pending_num_rows: usize, + /// Pending list column writes (column_index -> PendingList) + pending_lists: std::collections::HashMap, + /// Pending struct column writes (column_index -> PendingStruct) + pending_structs: std::collections::HashMap, +} + +impl VortexWriter { + /// Create a new writer for the given output path. + pub fn new(output_path: &str) -> VortexResult { + if output_path.is_empty() { + vortex_bail!("Output path cannot be empty"); + } + + Ok(Self { + output_path: output_path.to_string(), + columns: Vec::new(), + state: WriterState::Building, + total_rows: 0, + pending_columns: Vec::new(), + pending_num_rows: 0, + pending_lists: std::collections::HashMap::new(), + pending_structs: std::collections::HashMap::new(), + }) + } + + /// Add a column to the schema. + pub fn add_column( + &mut self, + name: &str, + clickhouse_type: &str, + nullable: bool, + ) -> VortexResult<()> { + if !matches!(self.state, WriterState::Building) { + vortex_bail!("Cannot add column after writing has started"); + } + + // Convert ClickHouse type to Vortex DType + let mut dtype = clickhouse_type_to_vortex(clickhouse_type)?; + + // Adjust nullability based on the nullable parameter + // If the ClickHouse type is already Nullable(...), respect that. + // Otherwise, use the nullable parameter to set nullability. + if !clickhouse_type.starts_with("Nullable(") { + use vortex::dtype::Nullability; + let target_nullability = if nullable { + Nullability::Nullable + } else { + Nullability::NonNullable + }; + dtype = dtype.with_nullability(target_nullability); + } + + self.columns.push(ColumnDef { + name: name.to_string(), + dtype, + }); + + Ok(()) + } + + /// Get the number of columns. + pub fn num_columns(&self) -> usize { + self.columns.len() + } + + /// Get the struct DType for the schema. + fn get_struct_dtype(&self) -> DType { + let field_names: Vec> = self + .columns + .iter() + .map(|c| Arc::from(c.name.as_str())) + .collect(); + let field_dtypes: Vec = self.columns.iter().map(|c| c.dtype.clone()).collect(); + + DType::Struct( + StructFields::new(FieldNames::from(field_names), field_dtypes), + Nullability::NonNullable, + ) + } + + /// Start the background writer if not already started. + fn ensure_writer_started(&mut self) -> VortexResult<()> { + if matches!(self.state, WriterState::Building) { + if self.columns.is_empty() { + vortex_bail!("Cannot start writing without any columns defined"); + } + + let struct_dtype = self.get_struct_dtype(); + let output_path = self.output_path.clone(); + + // Create channel for streaming data + let (sender, receiver) = mpsc::channel(CHANNEL_BUFFER_SIZE); + + // Create array stream from channel receiver + let array_stream = ArrayStreamAdapter::new(struct_dtype, receiver.map(|r| r)); + + // Spawn background writer task + let writer_task = SESSION.handle().spawn(async move { + let mut file = async_fs::File::create(&output_path).await.map_err(|e| { + vortex_err!("Failed to create output file '{}': {}", output_path, e) + })?; + SESSION.write_options().write(&mut file, array_stream).await + }); + + self.state = WriterState::Active { + sender, + writer_task, + }; + } + Ok(()) + } + + /// Send an array to the background writer. + fn send_array(&mut self, array: ArrayRef) -> VortexResult<()> { + self.ensure_writer_started()?; + + match &mut self.state { + WriterState::Active { sender, .. } => { + RUNTIME + .block_on(sender.send(Ok(array))) + .map_err(|e| vortex_err!("Failed to send array to writer: {}", e))?; + Ok(()) + } + WriterState::Finalized => { + vortex_bail!("Cannot write after finalization") + } + WriterState::Building => { + // This shouldn't happen as ensure_writer_started was called + vortex_bail!("Writer not started") + } + } + } + + /// Begin writing a new batch with the given number of rows. + /// + /// This prepares the writer to accept column data via write_column_*() methods. + pub fn begin_batch(&mut self, num_rows: usize) -> VortexResult<()> { + if matches!(self.state, WriterState::Finalized) { + vortex_bail!("Cannot write after finalization"); + } + + if !self.pending_columns.is_empty() { + vortex_bail!("Previous batch not completed. Call end_batch() first."); + } + + self.pending_columns = vec![None; self.columns.len()]; + self.pending_num_rows = num_rows; + Ok(()) + } + + /// Write a primitive column by index. + pub fn write_column_primitive( + &mut self, + column_index: usize, + data: *const c_void, + num_rows: usize, + ) -> VortexResult<()> { + self.write_column_primitive_with_validity(column_index, data, ptr::null(), num_rows) + } + + /// Write a primitive column by index with optional validity bitmap. + /// + /// The validity bitmap uses ClickHouse's convention where 0 = null, 1 = valid. + /// Each byte contains 8 validity bits in LSB order. + pub fn write_column_primitive_with_validity( + &mut self, + column_index: usize, + data: *const c_void, + validity_bitmap: *const u8, + num_rows: usize, + ) -> VortexResult<()> { + if column_index >= self.columns.len() { + vortex_bail!("Column index {} out of bounds", column_index); + } + + if num_rows != self.pending_num_rows { + vortex_bail!( + "Row count mismatch: expected {}, got {}", + self.pending_num_rows, + num_rows + ); + } + + let dtype = &self.columns[column_index].dtype; + let array = build_array_from_raw_with_validity(dtype, data, validity_bitmap, num_rows)?; + self.pending_columns[column_index] = Some(array); + Ok(()) + } + + /// Write a string column by index. + /// + /// The strings are provided as concatenated data with offsets. + pub fn write_column_strings( + &mut self, + column_index: usize, + data: *const u8, + offsets: *const u64, + num_rows: usize, + ) -> VortexResult<()> { + self.write_column_strings_with_validity(column_index, data, offsets, ptr::null(), num_rows) + } + + /// Write a string column by index with optional validity bitmap. + /// + /// The strings are provided as concatenated data with offsets. + /// The null_map uses ClickHouse's convention: one byte per row, 0 = valid, 1 = null. + pub fn write_column_strings_with_validity( + &mut self, + column_index: usize, + data: *const u8, + offsets: *const u64, + null_map: *const u8, + num_rows: usize, + ) -> VortexResult<()> { + if column_index >= self.columns.len() { + vortex_bail!("Column index {} out of bounds", column_index); + } + + if num_rows != self.pending_num_rows { + vortex_bail!( + "Row count mismatch: expected {}, got {}", + self.pending_num_rows, + num_rows + ); + } + + // Build string array from offsets + let offsets_slice = unsafe { std::slice::from_raw_parts(offsets, num_rows + 1) }; + + // Total data size is the last offset + let total_data_len = offsets_slice[num_rows] as usize; + let data_slice = if total_data_len > 0 { + unsafe { std::slice::from_raw_parts(data, total_data_len) } + } else { + &[] + }; + + // Check if we need to handle nullability + let dtype = &self.columns[column_index].dtype; + let is_nullable = match dtype { + DType::Utf8(n) | DType::Binary(n) => *n == Nullability::Nullable, + DType::Extension(ext) => ext.storage_dtype().is_nullable(), + _ => false, + }; + + // Check if the storage is binary (not UTF-8 text) + let is_binary = match dtype { + DType::Binary(_) => true, + DType::Extension(ext) => matches!(ext.storage_dtype(), DType::Binary(_)), + _ => false, + }; + + let array = if is_binary { + // Binary data path: treat bytes as-is, no UTF-8 validation + if is_nullable && !null_map.is_null() { + let null_slice = unsafe { std::slice::from_raw_parts(null_map, num_rows) }; + let bins: Vec> = (0..num_rows) + .map(|i| { + if null_slice[i] != 0 { + None + } else { + let start = offsets_slice[i] as usize; + let end = offsets_slice[i + 1] as usize; + Some(&data_slice[start..end]) + } + }) + .collect(); + VarBinViewArray::from_iter_nullable_bin(bins).into_array() + } else { + let bins: Vec<&[u8]> = (0..num_rows) + .map(|i| { + let start = offsets_slice[i] as usize; + let end = offsets_slice[i + 1] as usize; + &data_slice[start..end] + }) + .collect(); + VarBinViewArray::from_iter_bin(bins).into_array() + } + } else if is_nullable && !null_map.is_null() { + // Build nullable string array + let null_slice = unsafe { std::slice::from_raw_parts(null_map, num_rows) }; + let strings: Vec> = (0..num_rows) + .map(|i| { + // ClickHouse null map: 0 = valid, 1 = null + if null_slice[i] != 0 { + None + } else { + let start = offsets_slice[i] as usize; + let end = offsets_slice[i + 1] as usize; + Some(std::str::from_utf8(&data_slice[start..end]).unwrap_or("")) + } + }) + .collect(); + VarBinViewArray::from_iter_nullable_str(strings).into_array() + } else { + // Build non-nullable string array + let strings: Vec<&str> = (0..num_rows) + .map(|i| { + let start = offsets_slice[i] as usize; + let end = offsets_slice[i + 1] as usize; + std::str::from_utf8(&data_slice[start..end]).unwrap_or("") + }) + .collect(); + VarBinViewArray::from_iter_str(strings).into_array() + }; + + // Wrap in ExtensionArray if the target dtype is an Extension type + let array = if let DType::Extension(ext) = dtype { + use vortex::array::arrays::ExtensionArray; + ExtensionArray::new(ext.clone(), array).into_array() + } else { + array + }; + + self.pending_columns[column_index] = Some(array); + Ok(()) + } + + /// End the current batch and commit it. + /// + /// This sends the batch to the background writer through the channel. + /// Memory is released as soon as the batch is sent (bounded by channel buffer). + pub fn end_batch(&mut self) -> VortexResult<()> { + if self.pending_columns.is_empty() { + return Ok(()); + } + + // Check all columns are filled + for (i, col) in self.pending_columns.iter().enumerate() { + if col.is_none() { + vortex_bail!("Column {} not written before end_batch()", i); + } + } + + let field_arrays: Vec = + self.pending_columns.drain(..).map(|c| c.unwrap()).collect(); + + let field_names: Vec> = self + .columns + .iter() + .map(|c| Arc::from(c.name.as_str())) + .collect(); + + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + field_arrays, + self.pending_num_rows, + Validity::NonNullable, + )?; + + // Send the batch to the background writer + self.send_array(struct_array.into_array())?; + self.total_rows += self.pending_num_rows; + self.pending_num_rows = 0; + + Ok(()) + } + + // ========================================================================= + // List (Array) column writing + // ========================================================================= + + /// Write offsets for a list column. The element data is written separately. + pub fn list_write_offsets( + &mut self, + column_index: usize, + offsets: *const u64, + null_map: *const u8, + num_rows: usize, + ) -> VortexResult<()> { + if column_index >= self.columns.len() { + vortex_bail!("Column index {} out of bounds", column_index); + } + if num_rows != self.pending_num_rows { + vortex_bail!( + "Row count mismatch: expected {}, got {}", + self.pending_num_rows, + num_rows + ); + } + + let offsets_slice = unsafe { std::slice::from_raw_parts(offsets, num_rows + 1) }; + let offsets_vec: Vec = offsets_slice.to_vec(); + + let dtype = &self.columns[column_index].dtype; + let is_nullable = dtype.is_nullable(); + let validity = if is_nullable { + null_map_to_validity(null_map, num_rows) + } else { + Validity::NonNullable + }; + + self.pending_lists.insert( + column_index, + PendingList { + offsets: offsets_vec, + validity, + num_rows, + elements: None, + }, + ); + + Ok(()) + } + + /// Write primitive element data for a list column. + pub fn list_write_element_column( + &mut self, + column_index: usize, + data: *const c_void, + num_elements: usize, + ) -> VortexResult<()> { + self.list_write_element_column_nullable(column_index, data, ptr::null(), num_elements) + } + + /// Write nullable primitive element data for a list column. + pub fn list_write_element_column_nullable( + &mut self, + column_index: usize, + data: *const c_void, + null_map: *const u8, + num_elements: usize, + ) -> VortexResult<()> { + let pending = self + .pending_lists + .get_mut(&column_index) + .ok_or_else(|| vortex_err!("No pending list for column {}", column_index))?; + + // Get element dtype from the List dtype + let elem_dtype = match &self.columns[column_index].dtype { + DType::List(elem, _) => elem.as_ref().clone(), + _ => vortex_bail!("Column {} is not a List type", column_index), + }; + + let array = build_array_from_raw_with_validity(&elem_dtype, data, null_map, num_elements)?; + pending.elements = Some(array); + Ok(()) + } + + /// Write string element data for a list column. + pub fn list_write_element_string_column( + &mut self, + column_index: usize, + data: *const u8, + offsets: *const u64, + num_elements: usize, + ) -> VortexResult<()> { + let pending = self + .pending_lists + .get_mut(&column_index) + .ok_or_else(|| vortex_err!("No pending list for column {}", column_index))?; + + // Get element dtype from the List dtype to match nullability + let elem_dtype = match &self.columns[column_index].dtype { + DType::List(elem, _) => elem.as_ref().clone(), + _ => DType::Utf8(Nullability::NonNullable), + }; + + // Build string array from offsets + let offsets_slice = unsafe { std::slice::from_raw_parts(offsets, num_elements + 1) }; + let total_data_len = offsets_slice[num_elements] as usize; + let data_slice = if total_data_len > 0 { + unsafe { std::slice::from_raw_parts(data, total_data_len) } + } else { + &[] + }; + + let mut builder = VarBinViewBuilder::with_capacity(elem_dtype, num_elements); + for i in 0..num_elements { + let start = offsets_slice[i] as usize; + let end = offsets_slice[i + 1] as usize; + let s = std::str::from_utf8(&data_slice[start..end]).unwrap_or(""); + builder.append_value(s); + } + let array = builder.finish_into_varbinview().into_array(); + pending.elements = Some(array); + Ok(()) + } + + /// Write nullable string element data for a list column. + pub fn list_write_element_string_column_nullable( + &mut self, + column_index: usize, + data: *const u8, + offsets: *const u64, + null_map: *const u8, + num_elements: usize, + ) -> VortexResult<()> { + let pending = self + .pending_lists + .get_mut(&column_index) + .ok_or_else(|| vortex_err!("No pending list for column {}", column_index))?; + + // Get element dtype from the List dtype to match nullability + let elem_dtype = match &self.columns[column_index].dtype { + DType::List(elem, _) => elem.as_ref().clone(), + _ => DType::Utf8(Nullability::Nullable), + }; + + // Build string array from offsets with null map + let offsets_slice = unsafe { std::slice::from_raw_parts(offsets, num_elements + 1) }; + let total_data_len = offsets_slice[num_elements] as usize; + let data_slice = if total_data_len > 0 { + unsafe { std::slice::from_raw_parts(data, total_data_len) } + } else { + &[] + }; + + let null_slice: Option<&[u8]> = if null_map.is_null() { + None + } else { + Some(unsafe { std::slice::from_raw_parts(null_map, num_elements) }) + }; + + let mut builder = VarBinViewBuilder::with_capacity(elem_dtype, num_elements); + for i in 0..num_elements { + let is_null = null_slice.map_or(false, |ns| ns[i] != 0); + if is_null { + builder.append_null(); + } else { + let start = offsets_slice[i] as usize; + let end = offsets_slice[i + 1] as usize; + let s = std::str::from_utf8(&data_slice[start..end]).unwrap_or(""); + builder.append_value(s); + } + } + let array = builder.finish_into_varbinview().into_array(); + pending.elements = Some(array); + Ok(()) + } + + /// Finalize a list column: build the `ListViewArray` from offsets + elements. + pub fn list_end(&mut self, column_index: usize) -> VortexResult<()> { + let pending = self + .pending_lists + .remove(&column_index) + .ok_or_else(|| vortex_err!("No pending list for column {}", column_index))?; + + let elements = pending + .elements + .ok_or_else(|| vortex_err!("List elements not written for column {}", column_index))?; + + // Build offsets and sizes arrays for ListViewArray + // offsets[i] = start index of list i + // sizes[i] = length of list i + let num_rows = pending.num_rows; + let mut lv_offsets: Vec = Vec::with_capacity(num_rows); + let mut lv_sizes: Vec = Vec::with_capacity(num_rows); + + for i in 0..num_rows { + let start = pending.offsets[i] as i64; + let end = pending.offsets[i + 1] as i64; + lv_offsets.push(start); + lv_sizes.push(end - start); + } + + let offsets_array = + PrimitiveArray::new(Buffer::::from(lv_offsets), Validity::NonNullable) + .into_array(); + let sizes_array = + PrimitiveArray::new(Buffer::::from(lv_sizes), Validity::NonNullable).into_array(); + + let list_array = + ListViewArray::try_new(elements, offsets_array, sizes_array, pending.validity)?; + self.pending_columns[column_index] = Some(list_array.into_array()); + Ok(()) + } + + // ========================================================================= + // Struct (Tuple) column writing + // ========================================================================= + + /// Extract StructFields from a column dtype. + /// Handles both direct `Struct(fields, _)` and `List(Struct(fields, _), _)` (Map case). + fn get_struct_fields_from_dtype(dtype: &DType) -> VortexResult<&StructFields> { + match dtype { + DType::Struct(fields, _) => Ok(fields), + DType::List(elem, _) => match elem.as_ref() { + DType::Struct(fields, _) => Ok(fields), + _ => vortex_bail!("List element is not a Struct type"), + }, + _ => vortex_bail!("Column is not a Struct or Map type, got {:?}", dtype), + } + } + + /// Begin writing a struct column. + pub fn struct_begin( + &mut self, + column_index: usize, + null_map: *const u8, + num_rows: usize, + ) -> VortexResult<()> { + if column_index >= self.columns.len() { + vortex_bail!("Column index {} out of bounds", column_index); + } + + let dtype = &self.columns[column_index].dtype; + let num_fields = Self::get_struct_fields_from_dtype(dtype)?.nfields(); + + let is_nullable = dtype.is_nullable(); + let validity = if is_nullable { + null_map_to_validity(null_map, num_rows) + } else { + Validity::NonNullable + }; + + self.pending_structs.insert( + column_index, + PendingStruct { + validity, + num_rows, + fields: vec![None; num_fields], + num_fields, + }, + ); + + Ok(()) + } + + /// Write a primitive field of a struct column. + pub fn struct_write_field( + &mut self, + column_index: usize, + field_index: usize, + data: *const c_void, + null_map: *const u8, + num_rows: usize, + ) -> VortexResult<()> { + let pending = self + .pending_structs + .get_mut(&column_index) + .ok_or_else(|| vortex_err!("No pending struct for column {}", column_index))?; + + if field_index >= pending.num_fields { + vortex_bail!( + "Field index {} out of bounds (struct has {} fields)", + field_index, + pending.num_fields + ); + } + + // Get field dtype + let field_dtype = Self::get_struct_fields_from_dtype(&self.columns[column_index].dtype)? + .field_by_index(field_index) + .unwrap() + .clone(); + + let array = build_array_from_raw_with_validity(&field_dtype, data, null_map, num_rows); + pending.fields[field_index] = Some(array?); + Ok(()) + } + + /// Write a string field of a struct column. + pub fn struct_write_field_string( + &mut self, + column_index: usize, + field_index: usize, + data: *const u8, + offsets: *const u64, + null_map: *const u8, + num_rows: usize, + ) -> VortexResult<()> { + let pending = self + .pending_structs + .get_mut(&column_index) + .ok_or_else(|| vortex_err!("No pending struct for column {}", column_index))?; + + if field_index >= pending.num_fields { + vortex_bail!( + "Field index {} out of bounds (struct has {} fields)", + field_index, + pending.num_fields + ); + } + + // Get field dtype + let field_dtype = Self::get_struct_fields_from_dtype(&self.columns[column_index].dtype)? + .field_by_index(field_index) + .unwrap() + .clone(); + + let is_nullable = field_dtype.is_nullable(); + + // Build string array + let offsets_slice = unsafe { std::slice::from_raw_parts(offsets, num_rows + 1) }; + let total_data_len = offsets_slice[num_rows] as usize; + let data_slice = if total_data_len > 0 { + unsafe { std::slice::from_raw_parts(data, total_data_len) } + } else { + &[] + }; + + let array = if is_nullable && !null_map.is_null() { + let null_slice = unsafe { std::slice::from_raw_parts(null_map, num_rows) }; + let strings: Vec> = (0..num_rows) + .map(|i| { + if null_slice[i] != 0 { + None + } else { + let start = offsets_slice[i] as usize; + let end = offsets_slice[i + 1] as usize; + Some(std::str::from_utf8(&data_slice[start..end]).unwrap_or("")) + } + }) + .collect(); + VarBinViewArray::from_iter_nullable_str(strings).into_array() + } else { + let strings: Vec<&str> = (0..num_rows) + .map(|i| { + let start = offsets_slice[i] as usize; + let end = offsets_slice[i + 1] as usize; + std::str::from_utf8(&data_slice[start..end]).unwrap_or("") + }) + .collect(); + VarBinViewArray::from_iter_str(strings).into_array() + }; + + pending.fields[field_index] = Some(array); + Ok(()) + } + + /// Finalize a struct column: build the `StructArray` from fields. + pub fn struct_end(&mut self, column_index: usize) -> VortexResult<()> { + let pending = self + .pending_structs + .remove(&column_index) + .ok_or_else(|| vortex_err!("No pending struct for column {}", column_index))?; + + // Get field names from the dtype + let field_names = { + let fields = Self::get_struct_fields_from_dtype(&self.columns[column_index].dtype)?; + let names: Vec> = fields + .names() + .iter() + .map(|n| Arc::from(n.as_ref())) + .collect(); + FieldNames::from(names) + }; + + // Check all fields are written + let mut field_arrays = Vec::with_capacity(pending.num_fields); + for (i, field) in pending.fields.into_iter().enumerate() { + match field { + Some(array) => field_arrays.push(array), + None => vortex_bail!("Struct field {} not written for column {}", i, column_index), + } + } + + let struct_array = StructArray::try_new( + field_names, + field_arrays, + pending.num_rows, + pending.validity, + )?; + + // For Map columns (DType::List(Struct(...), _)), the struct is the list element, + // so store it in the pending list's elements rather than directly as the column. + if matches!(&self.columns[column_index].dtype, DType::List(_, _)) { + if let Some(pending_list) = self.pending_lists.get_mut(&column_index) { + pending_list.elements = Some(struct_array.into_array()); + } else { + vortex_bail!( + "No pending list for Map column {}; list_write_offsets must be called before struct_begin", + column_index + ); + } + } else { + self.pending_columns[column_index] = Some(struct_array.into_array()); + } + Ok(()) + } + + /// Write a batch of data (simplified API for primitive-only columns). + /// + /// This method takes raw column data and constructs a Vortex struct array. + pub fn write_batch( + &mut self, + column_data: &[*const c_void], + num_rows: usize, + ) -> VortexResult<()> { + if matches!(self.state, WriterState::Finalized) { + vortex_bail!("Cannot write after finalization"); + } + + if column_data.len() != self.columns.len() { + vortex_bail!( + "Column count mismatch: expected {}, got {}", + self.columns.len(), + column_data.len() + ); + } + + if num_rows == 0 { + return Ok(()); + } + + // Build arrays for each column + let mut field_arrays: Vec = Vec::with_capacity(self.columns.len()); + + for (i, col_def) in self.columns.iter().enumerate() { + let data_ptr = column_data[i]; + if data_ptr.is_null() { + vortex_bail!("Column {} data pointer is null", i); + } + + let array = build_array_from_raw(&col_def.dtype, data_ptr, num_rows)?; + field_arrays.push(array); + } + + // Create struct array + let field_names: Vec> = self + .columns + .iter() + .map(|c| Arc::from(c.name.as_str())) + .collect(); + + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + field_arrays, + num_rows, + Validity::NonNullable, + )?; + + // Send the batch to the background writer + self.send_array(struct_array.into_array())?; + self.total_rows += num_rows; + + Ok(()) + } + + /// Finalize the writer and flush all data to disk. + /// + /// This closes the channel and waits for the background writer to complete. + pub fn finalize(&mut self) -> VortexResult<()> { + // Commit any pending batch + if !self.pending_columns.is_empty() { + self.end_batch()?; + } + + // Take ownership of the state + let state = std::mem::replace(&mut self.state, WriterState::Finalized); + + match state { + WriterState::Building => { + // No data was written, nothing to finalize + if self.total_rows == 0 { + vortex_bail!("No data to write"); + } + Ok(()) + } + WriterState::Active { + sender, + writer_task, + } => { + // Close the sender to signal end of stream + drop(sender); + + // Wait for the writer task to complete + RUNTIME.block_on(async { + writer_task + .await + .map_err(|e| vortex_err!("Write failed: {}", e))?; + Ok(()) + }) + } + WriterState::Finalized => { + // Already finalized, nothing to do + Ok(()) + } + } + } + + /// Get the output path. + pub fn output_path(&self) -> &str { + &self.output_path + } + + /// Get the total number of rows written. + pub fn total_rows(&self) -> usize { + self.total_rows + } +} + +/// Convert ClickHouse null map (0=valid, 1=null) to Vortex Validity. +/// +/// ClickHouse uses UInt8 per row where 0 means not null, 1 means null. +/// Vortex uses a bitmask where 1 means valid, 0 means null. +fn null_map_to_validity(null_map: *const u8, num_rows: usize) -> Validity { + if null_map.is_null() || num_rows == 0 { + return Validity::AllValid; + } + + let null_slice = unsafe { std::slice::from_raw_parts(null_map, num_rows) }; + + // Check if all valid (all zeros) or all invalid (all ones) + let null_count = null_slice.iter().filter(|&&v| v != 0).count(); + + if null_count == 0 { + return Validity::AllValid; + } + if null_count == num_rows { + return Validity::AllInvalid; + } + + // Build validity bitmap (invert ClickHouse's null map) + // Vortex validity: 1 = valid, 0 = null + // ClickHouse null map: 0 = valid, 1 = null + let bitmap_bytes = (num_rows + 7) / 8; + let mut validity_bitmap = vec![0u8; bitmap_bytes]; + + for (i, &is_null) in null_slice.iter().enumerate() { + if is_null == 0 { + // Valid - set bit to 1 + let byte_idx = i / 8; + let bit_idx = i % 8; + validity_bitmap[byte_idx] |= 1 << bit_idx; + } + } + + // Create a BoolArray for validity + use vortex::array::arrays::BoolArray; + use vortex::buffer::{BitBuffer, ByteBuffer}; + + let byte_buffer = ByteBuffer::from(validity_bitmap); + let bit_buffer = BitBuffer::new(byte_buffer, num_rows); + let bool_array = BoolArray::from(bit_buffer).into_array(); + + Validity::Array(bool_array) +} + +/// Build a Vortex array from raw data pointer (non-nullable). +fn build_array_from_raw( + dtype: &DType, + data: *const c_void, + num_rows: usize, +) -> VortexResult { + build_array_from_raw_with_validity(dtype, data, ptr::null(), num_rows) +} + +/// Build a Vortex array from raw data pointer with optional validity. +fn build_array_from_raw_with_validity( + dtype: &DType, + data: *const c_void, + null_map: *const u8, + num_rows: usize, +) -> VortexResult { + match dtype { + DType::Primitive(ptype, nullability) => { + let validity = if *nullability == Nullability::Nullable { + null_map_to_validity(null_map, num_rows) + } else { + Validity::NonNullable + }; + + macro_rules! build_primitive { + ($rust_ty:ty) => {{ + let slice = + unsafe { std::slice::from_raw_parts(data as *const $rust_ty, num_rows) }; + let buffer: Buffer<$rust_ty> = slice.to_vec().into(); + PrimitiveArray::new(buffer, validity).into_array() + }}; + } + + let array = match ptype { + PType::I8 => build_primitive!(i8), + PType::I16 => build_primitive!(i16), + PType::I32 => build_primitive!(i32), + PType::I64 => build_primitive!(i64), + PType::U8 => build_primitive!(u8), + PType::U16 => build_primitive!(u16), + PType::U32 => build_primitive!(u32), + PType::U64 => build_primitive!(u64), + PType::F32 => build_primitive!(f32), + PType::F64 => build_primitive!(f64), + PType::F16 => vortex_bail!("F16 not supported"), + }; + + Ok(array) + } + DType::Utf8(_) | DType::Binary(_) => { + // For strings, we need a different approach + // The data should be an array of string pointers and lengths + // For now, return an error + vortex_bail!("String columns must be written using vortex_writer_write_string_column") + } + DType::Bool(nullability) => { + use vortex::array::arrays::BoolArray; + use vortex::buffer::BitBuffer; + + // Bool is stored as u8 in ClickHouse (0 = false, 1 = true) + let slice = unsafe { std::slice::from_raw_parts(data as *const u8, num_rows) }; + + let validity = if *nullability == Nullability::Nullable { + null_map_to_validity(null_map, num_rows) + } else { + Validity::NonNullable + }; + + let bits = BitBuffer::from_iter(slice.iter().map(|&v| v != 0)); + Ok(BoolArray::new(bits, validity).into_array()) + } + DType::Decimal(decimal_dtype, nullability) => { + let validity = if *nullability == Nullability::Nullable { + null_map_to_validity(null_map, num_rows) + } else { + Validity::NonNullable + }; + + // ClickHouse always sends data in its fixed-width storage format: + // precision 1-9 -> Int32 (4 bytes, Decimal32) + // precision 10-18 -> Int64 (8 bytes, Decimal64) + // precision 19-38 -> Int128 (16 bytes, Decimal128) + // precision 39-76 -> Int256 (32 bytes, Decimal256) + // We must use the ClickHouse storage type (not the smallest possible type) + // to correctly interpret the raw bytes from ClickHouse. + let values_type = + crate::exporter::decimal::clickhouse_decimal_type(decimal_dtype.precision()); + + let array = build_decimal_array(data, num_rows, *decimal_dtype, values_type, validity)?; + Ok(array) + } + DType::FixedSizeList(elem_dtype, size, nullability) => { + // Verify element type is u8 (for big integers) + if !matches!(elem_dtype.as_ref(), DType::Primitive(PType::U8, _)) { + vortex_bail!( + "Only FixedSizeList is supported for raw write, got FixedSizeList<{:?}, {}>", + elem_dtype, + size + ); + } + + // Only support 16 (Int128/UInt128) and 32 (Int256/UInt256) byte sizes + if *size != 16 && *size != 32 { + vortex_bail!( + "Only FixedSizeList with size 16 or 32 is supported, got {}", + size + ); + } + + let validity = if *nullability == Nullability::Nullable { + null_map_to_validity(null_map, num_rows) + } else { + Validity::NonNullable + }; + + // Build the flat byte array (num_rows * size bytes) + let size_usize = *size as usize; + let total_bytes = num_rows * size_usize; + let slice = unsafe { std::slice::from_raw_parts(data as *const u8, total_bytes) }; + let buffer: Buffer = slice.to_vec().into(); + let values = PrimitiveArray::new(buffer, Validity::NonNullable); + + // Create the FixedSizeListArray (needs 4 args: elements, list_size, validity, len) + let array = + FixedSizeListArray::try_new(values.into_array(), *size, validity, num_rows)?; + Ok(array.into_array()) + } + DType::Extension(ext) => { + // Build the storage array from raw data, then wrap in ExtensionArray + use vortex::array::arrays::ExtensionArray; + let storage_array = + build_array_from_raw_with_validity(ext.storage_dtype(), data, null_map, num_rows)?; + Ok(ExtensionArray::new(ext.clone(), storage_array).into_array()) + } + _ => vortex_bail!("Unsupported dtype for raw write: {:?}", dtype), + } +} + +/// Build a DecimalArray from raw data pointer. +fn build_decimal_array( + data: *const c_void, + num_rows: usize, + decimal_dtype: DecimalDType, + values_type: DecimalType, + validity: Validity, +) -> VortexResult { + match values_type { + DecimalType::I8 => { + let slice = unsafe { std::slice::from_raw_parts(data as *const i8, num_rows) }; + let buffer: Buffer = slice.to_vec().into(); + Ok(DecimalArray::new(buffer, decimal_dtype, validity).into_array()) + } + DecimalType::I16 => { + let slice = unsafe { std::slice::from_raw_parts(data as *const i16, num_rows) }; + let buffer: Buffer = slice.to_vec().into(); + Ok(DecimalArray::new(buffer, decimal_dtype, validity).into_array()) + } + DecimalType::I32 => { + let slice = unsafe { std::slice::from_raw_parts(data as *const i32, num_rows) }; + let buffer: Buffer = slice.to_vec().into(); + Ok(DecimalArray::new(buffer, decimal_dtype, validity).into_array()) + } + DecimalType::I64 => { + let slice = unsafe { std::slice::from_raw_parts(data as *const i64, num_rows) }; + let buffer: Buffer = slice.to_vec().into(); + Ok(DecimalArray::new(buffer, decimal_dtype, validity).into_array()) + } + DecimalType::I128 => { + let slice = unsafe { std::slice::from_raw_parts(data as *const i128, num_rows) }; + let buffer: Buffer = slice.to_vec().into(); + Ok(DecimalArray::new(buffer, decimal_dtype, validity).into_array()) + } + DecimalType::I256 => { + use vortex::dtype::i256; + let slice = unsafe { std::slice::from_raw_parts(data as *const i256, num_rows) }; + let buffer: Buffer = slice.to_vec().into(); + Ok(DecimalArray::new(buffer, decimal_dtype, validity).into_array()) + } + } +} + +// ============================================================================= +// FFI Exports for C++ +// ============================================================================= + +/// Create a new Vortex writer. +/// +/// # Safety +/// The `path` parameter must be a valid null-terminated C string. +/// Returns NULL on error. Call `vortex_get_last_error()` for error details. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_new(path: *const c_char) -> *mut VortexWriter { + clear_last_error(); + + if path.is_null() { + set_last_error("vortex_writer_new: path is null"); + return ptr::null_mut(); + } + + let path_str = match unsafe { CStr::from_ptr(path) }.to_str() { + Ok(s) => s, + Err(e) => { + set_last_error(&format!("vortex_writer_new: invalid UTF-8 in path: {}", e)); + return ptr::null_mut(); + } + }; + + match VortexWriter::new(path_str) { + Ok(writer) => Box::into_raw(Box::new(writer)), + Err(e) => { + set_last_error(&format!("vortex_writer_new: {}", e)); + ptr::null_mut() + } + } +} + +/// Free a Vortex writer. +/// +/// # Safety +/// The `writer` parameter must be a valid pointer returned by `vortex_writer_new`, +/// or NULL (which is safely ignored). +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_free(writer: *mut VortexWriter) { + if !writer.is_null() { + drop(unsafe { Box::from_raw(writer) }); + } +} + +/// Add a column to the writer's schema. +/// +/// # Safety +/// - The `writer` parameter must be a valid pointer. +/// - The `name` and `clickhouse_type` must be valid null-terminated C strings. +/// Returns 0 on success, negative error code on failure. +/// Call `vortex_get_last_error()` for error details. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_add_column( + writer: *mut VortexWriter, + name: *const c_char, + clickhouse_type: *const c_char, + nullable: i32, +) -> i32 { + clear_last_error(); + + if writer.is_null() { + set_last_error("vortex_writer_add_column: writer is null"); + return -1; + } + if name.is_null() { + set_last_error("vortex_writer_add_column: name is null"); + return -1; + } + if clickhouse_type.is_null() { + set_last_error("vortex_writer_add_column: clickhouse_type is null"); + return -1; + } + + let writer = unsafe { &mut *writer }; + + let name_str = match unsafe { CStr::from_ptr(name) }.to_str() { + Ok(s) => s, + Err(e) => { + set_last_error(&format!( + "vortex_writer_add_column: invalid UTF-8 in name: {}", + e + )); + return -2; + } + }; + + let type_str = match unsafe { CStr::from_ptr(clickhouse_type) }.to_str() { + Ok(s) => s, + Err(e) => { + set_last_error(&format!( + "vortex_writer_add_column: invalid UTF-8 in clickhouse_type: {}", + e + )); + return -3; + } + }; + + match writer.add_column(name_str, type_str, nullable != 0) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_add_column: {}", e)); + -4 + } + } +} + +/// Begin writing a new batch with the given number of rows. +/// +/// After calling this, use `vortex_writer_write_column_*` functions to write each column, +/// then call `vortex_writer_end_batch()` to commit the batch. +/// +/// # Safety +/// The `writer` parameter must be a valid pointer. +/// Returns 0 on success, negative error code on failure. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_begin_batch( + writer: *mut VortexWriter, + num_rows: usize, +) -> i32 { + clear_last_error(); + + if writer.is_null() { + set_last_error("vortex_writer_begin_batch: writer is null"); + return -1; + } + + let writer = unsafe { &mut *writer }; + + match writer.begin_batch(num_rows) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_begin_batch: {}", e)); + -2 + } + } +} + +/// Write a primitive column by index. +/// +/// Must be called between `vortex_writer_begin_batch()` and `vortex_writer_end_batch()`. +/// +/// # Safety +/// - The `writer` parameter must be a valid pointer. +/// - The `data` must point to an array of `num_rows` elements of the appropriate type. +/// Returns 0 on success, negative error code on failure. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_write_column( + writer: *mut VortexWriter, + column_index: usize, + data: *const c_void, + num_rows: usize, +) -> i32 { + clear_last_error(); + + if writer.is_null() { + set_last_error("vortex_writer_write_column: writer is null"); + return -1; + } + if data.is_null() { + set_last_error("vortex_writer_write_column: data is null"); + return -1; + } + + let writer = unsafe { &mut *writer }; + + match writer.write_column_primitive(column_index, data, num_rows) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_write_column: {}", e)); + -2 + } + } +} + +/// Write a nullable primitive column by index with validity bitmap. +/// +/// The null_map uses ClickHouse's convention: one byte per row, 0 = valid, 1 = null. +/// Must be called between `vortex_writer_begin_batch()` and `vortex_writer_end_batch()`. +/// +/// # Safety +/// - The `writer` parameter must be a valid pointer. +/// - The `data` must point to an array of `num_rows` elements of the appropriate type. +/// - The `null_map` must point to an array of `num_rows` bytes, or NULL for all-valid. +/// Returns 0 on success, negative error code on failure. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_write_column_nullable( + writer: *mut VortexWriter, + column_index: usize, + data: *const c_void, + null_map: *const u8, + num_rows: usize, +) -> i32 { + clear_last_error(); + + if writer.is_null() { + set_last_error("vortex_writer_write_column_nullable: writer is null"); + return -1; + } + if data.is_null() { + set_last_error("vortex_writer_write_column_nullable: data is null"); + return -1; + } + + let writer = unsafe { &mut *writer }; + + match writer.write_column_primitive_with_validity(column_index, data, null_map, num_rows) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_write_column_nullable: {}", e)); + -2 + } + } +} + +/// Write a string column by index. +/// +/// The strings are provided as concatenated data with offsets. +/// The `offsets` array must have `num_rows + 1` elements, where: +/// - `offsets[i]` is the start offset of string i +/// - `offsets[num_rows]` is the total data length +/// +/// Must be called between `vortex_writer_begin_batch()` and `vortex_writer_end_batch()`. +/// +/// # Safety +/// - The `writer` parameter must be a valid pointer. +/// - The `data` must point to the concatenated string data. +/// - The `offsets` must point to an array of `num_rows + 1` uint64_t values. +/// Returns 0 on success, negative error code on failure. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_write_string_column( + writer: *mut VortexWriter, + column_index: usize, + data: *const u8, + offsets: *const u64, + num_rows: usize, +) -> i32 { + clear_last_error(); + + if writer.is_null() { + set_last_error("vortex_writer_write_string_column: writer is null"); + return -1; + } + if data.is_null() && num_rows > 0 { + set_last_error("vortex_writer_write_string_column: data is null"); + return -1; + } + if offsets.is_null() { + set_last_error("vortex_writer_write_string_column: offsets is null"); + return -1; + } + + let writer = unsafe { &mut *writer }; + + match writer.write_column_strings(column_index, data, offsets, num_rows) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_write_string_column: {}", e)); + -2 + } + } +} + +/// Write a nullable string column by index with validity bitmap. +/// +/// The strings are provided as concatenated data with offsets. +/// The `null_map` uses ClickHouse's convention: one byte per row, 0 = valid, 1 = null. +/// +/// Must be called between `vortex_writer_begin_batch()` and `vortex_writer_end_batch()`. +/// +/// # Safety +/// - The `writer` parameter must be a valid pointer. +/// - The `data` must point to the concatenated string data. +/// - The `offsets` must point to an array of `num_rows + 1` uint64_t values. +/// - The `null_map` must point to an array of `num_rows` bytes, or NULL for all-valid. +/// Returns 0 on success, negative error code on failure. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_write_string_column_nullable( + writer: *mut VortexWriter, + column_index: usize, + data: *const u8, + offsets: *const u64, + null_map: *const u8, + num_rows: usize, +) -> i32 { + clear_last_error(); + + if writer.is_null() { + set_last_error("vortex_writer_write_string_column_nullable: writer is null"); + return -1; + } + if offsets.is_null() { + set_last_error("vortex_writer_write_string_column_nullable: offsets is null"); + return -1; + } + + let writer = unsafe { &mut *writer }; + + match writer.write_column_strings_with_validity(column_index, data, offsets, null_map, num_rows) + { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!( + "vortex_writer_write_string_column_nullable: {}", + e + )); + -2 + } + } +} + +/// End the current batch and commit it. +/// +/// # Safety +/// The `writer` parameter must be a valid pointer. +/// Returns 0 on success, negative error code on failure. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_end_batch(writer: *mut VortexWriter) -> i32 { + clear_last_error(); + + if writer.is_null() { + set_last_error("vortex_writer_end_batch: writer is null"); + return -1; + } + + let writer = unsafe { &mut *writer }; + + match writer.end_batch() { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_end_batch: {}", e)); + -2 + } + } +} + +// ============================================================================= +// List (Array) Column FFI +// ============================================================================= + +/// Write offsets for a list (Array) column. +/// +/// # Safety +/// All pointers must be valid. `offsets` must have `num_rows + 1` elements. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_list_write_offsets( + writer: *mut VortexWriter, + column_index: usize, + offsets: *const u64, + null_map: *const u8, + num_rows: usize, +) -> i32 { + clear_last_error(); + if writer.is_null() || offsets.is_null() { + set_last_error("vortex_writer_list_write_offsets: null pointer"); + return -1; + } + let writer = unsafe { &mut *writer }; + match writer.list_write_offsets(column_index, offsets, null_map, num_rows) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_list_write_offsets: {}", e)); + -2 + } + } +} + +/// Write primitive element data for a list column. +/// +/// # Safety +/// All pointers must be valid. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_list_write_element_column( + writer: *mut VortexWriter, + column_index: usize, + data: *const c_void, + num_elements: usize, +) -> i32 { + clear_last_error(); + if writer.is_null() { + set_last_error("vortex_writer_list_write_element_column: null pointer"); + return -1; + } + let writer = unsafe { &mut *writer }; + match writer.list_write_element_column(column_index, data, num_elements) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_list_write_element_column: {}", e)); + -2 + } + } +} + +/// Write string element data for a list column. +/// +/// # Safety +/// All pointers must be valid. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_list_write_element_string_column( + writer: *mut VortexWriter, + column_index: usize, + data: *const u8, + offsets: *const u64, + num_elements: usize, +) -> i32 { + clear_last_error(); + if writer.is_null() || offsets.is_null() { + set_last_error("vortex_writer_list_write_element_string_column: null pointer"); + return -1; + } + let writer = unsafe { &mut *writer }; + match writer.list_write_element_string_column(column_index, data, offsets, num_elements) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!( + "vortex_writer_list_write_element_string_column: {}", + e + )); + -2 + } + } +} + +/// Write nullable primitive element data for a list column. +/// +/// # Safety +/// All pointers must be valid. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_list_write_element_column_nullable( + writer: *mut VortexWriter, + column_index: usize, + data: *const c_void, + null_map: *const u8, + num_elements: usize, +) -> i32 { + clear_last_error(); + if writer.is_null() { + set_last_error("vortex_writer_list_write_element_column_nullable: null pointer"); + return -1; + } + let writer = unsafe { &mut *writer }; + match writer.list_write_element_column_nullable(column_index, data, null_map, num_elements) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!( + "vortex_writer_list_write_element_column_nullable: {}", + e + )); + -2 + } + } +} + +/// Write nullable string element data for a list column. +/// +/// # Safety +/// All pointers must be valid. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_list_write_element_string_column_nullable( + writer: *mut VortexWriter, + column_index: usize, + data: *const u8, + offsets: *const u64, + null_map: *const u8, + num_elements: usize, +) -> i32 { + clear_last_error(); + if writer.is_null() || offsets.is_null() { + set_last_error("vortex_writer_list_write_element_string_column_nullable: null pointer"); + return -1; + } + let writer = unsafe { &mut *writer }; + match writer.list_write_element_string_column_nullable( + column_index, + data, + offsets, + null_map, + num_elements, + ) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!( + "vortex_writer_list_write_element_string_column_nullable: {}", + e + )); + -2 + } + } +} + +/// Finalize a list column. +/// +/// # Safety +/// The `writer` parameter must be a valid pointer. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_list_end( + writer: *mut VortexWriter, + column_index: usize, +) -> i32 { + clear_last_error(); + if writer.is_null() { + set_last_error("vortex_writer_list_end: null pointer"); + return -1; + } + let writer = unsafe { &mut *writer }; + match writer.list_end(column_index) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_list_end: {}", e)); + -2 + } + } +} + +// ============================================================================= +// Struct (Tuple) Column FFI +// ============================================================================= + +/// Begin writing a struct (Tuple) column. +/// +/// # Safety +/// The `writer` parameter must be a valid pointer. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_struct_begin( + writer: *mut VortexWriter, + column_index: usize, + null_map: *const u8, + num_rows: usize, +) -> i32 { + clear_last_error(); + if writer.is_null() { + set_last_error("vortex_writer_struct_begin: null pointer"); + return -1; + } + let writer = unsafe { &mut *writer }; + match writer.struct_begin(column_index, null_map, num_rows) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_struct_begin: {}", e)); + -2 + } + } +} + +/// Write a primitive field of a struct column. +/// +/// # Safety +/// All pointers must be valid. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_struct_write_field( + writer: *mut VortexWriter, + column_index: usize, + field_index: usize, + data: *const c_void, + null_map: *const u8, + num_rows: usize, +) -> i32 { + clear_last_error(); + if writer.is_null() { + set_last_error("vortex_writer_struct_write_field: null pointer"); + return -1; + } + let writer = unsafe { &mut *writer }; + match writer.struct_write_field(column_index, field_index, data, null_map, num_rows) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_struct_write_field: {}", e)); + -2 + } + } +} + +/// Write a string field of a struct column. +/// +/// # Safety +/// All pointers must be valid. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_struct_write_field_string( + writer: *mut VortexWriter, + column_index: usize, + field_index: usize, + data: *const u8, + offsets: *const u64, + null_map: *const u8, + num_rows: usize, +) -> i32 { + clear_last_error(); + if writer.is_null() || offsets.is_null() { + set_last_error("vortex_writer_struct_write_field_string: null pointer"); + return -1; + } + let writer = unsafe { &mut *writer }; + match writer.struct_write_field_string( + column_index, + field_index, + data, + offsets, + null_map, + num_rows, + ) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_struct_write_field_string: {}", e)); + -2 + } + } +} + +/// Finalize a struct column. +/// +/// # Safety +/// The `writer` parameter must be a valid pointer. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_struct_end( + writer: *mut VortexWriter, + column_index: usize, +) -> i32 { + clear_last_error(); + if writer.is_null() { + set_last_error("vortex_writer_struct_end: null pointer"); + return -1; + } + let writer = unsafe { &mut *writer }; + match writer.struct_end(column_index) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_struct_end: {}", e)); + -2 + } + } +} + +/// Write a batch of data (simplified API for primitive-only columns). +/// +/// # Safety +/// - The `writer` parameter must be a valid pointer. +/// - The `data` array must contain `num_columns` valid pointers to column data. +/// - Each column data pointer must point to an array of `num_rows` elements. +/// Returns 0 on success, negative error code on failure. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_write_batch( + writer: *mut VortexWriter, + data: *const *const c_void, + num_columns: usize, + num_rows: usize, +) -> i32 { + clear_last_error(); + + if writer.is_null() { + set_last_error("vortex_writer_write_batch: writer is null"); + return -1; + } + if data.is_null() && num_columns > 0 { + set_last_error("vortex_writer_write_batch: data is null"); + return -1; + } + + let writer = unsafe { &mut *writer }; + + let column_data = if num_columns > 0 { + unsafe { std::slice::from_raw_parts(data, num_columns) } + } else { + &[] + }; + + match writer.write_batch(column_data, num_rows) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_write_batch: {}", e)); + -2 + } + } +} + +/// Finalize the writer and flush all data. +/// +/// # Safety +/// The `writer` parameter must be a valid pointer. +/// Returns 0 on success, negative error code on failure. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_finalize(writer: *mut VortexWriter) -> i32 { + clear_last_error(); + + if writer.is_null() { + set_last_error("vortex_writer_finalize: writer is null"); + return -1; + } + + let writer = unsafe { &mut *writer }; + + match writer.finalize() { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_writer_finalize: {}", e)); + -2 + } + } +} + +/// Get the number of columns in the writer's schema. +/// +/// # Safety +/// The `writer` parameter must be a valid pointer. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_num_columns(writer: *const VortexWriter) -> usize { + if writer.is_null() { + return 0; + } + unsafe { &*writer }.num_columns() +} + +/// Get the total number of rows written. +/// +/// # Safety +/// The `writer` parameter must be a valid pointer. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_writer_total_rows(writer: *const VortexWriter) -> usize { + if writer.is_null() { + return 0; + } + unsafe { &*writer }.total_rows() +} + +// ============================================================================= +// Tests +// ============================================================================= + +#[cfg(test)] +mod tests { + use super::*; + use std::ffi::CString; + use tempfile::NamedTempFile; + + #[test] + fn test_writer_new() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let writer = VortexWriter::new(&path).expect("Failed to create writer"); + assert_eq!(writer.output_path(), path); + assert_eq!(writer.num_columns(), 0); + } + + #[test] + fn test_writer_new_empty_path() { + let result = VortexWriter::new(""); + assert!(result.is_err()); + } + + #[test] + fn test_writer_add_column() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + writer + .add_column("value", "Float64", false) + .expect("Failed to add column"); + + assert_eq!(writer.num_columns(), 2); + } + + #[test] + fn test_writer_ffi_new_free() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path).expect("CString failed"); + + let writer = unsafe { vortex_writer_new(c_path.as_ptr()) }; + assert!(!writer.is_null()); + + unsafe { vortex_writer_free(writer) }; + } + + #[test] + fn test_writer_ffi_null_path() { + let writer = unsafe { vortex_writer_new(ptr::null()) }; + assert!(writer.is_null()); + } + + #[test] + fn test_writer_ffi_add_column() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path).expect("CString failed"); + + let writer = unsafe { vortex_writer_new(c_path.as_ptr()) }; + assert!(!writer.is_null()); + + let name = CString::new("id").unwrap(); + let ch_type = CString::new("Int64").unwrap(); + + let result = + unsafe { vortex_writer_add_column(writer, name.as_ptr(), ch_type.as_ptr(), 0) }; + assert_eq!(result, 0); + + let num_cols = unsafe { vortex_writer_num_columns(writer) }; + assert_eq!(num_cols, 1); + + unsafe { vortex_writer_free(writer) }; + } + + #[test] + fn test_writer_write_and_finalize() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + + // Create test data + let data: Vec = vec![1, 2, 3, 4, 5]; + let data_ptr = data.as_ptr() as *const c_void; + + writer + .write_batch(&[data_ptr], 5) + .expect("Failed to write batch"); + + assert_eq!(writer.total_rows(), 5); + + writer.finalize().expect("Failed to finalize"); + + // Verify the file was created + assert!(std::path::Path::new(&path).exists()); + } + + #[test] + fn test_writer_string_column() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + writer + .add_column("name", "String", false) + .expect("Failed to add column"); + + // Begin batch + writer.begin_batch(3).expect("Failed to begin batch"); + + // Write id column + let ids: Vec = vec![1, 2, 3]; + writer + .write_column_primitive(0, ids.as_ptr() as *const c_void, 3) + .expect("Failed to write id column"); + + // Write string column + let strings = "AliceBobCharlie"; + let offsets: Vec = vec![0, 5, 8, 15]; // Alice(5), Bob(3), Charlie(7) + writer + .write_column_strings(1, strings.as_ptr(), offsets.as_ptr(), 3) + .expect("Failed to write string column"); + + // End batch + writer.end_batch().expect("Failed to end batch"); + + // Finalize + writer.finalize().expect("Failed to finalize"); + + // Verify the file was created + assert!(std::path::Path::new(&path).exists()); + } + + #[test] + fn test_writer_multiple_batches() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + writer + .add_column("value", "Float64", false) + .expect("Failed to add column"); + + // Write first batch (3 rows) + let ids1: Vec = vec![1, 2, 3]; + let values1: Vec = vec![1.1, 2.2, 3.3]; + writer.begin_batch(3).expect("Failed to begin batch 1"); + writer + .write_column_primitive(0, ids1.as_ptr() as *const c_void, 3) + .expect("Failed to write id column"); + writer + .write_column_primitive(1, values1.as_ptr() as *const c_void, 3) + .expect("Failed to write value column"); + writer.end_batch().expect("Failed to end batch 1"); + + // Write second batch (2 rows) + let ids2: Vec = vec![4, 5]; + let values2: Vec = vec![4.4, 5.5]; + writer.begin_batch(2).expect("Failed to begin batch 2"); + writer + .write_column_primitive(0, ids2.as_ptr() as *const c_void, 2) + .expect("Failed to write id column"); + writer + .write_column_primitive(1, values2.as_ptr() as *const c_void, 2) + .expect("Failed to write value column"); + writer.end_batch().expect("Failed to end batch 2"); + + // Write third batch (4 rows) + let ids3: Vec = vec![6, 7, 8, 9]; + let values3: Vec = vec![6.6, 7.7, 8.8, 9.9]; + writer.begin_batch(4).expect("Failed to begin batch 3"); + writer + .write_column_primitive(0, ids3.as_ptr() as *const c_void, 4) + .expect("Failed to write id column"); + writer + .write_column_primitive(1, values3.as_ptr() as *const c_void, 4) + .expect("Failed to write value column"); + writer.end_batch().expect("Failed to end batch 3"); + + // Verify total rows before finalize + assert_eq!(writer.total_rows(), 9); // 3 + 2 + 4 = 9 + + // Finalize + writer.finalize().expect("Failed to finalize"); + + // Verify the file was created + assert!(std::path::Path::new(&path).exists()); + + // Verify file is not empty + let metadata = std::fs::metadata(&path).expect("Failed to get file metadata"); + assert!(metadata.len() > 0, "File should not be empty"); + } + + // ============================================================================= + // Edge Case Tests - Boundary Conditions + // ============================================================================= + + #[test] + fn test_writer_empty_batch() { + // Test writing a batch with 0 rows + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + + // Write empty batch (should be a no-op) + let data: Vec = vec![]; + let data_ptr = data.as_ptr() as *const c_void; + writer + .write_batch(&[data_ptr], 0) + .expect("Empty batch should succeed"); + + assert_eq!(writer.total_rows(), 0); + + // Now write actual data + let data2: Vec = vec![1, 2, 3]; + writer + .write_batch(&[data2.as_ptr() as *const c_void], 3) + .expect("Failed to write batch"); + + writer.finalize().expect("Failed to finalize"); + assert!(std::path::Path::new(&path).exists()); + } + + #[test] + fn test_writer_single_row_batch() { + // Test writing a batch with exactly 1 row + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + + let data: Vec = vec![42]; + writer + .write_batch(&[data.as_ptr() as *const c_void], 1) + .expect("Failed to write single row batch"); + + assert_eq!(writer.total_rows(), 1); + writer.finalize().expect("Failed to finalize"); + } + + #[test] + fn test_writer_large_batch() { + // Test writing a large batch (stress test) + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + + // 1 million rows + let num_rows = 1_000_000; + let data: Vec = (0..num_rows as i64).collect(); + writer + .write_batch(&[data.as_ptr() as *const c_void], num_rows) + .expect("Failed to write large batch"); + + assert_eq!(writer.total_rows(), num_rows); + writer.finalize().expect("Failed to finalize"); + + let metadata = std::fs::metadata(&path).expect("Failed to get file metadata"); + assert!(metadata.len() > 0, "File should not be empty"); + } + + #[test] + fn test_writer_many_small_batches() { + // Test writing many small batches (tests channel backpressure) + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + + // Write 100 batches of 10 rows each (exceeds channel buffer of 32) + let num_batches = 100; + let rows_per_batch = 10; + + for i in 0..num_batches { + let start = (i * rows_per_batch) as i64; + let data: Vec = (start..start + rows_per_batch as i64).collect(); + writer + .write_batch(&[data.as_ptr() as *const c_void], rows_per_batch) + .expect(&format!("Failed to write batch {}", i)); + } + + assert_eq!(writer.total_rows(), num_batches * rows_per_batch); + writer.finalize().expect("Failed to finalize"); + } + + #[test] + fn test_writer_finalize_no_data() { + // Test finalizing without writing any data + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + + // Finalize without writing any data should fail + let result = writer.finalize(); + assert!(result.is_err(), "Finalize with no data should fail"); + } + + #[test] + fn test_writer_double_finalize() { + // Test calling finalize twice + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + + let data: Vec = vec![1, 2, 3]; + writer + .write_batch(&[data.as_ptr() as *const c_void], 3) + .expect("Failed to write batch"); + + writer.finalize().expect("First finalize should succeed"); + + // Second finalize should succeed (idempotent) + let result = writer.finalize(); + assert!(result.is_ok(), "Second finalize should be idempotent"); + } + + #[test] + fn test_writer_write_after_finalize() { + // Test writing after finalization + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + + let data: Vec = vec![1, 2, 3]; + writer + .write_batch(&[data.as_ptr() as *const c_void], 3) + .expect("Failed to write batch"); + + writer.finalize().expect("Failed to finalize"); + + // Writing after finalize should fail + let result = writer.write_batch(&[data.as_ptr() as *const c_void], 3); + assert!(result.is_err(), "Write after finalize should fail"); + } + + #[test] + fn test_writer_add_column_after_write() { + // Test adding column after data has been written + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + + let data: Vec = vec![1, 2, 3]; + writer + .write_batch(&[data.as_ptr() as *const c_void], 3) + .expect("Failed to write batch"); + + // Adding column after writing should fail + let result = writer.add_column("value", "Float64", false); + assert!(result.is_err(), "Add column after writing should fail"); + } + + #[test] + fn test_writer_column_count_mismatch() { + // Test writing with wrong number of columns + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + writer + .add_column("value", "Float64", false) + .expect("Failed to add column"); + + // Try to write with only one column (should fail) + let data: Vec = vec![1, 2, 3]; + let result = writer.write_batch(&[data.as_ptr() as *const c_void], 3); + assert!(result.is_err(), "Column count mismatch should fail"); + } + + #[test] + fn test_writer_incomplete_batch() { + // Test begin_batch without end_batch, then finalize + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + writer + .add_column("name", "String", false) + .expect("Failed to add column"); + + // Begin batch but only write one column + writer.begin_batch(3).expect("Failed to begin batch"); + + let ids: Vec = vec![1, 2, 3]; + writer + .write_column_primitive(0, ids.as_ptr() as *const c_void, 3) + .expect("Failed to write id column"); + + // Try to end batch without writing all columns - should fail + let result = writer.end_batch(); + assert!(result.is_err(), "Incomplete batch should fail on end_batch"); + } + + #[test] + fn test_writer_begin_batch_twice() { + // Test calling begin_batch twice without end_batch + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + + writer + .begin_batch(3) + .expect("First begin_batch should succeed"); + + // Second begin_batch without end_batch should fail + let result = writer.begin_batch(3); + assert!(result.is_err(), "Second begin_batch should fail"); + } + + #[test] + fn test_writer_column_index_out_of_bounds() { + // Test writing to invalid column index + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + + writer.begin_batch(3).expect("Failed to begin batch"); + + let data: Vec = vec![1, 2, 3]; + // Try to write to column index 5 (only have 1 column) + let result = writer.write_column_primitive(5, data.as_ptr() as *const c_void, 3); + assert!(result.is_err(), "Out of bounds column index should fail"); + } + + #[test] + fn test_writer_row_count_mismatch_in_batch() { + // Test writing column with different row count than begin_batch specified + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("id", "Int64", false) + .expect("Failed to add column"); + + writer.begin_batch(3).expect("Failed to begin batch"); + + let data: Vec = vec![1, 2, 3, 4, 5]; // 5 rows instead of 3 + let result = writer.write_column_primitive(0, data.as_ptr() as *const c_void, 5); + assert!(result.is_err(), "Row count mismatch should fail"); + } + + #[test] + fn test_writer_empty_string_column() { + // Test writing empty strings + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("name", "String", false) + .expect("Failed to add column"); + + writer.begin_batch(3).expect("Failed to begin batch"); + + // Three empty strings + let strings = ""; + let offsets: Vec = vec![0, 0, 0, 0]; + writer + .write_column_strings(0, strings.as_ptr(), offsets.as_ptr(), 3) + .expect("Failed to write empty strings"); + + writer.end_batch().expect("Failed to end batch"); + writer.finalize().expect("Failed to finalize"); + } + + #[test] + fn test_writer_mixed_empty_and_nonempty_strings() { + // Test writing mix of empty and non-empty strings + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + writer + .add_column("name", "String", false) + .expect("Failed to add column"); + + writer.begin_batch(4).expect("Failed to begin batch"); + + // "Hello", "", "World", "" + let strings = "HelloWorld"; + let offsets: Vec = vec![0, 5, 5, 10, 10]; // Hello(5), empty, World(5), empty + writer + .write_column_strings(0, strings.as_ptr(), offsets.as_ptr(), 4) + .expect("Failed to write mixed strings"); + + writer.end_batch().expect("Failed to end batch"); + writer.finalize().expect("Failed to finalize"); + } + + #[test] + fn test_writer_all_data_types() { + // Test all supported primitive types + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + + let mut writer = VortexWriter::new(&path).expect("Failed to create writer"); + + // Add columns of various types + writer.add_column("i8", "Int8", false).expect("Failed"); + writer.add_column("i16", "Int16", false).expect("Failed"); + writer.add_column("i32", "Int32", false).expect("Failed"); + writer.add_column("i64", "Int64", false).expect("Failed"); + writer.add_column("u8", "UInt8", false).expect("Failed"); + writer.add_column("u16", "UInt16", false).expect("Failed"); + writer.add_column("u32", "UInt32", false).expect("Failed"); + writer.add_column("u64", "UInt64", false).expect("Failed"); + writer.add_column("f32", "Float32", false).expect("Failed"); + writer.add_column("f64", "Float64", false).expect("Failed"); + + writer.begin_batch(2).expect("Failed to begin batch"); + + let i8_data: Vec = vec![1, 2]; + let i16_data: Vec = vec![100, 200]; + let i32_data: Vec = vec![1000, 2000]; + let i64_data: Vec = vec![10000, 20000]; + let u8_data: Vec = vec![1, 2]; + let u16_data: Vec = vec![100, 200]; + let u32_data: Vec = vec![1000, 2000]; + let u64_data: Vec = vec![10000, 20000]; + let f32_data: Vec = vec![1.5, 2.5]; + let f64_data: Vec = vec![1.5, 2.5]; + + writer + .write_column_primitive(0, i8_data.as_ptr() as *const c_void, 2) + .expect("Failed"); + writer + .write_column_primitive(1, i16_data.as_ptr() as *const c_void, 2) + .expect("Failed"); + writer + .write_column_primitive(2, i32_data.as_ptr() as *const c_void, 2) + .expect("Failed"); + writer + .write_column_primitive(3, i64_data.as_ptr() as *const c_void, 2) + .expect("Failed"); + writer + .write_column_primitive(4, u8_data.as_ptr() as *const c_void, 2) + .expect("Failed"); + writer + .write_column_primitive(5, u16_data.as_ptr() as *const c_void, 2) + .expect("Failed"); + writer + .write_column_primitive(6, u32_data.as_ptr() as *const c_void, 2) + .expect("Failed"); + writer + .write_column_primitive(7, u64_data.as_ptr() as *const c_void, 2) + .expect("Failed"); + writer + .write_column_primitive(8, f32_data.as_ptr() as *const c_void, 2) + .expect("Failed"); + writer + .write_column_primitive(9, f64_data.as_ptr() as *const c_void, 2) + .expect("Failed"); + + writer.end_batch().expect("Failed to end batch"); + writer.finalize().expect("Failed to finalize"); + } + + /// Test that low-precision Decimal types (precision 1-4) are correctly handled + /// on the write path. ClickHouse always sends Decimal32 as 4-byte Int32, even + /// for low precisions like Decimal(3, 2). The write path must use the ClickHouse + /// storage width (i32) rather than the smallest possible width (i16 for precision 3). + #[test] + fn test_decimal_low_precision_write_read_roundtrip() { + use crate::exporter::ColumnExporter; + use crate::exporter::decimal::DecimalExporter; + + // Simulate what ClickHouse does for Decimal(3, 2): + // It sends raw i32 values (unscaled): 1.23 -> 123, -4.56 -> -456, 9.99 -> 999 + let clickhouse_data: Vec = vec![123, -456, 999]; + let num_rows = clickhouse_data.len(); + + // Parse the type string the same way the writer does + let dtype = + clickhouse_type_to_vortex("Decimal(3, 2)").expect("Failed to parse Decimal(3, 2)"); + + // Build array from raw data - this is the critical function being tested + let array = build_array_from_raw_with_validity( + &dtype, + clickhouse_data.as_ptr() as *const c_void, + ptr::null(), + num_rows, + ) + .expect("Failed to build decimal array from raw data"); + + // Now export back to ClickHouse format via DecimalExporter + let mut exporter = DecimalExporter::new(array).expect("Failed to create exporter"); + + let mut output = vec![0i32; num_rows]; + let exported = exporter + .export( + output.as_mut_ptr() as *mut c_void, + size_of_val(output.as_slice()), + num_rows, + ) + .expect("Export failed"); + + assert_eq!(exported, num_rows); + assert_eq!(output, vec![123, -456, 999]); + } + + /// Same test for precision 1-2 (would map to I8 with smallest_decimal_value_type). + #[test] + fn test_decimal_precision_2_write_read_roundtrip() { + use crate::exporter::ColumnExporter; + use crate::exporter::decimal::DecimalExporter; + + // Decimal(2, 1): ClickHouse sends raw i32 values: 1.2 -> 12, -9.9 -> -99 + let clickhouse_data: Vec = vec![12, -99]; + let num_rows = clickhouse_data.len(); + + let dtype = + clickhouse_type_to_vortex("Decimal(2, 1)").expect("Failed to parse Decimal(2, 1)"); + + let array = build_array_from_raw_with_validity( + &dtype, + clickhouse_data.as_ptr() as *const c_void, + ptr::null(), + num_rows, + ) + .expect("Failed to build decimal array from raw data"); + + let mut exporter = DecimalExporter::new(array).expect("Failed to create exporter"); + + let mut output = vec![0i32; num_rows]; + let exported = exporter + .export( + output.as_mut_ptr() as *mut c_void, + size_of_val(output.as_slice()), + num_rows, + ) + .expect("Export failed"); + + assert_eq!(exported, num_rows); + assert_eq!(output, vec![12, -99]); + } + + #[test] + fn test_writer_ffi_workflow_complete() { + // Complete FFI workflow test + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path.clone()).expect("CString failed"); + + unsafe { + // Create writer + let writer = vortex_writer_new(c_path.as_ptr()); + assert!(!writer.is_null()); + + // Add columns + let name_col = CString::new("id").unwrap(); + let type_col = CString::new("Int64").unwrap(); + let result = vortex_writer_add_column(writer, name_col.as_ptr(), type_col.as_ptr(), 0); + assert_eq!(result, 0); + + // Begin batch + let result = vortex_writer_begin_batch(writer, 3); + assert_eq!(result, 0); + + // Write column + let data: Vec = vec![1, 2, 3]; + let result = vortex_writer_write_column(writer, 0, data.as_ptr() as *const c_void, 3); + assert_eq!(result, 0); + + // End batch + let result = vortex_writer_end_batch(writer); + assert_eq!(result, 0); + + // Verify row count + let rows = vortex_writer_total_rows(writer); + assert_eq!(rows, 3); + + // Finalize + let result = vortex_writer_finalize(writer); + assert_eq!(result, 0); + + // Free + vortex_writer_free(writer); + } + + assert!(std::path::Path::new(&path).exists()); + } +} diff --git a/vortex-clickhouse/src/cpp.rs b/vortex-clickhouse/src/cpp.rs new file mode 100644 index 00000000000..cbf1fe69be2 --- /dev/null +++ b/vortex-clickhouse/src/cpp.rs @@ -0,0 +1,23 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! FFI bindings for ClickHouse. +//! +//! This module is auto-generated by bindgen when ClickHouse headers are available. +//! For now, it contains placeholder definitions. + +#![allow(dead_code)] +#![allow(non_camel_case_types)] +#![allow(non_upper_case_globals)] +#![allow(non_snake_case)] + +// Placeholder types - will be replaced by bindgen-generated code +// when ClickHouse headers are available. + +/// Result status for operations. +#[repr(C)] +#[derive(Debug, Copy, Clone, PartialEq, Eq)] +pub enum vortex_ch_state { + Success = 0, + Error = 1, +} diff --git a/vortex-clickhouse/src/e2e_test.rs b/vortex-clickhouse/src/e2e_test.rs new file mode 100644 index 00000000000..71374625af6 --- /dev/null +++ b/vortex-clickhouse/src/e2e_test.rs @@ -0,0 +1,281 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! End-to-end tests for vortex-clickhouse. +//! +//! These tests verify the full pipeline of reading and writing Vortex files +//! through ClickHouse. + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use vortex::dtype::Nullability::{NonNullable, Nullable}; + use vortex::dtype::{DType, FieldName, FieldNames, PType, StructFields}; + + use crate::convert::dtype::{clickhouse_type_to_vortex, vortex_to_clickhouse_type}; + + #[test] + fn test_type_roundtrip() { + // Test that types can be converted back and forth + let test_types = vec![ + ("Int32", DType::Primitive(PType::I32, NonNullable)), + ("UInt64", DType::Primitive(PType::U64, NonNullable)), + ("Float64", DType::Primitive(PType::F64, NonNullable)), + ("String", DType::Utf8(NonNullable)), + ("Bool", DType::Bool(NonNullable)), + ]; + + for (ch_type, expected_dtype) in test_types { + let converted = clickhouse_type_to_vortex(ch_type).unwrap(); + assert_eq!( + converted, expected_dtype, + "Failed for ClickHouse type: {}", + ch_type + ); + } + } + + #[test] + fn test_vortex_to_clickhouse_types() { + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::I32, NonNullable)).unwrap(), + "Int32" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Primitive(PType::I32, Nullable)).unwrap(), + "Nullable(Int32)" + ); + assert_eq!( + vortex_to_clickhouse_type(&DType::Utf8(NonNullable)).unwrap(), + "String" + ); + } + + /// Test a realistic table schema typical for analytics workloads. + #[test] + fn test_analytics_table_schema() { + // Simulate a typical ClickHouse analytics table schema + let ch_schema = vec![ + ("event_id", "UInt64"), + ("event_time", "DateTime64(3)"), + ("user_id", "Nullable(UInt64)"), + ("event_type", "LowCardinality(String)"), + ("properties", "String"), + ("amount", "Nullable(Float64)"), + ]; + + for (col_name, ch_type) in ch_schema { + let result = clickhouse_type_to_vortex(ch_type); + assert!( + result.is_ok(), + "Failed to convert column '{}' with type '{}'", + col_name, + ch_type + ); + } + } + + /// Test a realistic log table schema. + #[test] + fn test_log_table_schema() { + let ch_schema = vec![ + ("timestamp", "DateTime64(6, 'UTC')"), + ("level", "LowCardinality(String)"), + ("service", "String"), + ("trace_id", "UUID"), + ("message", "String"), + ("tags", "Array(String)"), + ( + "metadata", + "Tuple(host String, pod String, container String)", + ), + ]; + + for (col_name, ch_type) in ch_schema { + let result = clickhouse_type_to_vortex(ch_type); + assert!( + result.is_ok(), + "Failed to convert column '{}' with type '{}': {:?}", + col_name, + ch_type, + result.err() + ); + } + } + + /// Test time-series data schema. + #[test] + fn test_timeseries_schema() { + let ch_schema = vec![ + ("metric_name", "String"), + ("timestamp", "DateTime"), + ("value", "Float64"), + ("tags", "Array(String)"), + ]; + + for (col_name, ch_type) in ch_schema { + let result = clickhouse_type_to_vortex(ch_type); + assert!( + result.is_ok(), + "Failed to convert column '{}' with type '{}'", + col_name, + ch_type + ); + } + } + + /// Test building a full Vortex schema and converting to ClickHouse. + #[test] + fn test_full_schema_conversion() { + // Build a Vortex schema representing a user events table + let names = FieldNames::from(vec![ + FieldName::from("user_id"), + FieldName::from("event_name"), + FieldName::from("timestamp"), + FieldName::from("properties"), + FieldName::from("tags"), + ]); + + let dtypes = vec![ + DType::Primitive(PType::I64, NonNullable), + DType::Utf8(NonNullable), + DType::Primitive(PType::I64, NonNullable), // Unix timestamp + DType::Utf8(Nullable), // JSON properties + DType::List(Arc::new(DType::Utf8(NonNullable)), Nullable), + ]; + + let schema = DType::Struct(StructFields::new(names, dtypes), NonNullable); + + let ch_type = vortex_to_clickhouse_type(&schema).unwrap(); + assert!(ch_type.starts_with("Tuple(")); + assert!(ch_type.contains("user_id Int64")); + assert!(ch_type.contains("event_name String")); + assert!(ch_type.contains("tags Nullable(Array(String))")); + } + + /// Test nested struct conversion for hierarchical data. + #[test] + fn test_nested_data_schema() { + // Address nested struct + let address_names = + FieldNames::from(vec![FieldName::from("city"), FieldName::from("country")]); + let address_dtypes = vec![DType::Utf8(NonNullable), DType::Utf8(NonNullable)]; + let address_struct = DType::Struct( + StructFields::new(address_names, address_dtypes), + NonNullable, + ); + + // User struct containing address + let user_names = FieldNames::from(vec![ + FieldName::from("id"), + FieldName::from("name"), + FieldName::from("address"), + ]); + let user_dtypes = vec![ + DType::Primitive(PType::I64, NonNullable), + DType::Utf8(NonNullable), + address_struct, + ]; + let user_struct = DType::Struct(StructFields::new(user_names, user_dtypes), NonNullable); + + let ch_type = vortex_to_clickhouse_type(&user_struct).unwrap(); + assert_eq!( + ch_type, + "Tuple(id Int64, name String, address Tuple(city String, country String))" + ); + } + + /// Test array of structs (common pattern in ClickHouse). + #[test] + fn test_array_of_structs() { + // Item struct + let item_names = FieldNames::from(vec![ + FieldName::from("product_id"), + FieldName::from("quantity"), + FieldName::from("price"), + ]); + let item_dtypes = vec![ + DType::Primitive(PType::I64, NonNullable), + DType::Primitive(PType::I32, NonNullable), + DType::Primitive(PType::F64, NonNullable), + ]; + let item_struct = DType::Struct(StructFields::new(item_names, item_dtypes), NonNullable); + + // Array of items + let items_list = DType::List(Arc::new(item_struct), NonNullable); + + let ch_type = vortex_to_clickhouse_type(&items_list).unwrap(); + assert_eq!( + ch_type, + "Array(Tuple(product_id Int64, quantity Int32, price Float64))" + ); + } + + /// Test all primitive types roundtrip. + #[test] + fn test_all_primitives_roundtrip() { + let ch_primitives = vec![ + "Int8", "Int16", "Int32", "Int64", "UInt8", "UInt16", "UInt32", "UInt64", "Float32", + "Float64", "Bool", "String", + ]; + + for ch_type in ch_primitives { + let vortex_dtype = clickhouse_type_to_vortex(ch_type).unwrap(); + let back_to_ch = vortex_to_clickhouse_type(&vortex_dtype).unwrap(); + // NonNullable types roundtrip back to the same type string + assert_eq!( + back_to_ch, ch_type, + "Roundtrip failed for {}: got {}", + ch_type, back_to_ch + ); + } + } + + /// Test empty struct handling. + #[test] + fn test_empty_struct() { + let empty_struct = DType::Struct( + StructFields::new(FieldNames::from(Vec::::new()), vec![]), + NonNullable, + ); + let ch_type = vortex_to_clickhouse_type(&empty_struct).unwrap(); + assert_eq!(ch_type, "Tuple()"); + } + + /// Test single-field struct. + #[test] + fn test_single_field_struct() { + let names = FieldNames::from(vec![FieldName::from("value")]); + let dtypes = vec![DType::Primitive(PType::I64, NonNullable)]; + let single_struct = DType::Struct(StructFields::new(names, dtypes), NonNullable); + + let ch_type = vortex_to_clickhouse_type(&single_struct).unwrap(); + assert_eq!(ch_type, "Tuple(value Int64)"); + } + + /// Test deeply nested array. + #[test] + fn test_deeply_nested_array() { + // Array(Array(Array(Int32))) + let dtype = clickhouse_type_to_vortex("Array(Array(Array(Int32)))").unwrap(); + + // Verify it's three levels deep + if let DType::List(l1, _) = dtype { + if let DType::List(l2, _) = l1.as_ref() { + if let DType::List(l3, _) = l2.as_ref() { + assert!(matches!( + l3.as_ref(), + DType::Primitive(PType::I32, NonNullable) + )); + } else { + panic!("Expected third level to be List"); + } + } else { + panic!("Expected second level to be List"); + } + } else { + panic!("Expected first level to be List"); + } + } +} diff --git a/vortex-clickhouse/src/error.rs b/vortex-clickhouse/src/error.rs new file mode 100644 index 00000000000..32924151810 --- /dev/null +++ b/vortex-clickhouse/src/error.rs @@ -0,0 +1,211 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Thread-local error handling for FFI. +//! +//! This module provides a mechanism to store the last error message in a thread-local +//! variable, which can be retrieved by the C++ side after an FFI call returns an error. +//! +//! # Usage +//! +//! On the Rust side: +//! ```rust,ignore +//! use crate::error::{set_last_error, clear_last_error}; +//! +//! pub extern "C" fn some_ffi_function() -> i32 { +//! clear_last_error(); +//! match do_something() { +//! Ok(_) => 0, +//! Err(e) => { +//! set_last_error(&e.to_string()); +//! -1 +//! } +//! } +//! } +//! ``` +//! +//! On the C++ side: +//! ```cpp +//! int result = some_ffi_function(); +//! if (result < 0) { +//! const char* error = vortex_get_last_error(); +//! if (error) { +//! std::cerr << "Error: " << error << std::endl; +//! vortex_free_string(error); +//! } +//! } +//! ``` + +use std::cell::RefCell; +use std::ffi::{CString, c_char}; +use std::ptr; + +thread_local! { + /// Thread-local storage for the last error message. + static LAST_ERROR: RefCell> = const { RefCell::new(None) }; +} + +/// Set the last error message. +/// +/// This function stores the error message in thread-local storage, +/// where it can be retrieved by `vortex_get_last_error()`. +pub fn set_last_error(message: &str) { + LAST_ERROR.with(|cell| { + *cell.borrow_mut() = Some(message.to_string()); + }); +} + +/// Clear the last error message. +/// +/// This should be called at the beginning of each FFI function to ensure +/// that any previous error is cleared. +pub fn clear_last_error() { + LAST_ERROR.with(|cell| { + *cell.borrow_mut() = None; + }); +} + +/// Get the last error message. +/// +/// Returns the last error message if one was set, or None. +pub fn get_last_error() -> Option { + LAST_ERROR.with(|cell| cell.borrow().clone()) +} + +/// Check if there is a last error. +pub fn has_last_error() -> bool { + LAST_ERROR.with(|cell| cell.borrow().is_some()) +} + +// ============================================================================= +// FFI Exports +// ============================================================================= + +/// Get the last error message. +/// +/// Returns a null-terminated C string with the last error message, +/// or NULL if no error was set. The returned string must be freed +/// by calling `vortex_free_string()`. +/// +/// # Safety +/// The returned pointer must be freed with `vortex_free_string()` after use. +/// The pointer is only valid until the next call to any vortex FFI function +/// from the same thread. +#[unsafe(no_mangle)] +pub extern "C" fn vortex_get_last_error() -> *mut c_char { + match get_last_error() { + Some(msg) => match CString::new(msg) { + Ok(c_string) => c_string.into_raw(), + Err(_) => ptr::null_mut(), + }, + None => ptr::null_mut(), + } +} + +/// Check if there is a last error. +/// +/// Returns 1 if an error is set, 0 otherwise. +#[unsafe(no_mangle)] +pub extern "C" fn vortex_has_error() -> i32 { + if has_last_error() { 1 } else { 0 } +} + +/// Clear the last error. +/// +/// Call this before starting a new operation if you want to ensure +/// no stale error messages are present. +#[unsafe(no_mangle)] +pub extern "C" fn vortex_clear_error() { + clear_last_error(); +} + +/// Free a string returned by vortex FFI functions. +/// +/// This function must be called to free strings returned by functions like +/// `vortex_get_last_error()`, `vortex_scanner_column_name()`, etc. +/// +/// # Safety +/// The `ptr` must be a valid pointer returned by a vortex FFI function, +/// or NULL (which is safely ignored). +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_free_string(ptr: *mut c_char) { + if !ptr.is_null() { + drop(unsafe { CString::from_raw(ptr) }); + } +} + +// ============================================================================= +// Macro for FFI error handling +// ============================================================================= + +/// Macro to wrap FFI function bodies with error handling. +/// +/// This macro: +/// 1. Clears the last error at the start +/// 2. Executes the body +/// 3. On error, sets the last error message and returns the error value +/// +/// # Example +/// ```rust,ignore +/// ffi_try! { +/// let result = some_operation()?; +/// Ok(result) +/// } or_return -1 +/// ``` +macro_rules! ffi_try { + ($body:expr, $error_ret:expr) => {{ + crate::error::clear_last_error(); + match (|| -> vortex::error::VortexResult<_> { $body })() { + Ok(val) => val, + Err(e) => { + crate::error::set_last_error(&e.to_string()); + return $error_ret; + } + } + }}; +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_error_set_and_get() { + clear_last_error(); + assert!(!has_last_error()); + assert!(get_last_error().is_none()); + + set_last_error("Test error message"); + assert!(has_last_error()); + assert_eq!(get_last_error(), Some("Test error message".to_string())); + + clear_last_error(); + assert!(!has_last_error()); + assert!(get_last_error().is_none()); + } + + #[test] + fn test_ffi_get_last_error() { + clear_last_error(); + + // No error set + let ptr = vortex_get_last_error(); + assert!(ptr.is_null()); + + // Set an error + set_last_error("FFI test error"); + let ptr = vortex_get_last_error(); + assert!(!ptr.is_null()); + + // Read the error + let c_str = unsafe { std::ffi::CStr::from_ptr(ptr) }; + assert_eq!(c_str.to_str().unwrap(), "FFI test error"); + + // Free the string + unsafe { vortex_free_string(ptr) }; + + // Clear the error + vortex_clear_error(); + assert_eq!(vortex_has_error(), 0); + } +} diff --git a/vortex-clickhouse/src/exporter/bigint.rs b/vortex-clickhouse/src/exporter/bigint.rs new file mode 100644 index 00000000000..714d3eb28ab --- /dev/null +++ b/vortex-clickhouse/src/exporter/bigint.rs @@ -0,0 +1,311 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Big integer array exporter for ClickHouse. +//! +//! This module exports Vortex FixedSizeList arrays (used to store big integers like +//! Int128, UInt128, Int256, UInt256) to ClickHouse column buffers. +//! +//! Big integers are stored as FixedSizeList where N is 16 (for 128-bit) or 32 (for 256-bit). + +use std::any::Any; +use std::ffi::c_void; + +use vortex::array::arrays::PrimitiveArray; +use vortex::array::{Array, ArrayRef, ToCanonical}; +use vortex::dtype::{DType, Nullability, PType}; +use vortex::error::{VortexResult, vortex_bail}; +use vortex::mask::Mask; + +use super::{ColumnExporter, ExporterKind}; + +/// The size of each big integer in bytes. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum BigIntSize { + /// 128-bit integer (16 bytes) + Bits128 = 16, + /// 256-bit integer (32 bytes) + Bits256 = 32, +} + +/// Exporter for big integer arrays (Int128, UInt128, Int256, UInt256). +/// +/// These are stored as FixedSizeList in Vortex, where N is 16 or 32. +pub struct BigIntExporter { + /// Cached flat byte slice from canonical form + bytes: PrimitiveArray, + /// Cached validity mask (None = non-nullable) + validity: Option, + /// Current export position + position: usize, + /// Position at the start of last export (for validity export) + last_export_start: usize, + /// Number of rows exported in last export call + last_export_count: usize, + /// Total length of the array (number of big integers) + len: usize, + /// Size of each big integer + bigint_size: BigIntSize, + /// Whether the array is nullable + nullable: bool, +} + +impl BigIntExporter { + /// Create a new big integer exporter for the given array. + pub fn new(array: ArrayRef) -> VortexResult { + let len = array.len(); + let (bigint_size, nullable) = match array.dtype() { + DType::FixedSizeList(elem_dtype, size, nullability) => { + // Verify the element type is u8 + if !matches!(elem_dtype.as_ref(), DType::Primitive(PType::U8, _)) { + vortex_bail!( + "BigIntExporter requires FixedSizeList, got FixedSizeList<{:?}, {}>", + elem_dtype, + size + ); + } + let bigint_size = match *size { + 16 => BigIntSize::Bits128, + 32 => BigIntSize::Bits256, + _ => vortex_bail!( + "BigIntExporter requires FixedSizeList with size 16 or 32, got {}", + size + ), + }; + (bigint_size, *nullability == Nullability::Nullable) + } + _ => vortex_bail!( + "BigIntExporter requires a FixedSizeList array, got {:?}", + array.dtype() + ), + }; + + let canonical = array.to_fixed_size_list(); + let validity = if nullable { + Some(canonical.validity_mask()?) + } else { + None + }; + let elements = canonical.elements(); + let bytes = elements.to_primitive(); + + Ok(Self { + bytes, + validity, + position: 0, + last_export_start: 0, + last_export_count: 0, + len, + bigint_size, + nullable, + }) + } + + /// Get the size of each big integer in bytes. + pub fn bigint_size(&self) -> BigIntSize { + self.bigint_size + } +} + +impl ColumnExporter for BigIntExporter { + fn kind(&self) -> ExporterKind { + ExporterKind::BigInt + } + + fn export( + &mut self, + column_ptr: *mut c_void, + buffer_size_bytes: usize, + max_rows: usize, + ) -> VortexResult { + if column_ptr.is_null() { + vortex_bail!("column_ptr is null"); + } + + let remaining = self.len - self.position; + let rows_to_export = remaining.min(max_rows); + + if rows_to_export == 0 { + return Ok(0); + } + + let byte_size = self.bigint_size as usize; + let required_bytes = rows_to_export * byte_size; + if buffer_size_bytes < required_bytes { + vortex_bail!( + "buffer too small: need {} bytes for {} BigInt({}) rows, got {}", + required_bytes, + rows_to_export, + byte_size, + buffer_size_bytes + ); + } + + // Record the start position for validity export + self.last_export_start = self.position; + self.last_export_count = rows_to_export; + + let bytes = self.bytes.as_slice::(); + + // Calculate byte range to export + let start_byte = self.position * byte_size; + let end_byte = (self.position + rows_to_export) * byte_size; + let slice = &bytes[start_byte..end_byte]; + + // Copy to destination + let dst = column_ptr as *mut u8; + unsafe { + std::ptr::copy_nonoverlapping(slice.as_ptr(), dst, slice.len()); + } + + self.position += rows_to_export; + Ok(rows_to_export) + } + + fn element_size_bytes(&self) -> usize { + self.bigint_size as usize + } + + fn has_more(&self) -> bool { + self.position < self.len + } + + fn len(&self) -> usize { + self.len + } + + fn is_nullable(&self) -> bool { + self.nullable + } + + fn export_validity(&mut self, bitmap: *mut u8, max_rows: usize) -> VortexResult { + super::export_validity_cached( + bitmap, + max_rows, + self.validity.as_ref(), + self.last_export_start, + self.last_export_count, + ) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn as_any_mut(&mut self) -> &mut dyn Any { + self + } +} + +#[cfg(test)] +mod tests { + use super::*; + use vortex::array::IntoArray; + use vortex::array::arrays::{FixedSizeListArray, PrimitiveArray}; + use vortex::array::validity::Validity; + use vortex::buffer::Buffer; + + #[test] + fn test_bigint_exporter_128bit() { + // Create test data: two 128-bit integers as byte arrays + let bytes: Vec = vec![ + 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, + 0x0f, 0x10, 0x11, 0x12, 0x13, 0x14, 0x15, 0x16, 0x17, 0x18, 0x19, 0x1a, 0x1b, 0x1c, + 0x1d, 0x1e, 0x1f, 0x20, + ]; + + let values = PrimitiveArray::new(Buffer::::from(bytes.clone()), Validity::NonNullable); + let array = FixedSizeListArray::try_new( + values.into_array(), + 16, // 128-bit = 16 bytes + Validity::NonNullable, + 2, // 2 elements + ) + .expect("Failed to create FixedSizeListArray"); + + let mut exporter = + BigIntExporter::new(array.into_array()).expect("Failed to create exporter"); + + assert!(exporter.has_more()); + assert_eq!(exporter.bigint_size(), BigIntSize::Bits128); + + // Export all data + let mut output = vec![0u8; 32]; + let exported = exporter + .export(output.as_mut_ptr() as *mut c_void, output.len(), 2) + .expect("Export failed"); + + assert_eq!(exported, 2); + assert_eq!(output, bytes); + assert!(!exporter.has_more()); + } + + #[test] + fn test_bigint_exporter_256bit() { + // Create test data: one 256-bit integer as byte array + let bytes: Vec = (0..32).collect(); + + let values = PrimitiveArray::new(Buffer::::from(bytes.clone()), Validity::NonNullable); + let array = FixedSizeListArray::try_new( + values.into_array(), + 32, // 256-bit = 32 bytes + Validity::NonNullable, + 1, // 1 element + ) + .expect("Failed to create FixedSizeListArray"); + + let mut exporter = + BigIntExporter::new(array.into_array()).expect("Failed to create exporter"); + + assert!(exporter.has_more()); + assert_eq!(exporter.bigint_size(), BigIntSize::Bits256); + + // Export all data + let mut output = vec![0u8; 32]; + let exported = exporter + .export(output.as_mut_ptr() as *mut c_void, output.len(), 1) + .expect("Export failed"); + + assert_eq!(exported, 1); + assert_eq!(output, bytes); + assert!(!exporter.has_more()); + } + + #[test] + fn test_bigint_exporter_partial() { + // Create test data: four 128-bit integers + let bytes: Vec = (0..64).collect(); + + let values = PrimitiveArray::new(Buffer::::from(bytes.clone()), Validity::NonNullable); + let array = FixedSizeListArray::try_new( + values.into_array(), + 16, + Validity::NonNullable, + 4, // 4 elements + ) + .expect("Failed to create FixedSizeListArray"); + + let mut exporter = + BigIntExporter::new(array.into_array()).expect("Failed to create exporter"); + + // Export first 2 integers + let mut output1 = vec![0u8; 32]; + let exported1 = exporter + .export(output1.as_mut_ptr() as *mut c_void, output1.len(), 2) + .expect("Export failed"); + + assert_eq!(exported1, 2); + assert_eq!(output1, bytes[0..32]); + assert!(exporter.has_more()); + + // Export remaining 2 integers + let mut output2 = vec![0u8; 32]; + let exported2 = exporter + .export(output2.as_mut_ptr() as *mut c_void, output2.len(), 5) // request more than available + .expect("Export failed"); + + assert_eq!(exported2, 2); + assert_eq!(output2, bytes[32..64]); + assert!(!exporter.has_more()); + } +} diff --git a/vortex-clickhouse/src/exporter/bool_.rs b/vortex-clickhouse/src/exporter/bool_.rs new file mode 100644 index 00000000000..148ba299255 --- /dev/null +++ b/vortex-clickhouse/src/exporter/bool_.rs @@ -0,0 +1,153 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Bool array exporter for ClickHouse. +//! +//! This module exports Vortex Bool arrays to ClickHouse UInt8 column buffers. +//! ClickHouse represents booleans as UInt8 (0 = false, 1 = true). + +use std::any::Any; +use std::ffi::c_void; + +use vortex::array::arrays::BoolArray; +use vortex::array::{Array, ArrayRef, ToCanonical}; +use vortex::dtype::Nullability; +use vortex::error::{VortexResult, vortex_bail}; +use vortex::mask::Mask; + +use super::{ColumnExporter, ExporterKind}; + +/// Exporter for Bool arrays. +pub struct BoolExporter { + /// Cached canonical bool array + canonical: BoolArray, + /// Cached validity mask (None = non-nullable) + validity: Option, + /// Current export position + position: usize, + /// Position at start of last export (for validity export) + last_export_start: usize, + /// Number of rows exported in last export call + last_export_count: usize, + /// Total length of the array + len: usize, + /// Whether the array is nullable + nullable: bool, +} + +impl BoolExporter { + /// Create a new bool exporter for the given array. + pub fn new(array: ArrayRef) -> VortexResult { + let len = array.len(); + + let nullable = match array.dtype() { + vortex::dtype::DType::Bool(nullability) => *nullability == Nullability::Nullable, + _ => vortex_bail!("BoolExporter requires a Bool array"), + }; + + let canonical = array.to_bool(); + let validity = if nullable { + Some(canonical.validity_mask()?) + } else { + None + }; + + Ok(Self { + canonical, + validity, + position: 0, + last_export_start: 0, + last_export_count: 0, + len, + nullable, + }) + } +} + +impl ColumnExporter for BoolExporter { + fn kind(&self) -> ExporterKind { + ExporterKind::Bool + } + + fn export( + &mut self, + buffer: *mut c_void, + buffer_size_bytes: usize, + max_rows: usize, + ) -> VortexResult { + if buffer.is_null() { + vortex_bail!("buffer is null"); + } + + let remaining = self.len - self.position; + let rows_to_export = remaining.min(max_rows); + + if rows_to_export == 0 { + return Ok(0); + } + + // ClickHouse booleans are exported as one byte (u8) per row. + if buffer_size_bytes < rows_to_export { + vortex_bail!( + "buffer too small: need {} bytes for {} Bool rows, got {}", + rows_to_export, + rows_to_export, + buffer_size_bytes + ); + } + + // Record start position for validity export + self.last_export_start = self.position; + self.last_export_count = rows_to_export; + + // Export as u8 (ClickHouse boolean representation). + // Read directly from the bit-packed buffer instead of per-element scalar_at. + let bits = self.canonical.to_bit_buffer(); + let output_slice = + unsafe { std::slice::from_raw_parts_mut(buffer as *mut u8, rows_to_export) }; + + for i in 0..rows_to_export { + let idx = self.position + i; + // Null values produce 0 (false) — the validity bitmap is exported separately. + output_slice[i] = u8::from(bits.value(idx)); + } + + self.position += rows_to_export; + Ok(rows_to_export) + } + + fn element_size_bytes(&self) -> usize { + // ClickHouse represents Bool as UInt8, one byte per row. + 1 + } + + fn has_more(&self) -> bool { + self.position < self.len + } + + fn len(&self) -> usize { + self.len + } + + fn is_nullable(&self) -> bool { + self.nullable + } + + fn export_validity(&mut self, bitmap: *mut u8, max_rows: usize) -> VortexResult { + super::export_validity_cached( + bitmap, + max_rows, + self.validity.as_ref(), + self.last_export_start, + self.last_export_count, + ) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn as_any_mut(&mut self) -> &mut dyn Any { + self + } +} diff --git a/vortex-clickhouse/src/exporter/decimal.rs b/vortex-clickhouse/src/exporter/decimal.rs new file mode 100644 index 00000000000..829be79d442 --- /dev/null +++ b/vortex-clickhouse/src/exporter/decimal.rs @@ -0,0 +1,471 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Decimal array exporter for ClickHouse. +//! +//! This module exports Vortex decimal arrays to ClickHouse column buffers. +//! It handles different decimal precisions (Decimal32, Decimal64, Decimal128, Decimal256). + +use std::any::Any; +use std::ffi::c_void; + +use vortex::array::arrays::DecimalArray; +use vortex::array::{Array, ArrayRef, ToCanonical}; +use vortex::dtype::{DecimalDType, DecimalType, Nullability}; +use vortex::error::{VortexResult, vortex_bail}; +use vortex::mask::Mask; + +use super::{ColumnExporter, ExporterKind}; + +/// Map ClickHouse precision to the fixed-width storage type that ClickHouse expects. +/// +/// ClickHouse always uses a fixed-width type for each decimal variant, regardless +/// of the actual magnitude of the stored values: +/// precision 1-9 -> Int32 (Decimal32) +/// precision 10-18 -> Int64 (Decimal64) +/// precision 19-38 -> Int128 (Decimal128) +/// precision 39-76 -> Int256 (Decimal256) +pub(crate) fn clickhouse_decimal_type(precision: u8) -> DecimalType { + match precision { + 1..=9 => DecimalType::I32, + 10..=18 => DecimalType::I64, + 19..=38 => DecimalType::I128, + 39..=76 => DecimalType::I256, + 0 => unreachable!("precision must be greater than 0"), + p => unreachable!("unsupported precision {p}"), + } +} + +/// Return the byte width of a `DecimalType` as exported to ClickHouse. +fn decimal_type_byte_width(dt: DecimalType) -> usize { + match dt { + DecimalType::I8 => 1, + DecimalType::I16 => 2, + DecimalType::I32 => 4, + DecimalType::I64 => 8, + DecimalType::I128 => 16, + DecimalType::I256 => 32, + } +} + +/// Exporter for decimal arrays. +pub struct DecimalExporter { + /// Cached canonical decimal array + canonical: DecimalArray, + /// Cached validity mask (None = non-nullable) + validity: Option, + /// Current export position + position: usize, + /// Position at the start of last export (for validity export) + last_export_start: usize, + /// Number of rows exported in last export call + last_export_count: usize, + /// Total length of the array + len: usize, + /// Decimal dtype (precision and scale) + decimal_dtype: DecimalDType, + /// Vortex internal values type (may be smaller than ClickHouse expects) + values_type: DecimalType, + /// ClickHouse-expected export type (determined by precision ranges) + export_type: DecimalType, + /// Whether the array is nullable + nullable: bool, +} + +impl DecimalExporter { + /// Create a new decimal exporter for the given array. + pub fn new(array: ArrayRef) -> VortexResult { + let len = array.len(); + let (decimal_dtype, nullable) = match array.dtype() { + vortex::dtype::DType::Decimal(decimal_dtype, nullability) => { + (*decimal_dtype, *nullability == Nullability::Nullable) + } + _ => vortex_bail!("DecimalExporter requires a decimal array"), + }; + + let canonical = array.to_decimal(); + let values_type = canonical.values_type(); + let validity = if nullable { + Some(canonical.validity_mask()?) + } else { + None + }; + + let export_type = clickhouse_decimal_type(decimal_dtype.precision()); + + Ok(Self { + canonical, + validity, + position: 0, + last_export_start: 0, + last_export_count: 0, + len, + decimal_dtype, + values_type, + export_type, + nullable, + }) + } + + /// Get the decimal dtype + pub fn decimal_dtype(&self) -> DecimalDType { + self.decimal_dtype + } + + /// Get the decimal values type + pub fn values_type(&self) -> DecimalType { + self.values_type + } +} + +impl ColumnExporter for DecimalExporter { + fn kind(&self) -> ExporterKind { + ExporterKind::Decimal + } + + fn export( + &mut self, + column_ptr: *mut c_void, + buffer_size_bytes: usize, + max_rows: usize, + ) -> VortexResult { + if column_ptr.is_null() { + vortex_bail!("column_ptr is null"); + } + + let remaining = self.len - self.position; + let rows_to_export = remaining.min(max_rows); + + if rows_to_export == 0 { + return Ok(0); + } + + let elem_bytes = decimal_type_byte_width(self.export_type); + let required_bytes = rows_to_export * elem_bytes; + if buffer_size_bytes < required_bytes { + vortex_bail!( + "buffer too small: need {} bytes for {} Decimal({:?}) rows, got {}", + required_bytes, + rows_to_export, + self.export_type, + buffer_size_bytes + ); + } + + // Record the start position for validity export + self.last_export_start = self.position; + self.last_export_count = rows_to_export; + + let start = self.position; + let end = start + rows_to_export; + + // Export using the ClickHouse-expected type (export_type), reading from + // the Vortex internal type (values_type) and widening if needed. + macro_rules! export_widening { + ($src_ty:ty, $dst_ty:ty) => {{ + let buffer = self.canonical.buffer::<$src_ty>(); + let slice = &buffer.as_slice()[start..end]; + let dst = column_ptr as *mut $dst_ty; + for (i, &val) in slice.iter().enumerate() { + unsafe { + *dst.add(i) = val as $dst_ty; + } + } + }}; + } + + macro_rules! export_direct { + ($ty:ty) => {{ + let buffer = self.canonical.buffer::<$ty>(); + let slice = &buffer.as_slice()[start..end]; + let dst = column_ptr as *mut $ty; + unsafe { + std::ptr::copy_nonoverlapping(slice.as_ptr(), dst, rows_to_export); + } + }}; + } + + // Dispatch based on (values_type -> export_type) + match (self.values_type, self.export_type) { + // Same type: direct copy + (DecimalType::I32, DecimalType::I32) => export_direct!(i32), + (DecimalType::I64, DecimalType::I64) => export_direct!(i64), + (DecimalType::I128, DecimalType::I128) => export_direct!(i128), + (DecimalType::I256, DecimalType::I256) => { + use vortex::dtype::i256; + export_direct!(i256); + } + // Widen to I32 + (DecimalType::I8, DecimalType::I32) => export_widening!(i8, i32), + (DecimalType::I16, DecimalType::I32) => export_widening!(i16, i32), + // Widen to I64 + (DecimalType::I8, DecimalType::I64) => export_widening!(i8, i64), + (DecimalType::I16, DecimalType::I64) => export_widening!(i16, i64), + (DecimalType::I32, DecimalType::I64) => export_widening!(i32, i64), + // Widen to I128 + (DecimalType::I8, DecimalType::I128) => export_widening!(i8, i128), + (DecimalType::I16, DecimalType::I128) => export_widening!(i16, i128), + (DecimalType::I32, DecimalType::I128) => export_widening!(i32, i128), + (DecimalType::I64, DecimalType::I128) => export_widening!(i64, i128), + // Widen to I256 + (DecimalType::I8, DecimalType::I256) + | (DecimalType::I16, DecimalType::I256) + | (DecimalType::I32, DecimalType::I256) + | (DecimalType::I64, DecimalType::I256) + | (DecimalType::I128, DecimalType::I256) => { + use vortex::dtype::i256; + match self.values_type { + DecimalType::I8 => { + let buffer = self.canonical.buffer::(); + let slice = &buffer.as_slice()[start..end]; + let dst = column_ptr as *mut i256; + for (i, &val) in slice.iter().enumerate() { + unsafe { + *dst.add(i) = i256::from_i128(val as i128); + } + } + } + DecimalType::I16 => { + let buffer = self.canonical.buffer::(); + let slice = &buffer.as_slice()[start..end]; + let dst = column_ptr as *mut i256; + for (i, &val) in slice.iter().enumerate() { + unsafe { + *dst.add(i) = i256::from_i128(val as i128); + } + } + } + DecimalType::I32 => { + let buffer = self.canonical.buffer::(); + let slice = &buffer.as_slice()[start..end]; + let dst = column_ptr as *mut i256; + for (i, &val) in slice.iter().enumerate() { + unsafe { + *dst.add(i) = i256::from_i128(val as i128); + } + } + } + DecimalType::I64 => { + let buffer = self.canonical.buffer::(); + let slice = &buffer.as_slice()[start..end]; + let dst = column_ptr as *mut i256; + for (i, &val) in slice.iter().enumerate() { + unsafe { + *dst.add(i) = i256::from_i128(val as i128); + } + } + } + DecimalType::I128 => { + let buffer = self.canonical.buffer::(); + let slice = &buffer.as_slice()[start..end]; + let dst = column_ptr as *mut i256; + for (i, &val) in slice.iter().enumerate() { + unsafe { + *dst.add(i) = i256::from_i128(val); + } + } + } + DecimalType::I256 => unreachable!(), + } + } + // Unsupported: narrowing (should not happen) + (src, dst) => vortex_bail!( + "Unsupported decimal export: internal type {:?} to ClickHouse type {:?}", + src, + dst + ), + } + + self.position += rows_to_export; + Ok(rows_to_export) + } + + fn element_size_bytes(&self) -> usize { + decimal_type_byte_width(self.export_type) + } + + fn has_more(&self) -> bool { + self.position < self.len + } + + fn len(&self) -> usize { + self.len + } + + fn is_nullable(&self) -> bool { + self.nullable + } + + fn export_validity(&mut self, bitmap: *mut u8, max_rows: usize) -> VortexResult { + super::export_validity_cached( + bitmap, + max_rows, + self.validity.as_ref(), + self.last_export_start, + self.last_export_count, + ) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn as_any_mut(&mut self) -> &mut dyn Any { + self + } +} + +#[cfg(test)] +mod tests { + use super::*; + use vortex::array::IntoArray; + use vortex::array::arrays::DecimalArray; + use vortex::array::validity::Validity; + use vortex::buffer::Buffer; + use vortex::dtype::DecimalDType; + + #[test] + fn test_decimal_exporter_i32() { + let decimal_dtype = DecimalDType::new(9, 2); + let values: Buffer = vec![12345i32, 67890, -12300].into(); + let array = DecimalArray::new(values, decimal_dtype, Validity::NonNullable).into_array(); + + let mut exporter = DecimalExporter::new(array).expect("Failed to create exporter"); + + assert!(exporter.has_more()); + assert_eq!(exporter.decimal_dtype().precision(), 9); + assert_eq!(exporter.decimal_dtype().scale(), 2); + assert_eq!(exporter.values_type(), DecimalType::I32); + + let mut output = vec![0i32; 3]; + let exported = exporter + .export( + output.as_mut_ptr() as *mut c_void, + size_of_val(output.as_slice()), + 3, + ) + .expect("Export failed"); + + assert_eq!(exported, 3); + assert_eq!(output, vec![12345, 67890, -12300]); + assert!(!exporter.has_more()); + } + + #[test] + fn test_decimal_exporter_i64() { + let decimal_dtype = DecimalDType::new(18, 4); + let values: Buffer = vec![1234567890i64, -9876543210].into(); + let array = DecimalArray::new(values, decimal_dtype, Validity::NonNullable).into_array(); + + let mut exporter = DecimalExporter::new(array).expect("Failed to create exporter"); + + assert_eq!(exporter.values_type(), DecimalType::I64); + + let mut output = vec![0i64; 2]; + let exported = exporter + .export( + output.as_mut_ptr() as *mut c_void, + size_of_val(output.as_slice()), + 2, + ) + .expect("Export failed"); + + assert_eq!(exported, 2); + assert_eq!(output, vec![1234567890, -9876543210]); + } + + #[test] + fn test_decimal_exporter_i128() { + let decimal_dtype = DecimalDType::new(38, 10); + let values: Buffer = vec![12345678901234567890i128, -9876543210987654321].into(); + let array = DecimalArray::new(values, decimal_dtype, Validity::NonNullable).into_array(); + + let mut exporter = DecimalExporter::new(array).expect("Failed to create exporter"); + + assert_eq!(exporter.values_type(), DecimalType::I128); + + let mut output = vec![0i128; 2]; + let exported = exporter + .export( + output.as_mut_ptr() as *mut c_void, + size_of_val(output.as_slice()), + 2, + ) + .expect("Export failed"); + + assert_eq!(exported, 2); + assert_eq!(output, vec![12345678901234567890i128, -9876543210987654321]); + } + + #[test] + fn test_decimal_exporter_partial() { + let decimal_dtype = DecimalDType::new(9, 2); + let values: Buffer = vec![100, 200, 300, 400, 500].into(); + let array = DecimalArray::new(values, decimal_dtype, Validity::NonNullable).into_array(); + + let mut exporter = DecimalExporter::new(array).expect("Failed to create exporter"); + + // Export first 2 rows + let mut output1 = vec![0i32; 2]; + let exported1 = exporter + .export( + output1.as_mut_ptr() as *mut c_void, + size_of_val(output1.as_slice()), + 2, + ) + .expect("Export failed"); + + assert_eq!(exported1, 2); + assert_eq!(output1, vec![100, 200]); + assert!(exporter.has_more()); + + // Export next 3 rows + let mut output2 = vec![0i32; 3]; + let exported2 = exporter + .export( + output2.as_mut_ptr() as *mut c_void, + size_of_val(output2.as_slice()), + 5, + ) // request more than available + .expect("Export failed"); + + assert_eq!(exported2, 3); + assert_eq!(output2, vec![300, 400, 500]); + assert!(!exporter.has_more()); + } + + #[test] + fn test_decimal_exporter_nullable() { + let decimal_dtype = DecimalDType::new(9, 2); + let array = + DecimalArray::from_option_iter(vec![Some(100i32), None, Some(300)], decimal_dtype) + .into_array(); + + let mut exporter = DecimalExporter::new(array).expect("Failed to create exporter"); + + assert!(exporter.is_nullable()); + + // Export data + let mut data = vec![0i32; 3]; + let exported = exporter + .export( + data.as_mut_ptr() as *mut c_void, + size_of_val(data.as_slice()), + 3, + ) + .expect("Export failed"); + + assert_eq!(exported, 3); + + // Export validity + let mut validity = vec![0u8; 1]; + let validity_rows = exporter + .export_validity(validity.as_mut_ptr(), 3) + .expect("Export validity failed"); + + assert_eq!(validity_rows, 3); + // Validity bitmap: bit 0 = valid, bit 1 = invalid, bit 2 = valid + // So we expect binary: 101 = 5 + assert_eq!(validity[0], 0b101); + } +} diff --git a/vortex-clickhouse/src/exporter/list.rs b/vortex-clickhouse/src/exporter/list.rs new file mode 100644 index 00000000000..7106ac74105 --- /dev/null +++ b/vortex-clickhouse/src/exporter/list.rs @@ -0,0 +1,589 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! List (Array) exporter for ClickHouse. +//! +//! This module exports Vortex List arrays to ClickHouse Array columns. +//! Since ClickHouse Arrays have a nested structure, this exporter provides +//! methods to export offsets and element data separately. +//! +//! Vortex uses `ListViewArray` internally (with offsets and sizes), but +//! ClickHouse expects offset-based arrays, so we need to compute the +//! cumulative offsets for ClickHouse. + +use std::any::Any; +use std::ffi::c_void; + +use vortex::array::arrays::{ListViewArray, PrimitiveArray}; +use vortex::array::{Array, ArrayRef, ToCanonical}; +use vortex::dtype::Nullability; +use vortex::error::{VortexResult, vortex_bail}; + +use super::{ColumnExporter, ExporterKind, new_exporter}; + +/// Exporter for list/array types. +/// +/// ClickHouse Arrays are stored as: +/// - Offsets array (UInt64): indicates where each array element ends (cumulative) +/// - Nested data: the flattened element values +/// +/// Note: ClickHouse uses end-offsets, not start-offsets. The first list ends at offsets[0], +/// the second list ends at offsets[1], etc. +pub struct ListExporter { + /// The canonicalized ListViewArray + listview: ListViewArray, + /// Cached sizes as a primitive array, computed once during construction. + list_sizes: PrimitiveArray, + /// Current export position (in rows, not elements) + position: usize, + /// Cumulative element offset (for multi-batch export) + /// This tracks how many elements have been exported so far. + element_offset: u64, + /// Total number of rows (arrays) + len: usize, + /// Whether the list is nullable + nullable: bool, + /// Element exporter (lazy initialized) + element_exporter: Option>, +} + +impl ListExporter { + /// Create a new list exporter for the given array. + pub fn new(array: ArrayRef) -> VortexResult { + let len = array.len(); + + // Verify this is a list type + let nullable = match array.dtype() { + vortex::dtype::DType::List(_, nullability) => *nullability == Nullability::Nullable, + vortex::dtype::DType::FixedSizeList(_, _, nullability) => { + *nullability == Nullability::Nullable + } + _ => vortex_bail!("ListExporter requires a List or FixedSizeList array"), + }; + + // Canonicalize to ListViewArray + let listview = array.to_listview(); + let list_sizes = listview.sizes().to_primitive(); + + Ok(Self { + listview, + list_sizes, + position: 0, + element_offset: 0, + len, + nullable, + element_exporter: None, + }) + } + + /// Get the total number of elements across all arrays. + pub fn total_elements(&self) -> VortexResult { + Ok(self.listview.elements().len()) + } + + /// Export offsets for the list arrays. + /// + /// For ClickHouse compatibility, we export cumulative end-offsets. + /// The offsets array has `num_rows + 1` elements, where: + /// - `offsets[0]` is the starting element offset for this batch + /// - `offsets[i]` for i > 0 is the end position of list i-1 (= start of list i) + /// - `offsets[num_rows]` is the total number of elements exported so far + /// + /// For multi-batch export, offsets are cumulative across batches. + /// Example with 6 rows split into 2 batches of 3: + /// - Batch 1: offsets = [0, 2, 3, 6] (elements 0-5) + /// - Batch 2: offsets = [6, 7, 9, 12] (elements 6-11) + /// + /// # Arguments + /// * `offsets` - Buffer to write offsets (must have space for `max_rows + 1` uint64_t values) + /// * `max_rows` - Maximum number of rows to export + /// + /// # Returns + /// Number of rows exported. + pub fn export_offsets(&mut self, offsets: *mut u64, max_rows: usize) -> VortexResult { + if offsets.is_null() { + vortex_bail!("offsets buffer is null"); + } + + let remaining = self.len - self.position; + let rows_to_export = remaining.min(max_rows); + + if rows_to_export == 0 { + return Ok(0); + } + + // Get the cached sizes array + let list_sizes = &self.list_sizes; + + // We need to compute cumulative offsets for ClickHouse + // ListView has per-row (offset, size), but ClickHouse wants cumulative offsets + // Start from the current element_offset for multi-batch support + let mut current_offset: u64 = self.element_offset; + + // Write the starting offset for this batch + unsafe { + *offsets = current_offset; + } + + // Determine the offset type and export + // We only use sizes to compute cumulative offsets for ClickHouse + macro_rules! export_offsets_impl { + ($sizes_ty:ty) => {{ + let sizes_slice = list_sizes.as_slice::<$sizes_ty>(); + + for i in 0..rows_to_export { + let idx = self.position + i; + let size = sizes_slice[idx] as u64; + current_offset += size; + unsafe { + *offsets.add(i + 1) = current_offset; + } + } + }}; + } + + // Try different size types + use vortex::dtype::PType; + match list_sizes.ptype() { + PType::U64 => export_offsets_impl!(u64), + PType::U32 => export_offsets_impl!(u32), + PType::I64 => export_offsets_impl!(i64), + PType::I32 => export_offsets_impl!(i32), + PType::U16 => export_offsets_impl!(u16), + PType::I16 => export_offsets_impl!(i16), + PType::U8 => export_offsets_impl!(u8), + PType::I8 => export_offsets_impl!(i8), + size_ptype => { + vortex_bail!("Unsupported size type: {:?}", size_ptype) + } + } + + // Update the cumulative element offset for next batch + self.element_offset = current_offset; + + // Advance the position after successful export + self.advance(rows_to_export); + + Ok(rows_to_export) + } + + /// Get an exporter for the element data. + /// + /// This returns an exporter for the flattened elements of all arrays. + /// The exporter type depends on the element dtype. + pub fn element_exporter(&mut self) -> VortexResult<&mut Box> { + if self.element_exporter.is_none() { + let elements = self.listview.elements().clone(); + self.element_exporter = Some(new_exporter(elements)?); + } + + Ok(self.element_exporter.as_mut().unwrap()) + } + + /// Take ownership of the element exporter. + pub fn take_element_exporter(&mut self) -> VortexResult> { + if self.element_exporter.is_none() { + let elements = self.listview.elements().clone(); + self.element_exporter = Some(new_exporter(elements)?); + } + + self.element_exporter + .take() + .ok_or_else(|| vortex::error::vortex_err!("Element exporter already taken")) + } + + /// Get the number of rows remaining. + pub fn remaining(&self) -> usize { + self.len - self.position + } + + /// Advance the position by the given number of rows. + pub fn advance(&mut self, rows: usize) { + self.position = (self.position + rows).min(self.len); + } +} + +impl ColumnExporter for ListExporter { + fn kind(&self) -> ExporterKind { + ExporterKind::List + } + + fn export( + &mut self, + _column_ptr: *mut c_void, + _buffer_size_bytes: usize, + _max_rows: usize, + ) -> VortexResult { + // List export requires separate handling for offsets and elements + vortex_bail!( + "ListExporter::export() not supported. Use export_offsets() and element_exporter() separately." + ) + } + + fn has_more(&self) -> bool { + self.position < self.len + } + + fn len(&self) -> usize { + self.len + } + + fn is_nullable(&self) -> bool { + self.nullable + } + + fn export_validity(&mut self, bitmap: *mut u8, max_rows: usize) -> VortexResult { + if bitmap.is_null() { + vortex_bail!("bitmap is null"); + } + + let remaining = self.len - self.position; + let rows_to_export = remaining.min(max_rows); + + if rows_to_export == 0 { + return Ok(0); + } + + let validity = self.listview.validity_mask()?; + + let bitmap_slice = + unsafe { std::slice::from_raw_parts_mut(bitmap, (rows_to_export + 7) / 8) }; + + super::write_validity_bitmap(bitmap_slice, &validity, self.position, rows_to_export); + + Ok(rows_to_export) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn as_any_mut(&mut self) -> &mut dyn Any { + self + } +} + +#[cfg(test)] +mod tests { + use super::*; + use std::sync::Arc; + use vortex::array::IntoArray; + use vortex::array::arrays::{ListViewArray, PrimitiveArray}; + use vortex::array::validity::Validity; + use vortex::buffer::buffer; + + #[test] + fn test_list_exporter_creation() { + // Create a list view array: [[1, 2], [3], [4, 5, 6]] + // Elements: [1, 2, 3, 4, 5, 6] + // Offsets: [0, 2, 3] (start positions) + // Sizes: [2, 1, 3] + let elements = buffer![1i32, 2, 3, 4, 5, 6].into_array(); + let offsets = buffer![0u32, 2, 3].into_array(); + let sizes = buffer![2u32, 1, 3].into_array(); + + let list_array = ListViewArray::new(elements, offsets, sizes, Validity::NonNullable); + + let exporter = + ListExporter::new(list_array.into_array()).expect("Failed to create exporter"); + + assert_eq!(exporter.len, 3); + assert!(exporter.has_more()); + } + + #[test] + fn test_list_export_offsets() { + // Create a list view array: [[1, 2], [3], [4, 5, 6]] + let elements = buffer![1i32, 2, 3, 4, 5, 6].into_array(); + let offsets = buffer![0u32, 2, 3].into_array(); + let sizes = buffer![2u32, 1, 3].into_array(); + + let list_array = ListViewArray::new(elements, offsets, sizes, Validity::NonNullable); + + let mut exporter = + ListExporter::new(list_array.into_array()).expect("Failed to create exporter"); + + assert!(exporter.has_more()); + + // Export all offsets + let mut out_offsets = vec![0u64; 4]; + let exported = exporter + .export_offsets(out_offsets.as_mut_ptr(), 3) + .expect("Failed to export offsets"); + + assert_eq!(exported, 3); + // Cumulative offsets: [0, 2, 3, 6] + assert_eq!(out_offsets, vec![0, 2, 3, 6]); + + // After exporting all rows, has_more should return false + assert!(!exporter.has_more()); + } + + #[test] + fn test_list_element_exporter() { + // Create a list view array: [[1, 2], [3], [4, 5, 6]] + let elements = buffer![1i32, 2, 3, 4, 5, 6].into_array(); + let offsets = buffer![0u32, 2, 3].into_array(); + let sizes = buffer![2u32, 1, 3].into_array(); + + let list_array = ListViewArray::new(elements, offsets, sizes, Validity::NonNullable); + + let mut exporter = + ListExporter::new(list_array.into_array()).expect("Failed to create exporter"); + + // Get element exporter + let elem_exporter = exporter + .take_element_exporter() + .expect("Failed to get element exporter"); + + assert!(elem_exporter.has_more()); + + // Export elements + let mut out_elements = vec![0i32; 6]; + let mut elem_exporter = elem_exporter; + let exported = elem_exporter + .export( + out_elements.as_mut_ptr() as *mut c_void, + size_of_val(out_elements.as_slice()), + 6, + ) + .expect("Failed to export elements"); + + assert_eq!(exported, 6); + assert_eq!(out_elements, vec![1, 2, 3, 4, 5, 6]); + } + + #[test] + fn test_list_total_elements() { + let elements = buffer![1i32, 2, 3, 4, 5, 6, 7, 8, 9, 10].into_array(); + let offsets = buffer![0u32, 3, 5].into_array(); + let sizes = buffer![3u32, 2, 5].into_array(); + + let list_array = ListViewArray::new(elements, offsets, sizes, Validity::NonNullable); + + let exporter = + ListExporter::new(list_array.into_array()).expect("Failed to create exporter"); + + assert_eq!(exporter.total_elements().unwrap(), 10); + } + + #[test] + fn test_list_export_offsets_multi_batch() { + // Create a list view array with 6 rows: [[1,2], [3], [4,5,6], [7], [8,9], [10,11,12]] + // We'll export in 2 batches of 3 rows each + let elements = buffer![1i32, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12].into_array(); + let offsets = buffer![0u32, 2, 3, 6, 7, 9].into_array(); + let sizes = buffer![2u32, 1, 3, 1, 2, 3].into_array(); + + let list_array = ListViewArray::new(elements, offsets, sizes, Validity::NonNullable); + + let mut exporter = + ListExporter::new(list_array.into_array()).expect("Failed to create exporter"); + + assert_eq!(exporter.len, 6); + assert!(exporter.has_more()); + + // Export first batch (rows 0-2): [[1,2], [3], [4,5,6]] + let mut batch1_offsets = vec![0u64; 4]; + let exported1 = exporter + .export_offsets(batch1_offsets.as_mut_ptr(), 3) + .expect("Failed to export offsets batch 1"); + + assert_eq!(exported1, 3); + // First batch: offsets start at 0 + // [0, 2, 3, 6] - cumulative offsets for elements [1,2], [3], [4,5,6] + assert_eq!(batch1_offsets, vec![0, 2, 3, 6]); + assert!(exporter.has_more()); + + // Export second batch (rows 3-5): [[7], [8,9], [10,11,12]] + let mut batch2_offsets = vec![0u64; 4]; + let exported2 = exporter + .export_offsets(batch2_offsets.as_mut_ptr(), 3) + .expect("Failed to export offsets batch 2"); + + assert_eq!(exported2, 3); + // Second batch: offsets continue from where batch 1 ended (6) + // [6, 7, 9, 12] - cumulative offsets for elements [7], [8,9], [10,11,12] + assert_eq!(batch2_offsets, vec![6, 7, 9, 12]); + assert!(!exporter.has_more()); + } + + /// This test simulates the complete C++ side usage pattern for multi-batch export. + /// It verifies that: + /// 1. Offsets are cumulative across batches + /// 2. Element counts can be correctly calculated from offsets + /// 3. The offset calculation matches what ClickHouse expects + #[test] + fn test_list_export_full_flow_multi_batch() { + // Data: [[10, 20], [30], [40, 50, 60], [70], [80, 90], [100, 110, 120]] + // 6 rows, export in 2 batches of 3 + let elements = buffer![10i32, 20, 30, 40, 50, 60, 70, 80, 90, 100, 110, 120].into_array(); + let vortex_offsets = buffer![0u32, 2, 3, 6, 7, 9].into_array(); + let sizes = buffer![2u32, 1, 3, 1, 2, 3].into_array(); + + let list_array = ListViewArray::new(elements, vortex_offsets, sizes, Validity::NonNullable); + + let mut exporter = + ListExporter::new(list_array.into_array()).expect("Failed to create exporter"); + + // Simulate ClickHouse accumulator + let mut clickhouse_offsets: Vec = Vec::new(); + let mut clickhouse_elements: Vec = Vec::new(); + + // === Batch 1: rows 0-2 === + let mut batch1_offsets = vec![0u64; 4]; + let rows1 = exporter + .export_offsets(batch1_offsets.as_mut_ptr(), 3) + .expect("batch 1 offsets"); + assert_eq!(rows1, 3); + + // Get element exporter and export elements for batch 1 + let mut elem_exporter = exporter + .take_element_exporter() + .expect("take element exporter"); + + // Calculate elements for this batch (C++ logic) + let batch1_start = batch1_offsets[0] as usize; + let batch1_end = batch1_offsets[rows1] as usize; + let elements_count1 = batch1_end - batch1_start; + assert_eq!(elements_count1, 6); // 2 + 1 + 3 + + let mut batch1_elements = vec![0i32; elements_count1]; + let exported_elems1 = elem_exporter + .export( + batch1_elements.as_mut_ptr() as *mut c_void, + size_of_val(batch1_elements.as_slice()), + elements_count1, + ) + .expect("export elements batch 1"); + assert_eq!(exported_elems1, 6); + assert_eq!(batch1_elements, vec![10, 20, 30, 40, 50, 60]); + + // Update ClickHouse offsets (C++ logic) + let base_offset1 = if clickhouse_offsets.is_empty() { + 0 + } else { + *clickhouse_offsets.last().unwrap() + }; + for i in 0..rows1 { + let relative_offset = batch1_offsets[i + 1] - batch1_start as u64; + clickhouse_offsets.push(base_offset1 + relative_offset); + } + clickhouse_elements.extend(batch1_elements); + + // Verify batch 1 result + assert_eq!(clickhouse_offsets, vec![2, 3, 6]); + assert_eq!(clickhouse_elements, vec![10, 20, 30, 40, 50, 60]); + + // === Batch 2: rows 3-5 === + let mut batch2_offsets = vec![0u64; 4]; + let rows2 = exporter + .export_offsets(batch2_offsets.as_mut_ptr(), 3) + .expect("batch 2 offsets"); + assert_eq!(rows2, 3); + + // Verify cumulative offsets for batch 2 + assert_eq!(batch2_offsets, vec![6, 7, 9, 12]); + + // Calculate elements for batch 2 + let batch2_start = batch2_offsets[0] as usize; + let batch2_end = batch2_offsets[rows2] as usize; + let elements_count2 = batch2_end - batch2_start; + assert_eq!(elements_count2, 6); // 1 + 2 + 3 + + let mut batch2_elements = vec![0i32; elements_count2]; + let exported_elems2 = elem_exporter + .export( + batch2_elements.as_mut_ptr() as *mut c_void, + size_of_val(batch2_elements.as_slice()), + elements_count2, + ) + .expect("export elements batch 2"); + assert_eq!(exported_elems2, 6); + assert_eq!(batch2_elements, vec![70, 80, 90, 100, 110, 120]); + + // Update ClickHouse offsets + let base_offset2 = *clickhouse_offsets.last().unwrap(); + for i in 0..rows2 { + let relative_offset = batch2_offsets[i + 1] - batch2_start as u64; + clickhouse_offsets.push(base_offset2 + relative_offset); + } + clickhouse_elements.extend(batch2_elements); + + // Verify final result + // ClickHouse offsets should be end positions: [2, 3, 6, 7, 9, 12] + assert_eq!(clickhouse_offsets, vec![2, 3, 6, 7, 9, 12]); + assert_eq!( + clickhouse_elements, + vec![10, 20, 30, 40, 50, 60, 70, 80, 90, 100, 110, 120] + ); + + // Verify has_more is false after all batches + assert!(!exporter.has_more()); + assert!(!elem_exporter.has_more()); + } + + /// Test single row batches to verify edge cases + #[test] + fn test_list_export_single_row_batches() { + // Data: [[1, 2, 3], [4], [5, 6]] + let elements = buffer![1i32, 2, 3, 4, 5, 6].into_array(); + let offsets = buffer![0u32, 3, 4].into_array(); + let sizes = buffer![3u32, 1, 2].into_array(); + + let list_array = ListViewArray::new(elements, offsets, sizes, Validity::NonNullable); + + let mut exporter = + ListExporter::new(list_array.into_array()).expect("Failed to create exporter"); + + // Export row by row + let mut all_offsets: Vec> = Vec::new(); + + for expected_start in [0u64, 3, 4] { + let mut offsets = vec![0u64; 2]; + let rows = exporter + .export_offsets(offsets.as_mut_ptr(), 1) + .expect("export single row"); + assert_eq!(rows, 1); + assert_eq!(offsets[0], expected_start); + all_offsets.push(offsets); + } + + // Verify all offsets + assert_eq!(all_offsets[0], vec![0, 3]); // [1, 2, 3] + assert_eq!(all_offsets[1], vec![3, 4]); // [4] + assert_eq!(all_offsets[2], vec![4, 6]); // [5, 6] + + assert!(!exporter.has_more()); + } + + /// Test empty arrays in list + #[test] + fn test_list_export_with_empty_arrays() { + // Data: [[], [1], [], [2, 3], []] + let elements = buffer![1i32, 2, 3].into_array(); + // offsets indicate starting position in elements array for each list + // sizes indicate how many elements in each list + // [], [1], [], [2, 3], [] + // ^ ^ ^ ^ ^ + // offset=0, offset=0, offset=1, offset=1, offset=3 + let offsets = buffer![0u32, 0, 1, 1, 3].into_array(); + let sizes = buffer![0u32, 1, 0, 2, 0].into_array(); + + let list_array = ListViewArray::new(elements, offsets, sizes, Validity::NonNullable); + + let mut exporter = + ListExporter::new(list_array.into_array()).expect("Failed to create exporter"); + + // Export all at once + let mut out_offsets = vec![0u64; 6]; + let rows = exporter + .export_offsets(out_offsets.as_mut_ptr(), 5) + .expect("export offsets"); + + assert_eq!(rows, 5); + // Cumulative offsets: [0, 0, 1, 1, 3, 3] + // [], [1], [], [2, 3], [] + assert_eq!(out_offsets, vec![0, 0, 1, 1, 3, 3]); + } +} diff --git a/vortex-clickhouse/src/exporter/mod.rs b/vortex-clickhouse/src/exporter/mod.rs new file mode 100644 index 00000000000..5046d33de64 --- /dev/null +++ b/vortex-clickhouse/src/exporter/mod.rs @@ -0,0 +1,329 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Vortex Array exporter for ClickHouse. +//! +//! This module provides functionality to export Vortex arrays to ClickHouse's +//! column format. It handles different array encodings and attempts to use +//! zero-copy paths when possible. + +pub mod bigint; +pub mod bool_; +pub mod decimal; +pub mod list; +pub mod primitive; +pub mod struct_; +pub mod varbinview; + +use std::any::Any; + +use vortex::array::ArrayRef; +use vortex::dtype::{DType, PType}; +use vortex::error::{VortexResult, vortex_bail}; +use vortex::mask::Mask; + +/// Type tag identifying the concrete exporter kind. +#[repr(u8)] +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum ExporterKind { + /// Primitive numeric types (i8..i64, u8..u64, f32, f64) + Primitive = 0, + /// Utf8/Binary string types + String = 1, + /// Struct/Tuple types + Struct = 2, + /// List/Array types + List = 3, + /// Big integer types (Int128, UInt128, Int256, UInt256) + BigInt = 4, + /// Boolean type + Bool = 5, + /// Decimal types + Decimal = 6, +} + +pub use self::bigint::BigIntExporter; +pub use self::bool_::BoolExporter; +pub use self::decimal::DecimalExporter; +pub use self::list::ListExporter; +pub use self::primitive::PrimitiveExporter; +pub use self::struct_::StructExporter; +pub use self::varbinview::VarBinViewExporter; + +/// Write a validity bitmap from a `Mask` into a caller-provided byte buffer. +/// +/// The buffer is zeroed first so that trailing bits in the last byte are +/// always 0 (= null), then only the "valid" bits are set. This avoids +/// leaking uninitialized memory to the C++ consumer. +/// +/// # Arguments +/// * `bitmap` - Mutable byte slice of at least `(count + 7) / 8` bytes. +/// * `validity` - The Vortex validity mask to read from. +/// * `start` - Starting row index in the validity mask. +/// * `count` - Number of rows to export. +pub(crate) fn write_validity_bitmap( + bitmap: &mut [u8], + validity: &Mask, + start: usize, + count: usize, +) { + // Zero the buffer so trailing bits in the last byte are deterministic. + bitmap.fill(0); + + for i in 0..count { + if validity.value(start + i) { + bitmap[i / 8] |= 1 << (i % 8); + } + } +} + +/// Shared `export_validity` implementation for exporters that cache their +/// validity mask and track `last_export_start` / `last_export_count`. +/// +/// This covers `PrimitiveExporter`, `VarBinViewExporter`, `BigIntExporter`, +/// `BoolExporter`, and `DecimalExporter`. +pub(crate) fn export_validity_cached( + bitmap: *mut u8, + max_rows: usize, + validity: Option<&Mask>, + last_export_start: usize, + last_export_count: usize, +) -> VortexResult { + if bitmap.is_null() { + vortex_bail!("bitmap is null"); + } + + let rows_to_export = last_export_count.min(max_rows); + + if rows_to_export == 0 { + return Ok(0); + } + + let validity = match validity { + Some(v) => v, + None => vortex_bail!("export_validity called on non-nullable exporter"), + }; + + let bitmap_slice = + unsafe { std::slice::from_raw_parts_mut(bitmap, rows_to_export.div_ceil(8)) }; + + write_validity_bitmap(bitmap_slice, validity, last_export_start, rows_to_export); + + Ok(rows_to_export) +} + +/// Trait for exporting Vortex arrays to ClickHouse columns. +pub trait ColumnExporter: Send { + /// Return the kind tag for this exporter. + fn kind(&self) -> ExporterKind; + + /// Export array data to the ClickHouse column buffer. + /// + /// # Arguments + /// * `column_ptr` - Pointer to the output buffer. + /// * `buffer_size_bytes` - Total size of the output buffer in bytes. The + /// implementation must verify that the buffer is large enough before + /// writing and return an error otherwise. + /// * `max_rows` - Maximum number of rows to export. + /// + /// Returns the number of rows exported. + fn export( + &mut self, + column_ptr: *mut std::ffi::c_void, + buffer_size_bytes: usize, + max_rows: usize, + ) -> VortexResult; + + /// Return the number of bytes each row occupies in the export buffer. + /// + /// For fixed-width types this is the element width (e.g. 4 for `i32`). + /// Variable-length exporters that do not use `export` (strings, lists) + /// return 0 to indicate that the caller should use a specialised export + /// path instead. + fn element_size_bytes(&self) -> usize { + 0 + } + + /// Check if there is more data to export. + fn has_more(&self) -> bool; + + /// Get the total number of rows in this exporter. + fn len(&self) -> usize; + + /// Check if this exporter handles nullable data. + fn is_nullable(&self) -> bool { + false + } + + /// Export validity bitmap for nullable columns. + /// + /// Returns the number of rows processed. + fn export_validity(&mut self, _bitmap: *mut u8, _max_rows: usize) -> VortexResult { + vortex_bail!("export_validity not supported for this exporter") + } + + /// Export string data (for string/binary exporters). + /// + /// Returns the number of rows exported. + fn export_strings( + &mut self, + _data: *mut u8, + _lengths: *mut u32, + _offsets: *mut u64, + _max_rows: usize, + ) -> VortexResult { + vortex_bail!("export_strings not supported for this exporter") + } + + /// Get the total size of string data for the remaining rows. + /// + /// This is useful for pre-allocating buffers on the C++ side. + /// Returns (total_bytes, num_rows_remaining). + fn string_data_size(&self) -> VortexResult<(usize, usize)> { + vortex_bail!("string_data_size not supported for this exporter") + } + + /// Get as Any for downcasting. + fn as_any(&self) -> &dyn Any; + + /// Get as Any mut for downcasting. + fn as_any_mut(&mut self) -> &mut dyn Any; +} + +/// Create a new column exporter for the given Vortex array. +/// +/// This factory function inspects the array's dtype and creates the appropriate +/// exporter implementation: +/// - Primitive types (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64) → PrimitiveExporter +/// - Utf8/Binary types → VarBinViewExporter +/// - Struct types → StructExporter +/// - List types → ListExporter +/// - FixedSizeList → BigIntExporter (for Int128/UInt128/Int256/UInt256) +/// - Extension types → Exporter based on storage dtype +pub fn new_exporter(array: ArrayRef) -> VortexResult> { + match array.dtype() { + DType::Primitive(_, _) => Ok(Box::new(PrimitiveExporter::new(array)?)), + DType::Utf8(_) | DType::Binary(_) => Ok(Box::new(VarBinViewExporter::new(array)?)), + DType::Struct(_, _) => Ok(Box::new(StructExporter::new(array)?)), + DType::List(_, _) => Ok(Box::new(ListExporter::new(array)?)), + DType::FixedSizeList(elem_dtype, size, _) => { + // Check if this is a big integer type (FixedSizeList) + if matches!(elem_dtype.as_ref(), DType::Primitive(PType::U8, _)) + && (*size == 16 || *size == 32) + { + Ok(Box::new(BigIntExporter::new(array)?)) + } else { + Ok(Box::new(ListExporter::new(array)?)) + } + } + DType::Bool(_) => Ok(Box::new(BoolExporter::new(array)?)), + DType::Null => { + vortex_bail!("Null type arrays not supported in exporter") + } + DType::Extension(_) => { + // For extension types, extract the storage array and recurse. + // ExtensionArray wraps a storage array with the same data but + // typed as the storage dtype (e.g., Utf8, Primitive, etc.) + use vortex::array::ToCanonical; + use vortex::array::arrays::ExtensionArray; + let ext_array = array.to_extension(); + let storage = ext_array.storage().clone(); + new_exporter(storage) + } + DType::Decimal(_, _) => { + // Decimal types have their own dedicated exporter + Ok(Box::new(DecimalExporter::new(array)?)) + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use std::sync::Arc; + use vortex::array::IntoArray; + use vortex::array::arrays::{PrimitiveArray, StructArray, VarBinViewArray}; + use vortex::array::validity::Validity; + use vortex::buffer::Buffer; + use vortex::dtype::FieldNames; + + #[test] + fn test_new_exporter_primitive() { + let buffer: Buffer = vec![1i64, 2, 3].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let exporter = new_exporter(array).expect("Failed to create exporter"); + assert!(exporter.has_more()); + } + + #[test] + fn test_new_exporter_string() { + let array = VarBinViewArray::from_iter_str(vec!["hello", "world"]).into_array(); + + let exporter = new_exporter(array).expect("Failed to create exporter"); + assert!(exporter.has_more()); + } + + #[test] + fn test_primitive_export() { + let buffer: Buffer = vec![10i32, 20, 30, 40, 50].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let mut exporter = PrimitiveExporter::new(array).expect("Failed to create exporter"); + + // Allocate output buffer + let mut output = vec![0i32; 3]; + + // Export first 3 elements + let exported = exporter + .export( + output.as_mut_ptr() as *mut std::ffi::c_void, + size_of_val(output.as_slice()), + 3, + ) + .expect("Export failed"); + + assert_eq!(exported, 3); + assert_eq!(output, vec![10, 20, 30]); + assert!(exporter.has_more()); + + // Export remaining 2 elements + let mut output2 = vec![0i32; 3]; + let exported2 = exporter + .export( + output2.as_mut_ptr() as *mut std::ffi::c_void, + size_of_val(output2.as_slice()), + 3, + ) + .expect("Export failed"); + + assert_eq!(exported2, 2); + assert_eq!(output2[0..2], vec![40, 50]); + assert!(!exporter.has_more()); + } + + #[test] + fn test_struct_exporter_creation() { + let id_array = { + let buffer: Buffer = vec![1i64, 2, 3].into(); + PrimitiveArray::new(buffer, Validity::NonNullable).into_array() + }; + + let name_array = + VarBinViewArray::from_iter_str(vec!["Alice", "Bob", "Charlie"]).into_array(); + + let field_names: Vec> = vec![Arc::from("id"), Arc::from("name")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![id_array, name_array], + 3, + Validity::NonNullable, + ) + .expect("Failed to create struct array"); + + let exporter = StructExporter::new(struct_array.into_array()) + .expect("Failed to create struct exporter"); + + assert_eq!(exporter.num_fields(), 2); + } +} diff --git a/vortex-clickhouse/src/exporter/primitive.rs b/vortex-clickhouse/src/exporter/primitive.rs new file mode 100644 index 00000000000..3fb2ac556ff --- /dev/null +++ b/vortex-clickhouse/src/exporter/primitive.rs @@ -0,0 +1,175 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Primitive array exporter for ClickHouse. +//! +//! This module exports Vortex primitive arrays (numeric types) to ClickHouse column buffers. +//! It attempts to use zero-copy paths when possible by directly copying the underlying buffer. + +use std::any::Any; +use std::ffi::c_void; + +use vortex::array::arrays::PrimitiveArray; +use vortex::array::{Array, ArrayRef, ToCanonical}; +use vortex::dtype::{Nullability, PType}; +use vortex::error::{VortexResult, vortex_bail}; +use vortex::mask::Mask; + +use super::{ColumnExporter, ExporterKind}; + +/// Exporter for primitive (numeric) arrays. +pub struct PrimitiveExporter { + /// Cached canonical primitive array + canonical: PrimitiveArray, + /// Cached validity mask (None = non-nullable) + validity: Option, + /// Current export position + position: usize, + /// Position at the start of last export (for validity export) + last_export_start: usize, + /// Number of rows exported in last export call + last_export_count: usize, + /// Total length of the array + len: usize, + /// Primitive type + ptype: PType, + /// Whether the array is nullable + nullable: bool, +} + +impl PrimitiveExporter { + /// Create a new primitive exporter for the given array. + pub fn new(array: ArrayRef) -> VortexResult { + let len = array.len(); + let (ptype, nullable) = match array.dtype() { + vortex::dtype::DType::Primitive(ptype, nullability) => { + (*ptype, *nullability == Nullability::Nullable) + } + _ => vortex_bail!("PrimitiveExporter requires a primitive array"), + }; + + let canonical = array.to_primitive(); + let validity = if nullable { + Some(canonical.validity_mask()?) + } else { + None + }; + + Ok(Self { + canonical, + validity, + position: 0, + last_export_start: 0, + last_export_count: 0, + len, + ptype, + nullable, + }) + } +} + +impl ColumnExporter for PrimitiveExporter { + fn kind(&self) -> ExporterKind { + ExporterKind::Primitive + } + + fn export( + &mut self, + column_ptr: *mut c_void, + buffer_size_bytes: usize, + max_rows: usize, + ) -> VortexResult { + if column_ptr.is_null() { + vortex_bail!("column_ptr is null"); + } + + let remaining = self.len - self.position; + let rows_to_export = remaining.min(max_rows); + + if rows_to_export == 0 { + return Ok(0); + } + + let required_bytes = rows_to_export * self.ptype.byte_width(); + if buffer_size_bytes < required_bytes { + vortex_bail!( + "buffer too small: need {} bytes for {} rows of {:?}, got {}", + required_bytes, + rows_to_export, + self.ptype, + buffer_size_bytes + ); + } + + // Record the start position for validity export + self.last_export_start = self.position; + self.last_export_count = rows_to_export; + + // Export based on primitive type using macro + macro_rules! export_primitive { + ($ptype:ident, $rust_ty:ty) => {{ + let buffer = self.canonical.as_slice::<$rust_ty>(); + let start = self.position; + let end = start + rows_to_export; + let slice = &buffer[start..end]; + + // Copy to destination + let dst = column_ptr as *mut $rust_ty; + unsafe { + std::ptr::copy_nonoverlapping(slice.as_ptr(), dst, rows_to_export); + } + }}; + } + + match self.ptype { + PType::I8 => export_primitive!(I8, i8), + PType::I16 => export_primitive!(I16, i16), + PType::I32 => export_primitive!(I32, i32), + PType::I64 => export_primitive!(I64, i64), + PType::U8 => export_primitive!(U8, u8), + PType::U16 => export_primitive!(U16, u16), + PType::U32 => export_primitive!(U32, u32), + PType::U64 => export_primitive!(U64, u64), + PType::F32 => export_primitive!(F32, f32), + PType::F64 => export_primitive!(F64, f64), + PType::F16 => vortex_bail!("F16 export not supported"), + } + + self.position += rows_to_export; + Ok(rows_to_export) + } + + fn element_size_bytes(&self) -> usize { + self.ptype.byte_width() + } + + fn has_more(&self) -> bool { + self.position < self.len + } + + fn len(&self) -> usize { + self.len + } + + fn is_nullable(&self) -> bool { + self.nullable + } + + fn export_validity(&mut self, bitmap: *mut u8, max_rows: usize) -> VortexResult { + super::export_validity_cached( + bitmap, + max_rows, + self.validity.as_ref(), + self.last_export_start, + self.last_export_count, + ) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn as_any_mut(&mut self) -> &mut dyn Any { + self + } +} diff --git a/vortex-clickhouse/src/exporter/struct_.rs b/vortex-clickhouse/src/exporter/struct_.rs new file mode 100644 index 00000000000..f3f982d4427 --- /dev/null +++ b/vortex-clickhouse/src/exporter/struct_.rs @@ -0,0 +1,122 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Struct array exporter for ClickHouse. +//! +//! This module exports Vortex struct arrays (tuples/records) to ClickHouse. +//! It recursively exports each field using the appropriate exporter. + +use std::any::Any; +use std::ffi::c_void; + +use vortex::array::{Array, ArrayRef, ToCanonical}; +use vortex::error::{VortexResult, vortex_bail}; + +use super::{ColumnExporter, ExporterKind, new_exporter}; + +/// Exporter for struct (tuple) arrays. +pub struct StructExporter { + /// Child exporters for each field + field_exporters: Vec>>, + /// Total length of the struct array + len: usize, + /// Whether all fields have been exported + done: bool, +} + +impl StructExporter { + /// Create a new struct exporter for the given array. + pub fn new(array: ArrayRef) -> VortexResult { + // Verify this is a struct type + match array.dtype() { + vortex::dtype::DType::Struct(_, _) => {} + _ => vortex_bail!("StructExporter requires a Struct array"), + } + + let len = array.len(); + + // Get struct array and create exporters for each field + let struct_array = array.to_struct(); + let mut field_exporters = Vec::new(); + + for field in struct_array.unmasked_fields().iter() { + let exporter = new_exporter(field.clone())?; + field_exporters.push(Some(exporter)); + } + + Ok(Self { + field_exporters, + len, + done: false, + }) + } + + /// Get the number of fields in the struct. + pub fn num_fields(&self) -> usize { + self.field_exporters.len() + } + + /// Get the field exporter at the given index (borrow). + pub fn field_exporter(&mut self, index: usize) -> Option<&mut Box> { + self.field_exporters + .get_mut(index) + .and_then(|opt| opt.as_mut()) + } + + /// Take the field exporter at the given index (ownership transfer). + /// Returns None if index is out of bounds or already taken. + pub fn take_field_exporter(&mut self, index: usize) -> Option> { + self.field_exporters + .get_mut(index) + .and_then(|opt| opt.take()) + } + + /// Get the length of the struct array. + pub fn len(&self) -> usize { + self.len + } + + /// Check if the struct array is empty. + pub fn is_empty(&self) -> bool { + self.len == 0 + } +} + +impl ColumnExporter for StructExporter { + fn kind(&self) -> ExporterKind { + ExporterKind::Struct + } + + fn export( + &mut self, + _column_ptr: *mut c_void, + _buffer_size_bytes: usize, + _max_rows: usize, + ) -> VortexResult { + // Struct export requires exporting each field separately + // The caller should iterate over fields using field_exporter() + vortex_bail!( + "StructExporter::export() not supported. Use field_exporter() to export individual fields." + ) + } + + fn has_more(&self) -> bool { + !self.done + && self + .field_exporters + .iter() + .any(|e| e.as_ref().map_or(false, |exp| exp.has_more())) + } + + fn len(&self) -> usize { + self.len + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn as_any_mut(&mut self) -> &mut dyn Any { + self + } +} diff --git a/vortex-clickhouse/src/exporter/varbinview.rs b/vortex-clickhouse/src/exporter/varbinview.rs new file mode 100644 index 00000000000..d2951a1aec8 --- /dev/null +++ b/vortex-clickhouse/src/exporter/varbinview.rs @@ -0,0 +1,289 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! VarBinView array exporter for ClickHouse (String type). +//! +//! This module exports Vortex VarBinView arrays (string/binary types) to ClickHouse column buffers. +//! It uses zero-copy `bytes_at` for direct buffer access instead of scalar extraction. + +use std::any::Any; +use std::cell::Cell; +use std::ffi::c_void; + +use vortex::array::arrays::VarBinViewArray; +use vortex::array::{Array, ArrayRef, ToCanonical}; +use vortex::dtype::Nullability; +use vortex::error::{VortexResult, vortex_bail}; +use vortex::mask::Mask; + +use super::{ColumnExporter, ExporterKind}; + +/// Callback function type for appending strings to ClickHouse column. +/// +/// The C++ side provides this callback which takes: +/// - column_ptr: Pointer to the ClickHouse column +/// - data: Pointer to string data +/// - len: Length of the string +pub type StringAppendFn = extern "C" fn(*mut c_void, *const u8, usize); + +/// Exporter for variable-length binary/string arrays. +pub struct VarBinViewExporter { + /// Cached canonical VarBinView array + canonical: VarBinViewArray, + /// Cached validity mask (None = non-nullable) + validity: Option, + /// Current export position + position: usize, + /// Position at the start of last export (for validity export) + last_export_start: usize, + /// Number of rows exported in last export call + last_export_count: usize, + /// Total length of the array + len: usize, + /// Whether the array is nullable + nullable: bool, + /// Cached total data size for remaining rows (lazily computed). + /// + /// Uses `Cell` for interior mutability so the `&self` trait method + /// `string_data_size` can populate the cache without `&mut self`. + cached_data_size: Cell>, +} + +impl VarBinViewExporter { + /// Create a new varbinview exporter for the given array. + pub fn new(array: ArrayRef) -> VortexResult { + let len = array.len(); + + // Verify this is a string or binary type + let nullable = match array.dtype() { + vortex::dtype::DType::Utf8(nullability) => *nullability == Nullability::Nullable, + vortex::dtype::DType::Binary(nullability) => *nullability == Nullability::Nullable, + _ => vortex_bail!("VarBinViewExporter requires a Utf8 or Binary array"), + }; + + let canonical = array.to_varbinview(); + let validity = if nullable { + Some(canonical.validity_mask()?) + } else { + None + }; + + Ok(Self { + canonical, + validity, + position: 0, + last_export_start: 0, + last_export_count: 0, + len, + nullable, + cached_data_size: Cell::new(None), + }) + } + + /// Check if the value at the given index is valid (non-null). + fn is_valid(&self, idx: usize) -> bool { + match &self.validity { + Some(mask) => mask.value(idx), + None => true, + } + } + + /// Export strings using a callback function. + /// + /// This is the preferred method for exporting strings to ClickHouse, + /// as it allows the C++ side to handle memory allocation. + pub fn export_with_callback( + &mut self, + column_ptr: *mut c_void, + max_rows: usize, + append_fn: StringAppendFn, + ) -> VortexResult { + if column_ptr.is_null() { + vortex_bail!("column_ptr is null"); + } + + let remaining = self.len - self.position; + let rows_to_export = remaining.min(max_rows); + + if rows_to_export == 0 { + return Ok(0); + } + + for i in self.position..(self.position + rows_to_export) { + if self.is_valid(i) { + let bytes = self.canonical.bytes_at(i); + let slice = bytes.as_ref(); + append_fn(column_ptr, slice.as_ptr(), slice.len()); + } else { + append_fn(column_ptr, std::ptr::null(), 0); + } + } + + self.position += rows_to_export; + // Invalidate cached data size since position changed + self.cached_data_size.set(None); + Ok(rows_to_export) + } + + /// Calculate the total size of string data for the remaining rows. + /// + /// This method caches the result for efficiency. + pub fn compute_total_data_size(&mut self) -> VortexResult { + // Return cached value if available + if let Some(size) = self.cached_data_size.get() { + return Ok(size); + } + + let remaining = self.len - self.position; + if remaining == 0 { + self.cached_data_size.set(Some(0)); + return Ok(0); + } + + let views = self.canonical.views(); + let mut total_size = 0usize; + + for i in self.position..self.len { + if self.is_valid(i) { + total_size += views[i].len() as usize; + } + } + + self.cached_data_size.set(Some(total_size)); + Ok(total_size) + } +} + +impl ColumnExporter for VarBinViewExporter { + fn kind(&self) -> ExporterKind { + ExporterKind::String + } + + fn export( + &mut self, + _column_ptr: *mut c_void, + _buffer_size_bytes: usize, + _max_rows: usize, + ) -> VortexResult { + // The default export method is not suitable for strings + // because ClickHouse strings need special handling. + // Use export_with_callback or export_strings instead. + vortex_bail!("VarBinViewExporter::export() not supported. Use export_strings() instead.") + } + + fn has_more(&self) -> bool { + self.position < self.len + } + + fn len(&self) -> usize { + self.len + } + + fn is_nullable(&self) -> bool { + self.nullable + } + + fn export_strings( + &mut self, + data: *mut u8, + lengths: *mut u32, + offsets: *mut u64, + max_rows: usize, + ) -> VortexResult { + if data.is_null() || lengths.is_null() || offsets.is_null() { + vortex_bail!("Buffer pointers cannot be null"); + } + + let remaining = self.len - self.position; + let rows_to_export = remaining.min(max_rows); + + if rows_to_export == 0 { + return Ok(0); + } + + // Record the start position for validity export + self.last_export_start = self.position; + self.last_export_count = rows_to_export; + + let mut current_offset: u64 = 0; + + for i in 0..rows_to_export { + let idx = self.position + i; + + // Write offset + unsafe { + *offsets.add(i) = current_offset; + } + + if self.is_valid(idx) { + let bytes = self.canonical.bytes_at(idx); + let slice = bytes.as_ref(); + let len = slice.len(); + + // Write string/binary data + unsafe { + std::ptr::copy_nonoverlapping( + slice.as_ptr(), + data.add(current_offset as usize), + len, + ); + *lengths.add(i) = len as u32; + } + + current_offset += len as u64; + } else { + unsafe { + *lengths.add(i) = 0; + } + } + } + + self.position += rows_to_export; + // Invalidate cached data size since position changed + self.cached_data_size.set(None); + Ok(rows_to_export) + } + + fn export_validity(&mut self, bitmap: *mut u8, max_rows: usize) -> VortexResult { + super::export_validity_cached( + bitmap, + max_rows, + self.validity.as_ref(), + self.last_export_start, + self.last_export_count, + ) + } + + fn string_data_size(&self) -> VortexResult<(usize, usize)> { + let remaining = self.len - self.position; + if remaining == 0 { + return Ok((0, 0)); + } + + // If we have cached value, use it + if let Some(size) = self.cached_data_size.get() { + return Ok((size, remaining)); + } + + // Calculate total data size from views (no scalar extraction) + let views = self.canonical.views(); + let mut total_size = 0usize; + + for i in self.position..self.len { + if self.is_valid(i) { + total_size += views[i].len() as usize; + } + } + + self.cached_data_size.set(Some(total_size)); + Ok((total_size, remaining)) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn as_any_mut(&mut self) -> &mut dyn Any { + self + } +} diff --git a/vortex-clickhouse/src/ext_types/bigint.rs b/vortex-clickhouse/src/ext_types/bigint.rs new file mode 100644 index 00000000000..936a5e1beef --- /dev/null +++ b/vortex-clickhouse/src/ext_types/bigint.rs @@ -0,0 +1,376 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! BigInt Extension Type for ClickHouse Int128/UInt128/Int256/UInt256. +//! +//! ClickHouse supports large integer types that don't have direct equivalents +//! in Vortex's primitive types. This extension type stores them as fixed-size +//! byte arrays while preserving the semantic type information. +//! +//! # Storage Layout +//! - Int128/UInt128: 16 bytes per value (little-endian) +//! - Int256/UInt256: 32 bytes per value (little-endian) +//! +//! # Example +//! ```ignore +//! use vortex_clickhouse::ext_types::{BigInt, BigIntType, BigIntMetadata}; +//! use vortex::dtype::{DType, ExtDType, Nullability}; +//! +//! let dtype = BigInt::dtype(BigIntType::Int128, Nullability::Nullable); +//! ``` + +use std::fmt::{Debug, Display, Formatter}; +use std::hash::Hash; + +use vortex::dtype::extension::{ExtDType, ExtDTypeVTable, ExtID}; +use vortex::dtype::{DType, Nullability, PType}; +use vortex::error::{VortexResult, vortex_bail}; + +/// The extension type ID for ClickHouse BigInt types. +pub const BIGINT_EXT_ID: &str = "clickhouse.bigint"; + +/// The concrete BigInt types supported. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[repr(u8)] +pub enum BigIntType { + /// Signed 128-bit integer (16 bytes) + Int128 = 0, + /// Unsigned 128-bit integer (16 bytes) + UInt128 = 1, + /// Signed 256-bit integer (32 bytes) + Int256 = 2, + /// Unsigned 256-bit integer (32 bytes) + UInt256 = 3, +} + +impl BigIntType { + /// Returns the byte size for this BigInt type. + pub const fn byte_size(&self) -> usize { + match self { + BigIntType::Int128 | BigIntType::UInt128 => 16, + BigIntType::Int256 | BigIntType::UInt256 => 32, + } + } + + /// Returns true if this is a signed type. + pub const fn is_signed(&self) -> bool { + matches!(self, BigIntType::Int128 | BigIntType::Int256) + } + + /// Returns the ClickHouse type name. + pub const fn clickhouse_type_name(&self) -> &'static str { + match self { + BigIntType::Int128 => "Int128", + BigIntType::UInt128 => "UInt128", + BigIntType::Int256 => "Int256", + BigIntType::UInt256 => "UInt256", + } + } + + /// Parse from ClickHouse type name. + pub fn from_clickhouse_type(name: &str) -> Option { + match name { + "Int128" => Some(BigIntType::Int128), + "UInt128" => Some(BigIntType::UInt128), + "Int256" => Some(BigIntType::Int256), + "UInt256" => Some(BigIntType::UInt256), + _ => None, + } + } +} + +impl TryFrom for BigIntType { + type Error = vortex::error::VortexError; + + fn try_from(value: u8) -> Result { + match value { + 0 => Ok(BigIntType::Int128), + 1 => Ok(BigIntType::UInt128), + 2 => Ok(BigIntType::Int256), + 3 => Ok(BigIntType::UInt256), + _ => vortex_bail!("Invalid BigIntType tag: {}", value), + } + } +} + +impl Display for BigIntType { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.clickhouse_type_name()) + } +} + +/// Metadata for BigInt extension type. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +pub struct BigIntMetadata { + /// The specific BigInt type. + pub bigint_type: BigIntType, +} + +impl BigIntMetadata { + /// Create new BigInt metadata. + pub fn new(bigint_type: BigIntType) -> Self { + Self { bigint_type } + } +} + +impl Display for BigIntMetadata { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.bigint_type) + } +} + +/// The BigInt extension type VTable. +#[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] +pub struct BigInt; + +impl BigInt { + /// Create a new BigInt extension dtype. + pub fn new(bigint_type: BigIntType, nullability: Nullability) -> ExtDType { + let metadata = BigIntMetadata::new(bigint_type); + // Storage: Primitive u8 array (will be wrapped in FixedSizeList by the array) + // For extension types, we use Primitive(U8) as the storage per element + let storage_dtype = DType::Primitive(PType::U8, Nullability::NonNullable); + ExtDType::try_with_vtable(Self, metadata, storage_dtype.with_nullability(nullability)) + .expect("BigInt storage dtype is always valid") + } + + /// Create a BigInt DType (type-erased). + pub fn dtype(bigint_type: BigIntType, nullability: Nullability) -> DType { + DType::Extension(Self::new(bigint_type, nullability).erased()) + } + + /// Check if a DType is a BigInt extension type. + pub fn is_bigint(dtype: &DType) -> bool { + if let DType::Extension(ext) = dtype { + ext.id().as_ref() == BIGINT_EXT_ID + } else { + false + } + } + + /// Try to extract BigIntType from a DType. + pub fn try_get_type(dtype: &DType) -> Option { + if let DType::Extension(ext) = dtype { + if ext.id().as_ref() == BIGINT_EXT_ID { + // Use the Matcher trait to get metadata + ext.metadata_opt::().map(|m| m.bigint_type) + } else { + None + } + } else { + None + } + } +} + +impl ExtDTypeVTable for BigInt { + type Metadata = BigIntMetadata; + + fn id(&self) -> ExtID { + ExtID::new_ref(BIGINT_EXT_ID) + } + + fn serialize(&self, metadata: &Self::Metadata) -> VortexResult> { + Ok(vec![metadata.bigint_type as u8]) + } + + fn deserialize(&self, data: &[u8]) -> VortexResult { + if data.is_empty() { + vortex_bail!("BigInt metadata is empty"); + } + let bigint_type = BigIntType::try_from(data[0])?; + Ok(BigIntMetadata::new(bigint_type)) + } + + fn validate_dtype( + &self, + _metadata: &Self::Metadata, + storage_dtype: &DType, + ) -> VortexResult<()> { + // Storage should be Primitive(U8) + match storage_dtype { + DType::Primitive(PType::U8, _) => Ok(()), + _ => vortex_bail!( + "BigInt extension requires Primitive(U8) storage, got {:?}", + storage_dtype + ), + } + } +} + +/// Helper trait for ExtDTypeRef to access BigInt metadata. +pub trait BigIntExt { + /// Get the BigIntType if this is a BigInt extension. + fn bigint_type(&self) -> Option; + + /// Get the ClickHouse type name if this is a BigInt extension. + fn bigint_clickhouse_type(&self) -> Option<&'static str>; +} + +impl BigIntExt for vortex::dtype::extension::ExtDTypeRef { + fn bigint_type(&self) -> Option { + if self.id().as_ref() == BIGINT_EXT_ID { + // Use the Matcher trait to get metadata + self.metadata_opt::().map(|m| m.bigint_type) + } else { + None + } + } + + fn bigint_clickhouse_type(&self) -> Option<&'static str> { + self.bigint_type().map(|t| t.clickhouse_type_name()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_bigint_type_roundtrip() { + for bigint_type in [ + BigIntType::Int128, + BigIntType::UInt128, + BigIntType::Int256, + BigIntType::UInt256, + ] { + let tag = bigint_type as u8; + let roundtrip = BigIntType::try_from(tag).unwrap(); + assert_eq!(bigint_type, roundtrip); + } + } + + #[test] + fn test_bigint_byte_size() { + assert_eq!(BigIntType::Int128.byte_size(), 16); + assert_eq!(BigIntType::UInt128.byte_size(), 16); + assert_eq!(BigIntType::Int256.byte_size(), 32); + assert_eq!(BigIntType::UInt256.byte_size(), 32); + } + + #[test] + fn test_bigint_dtype_creation() { + let dtype = BigInt::dtype(BigIntType::Int128, Nullability::Nullable); + assert!(BigInt::is_bigint(&dtype)); + + if let DType::Extension(ext) = &dtype { + assert_eq!(ext.id().as_ref(), BIGINT_EXT_ID); + } else { + panic!("Expected Extension dtype"); + } + } + + #[test] + fn test_clickhouse_type_names() { + assert_eq!(BigIntType::Int128.clickhouse_type_name(), "Int128"); + assert_eq!(BigIntType::UInt128.clickhouse_type_name(), "UInt128"); + assert_eq!(BigIntType::Int256.clickhouse_type_name(), "Int256"); + assert_eq!(BigIntType::UInt256.clickhouse_type_name(), "UInt256"); + } + + #[test] + fn test_from_clickhouse_type() { + assert_eq!( + BigIntType::from_clickhouse_type("Int128"), + Some(BigIntType::Int128) + ); + assert_eq!( + BigIntType::from_clickhouse_type("UInt256"), + Some(BigIntType::UInt256) + ); + assert_eq!(BigIntType::from_clickhouse_type("String"), None); + } + + #[test] + fn test_bigint_file_roundtrip() { + use std::io::Write; + use std::sync::Arc; + use tempfile::NamedTempFile; + use vortex::array::IntoArray; + use vortex::array::arrays::{FixedSizeListArray, PrimitiveArray, StructArray}; + use vortex::array::stream::ArrayStreamExt; + use vortex::array::validity::Validity; + use vortex::buffer::Buffer; + use vortex::dtype::FieldNames; + use vortex::file::{OpenOptionsSessionExt, WriteOptionsSessionExt}; + use vortex::io::runtime::BlockingRuntime; + + use crate::{RUNTIME, SESSION}; + + // Create test data - 2 Int128 values as FixedSizeList + let bytes: Vec = vec![ + // First Int128 value: 100 (little-endian) + 100, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + // Second Int128 value: 200 (little-endian) + 200, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + ]; + + // Create FixedSizeList array for Int128 (16 bytes per element) + let values = PrimitiveArray::new(Buffer::::from(bytes.clone()), Validity::NonNullable); + let fsl_array = FixedSizeListArray::try_new( + values.into_array(), + 16, // element size in bytes + Validity::NonNullable, + 2, // number of elements + ) + .expect("Failed to create FixedSizeList"); + + // Wrap in struct + let field_names: Vec> = vec![Arc::from("val")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![fsl_array.into_array()], + 2, + Validity::NonNullable, + ) + .expect("Failed to create struct"); + + // Write to temp file + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_str().unwrap().to_string(); + + let mut buf = Vec::new(); + (*RUNTIME).block_on(async { + SESSION + .write_options() + .write(&mut buf, struct_array.to_array_stream()) + .await + .expect("Failed to write"); + }); + + // Write buffer to temp file + let mut file = std::fs::File::create(&path).expect("Failed to create file"); + file.write_all(&buf).expect("Failed to write to file"); + file.flush().expect("Failed to flush"); + drop(file); + + // Read back and verify dtype + let read_dtype = (*RUNTIME).block_on(async { + let vortex_file = SESSION + .open_options() + .open_path(&path) + .await + .expect("Failed to open"); + vortex_file.dtype().clone() + }); + + // Verify the dtype is a Struct with a FixedSizeList field + if let DType::Struct(fields, _) = &read_dtype { + let val_dtype = fields.fields().next().expect("Expected field"); + // Check that it's FixedSizeList (represents Int128) + match &val_dtype { + DType::FixedSizeList(elem, size, _) => { + assert_eq!(*size, 16, "Expected size 16 for Int128"); + assert!( + matches!(elem.as_ref(), DType::Primitive(PType::U8, _)), + "Expected Primitive(U8), got {:?}", + elem + ); + } + _ => panic!("Expected FixedSizeList dtype, got {:?}", val_dtype), + } + } else { + panic!("Expected Struct dtype, got {:?}", read_dtype); + } + } +} diff --git a/vortex-clickhouse/src/ext_types/date.rs b/vortex-clickhouse/src/ext_types/date.rs new file mode 100644 index 00000000000..fd801ecca2f --- /dev/null +++ b/vortex-clickhouse/src/ext_types/date.rs @@ -0,0 +1,129 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! ClickHouse Date Extension Type. +//! +//! Preserves `Date` and `Date32` semantics in Vortex files. +//! +//! # Storage +//! - `Date`: `Primitive(U16)` — days since 1970-01-01 +//! - `Date32`: `Primitive(I32)` — days since 1970-01-01 (wider range) + +use std::fmt::{Display, Formatter}; +use std::hash::Hash; + +use vortex::dtype::extension::{ExtDType, ExtDTypeVTable, ExtID}; +use vortex::dtype::{DType, Nullability, PType}; +use vortex::error::{VortexResult, vortex_bail}; + +/// Extension type ID. +pub const DATE_EXT_ID: &str = "clickhouse.date"; + +/// Metadata for the date extension type. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +pub struct DateMetadata { + /// `false` for `Date` (U16), `true` for `Date32` (I32). + pub is_date32: bool, +} + +impl Display for DateMetadata { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + if self.is_date32 { + write!(f, "Date32") + } else { + write!(f, "Date") + } + } +} + +/// The Date extension type VTable. +#[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] +pub struct ClickHouseDate; + +impl ClickHouseDate { + /// Create a ClickHouse Date DType (type-erased). + pub fn dtype(is_date32: bool, nullability: Nullability) -> DType { + let storage = if is_date32 { + DType::Primitive(PType::I32, Nullability::NonNullable) + } else { + DType::Primitive(PType::U16, Nullability::NonNullable) + }; + let metadata = DateMetadata { is_date32 }; + let ext = ExtDType::try_with_vtable(Self, metadata, storage.with_nullability(nullability)) + .expect("Date storage dtype is always valid"); + DType::Extension(ext.erased()) + } + + /// Try to extract `DateMetadata` from a DType. + pub fn try_get_metadata(dtype: &DType) -> Option { + if let DType::Extension(ext) = dtype { + if ext.id().as_ref() == DATE_EXT_ID { + return ext.metadata_opt::().copied(); + } + } + None + } + + /// Reconstruct the ClickHouse type string. + pub fn to_clickhouse_type(metadata: &DateMetadata) -> String { + format!("{}", metadata) + } +} + +impl ExtDTypeVTable for ClickHouseDate { + type Metadata = DateMetadata; + + fn id(&self) -> ExtID { + ExtID::new_ref(DATE_EXT_ID) + } + + fn serialize(&self, metadata: &Self::Metadata) -> VortexResult> { + Ok(vec![metadata.is_date32 as u8]) + } + + fn deserialize(&self, data: &[u8]) -> VortexResult { + if data.is_empty() { + vortex_bail!("Date metadata is empty"); + } + Ok(DateMetadata { + is_date32: data[0] != 0, + }) + } + + fn validate_dtype(&self, metadata: &Self::Metadata, storage_dtype: &DType) -> VortexResult<()> { + if metadata.is_date32 { + match storage_dtype { + DType::Primitive(PType::I32, _) => Ok(()), + _ => vortex_bail!( + "Date32 requires Primitive(I32) storage, got {:?}", + storage_dtype + ), + } + } else { + match storage_dtype { + DType::Primitive(PType::U16, _) => Ok(()), + _ => vortex_bail!( + "Date requires Primitive(U16) storage, got {:?}", + storage_dtype + ), + } + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_date_metadata_roundtrip() -> VortexResult<()> { + let vtable = ClickHouseDate; + for is_date32 in [false, true] { + let metadata = DateMetadata { is_date32 }; + let serialized = vtable.serialize(&metadata)?; + let deserialized = vtable.deserialize(&serialized)?; + assert_eq!(metadata, deserialized); + } + Ok(()) + } +} diff --git a/vortex-clickhouse/src/ext_types/datetime.rs b/vortex-clickhouse/src/ext_types/datetime.rs new file mode 100644 index 00000000000..0b76e3e630a --- /dev/null +++ b/vortex-clickhouse/src/ext_types/datetime.rs @@ -0,0 +1,195 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! ClickHouse DateTime Extension Type. +//! +//! Preserves `DateTime`/`DateTime64` precision and timezone in Vortex files. +//! +//! # Storage +//! - `DateTime` (precision=0): `Primitive(U32)` — seconds since epoch +//! - `DateTime64(p)` (precision>0): `Primitive(I64)` — sub-second ticks since epoch + +use std::fmt::{Display, Formatter}; +use std::hash::Hash; + +use vortex::dtype::extension::{ExtDType, ExtDTypeVTable, ExtID}; +use vortex::dtype::{DType, Nullability, PType}; +use vortex::error::{VortexResult, vortex_bail}; + +/// Extension type ID. +pub const DATETIME_EXT_ID: &str = "clickhouse.datetime"; + +/// Metadata for the datetime extension type. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct DateTimeMetadata { + /// 0 for `DateTime`, 1–9 for `DateTime64(p)`. + pub precision: u8, + /// Optional timezone string, e.g. `"UTC"`. + pub timezone: Option, +} + +impl Display for DateTimeMetadata { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + if self.precision == 0 { + if let Some(tz) = &self.timezone { + write!(f, "DateTime('{}')", tz) + } else { + write!(f, "DateTime") + } + } else if let Some(tz) = &self.timezone { + write!(f, "DateTime64({}, '{}')", self.precision, tz) + } else { + write!(f, "DateTime64({})", self.precision) + } + } +} + +/// The DateTime extension type VTable. +#[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] +pub struct ClickHouseDateTime; + +impl ClickHouseDateTime { + /// Create a ClickHouse DateTime DType (type-erased). + pub fn dtype(metadata: DateTimeMetadata, nullability: Nullability) -> DType { + let storage = if metadata.precision == 0 { + DType::Primitive(PType::U32, Nullability::NonNullable) + } else { + DType::Primitive(PType::I64, Nullability::NonNullable) + }; + let ext = ExtDType::try_with_vtable(Self, metadata, storage.with_nullability(nullability)) + .expect("DateTime storage dtype is always valid"); + DType::Extension(ext.erased()) + } + + /// Try to extract `DateTimeMetadata` from a DType. + pub fn try_get_metadata(dtype: &DType) -> Option { + if let DType::Extension(ext) = dtype { + if ext.id().as_ref() == DATETIME_EXT_ID { + return ext.metadata_opt::().cloned(); + } + } + None + } + + /// Reconstruct the ClickHouse type string. + pub fn to_clickhouse_type(metadata: &DateTimeMetadata) -> String { + format!("{}", metadata) + } +} + +impl ExtDTypeVTable for ClickHouseDateTime { + type Metadata = DateTimeMetadata; + + fn id(&self) -> ExtID { + ExtID::new_ref(DATETIME_EXT_ID) + } + + fn serialize(&self, metadata: &Self::Metadata) -> VortexResult> { + // [precision: u8] [tz_len: u16] [tz_bytes...] + let mut buf = vec![metadata.precision]; + if let Some(tz) = &metadata.timezone { + let tz_bytes = tz.as_bytes(); + let tz_len = tz_bytes.len() as u16; + buf.extend_from_slice(&tz_len.to_le_bytes()); + buf.extend_from_slice(tz_bytes); + } else { + buf.extend_from_slice(&0u16.to_le_bytes()); + } + Ok(buf) + } + + fn deserialize(&self, data: &[u8]) -> VortexResult { + if data.len() < 3 { + vortex_bail!("DateTime metadata too short"); + } + let precision = data[0]; + let tz_len = u16::from_le_bytes([data[1], data[2]]) as usize; + let timezone = if tz_len > 0 { + if data.len() < 3 + tz_len { + vortex_bail!("Truncated DateTime metadata"); + } + Some(String::from_utf8_lossy(&data[3..3 + tz_len]).to_string()) + } else { + None + }; + Ok(DateTimeMetadata { + precision, + timezone, + }) + } + + fn validate_dtype(&self, metadata: &Self::Metadata, storage_dtype: &DType) -> VortexResult<()> { + if metadata.precision == 0 { + match storage_dtype { + DType::Primitive(PType::U32, _) => Ok(()), + _ => vortex_bail!( + "DateTime (precision 0) requires Primitive(U32) storage, got {:?}", + storage_dtype + ), + } + } else { + match storage_dtype { + DType::Primitive(PType::I64, _) => Ok(()), + _ => vortex_bail!( + "DateTime64 requires Primitive(I64) storage, got {:?}", + storage_dtype + ), + } + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_datetime_metadata_roundtrip() -> VortexResult<()> { + let vtable = ClickHouseDateTime; + let metadata = DateTimeMetadata { + precision: 3, + timezone: Some("UTC".to_string()), + }; + let serialized = vtable.serialize(&metadata)?; + let deserialized = vtable.deserialize(&serialized)?; + assert_eq!(metadata, deserialized); + Ok(()) + } + + #[test] + fn test_datetime_no_timezone() -> VortexResult<()> { + let vtable = ClickHouseDateTime; + let metadata = DateTimeMetadata { + precision: 0, + timezone: None, + }; + let serialized = vtable.serialize(&metadata)?; + let deserialized = vtable.deserialize(&serialized)?; + assert_eq!(metadata, deserialized); + Ok(()) + } + + #[test] + fn test_datetime_display() { + assert_eq!( + format!( + "{}", + DateTimeMetadata { + precision: 0, + timezone: None + } + ), + "DateTime" + ); + assert_eq!( + format!( + "{}", + DateTimeMetadata { + precision: 3, + timezone: Some("UTC".into()) + } + ), + "DateTime64(3, 'UTC')" + ); + } +} diff --git a/vortex-clickhouse/src/ext_types/enum_.rs b/vortex-clickhouse/src/ext_types/enum_.rs new file mode 100644 index 00000000000..95865396abb --- /dev/null +++ b/vortex-clickhouse/src/ext_types/enum_.rs @@ -0,0 +1,216 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! ClickHouse Enum Extension Type. +//! +//! Preserves `Enum8`/`Enum16` name→value mappings in Vortex files so they can be +//! reconstructed on read when `output_clickhouse_types` is enabled. +//! +//! # Storage +//! - `Enum8`: `Primitive(I8)` +//! - `Enum16`: `Primitive(I16)` + +use std::fmt::{Display, Formatter}; +use std::hash::Hash; + +use vortex::dtype::extension::{ExtDType, ExtDTypeVTable, ExtID}; +use vortex::dtype::{DType, Nullability, PType}; +use vortex::error::{VortexResult, vortex_bail}; + +/// Extension type ID. +pub const ENUM_EXT_ID: &str = "clickhouse.enum"; + +/// Whether this is an 8-bit or 16-bit enum. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[repr(u8)] +pub enum EnumSize { + Enum8 = 8, + Enum16 = 16, +} + +/// A single enum entry: `(name, value)`. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct EnumEntry { + pub name: String, + pub value: i16, +} + +/// Metadata for the enum extension type. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct EnumMetadata { + pub enum_size: EnumSize, + pub entries: Vec, +} + +impl Display for EnumMetadata { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "Enum{}(", self.enum_size as u8)?; + for (i, entry) in self.entries.iter().enumerate() { + if i > 0 { + write!(f, ", ")?; + } + write!(f, "'{}' = {}", entry.name, entry.value)?; + } + write!(f, ")") + } +} + +/// The Enum extension type VTable. +#[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] +pub struct ClickHouseEnum; + +impl ClickHouseEnum { + /// Create a ClickHouse Enum DType (type-erased). + pub fn dtype(metadata: EnumMetadata, nullability: Nullability) -> DType { + let storage = match metadata.enum_size { + EnumSize::Enum8 => DType::Primitive(PType::I8, Nullability::NonNullable), + EnumSize::Enum16 => DType::Primitive(PType::I16, Nullability::NonNullable), + }; + let ext = ExtDType::try_with_vtable(Self, metadata, storage.with_nullability(nullability)) + .expect("Enum storage dtype is always valid"); + DType::Extension(ext.erased()) + } + + /// Try to extract `EnumMetadata` from a DType. + pub fn try_get_metadata(dtype: &DType) -> Option { + if let DType::Extension(ext) = dtype { + if ext.id().as_ref() == ENUM_EXT_ID { + return ext.metadata_opt::().cloned(); + } + } + None + } + + /// Reconstruct the ClickHouse type string, e.g. `Enum8('a' = 1, 'b' = 2)`. + pub fn to_clickhouse_type(metadata: &EnumMetadata) -> String { + let mut result = match metadata.enum_size { + EnumSize::Enum8 => "Enum8(".to_string(), + EnumSize::Enum16 => "Enum16(".to_string(), + }; + for (i, entry) in metadata.entries.iter().enumerate() { + if i > 0 { + result.push_str(", "); + } + result.push_str(&format!("'{}' = {}", entry.name, entry.value)); + } + result.push(')'); + result + } +} + +impl ExtDTypeVTable for ClickHouseEnum { + type Metadata = EnumMetadata; + + fn id(&self) -> ExtID { + ExtID::new_ref(ENUM_EXT_ID) + } + + fn serialize(&self, metadata: &Self::Metadata) -> VortexResult> { + // Simple binary format: + // [enum_size: u8] [num_entries: u16] [entry...] + // entry = [name_len: u16] [name_bytes...] [value: i16] + let mut buf = Vec::new(); + buf.push(metadata.enum_size as u8); + let n = metadata.entries.len() as u16; + buf.extend_from_slice(&n.to_le_bytes()); + for entry in &metadata.entries { + let name_bytes = entry.name.as_bytes(); + let name_len = name_bytes.len() as u16; + buf.extend_from_slice(&name_len.to_le_bytes()); + buf.extend_from_slice(name_bytes); + buf.extend_from_slice(&entry.value.to_le_bytes()); + } + Ok(buf) + } + + fn deserialize(&self, data: &[u8]) -> VortexResult { + if data.len() < 3 { + vortex_bail!("Enum metadata too short"); + } + let enum_size = match data[0] { + 8 => EnumSize::Enum8, + 16 => EnumSize::Enum16, + other => vortex_bail!("Invalid enum size: {}", other), + }; + let n = u16::from_le_bytes([data[1], data[2]]) as usize; + let mut offset = 3; + let mut entries = Vec::with_capacity(n); + for _ in 0..n { + if offset + 2 > data.len() { + vortex_bail!("Truncated enum metadata"); + } + let name_len = u16::from_le_bytes([data[offset], data[offset + 1]]) as usize; + offset += 2; + if offset + name_len + 2 > data.len() { + vortex_bail!("Truncated enum metadata"); + } + let name = String::from_utf8_lossy(&data[offset..offset + name_len]).to_string(); + offset += name_len; + let value = i16::from_le_bytes([data[offset], data[offset + 1]]); + offset += 2; + entries.push(EnumEntry { name, value }); + } + Ok(EnumMetadata { enum_size, entries }) + } + + fn validate_dtype(&self, metadata: &Self::Metadata, storage_dtype: &DType) -> VortexResult<()> { + match (metadata.enum_size, storage_dtype) { + (EnumSize::Enum8, DType::Primitive(PType::I8, _)) => Ok(()), + (EnumSize::Enum16, DType::Primitive(PType::I16, _)) => Ok(()), + _ => vortex_bail!( + "Enum{} requires Primitive(I{}) storage, got {:?}", + metadata.enum_size as u8, + metadata.enum_size as u8, + storage_dtype + ), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_enum_metadata_roundtrip() -> VortexResult<()> { + let vtable = ClickHouseEnum; + let metadata = EnumMetadata { + enum_size: EnumSize::Enum8, + entries: vec![ + EnumEntry { + name: "a".into(), + value: 1, + }, + EnumEntry { + name: "b".into(), + value: 2, + }, + ], + }; + let serialized = vtable.serialize(&metadata)?; + let deserialized = vtable.deserialize(&serialized)?; + assert_eq!(metadata, deserialized); + Ok(()) + } + + #[test] + fn test_enum_clickhouse_type_string() { + let metadata = EnumMetadata { + enum_size: EnumSize::Enum8, + entries: vec![ + EnumEntry { + name: "hello".into(), + value: 1, + }, + EnumEntry { + name: "world".into(), + value: 2, + }, + ], + }; + assert_eq!( + ClickHouseEnum::to_clickhouse_type(&metadata), + "Enum8('hello' = 1, 'world' = 2)" + ); + } +} diff --git a/vortex-clickhouse/src/ext_types/fixedstring.rs b/vortex-clickhouse/src/ext_types/fixedstring.rs new file mode 100644 index 00000000000..22ea529ea07 --- /dev/null +++ b/vortex-clickhouse/src/ext_types/fixedstring.rs @@ -0,0 +1,107 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! ClickHouse FixedString Extension Type. +//! +//! Preserves `FixedString(N)` semantics in Vortex files. +//! +//! # Storage +//! `Utf8` — the fixed-length padding is not stored; only the actual content is kept. + +use std::fmt::{Display, Formatter}; +use std::hash::Hash; + +use vortex::dtype::extension::{ExtDType, ExtDTypeVTable, ExtID}; +use vortex::dtype::{DType, Nullability}; +use vortex::error::{VortexResult, vortex_bail}; + +/// Extension type ID. +pub const FIXEDSTRING_EXT_ID: &str = "clickhouse.fixedstring"; + +/// Metadata: the N value from `FixedString(N)`. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +pub struct FixedStringMetadata { + pub n: u32, +} + +impl Display for FixedStringMetadata { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "FixedString({})", self.n) + } +} + +/// The FixedString extension type VTable. +#[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] +pub struct ClickHouseFixedString; + +impl ClickHouseFixedString { + /// Create a ClickHouse FixedString DType (type-erased). + pub fn dtype(n: u32, nullability: Nullability) -> DType { + let metadata = FixedStringMetadata { n }; + let storage = DType::Utf8(Nullability::NonNullable); + let ext = ExtDType::try_with_vtable(Self, metadata, storage.with_nullability(nullability)) + .expect("FixedString storage dtype is always valid"); + DType::Extension(ext.erased()) + } + + /// Try to extract metadata from a DType. + pub fn try_get_metadata(dtype: &DType) -> Option { + if let DType::Extension(ext) = dtype { + if ext.id().as_ref() == FIXEDSTRING_EXT_ID { + return ext.metadata_opt::().copied(); + } + } + None + } + + /// Reconstruct the ClickHouse type string. + pub fn to_clickhouse_type(metadata: &FixedStringMetadata) -> String { + format!("FixedString({})", metadata.n) + } +} + +impl ExtDTypeVTable for ClickHouseFixedString { + type Metadata = FixedStringMetadata; + + fn id(&self) -> ExtID { + ExtID::new_ref(FIXEDSTRING_EXT_ID) + } + + fn serialize(&self, metadata: &Self::Metadata) -> VortexResult> { + Ok(metadata.n.to_le_bytes().to_vec()) + } + + fn deserialize(&self, data: &[u8]) -> VortexResult { + if data.len() < 4 { + vortex_bail!("FixedString metadata too short"); + } + let n = u32::from_le_bytes([data[0], data[1], data[2], data[3]]); + Ok(FixedStringMetadata { n }) + } + + fn validate_dtype( + &self, + _metadata: &Self::Metadata, + storage_dtype: &DType, + ) -> VortexResult<()> { + match storage_dtype { + DType::Utf8(_) => Ok(()), + _ => vortex_bail!("FixedString requires Utf8 storage, got {:?}", storage_dtype), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_fixedstring_metadata_roundtrip() -> VortexResult<()> { + let vtable = ClickHouseFixedString; + let metadata = FixedStringMetadata { n: 16 }; + let serialized = vtable.serialize(&metadata)?; + let deserialized = vtable.deserialize(&serialized)?; + assert_eq!(metadata, deserialized); + Ok(()) + } +} diff --git a/vortex-clickhouse/src/ext_types/geo.rs b/vortex-clickhouse/src/ext_types/geo.rs new file mode 100644 index 00000000000..ec88ed77550 --- /dev/null +++ b/vortex-clickhouse/src/ext_types/geo.rs @@ -0,0 +1,255 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Geo Extension Type for ClickHouse Point/LineString/Ring/Polygon/MultiLineString/MultiPolygon. +//! +//! ClickHouse supports geographic types that are stored as nested Tuple/Array structures: +//! - Point = Tuple(Float64, Float64) +//! - Ring = Array(Point) +//! - LineString = Array(Point) +//! - Polygon = Array(Ring) +//! - MultiLineString = Array(LineString) +//! - MultiPolygon = Array(Polygon) +//! +//! In Vortex, these are stored as WKB-encoded binary strings. The C++ side handles +//! conversion between ClickHouse GEO columns and WKB binary format. +//! +//! This extension type preserves the GEO type name through the Vortex file format +//! so the read side can reconstruct the correct ClickHouse GEO type. + +use std::fmt::{Debug, Display, Formatter}; +use std::hash::Hash; + +use vortex::dtype::extension::{ExtDType, ExtDTypeVTable, ExtID}; +use vortex::dtype::{DType, Nullability}; +use vortex::error::{VortexResult, vortex_bail}; + +/// The extension type ID for ClickHouse Geo types. +pub const GEO_EXT_ID: &str = "clickhouse.geo"; + +/// The concrete GEO types supported. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[repr(u8)] +pub enum GeoType { + Point = 0, + LineString = 1, + Ring = 2, + Polygon = 3, + MultiLineString = 4, + MultiPolygon = 5, +} + +impl GeoType { + /// Returns the ClickHouse type name. + pub const fn clickhouse_type_name(&self) -> &'static str { + match self { + GeoType::Point => "Point", + GeoType::LineString => "LineString", + GeoType::Ring => "Ring", + GeoType::Polygon => "Polygon", + GeoType::MultiLineString => "MultiLineString", + GeoType::MultiPolygon => "MultiPolygon", + } + } + + /// Parse from ClickHouse type name. + pub fn from_clickhouse_type(name: &str) -> Option { + match name { + "Point" => Some(GeoType::Point), + "LineString" => Some(GeoType::LineString), + "Ring" => Some(GeoType::Ring), + "Polygon" => Some(GeoType::Polygon), + "MultiLineString" => Some(GeoType::MultiLineString), + "MultiPolygon" => Some(GeoType::MultiPolygon), + _ => None, + } + } +} + +impl TryFrom for GeoType { + type Error = vortex::error::VortexError; + + fn try_from(value: u8) -> Result { + match value { + 0 => Ok(GeoType::Point), + 1 => Ok(GeoType::LineString), + 2 => Ok(GeoType::Ring), + 3 => Ok(GeoType::Polygon), + 4 => Ok(GeoType::MultiLineString), + 5 => Ok(GeoType::MultiPolygon), + _ => vortex_bail!("Invalid GeoType tag: {}", value), + } + } +} + +impl Display for GeoType { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.clickhouse_type_name()) + } +} + +/// Metadata for Geo extension type. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +pub struct GeoMetadata { + /// The specific GEO type. + pub geo_type: GeoType, +} + +impl GeoMetadata { + /// Create new Geo metadata. + pub fn new(geo_type: GeoType) -> Self { + Self { geo_type } + } +} + +impl Display for GeoMetadata { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.geo_type) + } +} + +/// The Geo extension type VTable. +#[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] +pub struct Geo; + +impl Geo { + /// Create a new Geo extension dtype. + pub fn new(geo_type: GeoType, nullability: Nullability) -> ExtDType { + let metadata = GeoMetadata::new(geo_type); + // Storage: Binary (WKB-encoded binary data) + let storage_dtype = DType::Binary(Nullability::NonNullable); + ExtDType::try_with_vtable(Self, metadata, storage_dtype.with_nullability(nullability)) + .expect("Geo storage dtype is always valid") + } + + /// Create a Geo DType (type-erased). + pub fn dtype(geo_type: GeoType, nullability: Nullability) -> DType { + DType::Extension(Self::new(geo_type, nullability).erased()) + } + + /// Check if a DType is a Geo extension type. + pub fn is_geo(dtype: &DType) -> bool { + if let DType::Extension(ext) = dtype { + ext.id().as_ref() == GEO_EXT_ID + } else { + false + } + } + + /// Try to extract GeoType from a DType. + pub fn try_get_type(dtype: &DType) -> Option { + if let DType::Extension(ext) = dtype { + if ext.id().as_ref() == GEO_EXT_ID { + ext.metadata_opt::().map(|m| m.geo_type) + } else { + None + } + } else { + None + } + } +} + +impl ExtDTypeVTable for Geo { + type Metadata = GeoMetadata; + + fn id(&self) -> ExtID { + ExtID::new_ref(GEO_EXT_ID) + } + + fn serialize(&self, metadata: &Self::Metadata) -> VortexResult> { + Ok(vec![metadata.geo_type as u8]) + } + + fn deserialize(&self, data: &[u8]) -> VortexResult { + if data.is_empty() { + vortex_bail!("Geo metadata is empty"); + } + let geo_type = GeoType::try_from(data[0])?; + Ok(GeoMetadata::new(geo_type)) + } + + fn validate_dtype( + &self, + _metadata: &Self::Metadata, + storage_dtype: &DType, + ) -> VortexResult<()> { + // Storage should be Binary (WKB-encoded binary data) + match storage_dtype { + DType::Binary(_) => Ok(()), + _ => vortex_bail!( + "Geo extension requires Binary storage, got {:?}", + storage_dtype + ), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_geo_type_roundtrip() { + for geo_type in [ + GeoType::Point, + GeoType::LineString, + GeoType::Ring, + GeoType::Polygon, + GeoType::MultiLineString, + GeoType::MultiPolygon, + ] { + let tag = geo_type as u8; + let roundtrip = GeoType::try_from(tag).unwrap(); + assert_eq!(geo_type, roundtrip); + } + } + + #[test] + fn test_geo_dtype_creation() { + let dtype = Geo::dtype(GeoType::Point, Nullability::Nullable); + assert!(Geo::is_geo(&dtype)); + + if let DType::Extension(ext) = &dtype { + assert_eq!(ext.id().as_ref(), GEO_EXT_ID); + } else { + panic!("Expected Extension dtype"); + } + } + + #[test] + fn test_clickhouse_type_names() { + assert_eq!(GeoType::Point.clickhouse_type_name(), "Point"); + assert_eq!(GeoType::LineString.clickhouse_type_name(), "LineString"); + assert_eq!(GeoType::Ring.clickhouse_type_name(), "Ring"); + assert_eq!(GeoType::Polygon.clickhouse_type_name(), "Polygon"); + assert_eq!( + GeoType::MultiLineString.clickhouse_type_name(), + "MultiLineString" + ); + assert_eq!(GeoType::MultiPolygon.clickhouse_type_name(), "MultiPolygon"); + } + + #[test] + fn test_from_clickhouse_type() { + assert_eq!(GeoType::from_clickhouse_type("Point"), Some(GeoType::Point)); + assert_eq!( + GeoType::from_clickhouse_type("Polygon"), + Some(GeoType::Polygon) + ); + assert_eq!( + GeoType::from_clickhouse_type("MultiPolygon"), + Some(GeoType::MultiPolygon) + ); + assert_eq!(GeoType::from_clickhouse_type("String"), None); + } + + #[test] + fn test_geo_try_get_type() { + let dtype = Geo::dtype(GeoType::Polygon, Nullability::NonNullable); + assert_eq!(Geo::try_get_type(&dtype), Some(GeoType::Polygon)); + + let non_geo = DType::Utf8(Nullability::NonNullable); + assert_eq!(Geo::try_get_type(&non_geo), None); + } +} diff --git a/vortex-clickhouse/src/ext_types/ip.rs b/vortex-clickhouse/src/ext_types/ip.rs new file mode 100644 index 00000000000..1c425740560 --- /dev/null +++ b/vortex-clickhouse/src/ext_types/ip.rs @@ -0,0 +1,398 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! IP Address Extension Types for ClickHouse IPv4/IPv6. +//! +//! ClickHouse supports IPv4 and IPv6 address types which are stored as: +//! - IPv4: UInt32 (4 bytes, network byte order internally but stored as native UInt32) +//! - IPv6: UInt128 (16 bytes, stored as fixed-size binary) +//! +//! # Storage Layout +//! - IPv4: Stored as `Primitive(U32)` - each value is 4 bytes +//! - IPv6: Stored as `FixedSizeList` - each value is 16 bytes +//! +//! This is consistent with how Parquet and Arrow handle these types. +//! +//! # Example +//! ```ignore +//! use vortex_clickhouse::ext_types::{IPAddress, IPAddressType}; +//! use vortex::dtype::Nullability; +//! +//! // Create IPv4 dtype +//! let ipv4_dtype = IPAddress::dtype(IPAddressType::IPv4, Nullability::Nullable); +//! +//! // Create IPv6 dtype +//! let ipv6_dtype = IPAddress::dtype(IPAddressType::IPv6, Nullability::Nullable); +//! ``` + +use std::fmt::{Debug, Display, Formatter}; +use std::hash::Hash; +use std::sync::Arc; + +use vortex::dtype::extension::{ExtDType, ExtDTypeVTable, ExtID}; +use vortex::dtype::{DType, Nullability, PType}; +use vortex::error::{VortexResult, vortex_bail}; + +/// The extension type ID for ClickHouse IP address types. +pub const IP_EXT_ID: &str = "clickhouse.ip"; + +/// The concrete IP address types supported. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[repr(u8)] +pub enum IPAddressType { + /// IPv4 address (4 bytes, stored as UInt32) + IPv4 = 0, + /// IPv6 address (16 bytes, stored as 16-byte binary) + IPv6 = 1, +} + +impl IPAddressType { + /// Returns the byte size for this IP address type. + pub const fn byte_size(&self) -> usize { + match self { + IPAddressType::IPv4 => 4, + IPAddressType::IPv6 => 16, + } + } + + /// Returns the ClickHouse type name. + pub const fn clickhouse_type_name(&self) -> &'static str { + match self { + IPAddressType::IPv4 => "IPv4", + IPAddressType::IPv6 => "IPv6", + } + } + + /// Parse from ClickHouse type name. + pub fn from_clickhouse_type(name: &str) -> Option { + match name { + "IPv4" => Some(IPAddressType::IPv4), + "IPv6" => Some(IPAddressType::IPv6), + _ => None, + } + } +} + +impl TryFrom for IPAddressType { + type Error = vortex::error::VortexError; + + fn try_from(value: u8) -> Result { + match value { + 0 => Ok(IPAddressType::IPv4), + 1 => Ok(IPAddressType::IPv6), + _ => vortex_bail!("Invalid IPAddressType tag: {}", value), + } + } +} + +impl Display for IPAddressType { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.clickhouse_type_name()) + } +} + +/// Metadata for IP address extension type. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +pub struct IPAddressMetadata { + /// The specific IP address type. + pub ip_type: IPAddressType, +} + +impl IPAddressMetadata { + /// Create new IP address metadata. + pub fn new(ip_type: IPAddressType) -> Self { + Self { ip_type } + } +} + +impl Display for IPAddressMetadata { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.ip_type) + } +} + +/// The IP address extension type VTable. +#[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] +pub struct IPAddress; + +impl IPAddress { + /// Create a new IP address extension dtype. + pub fn new(ip_type: IPAddressType, nullability: Nullability) -> ExtDType { + let metadata = IPAddressMetadata::new(ip_type); + let storage_dtype = match ip_type { + // IPv4: stored as UInt32 (4 bytes) + IPAddressType::IPv4 => DType::Primitive(PType::U32, Nullability::NonNullable), + // IPv6: stored as 16 bytes (we use U8 as storage element, will be wrapped) + IPAddressType::IPv6 => DType::Primitive(PType::U8, Nullability::NonNullable), + }; + ExtDType::try_with_vtable(Self, metadata, storage_dtype.with_nullability(nullability)) + .expect("IPAddress storage dtype is always valid") + } + + /// Create an IP address DType (type-erased). + pub fn dtype(ip_type: IPAddressType, nullability: Nullability) -> DType { + DType::Extension(Self::new(ip_type, nullability).erased()) + } + + /// Check if a DType is an IP address extension type. + pub fn is_ip_address(dtype: &DType) -> bool { + if let DType::Extension(ext) = dtype { + ext.id().as_ref() == IP_EXT_ID + } else { + false + } + } + + /// Try to extract IPAddressType from a DType. + pub fn try_get_type(dtype: &DType) -> Option { + if let DType::Extension(ext) = dtype { + if ext.id().as_ref() == IP_EXT_ID { + ext.metadata_opt::().map(|m| m.ip_type) + } else { + None + } + } else { + None + } + } + + /// Create DType for storing IPv4 values (as Primitive U32). + /// This is used for actual storage in Vortex files. + pub fn ipv4_storage_dtype(nullability: Nullability) -> DType { + DType::Primitive(PType::U32, nullability) + } + + /// Create DType for storing IPv6 values (as FixedSizeList). + /// This is used for actual storage in Vortex files. + pub fn ipv6_storage_dtype(nullability: Nullability) -> DType { + DType::FixedSizeList( + Arc::new(DType::Primitive(PType::U8, Nullability::NonNullable)), + 16, + nullability, + ) + } +} + +impl ExtDTypeVTable for IPAddress { + type Metadata = IPAddressMetadata; + + fn id(&self) -> ExtID { + ExtID::new_ref(IP_EXT_ID) + } + + fn serialize(&self, metadata: &Self::Metadata) -> VortexResult> { + Ok(vec![metadata.ip_type as u8]) + } + + fn deserialize(&self, data: &[u8]) -> VortexResult { + if data.is_empty() { + vortex_bail!("IPAddress metadata is empty"); + } + let ip_type = IPAddressType::try_from(data[0])?; + Ok(IPAddressMetadata::new(ip_type)) + } + + fn validate_dtype(&self, metadata: &Self::Metadata, storage_dtype: &DType) -> VortexResult<()> { + match metadata.ip_type { + IPAddressType::IPv4 => { + // IPv4 storage should be Primitive(U32) + match storage_dtype { + DType::Primitive(PType::U32, _) => Ok(()), + _ => vortex_bail!( + "IPv4 extension requires Primitive(U32) storage, got {:?}", + storage_dtype + ), + } + } + IPAddressType::IPv6 => { + // IPv6 storage should be Primitive(U8) - the actual bytes + match storage_dtype { + DType::Primitive(PType::U8, _) => Ok(()), + _ => vortex_bail!( + "IPv6 extension requires Primitive(U8) storage, got {:?}", + storage_dtype + ), + } + } + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_ip_type_byte_size() { + assert_eq!(IPAddressType::IPv4.byte_size(), 4); + assert_eq!(IPAddressType::IPv6.byte_size(), 16); + } + + #[test] + fn test_ip_type_names() { + assert_eq!(IPAddressType::IPv4.clickhouse_type_name(), "IPv4"); + assert_eq!(IPAddressType::IPv6.clickhouse_type_name(), "IPv6"); + } + + #[test] + fn test_from_clickhouse_type() { + assert_eq!( + IPAddressType::from_clickhouse_type("IPv4"), + Some(IPAddressType::IPv4) + ); + assert_eq!( + IPAddressType::from_clickhouse_type("IPv6"), + Some(IPAddressType::IPv6) + ); + assert_eq!(IPAddressType::from_clickhouse_type("String"), None); + } + + #[test] + fn test_ip_type_roundtrip() { + for ip_type in [IPAddressType::IPv4, IPAddressType::IPv6] { + let tag = ip_type as u8; + let recovered = IPAddressType::try_from(tag).expect("roundtrip"); + assert_eq!(recovered, ip_type); + } + } + + #[test] + fn test_ip_dtype_creation() { + // Test IPv4 dtype + let ipv4_dtype = IPAddress::dtype(IPAddressType::IPv4, Nullability::Nullable); + assert!(IPAddress::is_ip_address(&ipv4_dtype)); + assert_eq!( + IPAddress::try_get_type(&ipv4_dtype), + Some(IPAddressType::IPv4) + ); + + // Test IPv6 dtype + let ipv6_dtype = IPAddress::dtype(IPAddressType::IPv6, Nullability::NonNullable); + assert!(IPAddress::is_ip_address(&ipv6_dtype)); + assert_eq!( + IPAddress::try_get_type(&ipv6_dtype), + Some(IPAddressType::IPv6) + ); + } + + #[test] + fn test_storage_dtypes() { + // IPv4 storage dtype + let ipv4_storage = IPAddress::ipv4_storage_dtype(Nullability::Nullable); + assert!(matches!(ipv4_storage, DType::Primitive(PType::U32, _))); + + // IPv6 storage dtype + let ipv6_storage = IPAddress::ipv6_storage_dtype(Nullability::Nullable); + if let DType::FixedSizeList(elem, size, _) = ipv6_storage { + assert_eq!(size, 16); + assert!(matches!(elem.as_ref(), DType::Primitive(PType::U8, _))); + } else { + panic!("Expected FixedSizeList"); + } + } + + #[test] + fn test_ip_file_roundtrip() { + use std::io::Write; + use std::sync::Arc; + use tempfile::NamedTempFile; + use vortex::array::IntoArray; + use vortex::array::arrays::{FixedSizeListArray, PrimitiveArray, StructArray}; + use vortex::array::stream::ArrayStreamExt; + use vortex::array::validity::Validity; + use vortex::buffer::Buffer; + use vortex::dtype::FieldNames; + use vortex::file::{OpenOptionsSessionExt, WriteOptionsSessionExt}; + use vortex::io::runtime::BlockingRuntime; + + use crate::{RUNTIME, SESSION}; + + // Create test data - 2 IPv4 values stored as U32 + let ipv4_values: Vec = vec![ + 0x7F000001, // 127.0.0.1 + 0xC0A80001, // 192.168.0.1 + ]; + let ipv4_array = + PrimitiveArray::new(Buffer::::from(ipv4_values), Validity::NonNullable); + + // Create test data - 2 IPv6 values as FixedSizeList + let ipv6_bytes: Vec = vec![ + // ::1 (loopback) + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, // fe80::1 (link-local) + 0xfe, 0x80, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, + ]; + let ipv6_values = + PrimitiveArray::new(Buffer::::from(ipv6_bytes), Validity::NonNullable); + let ipv6_array = + FixedSizeListArray::try_new(ipv6_values.into_array(), 16, Validity::NonNullable, 2) + .expect("Failed to create IPv6 array"); + + // Create struct with both fields + let field_names: Vec> = vec![Arc::from("ipv4"), Arc::from("ipv6")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![ipv4_array.into_array(), ipv6_array.into_array()], + 2, + Validity::NonNullable, + ) + .expect("Failed to create struct"); + + // Write to temp file + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_str().unwrap().to_string(); + + let mut buf = Vec::new(); + (*RUNTIME).block_on(async { + SESSION + .write_options() + .write(&mut buf, struct_array.to_array_stream()) + .await + .expect("Failed to write"); + }); + + // Write buffer to temp file + let mut file = std::fs::File::create(&path).expect("Failed to create file"); + file.write_all(&buf).expect("Failed to write to file"); + file.flush().expect("Failed to flush"); + drop(file); + + // Read back and verify dtype + let read_dtype = (*RUNTIME).block_on(async { + let vortex_file = SESSION + .open_options() + .open_path(&path) + .await + .expect("Failed to open"); + vortex_file.dtype().clone() + }); + + // Verify the dtype structure + if let DType::Struct(fields, _) = &read_dtype { + let field_iter = fields.fields(); + let dtypes: Vec<_> = field_iter.collect(); + assert_eq!(dtypes.len(), 2); + + // IPv4 field should be Primitive(U32) + assert!( + matches!(&dtypes[0], DType::Primitive(PType::U32, _)), + "Expected Primitive(U32) for IPv4, got {:?}", + dtypes[0] + ); + + // IPv6 field should be FixedSizeList + match &dtypes[1] { + DType::FixedSizeList(elem, size, _) => { + assert_eq!(*size, 16, "Expected size 16 for IPv6"); + assert!( + matches!(elem.as_ref(), DType::Primitive(PType::U8, _)), + "Expected Primitive(U8), got {:?}", + elem + ); + } + _ => panic!("Expected FixedSizeList for IPv6, got {:?}", dtypes[1]), + } + } else { + panic!("Expected Struct dtype, got {:?}", read_dtype); + } + } +} diff --git a/vortex-clickhouse/src/ext_types/lowcardinality.rs b/vortex-clickhouse/src/ext_types/lowcardinality.rs new file mode 100644 index 00000000000..60890cdb4f4 --- /dev/null +++ b/vortex-clickhouse/src/ext_types/lowcardinality.rs @@ -0,0 +1,106 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! ClickHouse LowCardinality Extension Type. +//! +//! Marker extension so the read side can reconstruct `LowCardinality(T)`. +//! The actual dictionary encoding is handled by Vortex compression. +//! +//! # Storage +//! Same as the inner type (e.g. `Utf8` for `LowCardinality(String)`). + +use std::fmt::{Display, Formatter}; +use std::hash::Hash; + +use vortex::dtype::extension::{ExtDType, ExtDTypeVTable, ExtID}; +use vortex::dtype::{DType, Nullability}; +use vortex::error::{VortexResult, vortex_bail}; + +/// Extension type ID. +pub const LOWCARDINALITY_EXT_ID: &str = "clickhouse.lowcardinality"; + +/// Metadata: stores the inner ClickHouse type string for reconstruction. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LowCardinalityMetadata { + /// The inner ClickHouse type string, e.g. `"String"`. + pub inner_type: String, +} + +impl Display for LowCardinalityMetadata { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "LowCardinality({})", self.inner_type) + } +} + +/// The LowCardinality extension type VTable. +#[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] +pub struct ClickHouseLowCardinality; + +impl ClickHouseLowCardinality { + /// Create a ClickHouse LowCardinality DType (type-erased). + pub fn dtype(inner_type: String, storage: DType, nullability: Nullability) -> DType { + let metadata = LowCardinalityMetadata { inner_type }; + let ext = ExtDType::try_with_vtable(Self, metadata, storage.with_nullability(nullability)) + .expect("LowCardinality storage dtype is always valid"); + DType::Extension(ext.erased()) + } + + /// Try to extract metadata from a DType. + pub fn try_get_metadata(dtype: &DType) -> Option { + if let DType::Extension(ext) = dtype { + if ext.id().as_ref() == LOWCARDINALITY_EXT_ID { + return ext.metadata_opt::().cloned(); + } + } + None + } + + /// Reconstruct the ClickHouse type string. + pub fn to_clickhouse_type(metadata: &LowCardinalityMetadata) -> String { + format!("LowCardinality({})", metadata.inner_type) + } +} + +impl ExtDTypeVTable for ClickHouseLowCardinality { + type Metadata = LowCardinalityMetadata; + + fn id(&self) -> ExtID { + ExtID::new_ref(LOWCARDINALITY_EXT_ID) + } + + fn serialize(&self, metadata: &Self::Metadata) -> VortexResult> { + Ok(metadata.inner_type.as_bytes().to_vec()) + } + + fn deserialize(&self, data: &[u8]) -> VortexResult { + Ok(LowCardinalityMetadata { + inner_type: String::from_utf8_lossy(data).to_string(), + }) + } + + fn validate_dtype( + &self, + _metadata: &Self::Metadata, + _storage_dtype: &DType, + ) -> VortexResult<()> { + // Any storage type is valid for LowCardinality marker + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_lowcardinality_metadata_roundtrip() -> VortexResult<()> { + let vtable = ClickHouseLowCardinality; + let metadata = LowCardinalityMetadata { + inner_type: "String".to_string(), + }; + let serialized = vtable.serialize(&metadata)?; + let deserialized = vtable.deserialize(&serialized)?; + assert_eq!(metadata, deserialized); + Ok(()) + } +} diff --git a/vortex-clickhouse/src/ext_types/mod.rs b/vortex-clickhouse/src/ext_types/mod.rs new file mode 100644 index 00000000000..ff2caa5713d --- /dev/null +++ b/vortex-clickhouse/src/ext_types/mod.rs @@ -0,0 +1,36 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! ClickHouse-specific Extension types for Vortex. +//! +//! This module provides Extension DTypes for ClickHouse types that don't have +//! direct Vortex equivalents, such as: +//! - Large integers (Int128, UInt128, Int256, UInt256) +//! - IP addresses (IPv4, IPv6) +//! - UUID +//! - Geo types (Point, Ring, LineString, Polygon, MultiLineString, MultiPolygon) +//! - Enum8/Enum16 (with name→value mappings) +//! - DateTime/DateTime64 (with precision and timezone) +//! - Date/Date32 +//! - LowCardinality marker +//! - FixedString(N) + +mod bigint; +pub mod date; +pub mod datetime; +pub mod enum_; +pub mod fixedstring; +mod geo; +mod ip; +pub mod lowcardinality; +mod uuid; + +pub use bigint::{BigInt, BigIntMetadata, BigIntType}; +pub use date::{ClickHouseDate, DATE_EXT_ID, DateMetadata}; +pub use datetime::{ClickHouseDateTime, DATETIME_EXT_ID, DateTimeMetadata}; +pub use enum_::{ClickHouseEnum, ENUM_EXT_ID, EnumEntry, EnumMetadata, EnumSize}; +pub use fixedstring::{ClickHouseFixedString, FIXEDSTRING_EXT_ID, FixedStringMetadata}; +pub use geo::{GEO_EXT_ID, Geo, GeoMetadata, GeoType}; +pub use ip::{IPAddress, IPAddressMetadata, IPAddressType}; +pub use lowcardinality::{ClickHouseLowCardinality, LOWCARDINALITY_EXT_ID, LowCardinalityMetadata}; +pub use uuid::{UUID, UUID_BYTE_SIZE, UUID_EXT_ID, UUIDMetadata}; diff --git a/vortex-clickhouse/src/ext_types/uuid.rs b/vortex-clickhouse/src/ext_types/uuid.rs new file mode 100644 index 00000000000..ce8eb5e02d1 --- /dev/null +++ b/vortex-clickhouse/src/ext_types/uuid.rs @@ -0,0 +1,342 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! UUID Extension Type for ClickHouse UUID. +//! +//! ClickHouse UUID is stored as UInt128 (16 bytes), which is internally represented +//! as two uint64_t values in little-endian format. +//! +//! # Storage Layout +//! - UUID: Stored as `FixedSizeList` - each value is 16 bytes +//! +//! This is consistent with how Parquet handles UUID (FIXED_LEN_BYTE_ARRAY(16)) +//! and Arrow (FixedSizeBinary(16)). +//! +//! # Byte Order Note +//! ClickHouse stores UUIDs in little-endian format internally. When reading/writing, +//! the raw bytes are preserved without any byte-order conversion. +//! +//! # Example +//! ```ignore +//! use vortex_clickhouse::ext_types::UUID; +//! use vortex::dtype::Nullability; +//! +//! // Create UUID dtype +//! let uuid_dtype = UUID::dtype(Nullability::Nullable); +//! ``` + +use std::fmt::{Debug, Display, Formatter}; +use std::hash::Hash; +use std::sync::Arc; + +use vortex::dtype::extension::{ExtDType, ExtDTypeVTable, ExtID}; +use vortex::dtype::{DType, Nullability, PType}; +use vortex::error::{VortexResult, vortex_bail}; + +/// The extension type ID for ClickHouse UUID type. +pub const UUID_EXT_ID: &str = "clickhouse.uuid"; + +/// Byte size of UUID (128 bits = 16 bytes). +pub const UUID_BYTE_SIZE: u32 = 16; + +/// Metadata for UUID extension type. +/// Currently empty as UUID has no configurable parameters. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Default)] +pub struct UUIDMetadata; + +impl Display for UUIDMetadata { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "UUID") + } +} + +/// The UUID extension type VTable. +#[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] +pub struct UUID; + +impl UUID { + /// Create a new UUID extension dtype. + pub fn new(nullability: Nullability) -> ExtDType { + // Storage dtype is Primitive(U8) as the underlying bytes + let storage_dtype = + DType::Primitive(PType::U8, Nullability::NonNullable).with_nullability(nullability); + ExtDType::try_with_vtable(Self, UUIDMetadata, storage_dtype) + .expect("UUID storage dtype is always valid") + } + + /// Create the DType for UUID (as Extension type). + pub fn dtype(nullability: Nullability) -> DType { + DType::Extension(Self::new(nullability).erased()) + } + + /// Get the storage dtype for UUID. + /// UUID is stored as FixedSizeList (16 bytes). + pub fn storage_dtype(nullability: Nullability) -> DType { + DType::FixedSizeList( + Arc::new(DType::Primitive(PType::U8, Nullability::NonNullable)), + UUID_BYTE_SIZE, + nullability, + ) + } + + /// Check if a dtype represents a UUID type. + /// This checks for either: + /// 1. Extension type with UUID ID + /// 2. FixedSizeList (the storage format) + pub fn is_uuid(dtype: &DType) -> bool { + match dtype { + DType::Extension(ext) => ext.id().as_ref() == UUID_EXT_ID, + DType::FixedSizeList(elem, size, _) => { + *size == UUID_BYTE_SIZE && matches!(elem.as_ref(), DType::Primitive(PType::U8, _)) + } + _ => false, + } + } + + /// Returns the ClickHouse type name. + pub const fn clickhouse_type_name() -> &'static str { + "UUID" + } + + /// Returns the byte size of UUID. + pub const fn byte_size() -> u32 { + UUID_BYTE_SIZE + } +} + +impl ExtDTypeVTable for UUID { + type Metadata = UUIDMetadata; + + fn id(&self) -> ExtID { + ExtID::new_ref(UUID_EXT_ID) + } + + fn serialize(&self, _metadata: &Self::Metadata) -> VortexResult> { + // No metadata to serialize for UUID + Ok(vec![]) + } + + fn deserialize(&self, _data: &[u8]) -> VortexResult { + // No metadata to deserialize + Ok(UUIDMetadata) + } + + fn validate_dtype( + &self, + _metadata: &Self::Metadata, + storage_dtype: &DType, + ) -> VortexResult<()> { + // UUID storage should be FixedSizeList or Primitive(U8) for the underlying bytes + match storage_dtype { + DType::FixedSizeList(elem, size, _) => { + if *size == UUID_BYTE_SIZE + && matches!(elem.as_ref(), DType::Primitive(PType::U8, _)) + { + Ok(()) + } else { + vortex_bail!( + "UUID extension requires FixedSizeList storage, got FixedSizeList with size {} and elem {:?}", + size, + elem + ) + } + } + DType::Primitive(PType::U8, _) => Ok(()), // Raw bytes storage + _ => vortex_bail!( + "UUID extension requires FixedSizeList or Primitive(U8) storage, got {:?}", + storage_dtype + ), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_uuid_byte_size() { + assert_eq!(UUID::byte_size(), 16); + } + + #[test] + fn test_uuid_type_name() { + assert_eq!(UUID::clickhouse_type_name(), "UUID"); + } + + #[test] + fn test_uuid_dtype_creation() { + let dtype = UUID::dtype(Nullability::Nullable); + assert!(UUID::is_uuid(&dtype)); + + let dtype = UUID::dtype(Nullability::NonNullable); + assert!(UUID::is_uuid(&dtype)); + } + + #[test] + fn test_uuid_storage_dtype() { + let storage = UUID::storage_dtype(Nullability::Nullable); + match storage { + DType::FixedSizeList(elem, size, nullability) => { + assert_eq!(size, 16); + assert_eq!(nullability, Nullability::Nullable); + assert!(matches!(elem.as_ref(), DType::Primitive(PType::U8, _))); + } + _ => panic!("Expected FixedSizeList, got {:?}", storage), + } + } + + #[test] + fn test_uuid_is_uuid_detection() { + // Test Extension type detection + let ext_dtype = UUID::dtype(Nullability::Nullable); + assert!(UUID::is_uuid(&ext_dtype)); + + // Test FixedSizeList detection (storage format) + let fsl_dtype = DType::FixedSizeList( + Arc::new(DType::Primitive(PType::U8, Nullability::NonNullable)), + 16, + Nullability::Nullable, + ); + assert!(UUID::is_uuid(&fsl_dtype)); + + // Test non-UUID types + let non_uuid = DType::Primitive(PType::U32, Nullability::Nullable); + assert!(!UUID::is_uuid(&non_uuid)); + + // Test wrong size FixedSizeList + let wrong_size = DType::FixedSizeList( + Arc::new(DType::Primitive(PType::U8, Nullability::NonNullable)), + 32, // Wrong size + Nullability::Nullable, + ); + assert!(!UUID::is_uuid(&wrong_size)); + } + + #[test] + fn test_uuid_metadata() { + let metadata = UUIDMetadata; + assert_eq!(format!("{}", metadata), "UUID"); + } + + #[test] + fn test_uuid_ext_vtable() { + use vortex::dtype::extension::ExtDTypeVTable; + + let uuid = UUID; + assert_eq!(uuid.id().as_ref(), UUID_EXT_ID); + + // Test serialize/deserialize metadata + let serialized = uuid.serialize(&UUIDMetadata).unwrap(); + assert!(serialized.is_empty()); // No metadata to serialize + + let deserialized = uuid.deserialize(&[]).unwrap(); + assert_eq!(deserialized, UUIDMetadata); + + // Test validate_dtype + let valid_dtype = DType::FixedSizeList( + Arc::new(DType::Primitive(PType::U8, Nullability::NonNullable)), + 16, + Nullability::Nullable, + ); + assert!(uuid.validate_dtype(&UUIDMetadata, &valid_dtype).is_ok()); + + // Invalid dtype should fail + let invalid_dtype = DType::Primitive(PType::U32, Nullability::Nullable); + assert!(uuid.validate_dtype(&UUIDMetadata, &invalid_dtype).is_err()); + } + + #[test] + fn test_uuid_file_roundtrip() { + use std::io::Write; + use tempfile::NamedTempFile; + use vortex::array::IntoArray; + use vortex::array::arrays::{FixedSizeListArray, PrimitiveArray, StructArray}; + use vortex::array::stream::ArrayStreamExt; + use vortex::array::validity::Validity; + use vortex::buffer::Buffer; + use vortex::dtype::FieldNames; + use vortex::file::{OpenOptionsSessionExt, WriteOptionsSessionExt}; + use vortex::io::runtime::BlockingRuntime; + + use crate::{RUNTIME, SESSION}; + + // Create test data - 2 UUIDs as FixedSizeList + // UUID format: xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx (16 bytes) + let bytes: Vec = vec![ + // First UUID: 550e8400-e29b-41d4-a716-446655440000 + 0x55, 0x0e, 0x84, 0x00, 0xe2, 0x9b, 0x41, 0xd4, 0xa7, 0x16, 0x44, 0x66, 0x55, 0x44, + 0x00, 0x00, // Second UUID: 6ba7b810-9dad-11d1-80b4-00c04fd430c8 + 0x6b, 0xa7, 0xb8, 0x10, 0x9d, 0xad, 0x11, 0xd1, 0x80, 0xb4, 0x00, 0xc0, 0x4f, 0xd4, + 0x30, 0xc8, + ]; + + // Create FixedSizeList array for UUID (16 bytes per element) + let values = PrimitiveArray::new(Buffer::::from(bytes.clone()), Validity::NonNullable); + let fsl_array = FixedSizeListArray::try_new( + values.into_array(), + 16, // element size in bytes + Validity::NonNullable, + 2, // number of elements + ) + .expect("Failed to create FixedSizeList"); + + // Wrap in struct + let field_names: Vec> = vec![Arc::from("uuid_col")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![fsl_array.into_array()], + 2, + Validity::NonNullable, + ) + .expect("Failed to create struct"); + + // Write to temp file + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_str().unwrap().to_string(); + + let mut buf = Vec::new(); + (*RUNTIME).block_on(async { + SESSION + .write_options() + .write(&mut buf, struct_array.to_array_stream()) + .await + .expect("Failed to write"); + }); + + // Write buffer to temp file + let mut file = std::fs::File::create(&path).expect("Failed to create file"); + file.write_all(&buf).expect("Failed to write to file"); + file.flush().expect("Failed to flush"); + drop(file); + + // Read back and verify dtype + let read_dtype = (*RUNTIME).block_on(async { + let vortex_file = SESSION + .open_options() + .open_path(&path) + .await + .expect("Failed to open"); + vortex_file.dtype().clone() + }); + + // Verify the dtype is a Struct with a FixedSizeList field + if let DType::Struct(fields, _) = &read_dtype { + let val_dtype = fields.fields().next().expect("Expected field"); + // Check that it's FixedSizeList (represents UUID) + match &val_dtype { + DType::FixedSizeList(elem, size, _) => { + assert_eq!(*size, 16, "Expected size 16 for UUID"); + assert!( + matches!(elem.as_ref(), DType::Primitive(PType::U8, _)), + "Expected Primitive(U8), got {:?}", + elem + ); + } + _ => panic!("Expected FixedSizeList dtype, got {:?}", val_dtype), + } + } else { + panic!("Expected Struct dtype, got {:?}", read_dtype); + } + } +} diff --git a/vortex-clickhouse/src/ffi_tests.rs b/vortex-clickhouse/src/ffi_tests.rs new file mode 100644 index 00000000000..3826e712599 --- /dev/null +++ b/vortex-clickhouse/src/ffi_tests.rs @@ -0,0 +1,1812 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! FFI and integration tests for vortex-clickhouse. +//! +//! These tests verify that the C ABI interface works correctly and that +//! memory management follows the "caller owns" principle. +//! +//! # Test Categories +//! +//! 1. **FFI Boundary Tests**: Verify C ABI function signatures and behavior +//! 2. **Memory Safety Tests**: Ensure proper allocation/deallocation patterns +//! 3. **End-to-End Tests**: Complete read/write workflows through FFI + +#[cfg(test)] +mod ffi_tests { + use std::ffi::CString; + use std::io::Write; + use std::ptr; + use std::sync::Arc; + + use tempfile::NamedTempFile; + use vortex::array::IntoArray; + use vortex::array::arrays::{PrimitiveArray, StructArray, VarBinViewArray}; + use vortex::array::stream::ArrayStreamExt; + use vortex::array::validity::Validity; + use vortex::buffer::{Buffer, ByteBufferMut}; + use vortex::dtype::FieldNames; + use vortex::file::WriteOptionsSessionExt; + use vortex::io::runtime::BlockingRuntime; + + use crate::scan::{vortex_scanner_free, vortex_scanner_new, vortex_scanner_num_columns}; + use crate::{RUNTIME, SESSION}; + + // ========================================================================= + // Test Fixtures and Helpers + // ========================================================================= + + /// Create a test Vortex file with sample data and return the path. + /// Uses a sync-friendly approach by writing to a temp file. + fn create_test_vortex_file() -> NamedTempFile { + // Create sample data + let id_buffer: Buffer = vec![1i64, 2, 3, 4, 5].into(); + let id_array = PrimitiveArray::new(id_buffer, Validity::NonNullable).into_array(); + + let value_buffer: Buffer = vec![1.1, 2.2, 3.3, 4.4, 5.5].into(); + let value_array = PrimitiveArray::new(value_buffer, Validity::NonNullable).into_array(); + + let name_array = + VarBinViewArray::from_iter_str(vec!["alice", "bob", "carol", "dave", "eve"]) + .into_array(); + + let field_names: Vec> = + vec![Arc::from("id"), Arc::from("value"), Arc::from("name")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![id_array, value_array, name_array], + 5, + Validity::NonNullable, + ) + .expect("Failed to create struct array"); + + // Write to in-memory buffer first + let mut buf = ByteBufferMut::empty(); + (*RUNTIME).block_on(async { + SESSION + .write_options() + .write(&mut buf, struct_array.to_array_stream()) + .await + .expect("Failed to write test file"); + }); + + // Write buffer to temp file + let mut temp_file = NamedTempFile::new().expect("Failed to create temp file"); + temp_file + .write_all(buf.as_ref()) + .expect("Failed to write to temp file"); + temp_file.flush().expect("Failed to flush temp file"); + + temp_file + } + + /// Create a test file with nullable columns. + fn create_nullable_test_file() -> NamedTempFile { + // Create data with nulls + let values: Vec> = vec![Some(1), None, Some(3), None, Some(5)]; + let id_array = PrimitiveArray::from_option_iter(values).into_array(); + + let str_values: Vec> = vec![Some("a"), None, Some("c"), Some("d"), None]; + let name_array = VarBinViewArray::from_iter_nullable_str(str_values).into_array(); + + let field_names: Vec> = vec![Arc::from("id"), Arc::from("name")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![id_array, name_array], + 5, + Validity::NonNullable, + ) + .expect("Failed to create struct array"); + + let mut buf = ByteBufferMut::empty(); + (*RUNTIME).block_on(async { + SESSION + .write_options() + .write(&mut buf, struct_array.to_array_stream()) + .await + .expect("Failed to write test file"); + }); + + let mut temp_file = NamedTempFile::new().expect("Failed to create temp file"); + temp_file + .write_all(buf.as_ref()) + .expect("Failed to write to temp file"); + temp_file.flush().expect("Failed to flush temp file"); + + temp_file + } + + /// Create a large test file for performance testing. + fn create_large_test_file(num_rows: usize) -> NamedTempFile { + let ids: Vec = (0..num_rows as i64).collect(); + let id_buffer: Buffer = ids.into(); + let id_array = PrimitiveArray::new(id_buffer, Validity::NonNullable).into_array(); + + let values: Vec = (0..num_rows).map(|i| i as f64 * 1.5).collect(); + let value_buffer: Buffer = values.into(); + let value_array = PrimitiveArray::new(value_buffer, Validity::NonNullable).into_array(); + + let field_names: Vec> = vec![Arc::from("id"), Arc::from("value")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![id_array, value_array], + num_rows, + Validity::NonNullable, + ) + .expect("Failed to create struct array"); + + let mut buf = ByteBufferMut::empty(); + (*RUNTIME).block_on(async { + SESSION + .write_options() + .write(&mut buf, struct_array.to_array_stream()) + .await + .expect("Failed to write test file"); + }); + + let mut temp_file = NamedTempFile::new().expect("Failed to create temp file"); + temp_file + .write_all(buf.as_ref()) + .expect("Failed to write to temp file"); + temp_file.flush().expect("Failed to flush temp file"); + + temp_file + } + + // ========================================================================= + // Scanner FFI Tests + // ========================================================================= + + #[test] + fn test_scanner_new_valid_file() { + let temp_file = create_test_vortex_file(); + let path = temp_file.path().to_string_lossy().to_string(); + + let c_path = CString::new(path).expect("Failed to create CString"); + let scanner = unsafe { vortex_scanner_new(c_path.as_ptr()) }; + + assert!( + !scanner.is_null(), + "Scanner should not be null for valid file" + ); + + // Verify column count + let num_cols = unsafe { vortex_scanner_num_columns(scanner) }; + assert_eq!(num_cols, 3, "Should have 3 columns"); + + // Clean up + unsafe { vortex_scanner_free(scanner) }; + } + + #[test] + fn test_scanner_new_invalid_path() { + let c_path = + CString::new("/nonexistent/path/file.vortex").expect("Failed to create CString"); + let scanner = unsafe { vortex_scanner_new(c_path.as_ptr()) }; + + assert!(scanner.is_null(), "Scanner should be null for invalid path"); + } + + #[test] + fn test_scanner_new_empty_path() { + let c_path = CString::new("").expect("Failed to create CString"); + let scanner = unsafe { vortex_scanner_new(c_path.as_ptr()) }; + + assert!(scanner.is_null(), "Scanner should be null for empty path"); + } + + #[test] + fn test_scanner_free_null() { + // Should not crash when freeing null pointer + unsafe { vortex_scanner_free(ptr::null_mut()) }; + } + + #[test] + fn test_scanner_num_columns_null() { + let num_cols = unsafe { vortex_scanner_num_columns(ptr::null()) }; + assert_eq!(num_cols, 0, "Should return 0 for null scanner"); + } + + // ========================================================================= + // Memory Management Tests + // ========================================================================= + + #[test] + fn test_scanner_memory_lifecycle() { + // Create multiple files + for _ in 0..5 { + let temp_file = create_test_vortex_file(); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path).expect("Failed to create CString"); + + let scanner = unsafe { vortex_scanner_new(c_path.as_ptr()) }; + assert!(!scanner.is_null()); + + // Use scanner + let _ = unsafe { vortex_scanner_num_columns(scanner) }; + + // Free scanner + unsafe { vortex_scanner_free(scanner) }; + } + // No memory leaks should occur (would be caught by ASAN/MSAN in CI) + } + + #[test] + fn test_scanner_stress_allocation() { + let temp_file = create_test_vortex_file(); + let path = temp_file.path().to_string_lossy().to_string(); + + for _ in 0..100 { + let c_path = CString::new(path.clone()).expect("Failed to create CString"); + let scanner = unsafe { vortex_scanner_new(c_path.as_ptr()) }; + assert!(!scanner.is_null()); + unsafe { vortex_scanner_free(scanner) }; + } + } + + // ========================================================================= + // Nullable Data Tests + // ========================================================================= + + #[test] + fn test_scanner_with_nullable_data() { + let temp_file = create_nullable_test_file(); + let path = temp_file.path().to_string_lossy().to_string(); + + let c_path = CString::new(path).expect("Failed to create CString"); + let scanner = unsafe { vortex_scanner_new(c_path.as_ptr()) }; + + assert!(!scanner.is_null(), "Scanner should handle nullable data"); + + let num_cols = unsafe { vortex_scanner_num_columns(scanner) }; + assert_eq!(num_cols, 2); + + unsafe { vortex_scanner_free(scanner) }; + } + + // ========================================================================= + // Large Data Tests + // ========================================================================= + + #[test] + fn test_scanner_large_file() { + let temp_file = create_large_test_file(100_000); + let path = temp_file.path().to_string_lossy().to_string(); + + let c_path = CString::new(path).expect("Failed to create CString"); + let scanner = unsafe { vortex_scanner_new(c_path.as_ptr()) }; + + assert!(!scanner.is_null()); + + let num_cols = unsafe { vortex_scanner_num_columns(scanner) }; + assert_eq!(num_cols, 2); + + unsafe { vortex_scanner_free(scanner) }; + } + + // ========================================================================= + // Schema Access Tests + // ========================================================================= + + #[test] + fn test_scanner_schema_struct() { + let temp_file = create_test_vortex_file(); + let path = temp_file.path().to_string_lossy().to_string(); + + let c_path = CString::new(path).expect("Failed to create CString"); + let scanner = unsafe { vortex_scanner_new(c_path.as_ptr()) }; + + assert!(!scanner.is_null()); + + // Schema should be a struct with 3 fields + let num_cols = unsafe { vortex_scanner_num_columns(scanner) }; + assert_eq!(num_cols, 3); + + unsafe { vortex_scanner_free(scanner) }; + } +} + +// ========================================================================= +// Exporter FFI Tests +// ========================================================================= + +#[cfg(test)] +mod exporter_ffi_tests { + use std::sync::Arc; + + use vortex::array::IntoArray; + use vortex::array::arrays::{PrimitiveArray, StructArray, VarBinViewArray}; + use vortex::array::validity::Validity; + use vortex::buffer::Buffer; + use vortex::dtype::FieldNames; + + use crate::exporter::{ + ColumnExporter, PrimitiveExporter, StructExporter, VarBinViewExporter, new_exporter, + }; + + #[test] + fn test_exporter_factory_primitive_i32() { + let buffer: Buffer = vec![1i32, 2, 3, 4, 5].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let exporter = new_exporter(array).expect("Should create exporter"); + assert!(exporter.has_more()); + } + + #[test] + fn test_exporter_factory_primitive_i64() { + let buffer: Buffer = vec![1i64, 2, 3, 4, 5].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let exporter = new_exporter(array).expect("Should create exporter"); + assert!(exporter.has_more()); + } + + #[test] + fn test_exporter_factory_primitive_f64() { + let buffer: Buffer = vec![1.1f64, 2.2, 3.3].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let exporter = new_exporter(array).expect("Should create exporter"); + assert!(exporter.has_more()); + } + + #[test] + fn test_exporter_factory_string() { + let array = VarBinViewArray::from_iter_str(vec!["hello", "world", "test"]).into_array(); + + let exporter = new_exporter(array).expect("Should create exporter"); + assert!(exporter.has_more()); + } + + #[test] + fn test_exporter_factory_struct() { + let id_buffer: Buffer = vec![1i64, 2, 3].into(); + let id_array = PrimitiveArray::new(id_buffer, Validity::NonNullable).into_array(); + + let name_array = VarBinViewArray::from_iter_str(vec!["a", "b", "c"]).into_array(); + + let field_names: Vec> = vec![Arc::from("id"), Arc::from("name")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![id_array, name_array], + 3, + Validity::NonNullable, + ) + .expect("Failed to create struct"); + + let exporter = new_exporter(struct_array.into_array()).expect("Should create exporter"); + assert!(exporter.has_more()); + } + + // ------------------------------------------------------------------------- + // Primitive Exporter Tests + // ------------------------------------------------------------------------- + + #[test] + fn test_primitive_export_all_at_once() { + let buffer: Buffer = vec![10i32, 20, 30, 40, 50].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let mut exporter = PrimitiveExporter::new(array).expect("Failed to create exporter"); + + let mut output = vec![0i32; 5]; + let exported = exporter + .export( + output.as_mut_ptr() as *mut std::ffi::c_void, + size_of_val(output.as_slice()), + 5, + ) + .expect("Export failed"); + + assert_eq!(exported, 5); + assert_eq!(output, vec![10, 20, 30, 40, 50]); + assert!(!exporter.has_more()); + } + + #[test] + fn test_primitive_export_in_chunks() { + let buffer: Buffer = vec![1i64, 2, 3, 4, 5, 6, 7, 8, 9, 10].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let mut exporter = PrimitiveExporter::new(array).expect("Failed to create exporter"); + + // Export in chunks of 3 + let mut total_exported = 0; + let mut all_values = Vec::new(); + + while exporter.has_more() { + let mut chunk = vec![0i64; 3]; + let exported = exporter + .export( + chunk.as_mut_ptr() as *mut std::ffi::c_void, + size_of_val(chunk.as_slice()), + 3, + ) + .expect("Export failed"); + total_exported += exported; + all_values.extend_from_slice(&chunk[..exported]); + } + + assert_eq!(total_exported, 10); + assert_eq!(all_values, vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); + } + + #[test] + fn test_primitive_export_f32() { + let buffer: Buffer = vec![1.5f32, 2.5, 3.5].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let mut exporter = PrimitiveExporter::new(array).expect("Failed to create exporter"); + + let mut output = vec![0.0f32; 3]; + let exported = exporter + .export( + output.as_mut_ptr() as *mut std::ffi::c_void, + size_of_val(output.as_slice()), + 3, + ) + .expect("Export failed"); + + assert_eq!(exported, 3); + assert_eq!(output, vec![1.5, 2.5, 3.5]); + } + + #[test] + fn test_primitive_export_empty() { + let buffer: Buffer = vec![].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let exporter = PrimitiveExporter::new(array).expect("Failed to create exporter"); + assert!(!exporter.has_more()); + } + + #[test] + fn test_primitive_export_null_ptr_error() { + let buffer: Buffer = vec![1i32, 2, 3].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let mut exporter = PrimitiveExporter::new(array).expect("Failed to create exporter"); + + let result = exporter.export(std::ptr::null_mut(), 0, 3); + assert!(result.is_err()); + } + + // ------------------------------------------------------------------------- + // Struct Exporter Tests + // ------------------------------------------------------------------------- + + #[test] + fn test_struct_exporter_field_count() { + let id_buffer: Buffer = vec![1i64, 2, 3].into(); + let id_array = PrimitiveArray::new(id_buffer, Validity::NonNullable).into_array(); + + let value_buffer: Buffer = vec![1.1, 2.2, 3.3].into(); + let value_array = PrimitiveArray::new(value_buffer, Validity::NonNullable).into_array(); + + let name_array = VarBinViewArray::from_iter_str(vec!["a", "b", "c"]).into_array(); + + let field_names: Vec> = + vec![Arc::from("id"), Arc::from("value"), Arc::from("name")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![id_array, value_array, name_array], + 3, + Validity::NonNullable, + ) + .expect("Failed to create struct"); + + let exporter = StructExporter::new(struct_array.into_array()) + .expect("Failed to create struct exporter"); + + assert_eq!(exporter.num_fields(), 3); + } + + // ------------------------------------------------------------------------- + // VarBinView Exporter Tests + // ------------------------------------------------------------------------- + + #[test] + fn test_varbinview_exporter_creation() { + let array = VarBinViewArray::from_iter_str(vec!["hello", "world"]).into_array(); + let exporter = VarBinViewExporter::new(array).expect("Should create exporter"); + assert!(exporter.has_more()); + } + + #[test] + fn test_varbinview_exporter_empty() { + let array = VarBinViewArray::from_iter_str(Vec::<&str>::new()).into_array(); + let exporter = VarBinViewExporter::new(array).expect("Should create exporter"); + assert!(!exporter.has_more()); + } +} + +// ========================================================================= +// Column Conversion Tests +// ========================================================================= + +#[cfg(test)] +mod column_conversion_tests { + use std::ffi::c_void; + + use vortex::array::arrays::PrimitiveArray; + use vortex::array::validity::Validity; + use vortex::array::{Array, IntoArray, ToCanonical}; + use vortex::buffer::Buffer; + + use crate::convert::column::{ + VortexColumnBuilder, clickhouse_column_to_vortex, vortex_to_clickhouse_column, + }; + + // ------------------------------------------------------------------------- + // ClickHouse -> Vortex Conversion Tests + // ------------------------------------------------------------------------- + + #[test] + fn test_clickhouse_to_vortex_i32() { + let data: Vec = vec![1, 2, 3, 4, 5]; + let array = + clickhouse_column_to_vortex(data.as_ptr() as *const c_void, data.len(), "Int32") + .expect("Conversion failed"); + + assert_eq!(array.len(), 5); + let primitive = array.to_primitive(); + let values = primitive.as_slice::(); + assert_eq!(values, &[1, 2, 3, 4, 5]); + } + + #[test] + fn test_clickhouse_to_vortex_i64() { + let data: Vec = vec![100, 200, 300]; + let array = + clickhouse_column_to_vortex(data.as_ptr() as *const c_void, data.len(), "Int64") + .expect("Conversion failed"); + + assert_eq!(array.len(), 3); + let primitive = array.to_primitive(); + let values = primitive.as_slice::(); + assert_eq!(values, &[100, 200, 300]); + } + + #[test] + fn test_clickhouse_to_vortex_f64() { + let data: Vec = vec![1.1, 2.2, 3.3]; + let array = + clickhouse_column_to_vortex(data.as_ptr() as *const c_void, data.len(), "Float64") + .expect("Conversion failed"); + + assert_eq!(array.len(), 3); + let primitive = array.to_primitive(); + let values = primitive.as_slice::(); + assert_eq!(values, &[1.1, 2.2, 3.3]); + } + + #[test] + fn test_clickhouse_to_vortex_bool() { + let data: Vec = vec![1, 0, 1, 1, 0]; + let array = clickhouse_column_to_vortex(data.as_ptr() as *const c_void, data.len(), "Bool") + .expect("Conversion failed"); + + assert_eq!(array.len(), 5); + for (i, expected) in [true, false, true, true, false].iter().enumerate() { + let scalar = array.scalar_at(i).unwrap(); + assert_eq!(scalar.as_bool().value().unwrap(), *expected); + } + } + + #[test] + fn test_clickhouse_to_vortex_empty() { + let data: Vec = vec![]; + let array = clickhouse_column_to_vortex(data.as_ptr() as *const c_void, 0, "Int32") + .expect("Conversion failed"); + + assert_eq!(array.len(), 0); + } + + #[test] + fn test_clickhouse_to_vortex_null_ptr_error() { + let result = clickhouse_column_to_vortex(std::ptr::null(), 5, "Int32"); + assert!(result.is_err()); + } + + // ------------------------------------------------------------------------- + // Vortex -> ClickHouse Conversion Tests + // ------------------------------------------------------------------------- + + #[test] + fn test_vortex_to_clickhouse_i32() { + let buffer: Buffer = vec![10, 20, 30].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let mut output = vec![0i32; 3]; + vortex_to_clickhouse_column(&array, output.as_mut_ptr() as *mut c_void) + .expect("Conversion failed"); + + assert_eq!(output, vec![10, 20, 30]); + } + + #[test] + fn test_vortex_to_clickhouse_f64() { + let buffer: Buffer = vec![1.5, 2.5, 3.5].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let mut output = vec![0.0f64; 3]; + vortex_to_clickhouse_column(&array, output.as_mut_ptr() as *mut c_void) + .expect("Conversion failed"); + + assert_eq!(output, vec![1.5, 2.5, 3.5]); + } + + #[test] + fn test_vortex_to_clickhouse_null_ptr_error() { + let buffer: Buffer = vec![1, 2, 3].into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let result = vortex_to_clickhouse_column(&array, std::ptr::null_mut()); + assert!(result.is_err()); + } + + // ------------------------------------------------------------------------- + // Column Builder Tests + // ------------------------------------------------------------------------- + + #[test] + fn test_builder_i64() { + let mut builder = VortexColumnBuilder::new("Nullable(Int64)", 5).unwrap(); + + builder.append_i64(10); + builder.append_i64(20); + builder.append_null(); + builder.append_i64(40); + builder.append_i64(50); + + let array = builder.finish().unwrap(); + assert_eq!(array.len(), 5); + + // Check non-null values + let scalar = array.scalar_at(0).unwrap(); + assert!(!scalar.is_null()); + + let scalar = array.scalar_at(2).unwrap(); + assert!(scalar.is_null()); + } + + #[test] + fn test_builder_f64() { + let mut builder = VortexColumnBuilder::new("Float64", 3).unwrap(); + + builder.append_f64(1.1); + builder.append_f64(2.2); + builder.append_f64(3.3); + + let array = builder.finish().unwrap(); + assert_eq!(array.len(), 3); + } + + #[test] + fn test_builder_string() { + let mut builder = VortexColumnBuilder::new("String", 4).unwrap(); + + builder.append_string("hello"); + builder.append_null(); + builder.append_string("world"); + builder.append_string("!"); + + let array = builder.finish().unwrap(); + assert_eq!(array.len(), 4); + + let scalar = array.scalar_at(0).unwrap(); + assert!(!scalar.is_null()); + + let scalar = array.scalar_at(1).unwrap(); + assert!(scalar.is_null()); + } + + #[test] + fn test_builder_nullable_int() { + let mut builder = VortexColumnBuilder::new("Nullable(Int32)", 3).unwrap(); + + // Nullable(Int32) should be I32 with nullable flag + builder.append_null(); + + let array = builder.finish().unwrap(); + assert_eq!(array.len(), 1); + } + + #[test] + fn test_builder_empty() { + let builder = VortexColumnBuilder::new("Int64", 0).unwrap(); + let array = builder.finish().unwrap(); + assert_eq!(array.len(), 0); + } +} + +// ========================================================================= +// End-to-End Integration Tests +// ========================================================================= + +#[cfg(test)] +mod e2e_integration_tests { + use std::ffi::CString; + use std::io::Write; + use std::sync::Arc; + + use tempfile::NamedTempFile; + use vortex::array::IntoArray; + use vortex::array::arrays::{PrimitiveArray, StructArray}; + use vortex::array::stream::ArrayStreamExt; + use vortex::array::validity::Validity; + use vortex::buffer::{Buffer, ByteBufferMut}; + use vortex::dtype::{DType, FieldNames}; + use vortex::file::WriteOptionsSessionExt; + use vortex::io::runtime::BlockingRuntime; + + use crate::convert::dtype::vortex_to_clickhouse_type; + use crate::exporter::{ColumnExporter, PrimitiveExporter}; + use crate::scan::{ + VortexScanner, vortex_scanner_free, vortex_scanner_new, vortex_scanner_num_columns, + }; + use crate::{RUNTIME, SESSION}; + + fn create_simple_test_file() -> NamedTempFile { + let original_data: Vec = vec![100, 200, 300, 400, 500]; + let id_buffer: Buffer = original_data.clone().into(); + let id_array = PrimitiveArray::new(id_buffer, Validity::NonNullable).into_array(); + + let field_names: Vec> = vec![Arc::from("id")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![id_array], + 5, + Validity::NonNullable, + ) + .expect("Failed to create struct"); + + let mut buf = ByteBufferMut::empty(); + (*RUNTIME).block_on(async { + SESSION + .write_options() + .write(&mut buf, struct_array.to_array_stream()) + .await + .expect("Failed to write"); + }); + + let mut temp_file = NamedTempFile::new().expect("Failed to create temp file"); + temp_file + .write_all(buf.as_ref()) + .expect("Failed to write to temp file"); + temp_file.flush().expect("Failed to flush temp file"); + + temp_file + } + + /// Test complete read workflow: Create file -> Open scanner -> Export data + #[test] + fn test_complete_read_workflow() { + let temp_file = create_simple_test_file(); + let path_str = temp_file.path().to_string_lossy().to_string(); + + // Open scanner via FFI + let c_path = CString::new(path_str.clone()).expect("CString failed"); + let scanner = unsafe { vortex_scanner_new(c_path.as_ptr()) }; + assert!(!scanner.is_null()); + + // Verify schema + let num_cols = unsafe { vortex_scanner_num_columns(scanner) }; + assert_eq!(num_cols, 1); + + // Read back via Rust API + let scanner_obj = VortexScanner::new(&path_str).expect("Failed to create scanner"); + let schema = scanner_obj.schema(); + assert!(matches!(schema, DType::Struct(..))); + + // Clean up + unsafe { vortex_scanner_free(scanner) }; + } + + /// Test complete write workflow: Build columns -> Write file -> Read back + #[test] + fn test_complete_write_workflow() { + // Simulate ClickHouse column data + let id_data: Vec = vec![1, 2, 3, 4, 5]; + let value_data: Vec = vec![1.1, 2.2, 3.3, 4.4, 5.5]; + + // Convert to Vortex arrays + let id_array = { + let buffer: Buffer = id_data.clone().into(); + PrimitiveArray::new(buffer, Validity::NonNullable).into_array() + }; + let value_array = { + let buffer: Buffer = value_data.clone().into(); + PrimitiveArray::new(buffer, Validity::NonNullable).into_array() + }; + + // Create struct and write + let field_names: Vec> = vec![Arc::from("id"), Arc::from("value")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![id_array, value_array], + 5, + Validity::NonNullable, + ) + .expect("Failed to create struct"); + + let mut buf = ByteBufferMut::empty(); + (*RUNTIME).block_on(async { + SESSION + .write_options() + .write(&mut buf, struct_array.to_array_stream()) + .await + .expect("Failed to write"); + }); + + let mut temp_file = NamedTempFile::new().expect("Failed to create temp file"); + temp_file + .write_all(buf.as_ref()) + .expect("Failed to write to temp file"); + temp_file.flush().expect("Failed to flush temp file"); + + // Read back and verify + let path_str = temp_file.path().to_string_lossy().to_string(); + let scanner = VortexScanner::new(&path_str).expect("Failed to create scanner"); + assert!(matches!(scanner.schema(), DType::Struct(..))); + } + + /// Test type conversion roundtrip with actual file I/O + #[test] + fn test_type_conversion_with_file_io() { + // Test various primitive types + let test_cases: Vec<(&str, Box vortex::array::ArrayRef>)> = vec![ + ( + "Int8", + Box::new(|| { + let buffer: Buffer = vec![1i8, -1, 127, -128].into(); + PrimitiveArray::new(buffer, Validity::NonNullable).into_array() + }), + ), + ( + "Int16", + Box::new(|| { + let buffer: Buffer = vec![1i16, -1, 32767].into(); + PrimitiveArray::new(buffer, Validity::NonNullable).into_array() + }), + ), + ( + "Int32", + Box::new(|| { + let buffer: Buffer = vec![1i32, -1, 2147483647].into(); + PrimitiveArray::new(buffer, Validity::NonNullable).into_array() + }), + ), + ( + "Int64", + Box::new(|| { + let buffer: Buffer = vec![1i64, -1, 9223372036854775807].into(); + PrimitiveArray::new(buffer, Validity::NonNullable).into_array() + }), + ), + ( + "UInt8", + Box::new(|| { + let buffer: Buffer = vec![0u8, 1, 255].into(); + PrimitiveArray::new(buffer, Validity::NonNullable).into_array() + }), + ), + ( + "UInt64", + Box::new(|| { + let buffer: Buffer = vec![0u64, 1, 18446744073709551615].into(); + PrimitiveArray::new(buffer, Validity::NonNullable).into_array() + }), + ), + ( + "Float32", + Box::new(|| { + let buffer: Buffer = vec![0.0f32, 1.5, -3.14].into(); + PrimitiveArray::new(buffer, Validity::NonNullable).into_array() + }), + ), + ( + "Float64", + Box::new(|| { + let buffer: Buffer = vec![0.0f64, 1.5, -3.14159265359].into(); + PrimitiveArray::new(buffer, Validity::NonNullable).into_array() + }), + ), + ]; + + for (type_name, create_array) in test_cases { + let array = create_array(); + let original_len = array.len(); + + // Wrap in struct + let field_names: Vec> = vec![Arc::from("col")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![array], + original_len, + Validity::NonNullable, + ) + .unwrap(); + + // Write to buffer + let mut buf = ByteBufferMut::empty(); + (*RUNTIME).block_on(async { + SESSION + .write_options() + .write(&mut buf, struct_array.to_array_stream()) + .await + .expect(&format!("Failed to write {}", type_name)); + }); + + // Write buffer to temp file + let mut temp_file = NamedTempFile::new() + .expect(&format!("Failed to create temp file for {}", type_name)); + temp_file + .write_all(buf.as_ref()) + .expect(&format!("Failed to write to temp file for {}", type_name)); + temp_file.flush().expect("Failed to flush"); + + // Read back + let path_str = temp_file.path().to_string_lossy().to_string(); + let scanner = VortexScanner::new(&path_str) + .expect(&format!("Failed to create scanner for {}", type_name)); + + let schema = scanner.schema(); + if let DType::Struct(fields, _) = schema { + let field_dtype = fields.field_by_index(0).unwrap(); + + // Verify type mapping + let ch_type = vortex_to_clickhouse_type(&field_dtype) + .expect(&format!("Failed to convert dtype for {}", type_name)); + + // Should match or be nullable version + assert!( + ch_type == type_name || ch_type == format!("Nullable({})", type_name), + "Type mismatch for {}: expected {} or Nullable({}), got {}", + type_name, + type_name, + type_name, + ch_type + ); + } + } + } + + /// Test exporter data integrity + #[test] + fn test_exporter_data_integrity() { + let original: Vec = (0..1000).collect(); + let buffer: Buffer = original.clone().into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let mut exporter = PrimitiveExporter::new(array).expect("Failed to create exporter"); + + let mut exported: Vec = Vec::with_capacity(1000); + while exporter.has_more() { + let mut chunk = vec![0i64; 100]; + let count = exporter + .export( + chunk.as_mut_ptr() as *mut std::ffi::c_void, + size_of_val(chunk.as_slice()), + 100, + ) + .expect("Export failed"); + exported.extend_from_slice(&chunk[..count]); + } + + assert_eq!(original, exported, "Data mismatch after export"); + } +} + +// ========================================================================= +// Error Handling Tests +// ========================================================================= + +#[cfg(test)] +mod error_handling_tests { + use crate::convert::column::VortexColumnBuilder; + use crate::convert::dtype::clickhouse_type_to_vortex; + use crate::scan::VortexScanner; + + #[test] + fn test_scanner_invalid_path() { + let result = VortexScanner::new("/this/path/does/not/exist.vortex"); + assert!(result.is_err()); + } + + #[test] + fn test_scanner_invalid_glob() { + // Invalid glob pattern + let result = VortexScanner::new("/path/[invalid"); + assert!(result.is_err()); + } + + #[test] + fn test_type_conversion_supported_special_types() { + // These ClickHouse types are all supported and should parse successfully + assert!(clickhouse_type_to_vortex("IPv4").is_ok()); + assert!(clickhouse_type_to_vortex("IPv6").is_ok()); + assert!(clickhouse_type_to_vortex("Enum8('a'=1)").is_ok()); + assert!(clickhouse_type_to_vortex("Map(String, Int32)").is_ok()); + } + + #[test] + fn test_type_conversion_unsupported() { + // Genuinely unsupported ClickHouse types + let unsupported = vec![ + "Nothing", + "SimpleAggregateFunction(sum, Int64)", + "AggregateFunction(uniq, String)", + ]; + + for type_str in unsupported { + let result = clickhouse_type_to_vortex(type_str); + assert!( + result.is_err(), + "Expected '{}' to be unsupported, but it succeeded", + type_str, + ); + } + } + + #[test] + fn test_builder_unsupported_type() { + // Complex types not yet supported in builder + let result = VortexColumnBuilder::new("Array(Int32)", 10); + assert!(result.is_err()); + + let result = VortexColumnBuilder::new("Tuple(Int32, String)", 10); + assert!(result.is_err()); + } +} + +// ========================================================================= +// Writer FFI Tests +// ========================================================================= + +#[cfg(test)] +mod writer_ffi_tests { + use std::ffi::{CString, c_void}; + use std::ptr; + + use tempfile::NamedTempFile; + + use crate::copy::{ + vortex_writer_add_column, vortex_writer_begin_batch, vortex_writer_end_batch, + vortex_writer_finalize, vortex_writer_free, vortex_writer_new, vortex_writer_num_columns, + vortex_writer_total_rows, vortex_writer_write_batch, vortex_writer_write_column, + vortex_writer_write_column_nullable, vortex_writer_write_string_column, + vortex_writer_write_string_column_nullable, + }; + use crate::scan::{vortex_scanner_free, vortex_scanner_new, vortex_scanner_num_columns}; + + // ------------------------------------------------------------------------- + // Null Pointer Safety + // ------------------------------------------------------------------------- + + #[test] + fn test_writer_ffi_null_path() { + let writer = unsafe { vortex_writer_new(ptr::null()) }; + assert!(writer.is_null()); + } + + #[test] + fn test_writer_ffi_free_null() { + // Should not panic when freeing null pointer + unsafe { vortex_writer_free(ptr::null_mut()) }; + } + + #[test] + fn test_writer_ffi_add_column_null_writer() { + let name = CString::new("id").unwrap(); + let ch_type = CString::new("Int64").unwrap(); + let result = unsafe { + vortex_writer_add_column(ptr::null_mut(), name.as_ptr(), ch_type.as_ptr(), 0) + }; + assert!(result < 0); + } + + #[test] + fn test_writer_ffi_begin_batch_null_writer() { + let result = unsafe { vortex_writer_begin_batch(ptr::null_mut(), 10) }; + assert!(result < 0); + } + + #[test] + fn test_writer_ffi_write_column_null_writer() { + let data: Vec = vec![1, 2, 3]; + let result = unsafe { + vortex_writer_write_column(ptr::null_mut(), 0, data.as_ptr() as *const c_void, 3) + }; + assert!(result < 0); + } + + #[test] + fn test_writer_ffi_write_column_null_data() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path).unwrap(); + + unsafe { + let writer = vortex_writer_new(c_path.as_ptr()); + assert!(!writer.is_null()); + + let name = CString::new("id").unwrap(); + let ch_type = CString::new("Int64").unwrap(); + vortex_writer_add_column(writer, name.as_ptr(), ch_type.as_ptr(), 0); + + vortex_writer_begin_batch(writer, 3); + let result = vortex_writer_write_column(writer, 0, ptr::null(), 3); + assert!(result < 0); + + vortex_writer_free(writer); + } + } + + #[test] + fn test_writer_ffi_finalize_null_writer() { + let result = unsafe { vortex_writer_finalize(ptr::null_mut()) }; + assert!(result < 0); + } + + #[test] + fn test_writer_ffi_num_columns_null() { + let num_cols = unsafe { vortex_writer_num_columns(ptr::null()) }; + assert_eq!(num_cols, 0); + } + + #[test] + fn test_writer_ffi_total_rows_null() { + let rows = unsafe { vortex_writer_total_rows(ptr::null()) }; + assert_eq!(rows, 0); + } + + // ------------------------------------------------------------------------- + // Write Workflow via FFI + // ------------------------------------------------------------------------- + + #[test] + fn test_writer_ffi_primitive_write_workflow() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path.clone()).unwrap(); + + unsafe { + let writer = vortex_writer_new(c_path.as_ptr()); + assert!(!writer.is_null()); + + // Add columns + let id_name = CString::new("id").unwrap(); + let id_type = CString::new("Int64").unwrap(); + assert_eq!( + vortex_writer_add_column(writer, id_name.as_ptr(), id_type.as_ptr(), 0), + 0 + ); + + let val_name = CString::new("value").unwrap(); + let val_type = CString::new("Float64").unwrap(); + assert_eq!( + vortex_writer_add_column(writer, val_name.as_ptr(), val_type.as_ptr(), 0), + 0 + ); + + assert_eq!(vortex_writer_num_columns(writer), 2); + + // Write batch + assert_eq!(vortex_writer_begin_batch(writer, 3), 0); + + let ids: Vec = vec![1, 2, 3]; + assert_eq!( + vortex_writer_write_column(writer, 0, ids.as_ptr() as *const c_void, 3), + 0 + ); + + let values: Vec = vec![1.1, 2.2, 3.3]; + assert_eq!( + vortex_writer_write_column(writer, 1, values.as_ptr() as *const c_void, 3), + 0 + ); + + assert_eq!(vortex_writer_end_batch(writer), 0); + assert_eq!(vortex_writer_total_rows(writer), 3); + + assert_eq!(vortex_writer_finalize(writer), 0); + vortex_writer_free(writer); + } + + assert!(std::path::Path::new(&path).exists()); + } + + #[test] + fn test_writer_ffi_string_column() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path.clone()).unwrap(); + + unsafe { + let writer = vortex_writer_new(c_path.as_ptr()); + assert!(!writer.is_null()); + + let name = CString::new("name").unwrap(); + let ch_type = CString::new("String").unwrap(); + assert_eq!( + vortex_writer_add_column(writer, name.as_ptr(), ch_type.as_ptr(), 0), + 0 + ); + + assert_eq!(vortex_writer_begin_batch(writer, 3), 0); + + let strings = b"AliceBobCharlie"; + let offsets: Vec = vec![0, 5, 8, 15]; + assert_eq!( + vortex_writer_write_string_column(writer, 0, strings.as_ptr(), offsets.as_ptr(), 3,), + 0 + ); + + assert_eq!(vortex_writer_end_batch(writer), 0); + assert_eq!(vortex_writer_finalize(writer), 0); + vortex_writer_free(writer); + } + + assert!(std::path::Path::new(&path).exists()); + } + + #[test] + fn test_writer_ffi_nullable_column() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path.clone()).unwrap(); + + unsafe { + let writer = vortex_writer_new(c_path.as_ptr()); + assert!(!writer.is_null()); + + let name = CString::new("value").unwrap(); + let ch_type = CString::new("Int64").unwrap(); + assert_eq!( + vortex_writer_add_column(writer, name.as_ptr(), ch_type.as_ptr(), 1), + 0 + ); + + assert_eq!(vortex_writer_begin_batch(writer, 4), 0); + + let data: Vec = vec![10, 0, 30, 0]; + // ClickHouse null_map convention: 0 = valid, 1 = null + let null_map: Vec = vec![0, 1, 0, 1]; + assert_eq!( + vortex_writer_write_column_nullable( + writer, + 0, + data.as_ptr() as *const c_void, + null_map.as_ptr(), + 4, + ), + 0 + ); + + assert_eq!(vortex_writer_end_batch(writer), 0); + assert_eq!(vortex_writer_total_rows(writer), 4); + assert_eq!(vortex_writer_finalize(writer), 0); + vortex_writer_free(writer); + } + + assert!(std::path::Path::new(&path).exists()); + } + + #[test] + fn test_writer_ffi_nullable_string_column() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path.clone()).unwrap(); + + unsafe { + let writer = vortex_writer_new(c_path.as_ptr()); + assert!(!writer.is_null()); + + let name = CString::new("name").unwrap(); + let ch_type = CString::new("String").unwrap(); + assert_eq!( + vortex_writer_add_column(writer, name.as_ptr(), ch_type.as_ptr(), 1), + 0 + ); + + assert_eq!(vortex_writer_begin_batch(writer, 3), 0); + + let strings = b"AliceCharlie"; + let offsets: Vec = vec![0, 5, 5, 12]; // "Alice", "", "Charlie" + let null_map: Vec = vec![0, 1, 0]; // second is null + assert_eq!( + vortex_writer_write_string_column_nullable( + writer, + 0, + strings.as_ptr(), + offsets.as_ptr(), + null_map.as_ptr(), + 3, + ), + 0 + ); + + assert_eq!(vortex_writer_end_batch(writer), 0); + assert_eq!(vortex_writer_finalize(writer), 0); + vortex_writer_free(writer); + } + + assert!(std::path::Path::new(&path).exists()); + } + + #[test] + fn test_writer_ffi_write_batch_simplified() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path.clone()).unwrap(); + + unsafe { + let writer = vortex_writer_new(c_path.as_ptr()); + assert!(!writer.is_null()); + + let id_name = CString::new("id").unwrap(); + let id_type = CString::new("Int64").unwrap(); + vortex_writer_add_column(writer, id_name.as_ptr(), id_type.as_ptr(), 0); + + let val_name = CString::new("value").unwrap(); + let val_type = CString::new("Float64").unwrap(); + vortex_writer_add_column(writer, val_name.as_ptr(), val_type.as_ptr(), 0); + + let ids: Vec = vec![1, 2, 3]; + let values: Vec = vec![1.1, 2.2, 3.3]; + let column_ptrs: Vec<*const c_void> = vec![ + ids.as_ptr() as *const c_void, + values.as_ptr() as *const c_void, + ]; + + assert_eq!( + vortex_writer_write_batch(writer, column_ptrs.as_ptr(), 2, 3), + 0 + ); + assert_eq!(vortex_writer_total_rows(writer), 3); + + assert_eq!(vortex_writer_finalize(writer), 0); + vortex_writer_free(writer); + } + + assert!(std::path::Path::new(&path).exists()); + } + + // ------------------------------------------------------------------------- + // Write-then-Read Roundtrip via FFI + // ------------------------------------------------------------------------- + + #[test] + fn test_writer_ffi_write_then_read_roundtrip() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path).unwrap(); + + // Write via FFI + unsafe { + let writer = vortex_writer_new(c_path.as_ptr()); + assert!(!writer.is_null()); + + let id_name = CString::new("id").unwrap(); + let id_type = CString::new("Int64").unwrap(); + vortex_writer_add_column(writer, id_name.as_ptr(), id_type.as_ptr(), 0); + + let data: Vec = vec![100, 200, 300, 400, 500]; + let column_ptrs: Vec<*const c_void> = vec![data.as_ptr() as *const c_void]; + assert_eq!( + vortex_writer_write_batch(writer, column_ptrs.as_ptr(), 1, 5), + 0 + ); + + assert_eq!(vortex_writer_finalize(writer), 0); + vortex_writer_free(writer); + } + + // Read back via FFI scanner + unsafe { + let scanner = vortex_scanner_new(c_path.as_ptr()); + assert!( + !scanner.is_null(), + "Scanner should open file written by writer" + ); + + let num_cols = vortex_scanner_num_columns(scanner); + assert_eq!(num_cols, 1); + + vortex_scanner_free(scanner); + } + } + + #[test] + fn test_writer_ffi_multiple_batches() { + let temp_file = NamedTempFile::new().expect("Failed to create temp file"); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path.clone()).unwrap(); + + unsafe { + let writer = vortex_writer_new(c_path.as_ptr()); + assert!(!writer.is_null()); + + let name = CString::new("id").unwrap(); + let ch_type = CString::new("Int64").unwrap(); + vortex_writer_add_column(writer, name.as_ptr(), ch_type.as_ptr(), 0); + + // Batch 1 + let data1: Vec = vec![1, 2, 3]; + let ptrs1: Vec<*const c_void> = vec![data1.as_ptr() as *const c_void]; + assert_eq!(vortex_writer_write_batch(writer, ptrs1.as_ptr(), 1, 3), 0); + + // Batch 2 + let data2: Vec = vec![4, 5]; + let ptrs2: Vec<*const c_void> = vec![data2.as_ptr() as *const c_void]; + assert_eq!(vortex_writer_write_batch(writer, ptrs2.as_ptr(), 1, 2), 0); + + assert_eq!(vortex_writer_total_rows(writer), 5); + assert_eq!(vortex_writer_finalize(writer), 0); + vortex_writer_free(writer); + } + + assert!(std::path::Path::new(&path).exists()); + } +} + +// ========================================================================= +// Nullable Export Validity FFI Tests +// ========================================================================= + +#[cfg(test)] +mod nullable_export_validity_tests { + use std::ffi::CString; + use std::io::Write; + use std::sync::Arc; + + use tempfile::NamedTempFile; + use vortex::array::IntoArray; + use vortex::array::arrays::{PrimitiveArray, StructArray, VarBinViewArray}; + use vortex::array::validity::Validity; + use vortex::buffer::ByteBufferMut; + use vortex::dtype::FieldNames; + use vortex::file::WriteOptionsSessionExt; + use vortex::io::runtime::BlockingRuntime; + + use crate::scan::{ + vortex_exporter_export, vortex_exporter_export_strings, vortex_exporter_export_validity, + vortex_exporter_free, vortex_exporter_get_field, vortex_exporter_is_nullable, + vortex_exporter_len, vortex_exporter_num_fields, vortex_exporter_string_data_size, + vortex_scanner_free, vortex_scanner_has_more, vortex_scanner_new, + vortex_scanner_read_batch, + }; + use crate::{RUNTIME, SESSION}; + + /// Create a test file with a nullable Int64 column and a nullable String column. + fn create_nullable_file() -> NamedTempFile { + let values: Vec> = vec![Some(10), None, Some(30), None, Some(50)]; + let int_array = PrimitiveArray::from_option_iter(values).into_array(); + + let str_values: Vec> = vec![Some("a"), None, Some("c"), Some("d"), None]; + let str_array = VarBinViewArray::from_iter_nullable_str(str_values).into_array(); + + let field_names: Vec> = vec![Arc::from("id"), Arc::from("name")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![int_array, str_array], + 5, + Validity::NonNullable, + ) + .expect("Failed to create struct array"); + + let mut buf = ByteBufferMut::empty(); + (*RUNTIME).block_on(async { + SESSION + .write_options() + .write(&mut buf, struct_array.to_array_stream()) + .await + .expect("Failed to write test file"); + }); + + let mut temp_file = NamedTempFile::new().expect("Failed to create temp file"); + temp_file + .write_all(buf.as_ref()) + .expect("Failed to write to temp file"); + temp_file.flush().expect("Failed to flush temp file"); + + temp_file + } + + #[test] + fn test_export_validity_nullable_primitive() { + let temp_file = create_nullable_file(); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path).unwrap(); + + unsafe { + let scanner = vortex_scanner_new(c_path.as_ptr()); + assert!(!scanner.is_null()); + + // Read a batch + assert_eq!(vortex_scanner_has_more(scanner), 1); + let batch = vortex_scanner_read_batch(scanner); + assert!(!batch.is_null()); + + // The batch is a struct exporter; get the first field (nullable Int64) + let num_fields = vortex_exporter_num_fields(batch); + assert!(num_fields >= 2); + + let int_exporter = vortex_exporter_get_field(batch, 0); + assert!(!int_exporter.is_null()); + + // Verify it is nullable + let is_nullable = vortex_exporter_is_nullable(int_exporter); + assert_eq!(is_nullable, 1, "Int64 column should be nullable"); + + let len = vortex_exporter_len(int_exporter); + assert_eq!(len, 5); + + // Export data first (required before export_validity) + let mut data_buf = vec![0i64; 5]; + let exported = vortex_exporter_export( + int_exporter, + data_buf.as_mut_ptr() as *mut std::ffi::c_void, + size_of_val(data_buf.as_slice()), + 5, + ); + assert_eq!(exported, 5); + + // Export validity bitmap + // The Vortex bitmap uses 1-bit per row packed into bytes. + let bitmap_size = 5_usize.div_ceil(8); + let mut validity_bitmap = vec![0u8; bitmap_size]; + let validity_rows = + vortex_exporter_export_validity(int_exporter, validity_bitmap.as_mut_ptr(), 5); + assert_eq!(validity_rows, 5); + + // Non-null values at index 0, 2, 4; null at index 1, 3 + // In a packed bitmap (LSB first): bit 0=1, bit 1=0, bit 2=1, bit 3=0, bit 4=1 + // = 0b00010101 = 0x15 + assert_eq!( + validity_bitmap[0], 0b00010101, + "Validity bitmap should mark indices 1 and 3 as null" + ); + + vortex_exporter_free(int_exporter); + vortex_exporter_free(batch); + vortex_scanner_free(scanner); + } + } + + #[test] + fn test_export_validity_nullable_string() { + let temp_file = create_nullable_file(); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path).unwrap(); + + unsafe { + let scanner = vortex_scanner_new(c_path.as_ptr()); + assert!(!scanner.is_null()); + + let batch = vortex_scanner_read_batch(scanner); + assert!(!batch.is_null()); + + // Get the second field (nullable String) + let str_exporter = vortex_exporter_get_field(batch, 1); + assert!(!str_exporter.is_null()); + + let is_nullable = vortex_exporter_is_nullable(str_exporter); + assert_eq!(is_nullable, 1, "String column should be nullable"); + + let len = vortex_exporter_len(str_exporter); + assert_eq!(len, 5); + + // For string exporters, we must export strings first to set the + // last_export_start/last_export_count used by export_validity. + let mut total_bytes: usize = 0; + let mut num_rows: usize = 0; + let size_result = vortex_exporter_string_data_size( + str_exporter, + &raw mut total_bytes, + &raw mut num_rows, + ); + assert_eq!(size_result, 0); + assert_eq!(num_rows, 5); + + let mut data_buf = vec![0u8; total_bytes.max(1)]; + let mut lengths_buf = vec![0u32; 5]; + let mut offsets_buf = vec![0u64; 5]; + let exported = vortex_exporter_export_strings( + str_exporter, + data_buf.as_mut_ptr(), + lengths_buf.as_mut_ptr(), + offsets_buf.as_mut_ptr(), + 5, + ); + assert_eq!(exported, 5); + + // Now export validity bitmap + let bitmap_size = 5_usize.div_ceil(8); + let mut validity_bitmap = vec![0u8; bitmap_size]; + let validity_rows = + vortex_exporter_export_validity(str_exporter, validity_bitmap.as_mut_ptr(), 5); + assert_eq!(validity_rows, 5); + + // Non-null at 0, 2, 3; null at 1, 4 + // Packed LSB first: bit 0=1, 1=0, 2=1, 3=1, 4=0 = 0b00001101 = 0x0D + assert_eq!( + validity_bitmap[0], 0b00001101, + "Validity bitmap should mark indices 1 and 4 as null" + ); + + vortex_exporter_free(str_exporter); + vortex_exporter_free(batch); + vortex_scanner_free(scanner); + } + } + + #[test] + fn test_export_validity_null_handle() { + let result = unsafe { + vortex_exporter_export_validity(std::ptr::null_mut(), std::ptr::null_mut(), 5) + }; + assert!(result < 0, "Should return error for null handle"); + } + + #[test] + fn test_export_validity_null_bitmap() { + let temp_file = create_nullable_file(); + let path = temp_file.path().to_string_lossy().to_string(); + let c_path = CString::new(path).unwrap(); + + unsafe { + let scanner = vortex_scanner_new(c_path.as_ptr()); + let batch = vortex_scanner_read_batch(scanner); + let int_exporter = vortex_exporter_get_field(batch, 0); + + // Pass null bitmap pointer + let result = vortex_exporter_export_validity(int_exporter, std::ptr::null_mut(), 5); + assert!(result < 0, "Should return error for null bitmap pointer"); + + vortex_exporter_free(int_exporter); + vortex_exporter_free(batch); + vortex_scanner_free(scanner); + } + } +} + +// ========================================================================= +// Performance / Stress Tests +// ========================================================================= + +#[cfg(test)] +mod performance_tests { + use std::io::Write; + use std::sync::Arc; + + use tempfile::NamedTempFile; + use vortex::array::IntoArray; + use vortex::array::arrays::{PrimitiveArray, StructArray}; + use vortex::array::stream::ArrayStreamExt; + use vortex::array::validity::Validity; + use vortex::buffer::{Buffer, ByteBufferMut}; + use vortex::dtype::FieldNames; + use vortex::file::WriteOptionsSessionExt; + use vortex::io::runtime::BlockingRuntime; + + use crate::exporter::{ColumnExporter, PrimitiveExporter}; + use crate::scan::VortexScanner; + use crate::{RUNTIME, SESSION}; + + #[test] + fn test_large_column_export() { + let num_rows = 1_000_000; + let data: Vec = (0..num_rows as i64).collect(); + let buffer: Buffer = data.into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let mut exporter = PrimitiveExporter::new(array).expect("Failed to create exporter"); + + let mut total = 0usize; + let chunk_size = 65536; + let mut output = vec![0i64; chunk_size]; + + while exporter.has_more() { + let exported = exporter + .export( + output.as_mut_ptr() as *mut std::ffi::c_void, + size_of_val(output.as_slice()), + chunk_size, + ) + .expect("Export failed"); + total += exported; + } + + assert_eq!(total, num_rows); + } + + #[test] + fn test_scanner_large_file() { + let num_rows = 500_000; + let data: Vec = (0..num_rows as i64).collect(); + let buffer: Buffer = data.into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let field_names: Vec> = vec![Arc::from("id")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![array], + num_rows, + Validity::NonNullable, + ) + .unwrap(); + + let mut buf = ByteBufferMut::empty(); + (*RUNTIME).block_on(async { + SESSION + .write_options() + .write(&mut buf, struct_array.to_array_stream()) + .await + .expect("Failed to write"); + }); + + let mut temp_file = NamedTempFile::new().expect("Failed to create temp file"); + temp_file + .write_all(buf.as_ref()) + .expect("Failed to write to temp file"); + temp_file.flush().expect("Failed to flush"); + + let path_str = temp_file.path().to_string_lossy().to_string(); + + // Time scanner creation + let start = std::time::Instant::now(); + let scanner = VortexScanner::new(&path_str).expect("Failed to create scanner"); + let elapsed = start.elapsed(); + + // Should be fast (< 1 second for just opening) + assert!( + elapsed.as_secs() < 2, + "Scanner creation too slow: {:?}", + elapsed + ); + + assert_eq!(scanner.file_paths().len(), 1); + } + + #[test] + fn test_repeated_scanner_creation() { + let data: Vec = vec![1, 2, 3, 4, 5]; + let buffer: Buffer = data.into(); + let array = PrimitiveArray::new(buffer, Validity::NonNullable).into_array(); + + let field_names: Vec> = vec![Arc::from("id")]; + let struct_array = StructArray::try_new( + FieldNames::from(field_names), + vec![array], + 5, + Validity::NonNullable, + ) + .unwrap(); + + let mut buf = ByteBufferMut::empty(); + (*RUNTIME).block_on(async { + SESSION + .write_options() + .write(&mut buf, struct_array.to_array_stream()) + .await + .expect("Failed to write"); + }); + + let mut temp_file = NamedTempFile::new().expect("Failed to create temp file"); + temp_file + .write_all(buf.as_ref()) + .expect("Failed to write to temp file"); + temp_file.flush().expect("Failed to flush"); + + let path_str = temp_file.path().to_string_lossy().to_string(); + + // Create many scanners repeatedly + for _ in 0..100 { + let scanner = VortexScanner::new(&path_str).expect("Failed to create scanner"); + assert_eq!(scanner.file_paths().len(), 1); + } + } +} diff --git a/vortex-clickhouse/src/lib.rs b/vortex-clickhouse/src/lib.rs new file mode 100644 index 00000000000..8d218ebe876 --- /dev/null +++ b/vortex-clickhouse/src/lib.rs @@ -0,0 +1,108 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Vortex ClickHouse Extension +//! +//! This crate provides ClickHouse integration for the Vortex columnar format. +//! It enables ClickHouse to read and write Vortex files with support for +//! predicate and projection pushdown. +//! +//! # Architecture +//! +//! The crate is organized into three layers: +//! +//! 1. **C++ Layer** (`cpp/`): Implements ClickHouse's `IInputFormat` and `IOutputFormat` +//! interfaces, which are the entry points for the ClickHouse format system. +//! +//! 2. **FFI Boundary**: Uses `bindgen` to import ClickHouse types and `cbindgen` to +//! export Rust functions to C++. +//! +//! 3. **Rust Core**: Implements the actual Vortex file reading/writing logic, +//! type conversion, and query optimization features. +//! +//! # Usage +//! +//! ```sql +//! -- Read from Vortex file +//! SELECT * FROM file('data.vortex', 'Vortex'); +//! +//! -- With predicate pushdown +//! SELECT * FROM file('data.vortex', 'Vortex') WHERE x > 100; +//! ``` + +#![allow(clippy::missing_safety_doc)] + +use std::sync::LazyLock; + +use vortex::VortexSessionDefault; +use vortex::dtype::session::DTypeSessionExt; +use vortex::io::runtime::BlockingRuntime; +use vortex::io::runtime::current::CurrentThreadRuntime; +use vortex::io::session::RuntimeSessionExt; +use vortex::session::VortexSession; + +use crate::ext_types::{ + BigInt, ClickHouseDate, ClickHouseDateTime, ClickHouseEnum, ClickHouseFixedString, + ClickHouseLowCardinality, Geo, IPAddress, UUID, +}; + +pub mod clickhouse; +pub mod convert; +mod copy; +pub mod error; +pub mod exporter; +pub mod ext_types; +mod scan; +mod utils; + +#[rustfmt::skip] +#[path = "./cpp.rs"] +/// This module provides the FFI interface to our C++ code exposing additional functionality +/// for ClickHouse, such as custom data types and functions. +/// cbindgen:ignore +mod cpp; + +#[cfg(test)] +mod e2e_test; + +#[cfg(test)] +mod ffi_tests; + +// A global runtime for Vortex operations within ClickHouse. +static RUNTIME: LazyLock = LazyLock::new(CurrentThreadRuntime::new); +static SESSION: LazyLock = LazyLock::new(|| { + let session = VortexSession::default().with_handle(RUNTIME.handle()); + // Register ClickHouse-specific extension types + session.dtypes().register(BigInt); + session.dtypes().register(Geo); + session.dtypes().register(ClickHouseEnum); + session.dtypes().register(ClickHouseDateTime); + session.dtypes().register(ClickHouseDate); + session.dtypes().register(ClickHouseLowCardinality); + session.dtypes().register(ClickHouseFixedString); + session.dtypes().register(UUID); + session.dtypes().register(IPAddress); + session +}); + +/// Get the global Vortex session used for ClickHouse operations. +pub fn session() -> &'static VortexSession { + &SESSION +} + +/// Get the global runtime handle for async operations. +pub fn runtime() -> &'static CurrentThreadRuntime { + &RUNTIME +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_session_initialization() { + // Just verify that the lazy static initializes without panicking + let _session = session(); + let _runtime = runtime(); + } +} diff --git a/vortex-clickhouse/src/scan.rs b/vortex-clickhouse/src/scan.rs new file mode 100644 index 00000000000..5f461d533e5 --- /dev/null +++ b/vortex-clickhouse/src/scan.rs @@ -0,0 +1,1398 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Vortex file scanning for ClickHouse. +//! +//! This module implements the read path for Vortex files in ClickHouse. +//! It provides the core logic for the `VortexBlockInputFormat` C++ class. +//! +//! # FFI Interface +//! +//! The following C functions are exported for ClickHouse to use: +//! +//! - `vortex_scanner_new` - Create a new scanner +//! - `vortex_scanner_free` - Free a scanner +//! - `vortex_scanner_num_columns` - Get number of columns +//! - `vortex_scanner_column_name` - Get column name by index +//! - `vortex_scanner_column_type` - Get ClickHouse type string for column +//! - `vortex_scanner_set_projection` - Set columns to read +//! - `vortex_scanner_read_batch` - Read a batch of data +//! +//! # Thread Safety +//! +//! A [`VortexScanner`] instance is **not** thread-safe. The FFI functions +//! dereference raw pointers into shared or exclusive references, so concurrent +//! calls on the same scanner handle from multiple threads cause undefined +//! behavior. The caller on the C++ side must serialize all access to a given +//! scanner handle, or create a separate scanner per thread. +//! +//! The error reporting functions (`vortex_get_last_error`, `vortex_has_error`, +//! `vortex_clear_error`) use thread-local storage and are safe to call from any +//! thread. +//! +//! # Remote File Support +//! +//! This module supports reading Vortex files from remote storage systems: +//! - S3 (`s3://bucket/path/to/file.vortex`) +//! - Google Cloud Storage (`gs://bucket/path/to/file.vortex`) +//! - Azure Blob Storage (`az://container/path/to/file.vortex`) +//! - HTTP/HTTPS (`https://example.com/path/to/file.vortex`) + +use std::collections::VecDeque; +use std::ffi::{CStr, CString, c_char, c_void}; +use std::path::Path; +use std::ptr; + +use futures::TryStreamExt; +use parking_lot::Mutex; +use vortex::array::ArrayRef; +use vortex::dtype::DType; +use vortex::error::{VortexResult, vortex_bail, vortex_err}; +use vortex::expr::{root, select}; +use vortex::file::{OpenOptionsSessionExt, VortexFile}; +use vortex::io::runtime::BlockingRuntime; + +use crate::convert::dtype::vortex_to_clickhouse_type; +use crate::error::{clear_last_error, set_last_error}; +use crate::exporter::{ColumnExporter, ExporterKind, new_exporter}; +use crate::utils::object_store::{is_remote_path, make_object_store}; +use crate::{RUNTIME, SESSION}; + +/// Vortex file scanner that implements the read logic for ClickHouse. +/// +/// This struct holds the state needed to read data from one or more Vortex files. +/// It manages file handles, schema information, projection settings, and +/// provides an iterator-like interface for reading data batches. +/// +/// # Thread Safety +/// +/// This type is **not** thread-safe. Although `cached_total_rows` is wrapped in +/// a [`Mutex`] (to allow interior mutability behind `&self`), the remaining +/// fields are unprotected mutable state. Concurrent access from multiple threads +/// — including through the FFI functions that dereference raw pointers — is +/// undefined behavior. The caller must serialize all access to a given instance. +pub struct VortexScanner { + /// Path or glob pattern for files to scan. + file_paths: Vec, + /// Index of the current file being scanned. + current_file_idx: usize, + /// The currently open file, if any. + current_file: Option, + /// Column indices to project (None = all columns). + projection: Option>, + /// Column names for projection. + projection_names: Option>, + /// The schema of the Vortex file. + schema: DType, + /// Cached column names from schema. + column_names: Vec, + /// Cached ClickHouse type strings. + column_types: Vec, + /// Current row offset within the file. + current_row_offset: u64, + /// Total rows read so far. + total_rows_read: u64, + /// Batch size for reading (rows per batch). + batch_size: usize, + /// Current batch being exported, if any. + current_batch: Option>, + /// Pending chunks from the last file read (avoids merging all chunks). + pending_chunks: VecDeque, + /// Whether we've finished reading all files. + finished: bool, + /// Cached total row count across all files. + cached_total_rows: Mutex>, +} + +impl VortexScanner { + /// Create a new scanner for the given file path or glob pattern. + /// + /// The path can be: + /// - A local file path: `/path/to/file.vortex` + /// - A local glob pattern: `/path/to/*.vortex` + /// - A remote URL: `s3://bucket/path/to/file.vortex` + /// - A remote URL with glob pattern: `s3://bucket/path/to/*.vortex` + pub fn new(path: &str) -> VortexResult { + if path.is_empty() { + vortex_bail!("Path cannot be empty"); + } + + let file_paths = expand_glob(path)?; + + // Read schema from first file + let schema = read_schema_from_file(&file_paths[0])?; + + // Extract column names and types from schema + let (column_names, column_types) = extract_column_info(&schema)?; + + Ok(Self { + file_paths, + current_file_idx: 0, + current_file: None, + projection: None, + projection_names: None, + schema, + column_names, + column_types, + current_row_offset: 0, + total_rows_read: 0, + batch_size: 65536, // Default batch size + current_batch: None, + pending_chunks: VecDeque::new(), + finished: false, + cached_total_rows: Mutex::new(None), + }) + } + + /// Set the columns to project by name. + pub fn set_projection(&mut self, columns: Vec) -> VortexResult<()> { + // Map column names to indices + let indices: VortexResult> = columns + .iter() + .map(|name| { + self.column_names + .iter() + .position(|n| n == name) + .ok_or_else(|| vortex_err!("Column not found: {}", name)) + }) + .collect(); + + self.projection = Some(indices?); + self.projection_names = Some(columns); + Ok(()) + } + + /// Set the columns to project by index. + pub fn set_projection_indices(&mut self, indices: Vec) -> VortexResult<()> { + // Validate indices + for &idx in &indices { + if idx >= self.column_names.len() { + vortex_bail!( + "Column index {} out of bounds (max: {})", + idx, + self.column_names.len() - 1 + ); + } + } + + let names: Vec = indices + .iter() + .map(|&idx| self.column_names[idx].clone()) + .collect(); + + self.projection_names = Some(names); + self.projection = Some(indices); + Ok(()) + } + + /// Set the batch size for reading. + pub fn set_batch_size(&mut self, batch_size: usize) { + self.batch_size = batch_size.max(1); + } + + /// Get the schema of the Vortex file. + pub fn schema(&self) -> &DType { + &self.schema + } + + /// Get the list of file paths to scan. + pub fn file_paths(&self) -> &[String] { + &self.file_paths + } + + /// Get the number of columns in the schema. + pub fn num_columns(&self) -> usize { + self.column_names.len() + } + + /// Get a column name by index. + pub fn column_name(&self, index: usize) -> Option<&str> { + self.column_names.get(index).map(|s| s.as_str()) + } + + /// Get a column's ClickHouse type string by index. + pub fn column_type(&self, index: usize) -> Option<&str> { + self.column_types.get(index).map(|s| s.as_str()) + } + + /// Check if there are more batches to read. + pub fn has_more(&self) -> bool { + !self.finished + } + + /// Read the next batch of data. + /// + /// Returns an exporter for the batch, or None if no more data. + pub fn read_next_batch(&mut self) -> VortexResult>> { + if self.finished { + return Ok(None); + } + + // Check if we have a current batch with remaining data + if let Some(ref batch) = self.current_batch { + if batch.has_more() { + return Ok(self.current_batch.take()); + } + } + + // Check if we have pending chunks from a previous read + if let Some(chunk) = self.pending_chunks.pop_front() { + self.total_rows_read += chunk.len() as u64; + let exporter = new_exporter(chunk)?; + return Ok(Some(exporter)); + } + + // Try to read from current or next file + loop { + // Open next file if needed + if self.current_file.is_none() { + if self.current_file_idx >= self.file_paths.len() { + self.finished = true; + return Ok(None); + } + + let path = &self.file_paths[self.current_file_idx]; + self.current_file = Some(open_vortex_file(path)?); + self.current_row_offset = 0; + } + + let file = self.current_file.as_ref().unwrap(); + let row_count = file.row_count(); + + // Check if we've read all rows in this file + if self.current_row_offset >= row_count { + self.current_file = None; + self.current_file_idx += 1; + continue; + } + + // Calculate the range to read + let start = self.current_row_offset; + let end = (start + self.batch_size as u64).min(row_count); + + // Read the batch — returns individual chunks without merging + let chunks = read_batch_from_file(file, start..end, self.projection_names.as_ref())?; + + self.current_row_offset = end; + + // Return the first chunk, queue the rest + if chunks.is_empty() { + continue; + } + + let mut chunks = VecDeque::from(chunks); + let first = chunks.pop_front().unwrap(); + self.total_rows_read += first.len() as u64; + self.pending_chunks = chunks; + + let exporter = new_exporter(first)?; + return Ok(Some(exporter)); + } + } + + /// Get the total number of rows read so far. + pub fn total_rows_read(&self) -> u64 { + self.total_rows_read + } + + /// Get the total row count across all files. + /// This reads metadata from all files to compute the total. + /// The result is cached after the first call. + pub fn total_row_count(&self) -> VortexResult { + let mut cached = self.cached_total_rows.lock(); + if let Some(total) = *cached { + return Ok(total); + } + let mut total: u64 = 0; + for path in &self.file_paths { + let file = open_vortex_file(path)?; + total += file.row_count(); + } + *cached = Some(total); + Ok(total) + } + + /// Get the number of files to scan. + pub fn num_files(&self) -> usize { + self.file_paths.len() + } + + /// Get the current file index. + pub fn current_file_index(&self) -> usize { + self.current_file_idx + } +} + +/// Extract column names and ClickHouse type strings from schema. +fn extract_column_info(schema: &DType) -> VortexResult<(Vec, Vec)> { + match schema { + DType::Struct(fields, _) => { + let mut names = Vec::with_capacity(fields.nfields()); + let mut types = Vec::with_capacity(fields.nfields()); + + for i in 0..fields.nfields() { + let name = fields + .field_name(i) + .map(|n| n.to_string()) + .unwrap_or_else(|| format!("_{}", i)); + names.push(name); + + let dtype = fields + .field_by_index(i) + .ok_or_else(|| vortex_err!("Failed to get field dtype at index {}", i))?; + let ch_type = vortex_to_clickhouse_type(&dtype)?; + types.push(ch_type); + } + + Ok((names, types)) + } + _ => { + // For non-struct types, treat as a single column + let ch_type = vortex_to_clickhouse_type(schema)?; + Ok((vec!["value".to_string()], vec![ch_type])) + } + } +} + +/// Open a Vortex file (local or remote). +fn open_vortex_file(path: &str) -> VortexResult { + if is_remote_path(path) { + open_remote_vortex_file_with_retry(path) + } else { + open_local_vortex_file(path) + } +} + +/// Open a local Vortex file. +fn open_local_vortex_file(path: &str) -> VortexResult { + (*RUNTIME).block_on(async { SESSION.open_options().open_path(path).await }) +} + +/// Retry configuration for remote operations. +const MAX_RETRIES: u32 = 3; +const INITIAL_BACKOFF_MS: u64 = 100; +const MAX_BACKOFF_MS: u64 = 5000; + +/// Open a remote Vortex file with retry logic. +fn open_remote_vortex_file_with_retry(path: &str) -> VortexResult { + (*RUNTIME).block_on(async { + let mut backoff_ms = INITIAL_BACKOFF_MS; + + for attempt in 0..MAX_RETRIES { + match open_remote_vortex_file_async(path).await { + Ok(file) => return Ok(file), + Err(e) => { + if !is_retryable_error(&e) || attempt == MAX_RETRIES - 1 { + return Err(e); + } + + tracing::warn!( + "Retrying remote file open for '{}' (attempt {}/{}): {}", + path, + attempt + 1, + MAX_RETRIES, + e + ); + + // Exponential backoff with simple jitter + let jitter_factor = (std::time::SystemTime::now() + .duration_since(std::time::UNIX_EPOCH) + .unwrap_or_default() + .subsec_nanos() as u64) + % (backoff_ms / 5 + 1); + let sleep_ms = backoff_ms + jitter_factor; + smol::Timer::after(std::time::Duration::from_millis(sleep_ms)).await; + + backoff_ms = (backoff_ms * 2).min(MAX_BACKOFF_MS); + } + } + } + + vortex_bail!("Max retries exceeded for '{}'", path) + }) +} + +/// Check if an error is retryable by walking the error source chain. +fn is_retryable_error(e: &vortex::error::VortexError) -> bool { + let mut source: Option<&dyn std::error::Error> = Some(e); + while let Some(err) = source { + // Check for std::io errors + if let Some(io_err) = err.downcast_ref::() { + return matches!( + io_err.kind(), + std::io::ErrorKind::ConnectionReset + | std::io::ErrorKind::ConnectionAborted + | std::io::ErrorKind::TimedOut + | std::io::ErrorKind::ConnectionRefused + ); + } + // Fallback: check the error message for common retryable patterns + let msg = err.to_string(); + if msg.contains("503") + || msg.contains("429") + || msg.contains("timeout") + || msg.contains("Timeout") + || msg.contains("connection reset") + || msg.contains("temporarily unavailable") + { + return true; + } + source = err.source(); + } + false +} + +/// Async helper for opening remote files. +async fn open_remote_vortex_file_async(path: &str) -> VortexResult { + let store_info = make_object_store(path)?; + SESSION + .open_options() + .open_object_store(&store_info.store, store_info.path.as_ref()) + .await +} + +/// Read a batch of rows from a Vortex file. +/// +/// Returns individual chunks without merging, to avoid unnecessary copies. +fn read_batch_from_file( + file: &VortexFile, + row_range: std::ops::Range, + projection: Option<&Vec>, +) -> VortexResult> { + (*RUNTIME).block_on(async { + let mut scan_builder = file.scan()?.with_row_range(row_range); + + // Apply projection if specified + if let Some(columns) = projection { + let column_names: Vec<&str> = columns.iter().map(|s| s.as_str()).collect(); + let projection_expr = select(column_names, root()); + scan_builder = scan_builder.with_projection(projection_expr); + } + + let stream = scan_builder.into_array_stream()?; + let chunks: Vec = stream.try_collect().await?; + + Ok(chunks) + }) +} + +/// Expand a glob pattern to a list of file paths. +fn expand_glob(pattern: &str) -> VortexResult> { + if is_remote_path(pattern) { + expand_remote_glob(pattern) + } else { + expand_local_glob(pattern) + } +} + +/// Expand a local glob pattern to a list of file paths. +fn expand_local_glob(pattern: &str) -> VortexResult> { + if pattern.contains('*') || pattern.contains('?') { + let paths: Vec<_> = glob::glob(pattern) + .map_err(|e| vortex_err!("Invalid glob pattern: {}", e))? + .filter_map(|r| r.ok()) + .map(|p| p.to_string_lossy().to_string()) + .collect(); + + if paths.is_empty() { + vortex_bail!("No files found matching pattern: {}", pattern); + } + + Ok(paths) + } else { + if !Path::new(pattern).exists() { + vortex_bail!("File not found: {}", pattern); + } + Ok(vec![pattern.to_string()]) + } +} + +/// Expand a remote glob pattern using object_store's list API. +fn expand_remote_glob(pattern: &str) -> VortexResult> { + (*RUNTIME).block_on(async { expand_remote_glob_async(pattern).await }) +} + +/// Async implementation of remote glob expansion. +async fn expand_remote_glob_async(pattern: &str) -> VortexResult> { + let has_glob = pattern.contains('*') || pattern.contains('?'); + + if !has_glob { + let store_info = make_object_store(pattern)?; + store_info + .store + .head(&store_info.path) + .await + .map_err(|e| vortex_err!("Remote file not found '{}': {}", pattern, e))?; + return Ok(vec![pattern.to_string()]); + } + + let (base_url, glob_pattern) = split_glob_pattern(pattern)?; + + let store_info = make_object_store(&base_url)?; + let prefix = store_info.path.clone(); + + let list_stream = store_info.store.list(Some(&prefix)); + let objects: Vec<_> = list_stream + .try_collect() + .await + .map_err(|e| vortex_err!("Failed to list remote directory '{}': {}", base_url, e))?; + + let glob_matcher = glob::Pattern::new(&glob_pattern) + .map_err(|e| vortex_err!("Invalid glob pattern '{}': {}", glob_pattern, e))?; + + let url = store_info.url; + let base = format!("{}://{}", url.scheme(), url.host_str().unwrap_or_default()); + + let matching_paths: Vec = objects + .into_iter() + .filter(|obj| glob_matcher.matches(obj.location.as_ref())) + .map(|obj| format!("{}/{}", base, obj.location)) + .collect(); + + if matching_paths.is_empty() { + vortex_bail!("No files found matching pattern: {}", pattern); + } + + Ok(matching_paths) +} + +/// Split a glob pattern into base URL and glob pattern. +fn split_glob_pattern(pattern: &str) -> VortexResult<(String, String)> { + let glob_pos = pattern.find(|c| c == '*' || c == '?'); + + match glob_pos { + Some(pos) => { + let base_end = pattern[..pos].rfind('/').unwrap_or(0); + let base_url = pattern[..base_end].to_string(); + let glob_pattern = pattern[base_end + 1..].to_string(); + Ok((base_url, glob_pattern)) + } + None => Ok((pattern.to_string(), String::new())), + } +} + +/// Read the schema from a Vortex file. +fn read_schema_from_file(path: &str) -> VortexResult { + if is_remote_path(path) { + read_schema_from_remote_file(path) + } else { + read_schema_from_local_file(path) + } +} + +/// Read the schema from a local Vortex file. +fn read_schema_from_local_file(path: &str) -> VortexResult { + (*RUNTIME).block_on(async { + let vortex_file = SESSION.open_options().open_path(path).await?; + Ok(vortex_file.dtype().clone()) + }) +} + +/// Read the schema from a remote Vortex file. +fn read_schema_from_remote_file(path: &str) -> VortexResult { + (*RUNTIME).block_on(async { + let store_info = make_object_store(path)?; + let vortex_file = SESSION + .open_options() + .open_object_store(&store_info.store, store_info.path.as_ref()) + .await?; + Ok(vortex_file.dtype().clone()) + }) +} + +// ============================================================================= +// FFI Exports for C++ +// ============================================================================= +// +// Thread safety: All FFI functions that take a `*const VortexScanner` or +// `*mut VortexScanner` assume exclusive, single-threaded access to that handle. +// The caller must never invoke any of these functions concurrently on the same +// scanner pointer. See the module-level and struct-level docs for details. + +/// Create a new Vortex scanner. +/// +/// # Safety +/// The `path` parameter must be a valid null-terminated C string. +/// Returns NULL on error. Call `vortex_get_last_error()` for error details. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_new(path: *const c_char) -> *mut VortexScanner { + clear_last_error(); + + if path.is_null() { + set_last_error("vortex_scanner_new: path is null"); + return ptr::null_mut(); + } + + let path_str = match unsafe { CStr::from_ptr(path) }.to_str() { + Ok(s) => s, + Err(e) => { + set_last_error(&format!("vortex_scanner_new: invalid UTF-8 in path: {}", e)); + return ptr::null_mut(); + } + }; + + match VortexScanner::new(path_str) { + Ok(scanner) => Box::into_raw(Box::new(scanner)), + Err(e) => { + set_last_error(&format!("vortex_scanner_new: {}", e)); + ptr::null_mut() + } + } +} + +/// Free a Vortex scanner. +/// +/// # Safety +/// The `scanner` parameter must be a valid pointer returned by `vortex_scanner_new`, +/// or NULL (which is safely ignored). +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_free(scanner: *mut VortexScanner) { + if !scanner.is_null() { + drop(unsafe { Box::from_raw(scanner) }); + } +} + +/// Get the number of columns in the schema. +/// +/// # Safety +/// The `scanner` parameter must be a valid pointer or NULL. +/// Returns 0 if scanner is NULL. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_num_columns(scanner: *const VortexScanner) -> usize { + if scanner.is_null() { + return 0; + } + unsafe { &*scanner }.num_columns() +} + +/// Get a column name by index. +/// +/// # Safety +/// - The `scanner` parameter must be a valid pointer. +/// - The returned string must be freed with `vortex_free_string()`. +/// - Returns NULL if index is out of bounds or scanner is NULL. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_column_name( + scanner: *const VortexScanner, + index: usize, +) -> *mut c_char { + if scanner.is_null() { + return ptr::null_mut(); + } + + unsafe { &*scanner } + .column_name(index) + .map(|name| { + CString::new(name) + .map(|c_string| c_string.into_raw()) + .unwrap_or(ptr::null_mut()) + }) + .unwrap_or(ptr::null_mut()) +} + +/// Get the ClickHouse type string for a column. +/// +/// # Safety +/// - The `scanner` parameter must be a valid pointer. +/// - The returned string must be freed with `vortex_free_string()`. +/// - Returns NULL if index is out of bounds or scanner is NULL. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_column_type( + scanner: *const VortexScanner, + index: usize, +) -> *mut c_char { + if scanner.is_null() { + return ptr::null_mut(); + } + + unsafe { &*scanner } + .column_type(index) + .map(|type_str| { + CString::new(type_str) + .map(|c_string| c_string.into_raw()) + .unwrap_or(ptr::null_mut()) + }) + .unwrap_or(ptr::null_mut()) +} + +/// Set the columns to project (by index). +/// +/// # Safety +/// - The `scanner` parameter must be a valid pointer. +/// - The `indices` parameter must point to an array of `num_indices` elements. +/// Returns 0 on success, non-zero on error. Call `vortex_get_last_error()` for details. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_set_projection( + scanner: *mut VortexScanner, + indices: *const usize, + num_indices: usize, +) -> i32 { + clear_last_error(); + + if scanner.is_null() { + set_last_error("vortex_scanner_set_projection: scanner is null"); + return -1; + } + + if indices.is_null() && num_indices > 0 { + set_last_error("vortex_scanner_set_projection: indices is null but num_indices > 0"); + return -2; + } + + let scanner = unsafe { &mut *scanner }; + + let indices_vec = if num_indices > 0 { + unsafe { std::slice::from_raw_parts(indices, num_indices) }.to_vec() + } else { + // Empty projection means select all columns + (0..scanner.num_columns()).collect() + }; + + match scanner.set_projection_indices(indices_vec) { + Ok(()) => 0, + Err(e) => { + set_last_error(&format!("vortex_scanner_set_projection: {}", e)); + -3 + } + } +} + +/// Set the batch size for reading. +/// +/// # Safety +/// The `scanner` parameter must be a valid pointer. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_set_batch_size( + scanner: *mut VortexScanner, + batch_size: usize, +) { + if !scanner.is_null() { + unsafe { &mut *scanner }.set_batch_size(batch_size); + } +} + +/// Check if there are more batches to read. +/// +/// # Safety +/// The `scanner` parameter must be a valid pointer. +/// Returns 0 if no more data, 1 if more data available. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_has_more(scanner: *const VortexScanner) -> i32 { + if scanner.is_null() { + return 0; + } + if unsafe { &*scanner }.has_more() { + 1 + } else { + 0 + } +} + +/// Opaque handle to a column exporter. +pub struct VortexExporterHandle { + kind: ExporterKind, + exporter: Box, +} + +impl VortexExporterHandle { + /// Create a new handle, reading the kind from the exporter itself. + fn new(exporter: Box) -> Self { + let kind = exporter.kind(); + Self { kind, exporter } + } +} + +/// Read the next batch of data. +/// +/// # Safety +/// - The `scanner` parameter must be a valid pointer. +/// - Returns NULL if no more data or on error. Call `vortex_get_last_error()` for details. +/// - The returned handle must be freed with `vortex_exporter_free`. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_read_batch( + scanner: *mut VortexScanner, +) -> *mut VortexExporterHandle { + clear_last_error(); + + if scanner.is_null() { + set_last_error("vortex_scanner_read_batch: scanner is null"); + return ptr::null_mut(); + } + + let scanner = unsafe { &mut *scanner }; + + match scanner.read_next_batch() { + Ok(Some(exporter)) => Box::into_raw(Box::new(VortexExporterHandle::new(exporter))), + Ok(None) => ptr::null_mut(), // No error, just no more data + Err(e) => { + set_last_error(&format!("vortex_scanner_read_batch: {}", e)); + ptr::null_mut() + } + } +} + +/// Get the number of files to scan. +/// +/// # Safety +/// The `scanner` parameter must be a valid pointer. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_num_files(scanner: *const VortexScanner) -> usize { + if scanner.is_null() { + return 0; + } + unsafe { &*scanner }.num_files() +} + +/// Get the current file index being scanned. +/// +/// # Safety +/// The `scanner` parameter must be a valid pointer. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_current_file_index(scanner: *const VortexScanner) -> usize { + if scanner.is_null() { + return 0; + } + unsafe { &*scanner }.current_file_index() +} + +/// Get the total number of rows read so far. +/// +/// # Safety +/// The `scanner` parameter must be a valid pointer. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_total_rows_read(scanner: *const VortexScanner) -> u64 { + if scanner.is_null() { + return 0; + } + unsafe { &*scanner }.total_rows_read() +} + +/// Get the total row count across all files. +/// +/// This function reads metadata from all files to compute the total row count. +/// Note: This may be slow for large file sets as it opens each file's metadata. +/// +/// # Safety +/// The `scanner` parameter must be a valid pointer. +/// Returns 0 on error. Call `vortex_get_last_error()` for error details. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_scanner_total_row_count(scanner: *const VortexScanner) -> u64 { + clear_last_error(); + + if scanner.is_null() { + set_last_error("vortex_scanner_total_row_count: scanner is null"); + return 0; + } + + match unsafe { &*scanner }.total_row_count() { + Ok(count) => count, + Err(e) => { + set_last_error(&format!("vortex_scanner_total_row_count: {}", e)); + 0 + } + } +} + +// ============================================================================= +// Exporter FFI Functions +// ============================================================================= + +/// Free an exporter handle. +/// +/// # Safety +/// The `handle` parameter must be a valid pointer returned by `vortex_scanner_read_batch`, +/// or NULL (which is safely ignored). +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_free(handle: *mut VortexExporterHandle) { + if !handle.is_null() { + drop(unsafe { Box::from_raw(handle) }); + } +} + +/// Check if the exporter has more data. +/// +/// # Safety +/// The `handle` parameter must be a valid pointer. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_has_more(handle: *const VortexExporterHandle) -> i32 { + if handle.is_null() { + return 0; + } + if unsafe { &*handle }.exporter.has_more() { + 1 + } else { + 0 + } +} + +/// Get the total number of rows in the exporter. +/// +/// # Safety +/// The `handle` parameter must be a valid pointer. +/// Returns 0 if handle is NULL. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_len(handle: *const VortexExporterHandle) -> usize { + if handle.is_null() { + return 0; + } + unsafe { &*handle }.exporter.len() +} + +/// Export data to a buffer. +/// +/// # Safety +/// - The `handle` parameter must be a valid pointer. +/// - The `buffer` must point to allocated memory of at least `buffer_size_bytes` bytes. +/// - `buffer_size_bytes` must be the total size of the buffer in bytes. The +/// exporter will refuse to write more data than fits in the buffer. +/// Use `vortex_exporter_element_size_bytes` to query the per-row size +/// and allocate `element_size * max_rows` bytes. +/// - Returns the number of rows exported, or negative on error. +/// Call `vortex_get_last_error()` for error details. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_export( + handle: *mut VortexExporterHandle, + buffer: *mut c_void, + buffer_size_bytes: usize, + max_rows: usize, +) -> i64 { + clear_last_error(); + + if handle.is_null() { + set_last_error("vortex_exporter_export: handle is null"); + return -1; + } + if buffer.is_null() { + set_last_error("vortex_exporter_export: buffer is null"); + return -1; + } + + let handle = unsafe { &mut *handle }; + + match handle.exporter.export(buffer, buffer_size_bytes, max_rows) { + Ok(rows) => rows as i64, + Err(e) => { + set_last_error(&format!("vortex_exporter_export: {}", e)); + -2 + } + } +} + +/// Get the number of fields in a struct exporter. +/// +/// # Safety +/// The `handle` parameter must be a valid pointer. +/// Returns 0 if not a struct exporter or handle is NULL. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_num_fields(handle: *const VortexExporterHandle) -> usize { + if handle.is_null() { + return 0; + } + + let handle = unsafe { &*handle }; + + // Try to downcast to StructExporter + if let Some(struct_exporter) = handle + .exporter + .as_any() + .downcast_ref::() + { + struct_exporter.num_fields() + } else { + 0 + } +} + +/// Get a field exporter from a struct exporter. +/// +/// # Safety +/// - The `handle` parameter must be a valid pointer to a struct exporter. +/// - Returns NULL if not a struct exporter or index is out of bounds. +/// - The returned handle must be freed with `vortex_exporter_free`. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_get_field( + handle: *mut VortexExporterHandle, + index: usize, +) -> *mut VortexExporterHandle { + clear_last_error(); + + if handle.is_null() { + set_last_error("vortex_exporter_get_field: handle is null"); + return ptr::null_mut(); + } + + let handle = unsafe { &mut *handle }; + + // Try to downcast to StructExporter + if let Some(struct_exporter) = handle + .exporter + .as_any_mut() + .downcast_mut::() + { + if let Some(field_exporter) = struct_exporter.take_field_exporter(index) { + Box::into_raw(Box::new(VortexExporterHandle::new(field_exporter))) + } else { + set_last_error(&format!( + "vortex_exporter_get_field: field index {} out of bounds or already taken", + index + )); + ptr::null_mut() + } + } else { + set_last_error("vortex_exporter_get_field: handle is not a struct exporter"); + ptr::null_mut() + } +} + +/// Check if the exporter is for nullable data. +/// +/// # Safety +/// The `handle` parameter must be a valid pointer. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_is_nullable(handle: *const VortexExporterHandle) -> i32 { + if handle.is_null() { + return 0; + } + + let handle = unsafe { &*handle }; + + if handle.exporter.is_nullable() { 1 } else { 0 } +} + +/// Export validity (null) bitmap. +/// +/// # Safety +/// - The `handle` parameter must be a valid pointer. +/// - The `validity_bitmap` must point to allocated memory of at least (max_rows + 7) / 8 bytes. +/// - Returns the number of rows, or negative on error. +/// Call `vortex_get_last_error()` for error details. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_export_validity( + handle: *mut VortexExporterHandle, + validity_bitmap: *mut u8, + max_rows: usize, +) -> i64 { + clear_last_error(); + + if handle.is_null() { + set_last_error("vortex_exporter_export_validity: handle is null"); + return -1; + } + if validity_bitmap.is_null() { + set_last_error("vortex_exporter_export_validity: validity_bitmap is null"); + return -1; + } + + let handle = unsafe { &mut *handle }; + + match handle.exporter.export_validity(validity_bitmap, max_rows) { + Ok(rows) => rows as i64, + Err(e) => { + set_last_error(&format!("vortex_exporter_export_validity: {}", e)); + -2 + } + } +} + +/// Export string data. +/// +/// # Safety +/// - The `handle` parameter must be a valid pointer to a string exporter. +/// - All buffer pointers must be valid and properly sized. +/// - Returns the number of rows exported, or negative on error. +/// Call `vortex_get_last_error()` for error details. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_export_strings( + handle: *mut VortexExporterHandle, + data: *mut u8, + lengths: *mut u32, + offsets: *mut u64, + max_rows: usize, +) -> i64 { + clear_last_error(); + + if handle.is_null() { + set_last_error("vortex_exporter_export_strings: handle is null"); + return -1; + } + if data.is_null() || lengths.is_null() || offsets.is_null() { + set_last_error("vortex_exporter_export_strings: one or more buffer pointers are null"); + return -1; + } + + let handle = unsafe { &mut *handle }; + + match handle + .exporter + .export_strings(data, lengths, offsets, max_rows) + { + Ok(rows) => rows as i64, + Err(e) => { + set_last_error(&format!("vortex_exporter_export_strings: {}", e)); + -2 + } + } +} + +/// Get the total size of string data for remaining rows in the exporter. +/// +/// This is useful for pre-allocating buffers before calling `vortex_exporter_export_strings`. +/// +/// # Safety +/// - The `handle` parameter must be a valid pointer to a string exporter. +/// - The `total_bytes` and `num_rows` parameters must be valid pointers. +/// - Returns 0 on success, negative on error. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_string_data_size( + handle: *const VortexExporterHandle, + total_bytes: *mut usize, + num_rows: *mut usize, +) -> i32 { + clear_last_error(); + + if handle.is_null() { + set_last_error("vortex_exporter_string_data_size: handle is null"); + return -1; + } + if total_bytes.is_null() || num_rows.is_null() { + set_last_error("vortex_exporter_string_data_size: output pointers are null"); + return -1; + } + + let handle = unsafe { &*handle }; + + match handle.exporter.string_data_size() { + Ok((bytes, rows)) => { + unsafe { + *total_bytes = bytes; + *num_rows = rows; + } + 0 + } + Err(e) => { + set_last_error(&format!("vortex_exporter_string_data_size: {}", e)); + -2 + } + } +} + +// ============================================================================= +// List/Array Exporter FFI Functions +// ============================================================================= + +/// Check if the exporter is a list (array) exporter. +/// +/// # Safety +/// The `handle` parameter must be a valid pointer. +/// Returns 1 if it's a list exporter, 0 otherwise. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_is_list(handle: *const VortexExporterHandle) -> i32 { + if handle.is_null() { + return 0; + } + + let handle = unsafe { &*handle }; + if handle.kind == ExporterKind::List { + 1 + } else { + 0 + } +} + +/// Get the exporter kind tag. +/// +/// # Safety +/// The `handle` parameter must be a valid pointer. +/// Returns the `ExporterKind` as a u8. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_kind(handle: *const VortexExporterHandle) -> u8 { + if handle.is_null() { + return 0; + } + unsafe { &*handle }.kind as u8 +} + +/// Get the number of bytes each row occupies in the export buffer. +/// +/// For fixed-width exporters (Primitive, Bool, BigInt, Decimal) this returns +/// the element width in bytes. Variable-length exporters (String, List, +/// Struct) return 0 because they use specialised export paths. +/// +/// The value returned by this function can be multiplied by the desired row +/// count to compute the `buffer_size_bytes` argument for +/// `vortex_exporter_export`. +/// +/// # Safety +/// The `handle` parameter must be a valid pointer. +/// Returns 0 if the handle is NULL. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_element_size_bytes( + handle: *const VortexExporterHandle, +) -> usize { + if handle.is_null() { + return 0; + } + unsafe { &*handle }.exporter.element_size_bytes() +} + +/// Export list offsets. +/// +/// The offsets array will have `num_rows + 1` elements written, where: +/// - `offsets[i]` is the start index of array i in the flattened elements +/// - `offsets[num_rows]` is the total number of elements +/// +/// # Safety +/// - The `handle` parameter must be a valid pointer to a list exporter. +/// - The `offsets` buffer must have space for `max_rows + 1` uint64_t values. +/// - Returns the number of rows (not elements) exported, or negative on error. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_export_list_offsets( + handle: *mut VortexExporterHandle, + offsets: *mut u64, + max_rows: usize, +) -> i64 { + clear_last_error(); + + if handle.is_null() { + set_last_error("vortex_exporter_export_list_offsets: handle is null"); + return -1; + } + if offsets.is_null() { + set_last_error("vortex_exporter_export_list_offsets: offsets is null"); + return -1; + } + + let handle = unsafe { &mut *handle }; + + // Try to downcast to ListExporter + if let Some(list_exporter) = handle + .exporter + .as_any_mut() + .downcast_mut::() + { + match list_exporter.export_offsets(offsets, max_rows) { + Ok(rows) => rows as i64, + Err(e) => { + set_last_error(&format!("vortex_exporter_export_list_offsets: {}", e)); + -2 + } + } + } else { + set_last_error("vortex_exporter_export_list_offsets: handle is not a list exporter"); + -3 + } +} + +/// Get the element exporter from a list exporter. +/// +/// This returns an exporter for the flattened elements of all arrays. +/// The returned exporter should be used to export the element data. +/// +/// # Safety +/// - The `handle` parameter must be a valid pointer to a list exporter. +/// - Returns NULL if not a list exporter or on error. +/// - The returned handle must be freed with `vortex_exporter_free`. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_get_list_elements( + handle: *mut VortexExporterHandle, +) -> *mut VortexExporterHandle { + clear_last_error(); + + if handle.is_null() { + set_last_error("vortex_exporter_get_list_elements: handle is null"); + return ptr::null_mut(); + } + + let handle = unsafe { &mut *handle }; + + // Try to downcast to ListExporter + if let Some(list_exporter) = handle + .exporter + .as_any_mut() + .downcast_mut::() + { + match list_exporter.take_element_exporter() { + Ok(elem_exporter) => Box::into_raw(Box::new(VortexExporterHandle::new(elem_exporter))), + Err(e) => { + set_last_error(&format!("vortex_exporter_get_list_elements: {}", e)); + ptr::null_mut() + } + } + } else { + set_last_error("vortex_exporter_get_list_elements: handle is not a list exporter"); + ptr::null_mut() + } +} + +/// Get the total number of elements in all arrays (for a list exporter). +/// +/// This is useful for pre-allocating the element buffer. +/// +/// # Safety +/// The `handle` parameter must be a valid pointer to a list exporter. +/// Returns 0 if not a list exporter or on error. +#[unsafe(no_mangle)] +pub unsafe extern "C" fn vortex_exporter_list_total_elements( + handle: *const VortexExporterHandle, +) -> usize { + if handle.is_null() { + return 0; + } + + let handle = unsafe { &*handle }; + + // Try to downcast to ListExporter + if let Some(list_exporter) = handle + .exporter + .as_any() + .downcast_ref::() + { + list_exporter.total_elements().unwrap_or(0) + } else { + 0 + } +} + +// ============================================================================= +// Tests +// ============================================================================= + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_split_glob_pattern() { + let (base, pattern) = split_glob_pattern("s3://bucket/data/*.vortex").unwrap(); + assert_eq!(base, "s3://bucket/data"); + assert_eq!(pattern, "*.vortex"); + + let (base, pattern) = split_glob_pattern("s3://bucket/data/prefix_*.vortex").unwrap(); + assert_eq!(base, "s3://bucket/data"); + assert_eq!(pattern, "prefix_*.vortex"); + + let (base, pattern) = split_glob_pattern("s3://bucket/a/b/c/*.vortex").unwrap(); + assert_eq!(base, "s3://bucket/a/b/c"); + assert_eq!(pattern, "*.vortex"); + } + + #[test] + fn test_is_glob_pattern() { + assert!("*.vortex".contains('*')); + assert!("prefix_?.vortex".contains('?')); + assert!(!"/path/to/file.vortex".contains('*')); + assert!(!"/path/to/file.vortex".contains('?')); + } + + #[test] + fn test_scanner_new_empty_path() { + let result = VortexScanner::new(""); + assert!(result.is_err()); + } + + #[test] + fn test_scanner_new_invalid_path() { + let result = VortexScanner::new("/nonexistent/path/file.vortex"); + assert!(result.is_err()); + } +} diff --git a/vortex-clickhouse/src/utils/mod.rs b/vortex-clickhouse/src/utils/mod.rs new file mode 100644 index 00000000000..98595e2645a --- /dev/null +++ b/vortex-clickhouse/src/utils/mod.rs @@ -0,0 +1,6 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Utility functions for vortex-clickhouse. + +pub mod object_store; diff --git a/vortex-clickhouse/src/utils/object_store.rs b/vortex-clickhouse/src/utils/object_store.rs new file mode 100644 index 00000000000..97e7f186e5a --- /dev/null +++ b/vortex-clickhouse/src/utils/object_store.rs @@ -0,0 +1,296 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Object store utilities for remote file access. +//! +//! This module provides utilities for: +//! - Detecting remote URLs (S3, GCS, Azure, HTTP) +//! - Creating `ObjectStore` instances from URLs +//! - Parsing and validating remote paths + +use std::str::FromStr; +use std::sync::Arc; +use std::sync::LazyLock; +use std::time::Duration; + +use object_store::aws::{AmazonS3Builder, AmazonS3ConfigKey}; +use object_store::azure::{AzureConfigKey, MicrosoftAzureBuilder}; +use object_store::gcp::{GoogleCloudStorageBuilder, GoogleConfigKey}; +use object_store::http::HttpBuilder; +use object_store::local::LocalFileSystem; +use object_store::path::Path as ObjectPath; +use object_store::{ClientOptions, ObjectStore, ObjectStoreScheme}; +use parking_lot::Mutex; +use url::Url; +use vortex::error::{VortexResult, vortex_bail, vortex_err}; +use vortex_utils::aliases::hash_map::HashMap; + +/// Cached object stores to avoid recreating connections. +static OBJECT_STORE_CACHE: LazyLock>>> = + LazyLock::new(|| Mutex::new(HashMap::new())); + +/// Parse a URL string into a `Url`. +pub fn parse_url(url_str: &str) -> VortexResult { + Url::parse(url_str).map_err(|e| vortex_err!("Invalid URL '{}': {}", url_str, e)) +} + +/// Check if a path is a remote URL (S3, HTTP, etc.). +pub fn is_remote_path(path: &str) -> bool { + path.starts_with("s3://") + || path.starts_with("http://") + || path.starts_with("https://") + || path.starts_with("gs://") + || path.starts_with("az://") + || path.starts_with("abfs://") + || path.starts_with("abfss://") + || path.starts_with("hdfs://") +} + +/// Extract the object path from a URL. +/// +/// For example, `s3://bucket/path/to/file.vortex` returns `path/to/file.vortex`. +pub fn extract_object_path(url: &Url) -> VortexResult { + ObjectPath::from_url_path(url.path()) + .map_err(|e| vortex_err!("Invalid object path in URL '{}': {}", url, e)) +} + +/// Result of creating an object store from a URL. +pub struct ObjectStoreWithPath { + /// The object store instance. + pub store: Arc, + /// The path within the object store. + pub path: ObjectPath, + /// The original URL. + pub url: Url, +} + +/// Generate a cache key from a URL. +/// +/// The cache key is based on scheme + host + port, so different paths +/// on the same server share the same ObjectStore instance. +fn url_cache_key(url: &Url) -> String { + format!( + "{}://{}", + url.scheme(), + &url[url::Position::BeforeHost..url::Position::AfterPort], + ) +} + +/// Create an `ObjectStore` from a URL. +/// +/// This function creates and caches object store instances. It supports: +/// - S3 (`s3://bucket/path`) +/// - Google Cloud Storage (`gs://bucket/path`) +/// - Azure Blob Storage (`az://container/path`, `abfs://`, `abfss://`) +/// - HTTP/HTTPS (`http://`, `https://`) +/// - Local filesystem (`file://` or plain paths) +/// +/// # Arguments +/// * `url_str` - The URL string to parse +/// +/// # Returns +/// An `ObjectStoreWithPath` containing the store, path, and parsed URL. +#[expect(clippy::cognitive_complexity)] +pub fn make_object_store(url_str: &str) -> VortexResult { + make_object_store_with_options(url_str, &HashMap::new()) +} + +/// Create an `ObjectStore` from a URL with custom configuration options. +/// +/// This function creates and caches object store instances with custom configuration. +/// +/// # Arguments +/// * `url_str` - The URL string to parse +/// * `properties` - Configuration options (e.g., AWS credentials, region, etc.) +/// +/// # Returns +/// An `ObjectStoreWithPath` containing the store, path, and parsed URL. +#[expect(clippy::cognitive_complexity)] +pub fn make_object_store_with_options( + url_str: &str, + properties: &HashMap, +) -> VortexResult { + let url = parse_url(url_str)?; + let path = extract_object_path(&url)?; + + let (scheme, _) = ObjectStoreScheme::parse(&url) + .map_err(|error| vortex_err!("Failed to parse object store scheme: {}", error))?; + + let cache_key = url_cache_key(&url); + + // Check cache first + { + if let Some(cached) = OBJECT_STORE_CACHE.lock().get(&cache_key) { + return Ok(ObjectStoreWithPath { + store: cached.clone(), + path, + url, + }); + } + } + + let store: Arc = match scheme { + ObjectStoreScheme::Local => { + tracing::trace!("using LocalFileSystem object store"); + Arc::new(LocalFileSystem::default()) + } + ObjectStoreScheme::AmazonS3 => { + tracing::trace!("using AmazonS3 object store for URL: {}", url); + let mut builder = AmazonS3Builder::new() + .with_url(url.to_string()) + .with_virtual_hosted_style_request(false); + + // Try to load credentials from environment if not provided in properties + if !properties.contains_key("access_key_id") { + if let Ok(access_key) = std::env::var("AWS_ACCESS_KEY_ID") { + builder = builder.with_access_key_id(access_key); + } + } + if !properties.contains_key("secret_access_key") { + if let Ok(secret_key) = std::env::var("AWS_SECRET_ACCESS_KEY") { + builder = builder.with_secret_access_key(secret_key); + } + } + if !properties.contains_key("region") { + if let Ok(region) = std::env::var("AWS_DEFAULT_REGION") { + builder = builder.with_region(region); + } + } + + for (key, val) in properties { + if let Ok(config_key) = AmazonS3ConfigKey::from_str(key.as_str()) { + builder = builder.with_config(config_key, val); + } else { + tracing::warn!("Skipping unknown Amazon S3 config key: {}", key); + } + } + + Arc::new(builder.build()?) + } + ObjectStoreScheme::MicrosoftAzure => { + tracing::trace!("using MicrosoftAzure object store for URL: {}", url); + + let client_opts = ClientOptions::new().with_timeout(Duration::from_secs(120)); + let mut builder = MicrosoftAzureBuilder::new() + .with_url(url.to_string()) + .with_client_options(client_opts); + + for (key, val) in properties { + if let Ok(config_key) = AzureConfigKey::from_str(key.as_str()) { + builder = builder.with_config(config_key, val); + } else { + tracing::warn!("Skipping unknown Azure config key: {}", key); + } + } + + Arc::new(builder.build()?) + } + ObjectStoreScheme::GoogleCloudStorage => { + tracing::trace!("using GoogleCloudStorage object store for URL: {}", url); + + let mut builder = GoogleCloudStorageBuilder::new().with_url(url.to_string()); + + for (key, val) in properties { + if let Ok(config_key) = GoogleConfigKey::from_str(key.as_str()) { + builder = builder.with_config(config_key, val); + } else { + tracing::warn!("Skipping unknown Google Cloud Storage config key: {}", key); + } + } + + Arc::new(builder.build()?) + } + ObjectStoreScheme::Http => { + tracing::trace!("using HTTP object store for URL: {}", url); + + let client_opts = ClientOptions::new().with_timeout(Duration::from_secs(120)); + let builder = HttpBuilder::new() + .with_url(url.to_string()) + .with_client_options(client_opts); + + Arc::new(builder.build()?) + } + store => { + vortex_bail!("Unsupported object store scheme: {:?}", store); + } + }; + + // Cache the store + { + OBJECT_STORE_CACHE.lock().insert(cache_key, store.clone()); + } + + Ok(ObjectStoreWithPath { store, path, url }) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_is_remote_path() { + // Remote paths + assert!(is_remote_path("s3://bucket/path/to/file.vortex")); + assert!(is_remote_path("https://example.com/data.vortex")); + assert!(is_remote_path("http://localhost:8080/data.vortex")); + assert!(is_remote_path("gs://my-bucket/file.vortex")); + assert!(is_remote_path("az://container/blob.vortex")); + assert!(is_remote_path("abfs://container/blob.vortex")); + assert!(is_remote_path("abfss://container/blob.vortex")); + + // Local paths + assert!(!is_remote_path("/path/to/file.vortex")); + assert!(!is_remote_path("./relative/path.vortex")); + assert!(!is_remote_path("file.vortex")); + assert!(!is_remote_path("C:\\Windows\\path.vortex")); + } + + #[test] + fn test_parse_url() { + let url = parse_url("s3://bucket/path/to/file.vortex").unwrap(); + assert_eq!(url.scheme(), "s3"); + assert_eq!(url.host_str(), Some("bucket")); + assert_eq!(url.path(), "/path/to/file.vortex"); + + let url = parse_url("https://example.com/data.vortex").unwrap(); + assert_eq!(url.scheme(), "https"); + assert_eq!(url.host_str(), Some("example.com")); + assert_eq!(url.path(), "/data.vortex"); + + // Invalid URL + assert!(parse_url("not a valid url").is_err()); + } + + #[test] + fn test_extract_object_path() { + let url = parse_url("s3://bucket/path/to/file.vortex").unwrap(); + let path = extract_object_path(&url).unwrap(); + assert_eq!(path.as_ref(), "path/to/file.vortex"); + + let url = parse_url("https://example.com/data/nested/file.vortex").unwrap(); + let path = extract_object_path(&url).unwrap(); + assert_eq!(path.as_ref(), "data/nested/file.vortex"); + } + + #[test] + fn test_url_cache_key() { + let url = parse_url("s3://bucket/path/to/file.vortex").unwrap(); + assert_eq!(url_cache_key(&url), "s3://bucket"); + + let url = parse_url("https://example.com:8080/data.vortex").unwrap(); + assert_eq!(url_cache_key(&url), "https://example.com:8080"); + + // Different paths on same server should have same cache key + let url1 = parse_url("s3://bucket/path1/file1.vortex").unwrap(); + let url2 = parse_url("s3://bucket/path2/file2.vortex").unwrap(); + assert_eq!(url_cache_key(&url1), url_cache_key(&url2)); + } + + #[test] + fn test_make_object_store_local() { + let result = make_object_store("file:///tmp/test.vortex"); + assert!(result.is_ok()); + let result = result.unwrap(); + assert_eq!(result.path.as_ref(), "tmp/test.vortex"); + } +}