From 941bd5984ed522157b0592efc02906cddf187124 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 23 Oct 2025 14:35:17 -0400 Subject: [PATCH 1/6] Vortex Compute Signed-off-by: Nicholas Gates --- Cargo.lock | 10 +++ Cargo.toml | 4 + vortex-buffer/benches/vortex_bitbuffer.rs | 9 +++ vortex-buffer/src/bit/buf_mut.rs | 17 +++- vortex-buffer/src/bit/ops.rs | 34 +++++++- vortex-compute/Cargo.toml | 32 ++++++++ vortex-compute/src/filter/bitbuffer.rs | 67 ++++++++++++++++ vortex-compute/src/filter/bool.rs | 13 ++++ vortex-compute/src/filter/buffer.rs | 38 +++++++++ vortex-compute/src/filter/mask.rs | 21 +++++ vortex-compute/src/filter/mod.rs | 23 ++++++ vortex-compute/src/lib.rs | 13 ++++ vortex-compute/src/logical/and.rs | 20 +++++ vortex-compute/src/logical/and_kleene.rs | 95 +++++++++++++++++++++++ vortex-compute/src/logical/and_not.rs | 20 +++++ vortex-compute/src/logical/mod.rs | 65 ++++++++++++++++ vortex-compute/src/logical/not.rs | 39 ++++++++++ vortex-compute/src/logical/or.rs | 20 +++++ vortex-compute/src/logical/or_kleene.rs | 95 +++++++++++++++++++++++ vortex-vector/src/bool/vector.rs | 5 ++ vortex-vector/src/bool/vector_mut.rs | 11 +++ 21 files changed, 647 insertions(+), 4 deletions(-) create mode 100644 vortex-compute/Cargo.toml create mode 100644 vortex-compute/src/filter/bitbuffer.rs create mode 100644 vortex-compute/src/filter/bool.rs create mode 100644 vortex-compute/src/filter/buffer.rs create mode 100644 vortex-compute/src/filter/mask.rs create mode 100644 vortex-compute/src/filter/mod.rs create mode 100644 vortex-compute/src/lib.rs create mode 100644 vortex-compute/src/logical/and.rs create mode 100644 vortex-compute/src/logical/and_kleene.rs create mode 100644 vortex-compute/src/logical/and_not.rs create mode 100644 vortex-compute/src/logical/mod.rs create mode 100644 vortex-compute/src/logical/not.rs create mode 100644 vortex-compute/src/logical/or.rs create mode 100644 vortex-compute/src/logical/or_kleene.rs diff --git a/Cargo.lock b/Cargo.lock index 9cd94baa0b5..32430a2b181 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8622,6 +8622,16 @@ dependencies = [ "vortex-scalar", ] +[[package]] +name = "vortex-compute" +version = "0.1.0" +dependencies = [ + "vortex-buffer", + "vortex-error", + "vortex-mask", + "vortex-vector", +] + [[package]] name = "vortex-cxx" version = "0.1.0" diff --git a/Cargo.toml b/Cargo.toml index 7423faf01bb..3411739247c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -8,6 +8,7 @@ members = [ "vortex-array", "vortex-btrblocks", "vortex-buffer", + "vortex-compute", "vortex-cxx", "vortex-datafusion", "vortex-dtype", @@ -112,6 +113,7 @@ dirs = "6.0.0" divan = { package = "codspeed-divan-compat", version = "4.0.4" } dyn-hash = "0.2.0" enum-iterator = "2.0.0" +enum-map = "2.7.3" erased-serde = "0.4" fastlanes = "0.5" flatbuffers = "25.2.10" @@ -216,6 +218,7 @@ vortex-array = { version = "0.1.0", path = "./vortex-array", default-features = vortex-btrblocks = { version = "0.1.0", path = "./vortex-btrblocks", default-features = false } vortex-buffer = { version = "0.1.0", path = "./vortex-buffer", default-features = false } vortex-bytebool = { version = "0.1.0", path = "./encodings/bytebool", default-features = false } +vortex-compute = { version = "0.1.0", path = "./vortex-compute", default-features = false } vortex-datafusion = { version = "0.1.0", path = "./vortex-datafusion", default-features = false } vortex-datetime-parts = { version = "0.1.0", path = "./encodings/datetime-parts", default-features = false } vortex-decimal-byte-parts = { version = "0.1.0", path = "encodings/decimal-byte-parts", default-features = false } @@ -242,6 +245,7 @@ vortex-sequence = { version = "0.1.0", path = "encodings/sequence", default-feat vortex-sparse = { version = "0.1.0", path = "./encodings/sparse", default-features = false } vortex-tui = { version = "0.1.0", path = "./vortex-tui", default-features = false } vortex-utils = { version = "0.1.0", path = "./vortex-utils", default-features = false } +vortex-vector = { version = "0.1.0", path = "./vortex-vector", default-features = false } vortex-zigzag = { version = "0.1.0", path = "./encodings/zigzag", default-features = false } vortex-zstd = { version = "0.1.0", path = "./encodings/zstd", default-features = false } # END crates published by this project diff --git a/vortex-buffer/benches/vortex_bitbuffer.rs b/vortex-buffer/benches/vortex_bitbuffer.rs index e82798c2e47..a0baa9e5b70 100644 --- a/vortex-buffer/benches/vortex_bitbuffer.rs +++ b/vortex-buffer/benches/vortex_bitbuffer.rs @@ -229,6 +229,15 @@ fn bitwise_not_vortex_buffer(bencher: Bencher, length: usize) { }); } +#[divan::bench(args = INPUT_SIZE)] +fn bitwise_not_vortex_buffer_mut(bencher: Bencher, length: usize) { + bencher + .with_inputs(|| BitBufferMut::from_iter((0..length).map(|i| i % 2 == 0))) + .bench_values(|buffer| { + divan::black_box(!buffer); + }); +} + #[divan::bench(args = INPUT_SIZE)] fn bitwise_not_arrow_buffer(bencher: Bencher, length: usize) { bencher diff --git a/vortex-buffer/src/bit/buf_mut.rs b/vortex-buffer/src/bit/buf_mut.rs index c0ff973feec..52c9f2a01a7 100644 --- a/vortex-buffer/src/bit/buf_mut.rs +++ b/vortex-buffer/src/bit/buf_mut.rs @@ -3,9 +3,10 @@ use arrow_buffer::bit_chunk_iterator::BitChunks; use bitvec::view::BitView; +use std::ops::Not; -use crate::bit::{get_bit_unchecked, set_bit_unchecked, unset_bit_unchecked}; -use crate::{BitBuffer, BufferMut, ByteBufferMut, buffer_mut}; +use crate::bit::{get_bit_unchecked, ops, set_bit_unchecked, unset_bit_unchecked}; +use crate::{buffer_mut, BitBuffer, BufferMut, ByteBufferMut}; /// A mutable bitset buffer that allows random access to individual bits for set and get. /// @@ -462,6 +463,16 @@ impl Default for BitBufferMut { } } +// Mutate-in-place implementation of bitwise NOT. +impl Not for BitBufferMut { + type Output = BitBufferMut; + + fn not(mut self) -> Self::Output { + ops::bitwise_unary_op_mut(&mut self, |b| !b); + self + } +} + impl From<&[bool]> for BitBufferMut { fn from(value: &[bool]) -> Self { let mut buf = BitBufferMut::new_unset(value.len()); @@ -507,7 +518,7 @@ impl FromIterator for BitBufferMut { #[cfg(test)] mod tests { use crate::bit::buf_mut::BitBufferMut; - use crate::{BufferMut, bitbuffer, bitbuffer_mut, buffer_mut}; + use crate::{bitbuffer, bitbuffer_mut, buffer_mut, BufferMut}; #[test] fn test_bits_mut() { diff --git a/vortex-buffer/src/bit/ops.rs b/vortex-buffer/src/bit/ops.rs index b46b40a3463..19a14548243 100644 --- a/vortex-buffer/src/bit/ops.rs +++ b/vortex-buffer/src/bit/ops.rs @@ -2,7 +2,7 @@ // SPDX-FileCopyrightText: Copyright the Vortex contributors use crate::trusted_len::TrustedLenExt; -use crate::{BitBuffer, Buffer}; +use crate::{BitBuffer, BitBufferMut, Buffer}; pub(super) fn bitwise_unary_op u64>(buffer: &BitBuffer, op: F) -> BitBuffer { let iter = buffer.chunks().iter_padded().map(op); @@ -13,6 +13,38 @@ pub(super) fn bitwise_unary_op u64>(buffer: &BitBuffer, op: F) BitBuffer::new(result, buffer.len()) } +pub(super) fn bitwise_unary_op_mut u64>(buffer: &mut BitBufferMut, mut op: F) { + let slice_mut = buffer.as_mut_slice(); + + // The number of complete u64 words in the buffer (unaligned) + let u64_len = slice_mut.len() / 8; + let remainder = slice_mut.len() % 8; + + // Create a pointer to the *unaligned* u64 words + let mut ptr = slice_mut.as_mut_ptr() as *mut u64; + for _ in 0..u64_len { + let value = unsafe { ptr.read_unaligned() }; + let value = op(value); + unsafe { ptr.write_unaligned(value) }; + ptr = unsafe { ptr.add(1) }; + } + + // Read remainder into a u64; + let mut remainder_u64 = 0u64; + let ptr = ptr as *mut u8; + for i in 0..remainder { + let byte = unsafe { ptr.add(i).read() }; + remainder_u64 |= (byte as u64) << (i * 8); + } + let remainder_u64 = op(remainder_u64); + + // Write back remainder + for i in 0..remainder { + let byte = ((remainder_u64 >> (i * 8)) & 0xFF) as u8; + unsafe { ptr.add(i).write(byte) }; + } +} + pub(super) fn bitwise_binary_op u64>( left: &BitBuffer, right: &BitBuffer, diff --git a/vortex-compute/Cargo.toml b/vortex-compute/Cargo.toml new file mode 100644 index 00000000000..a38f1e45fe8 --- /dev/null +++ b/vortex-compute/Cargo.toml @@ -0,0 +1,32 @@ +[package] +name = "vortex-compute" +authors = { workspace = true } +categories = { workspace = true } +description = "Compute functions that operator over Vortex vectors, buffers, and masks" +edition = { workspace = true } +homepage = { workspace = true } +include = { workspace = true } +keywords = { workspace = true } +license = { workspace = true } +readme = { workspace = true } +repository = { workspace = true } +rust-version = { workspace = true } +version = { workspace = true } + +[package.metadata.docs.rs] +all-features = true + +[lints] +workspace = true + +[dependencies] +vortex-buffer = { workspace = true } +vortex-error = { workspace = true } +vortex-mask = { workspace = true } +vortex-vector = { workspace = true } + +[features] +default = ["filter", "logical"] + +filter = [] +logical = [] diff --git a/vortex-compute/src/filter/bitbuffer.rs b/vortex-compute/src/filter/bitbuffer.rs new file mode 100644 index 00000000000..c6dc2a9d423 --- /dev/null +++ b/vortex-compute/src/filter/bitbuffer.rs @@ -0,0 +1,67 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use vortex_buffer::{BitBuffer, BitBufferMut, get_bit}; +use vortex_mask::{Mask, MaskIter}; + +use crate::filter::Filter; + +/// If the filter density is above 80%, we use slices to filter the array instead of indices. +// TODO(ngates): we need more experimentation to determine the best threshold here. +const FILTER_SLICES_DENSITY_THRESHOLD: f64 = 0.8; + +impl Filter for BitBuffer { + fn filter(&self, mask: &Mask) -> Self { + assert_eq!(mask.len(), self.len()); + match mask { + Mask::AllTrue(_) => self.clone(), + Mask::AllFalse(_) => Self::empty(), + Mask::Values(v) => match v.threshold_iter(FILTER_SLICES_DENSITY_THRESHOLD) { + MaskIter::Indices(indices) => filter_indices(self, indices), + MaskIter::Slices(slices) => filter_slices(self, mask.true_count(), slices), + }, + } + } +} + +fn filter_indices(bools: &BitBuffer, indices: &[usize]) -> BitBuffer { + let buffer = bools.inner().as_ref(); + BitBuffer::collect_bool(indices.len(), |idx| { + let idx = *unsafe { indices.get_unchecked(idx) }; + get_bit(buffer, bools.offset() + idx) + }) +} + +fn filter_slices(buffer: &BitBuffer, output_len: usize, slices: &[(usize, usize)]) -> BitBuffer { + let mut builder = BitBufferMut::with_capacity(output_len); + for (start, end) in slices { + // TODO(ngates): we probably want a borrowed slice for things like this. + builder.append_buffer(&buffer.slice(*start..*end)); + } + builder.freeze() +} + +#[cfg(test)] +mod test { + use vortex_buffer::bitbuffer; + + use super::*; + + #[test] + fn filter_bool_by_slice_test() { + let bits = bitbuffer![true, true, false]; + + let filtered = filter_slices(&bits, 2, &[(0, 1), (2, 3)]); + assert_eq!(2, filtered.len()); + + assert_eq!(filtered, bitbuffer![true, false]) + } + + #[test] + fn filter_bool_by_index_test() { + let buf = bitbuffer![true, true, false]; + let filtered = filter_indices(&buf, &[0, 2]); + assert_eq!(2, filtered.len()); + assert_eq!(bitbuffer![true, false], filtered) + } +} diff --git a/vortex-compute/src/filter/bool.rs b/vortex-compute/src/filter/bool.rs new file mode 100644 index 00000000000..84c626d9953 --- /dev/null +++ b/vortex-compute/src/filter/bool.rs @@ -0,0 +1,13 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use vortex_mask::Mask; +use vortex_vector::{BoolVector, VectorOps}; + +use crate::filter::Filter; + +impl Filter for BoolVector { + fn filter(&self, mask: &Mask) -> Self { + Self::new(self.bits().filter(mask), self.validity().filter(mask)) + } +} diff --git a/vortex-compute/src/filter/buffer.rs b/vortex-compute/src/filter/buffer.rs new file mode 100644 index 00000000000..572f128bdca --- /dev/null +++ b/vortex-compute/src/filter/buffer.rs @@ -0,0 +1,38 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use vortex_buffer::{Buffer, BufferMut}; +use vortex_mask::{Mask, MaskIter}; + +use crate::filter::Filter; + +// This is modeled after the constant with the equivalent name in arrow-rs. +const FILTER_SLICES_SELECTIVITY_THRESHOLD: f64 = 0.8; + +impl Filter for Buffer { + fn filter(&self, mask: &Mask) -> Self { + assert_eq!(mask.len(), self.len()); + match mask { + Mask::AllTrue(_) => self.clone(), + Mask::AllFalse(_) => Self::empty(), + Mask::Values(v) => match v.threshold_iter(FILTER_SLICES_SELECTIVITY_THRESHOLD) { + MaskIter::Indices(indices) => filter_indices(self.as_slice(), indices), + MaskIter::Slices(slices) => { + filter_slices(self.as_slice(), mask.true_count(), slices) + } + }, + } + } +} + +fn filter_indices(values: &[T], indices: &[usize]) -> Buffer { + Buffer::::from_trusted_len_iter(indices.iter().map(|&idx| values[idx])) +} + +fn filter_slices(values: &[T], output_len: usize, slices: &[(usize, usize)]) -> Buffer { + let mut out = BufferMut::::with_capacity(output_len); + for (start, end) in slices { + out.extend_from_slice(&values[*start..*end]); + } + out.freeze() +} diff --git a/vortex-compute/src/filter/mask.rs b/vortex-compute/src/filter/mask.rs new file mode 100644 index 00000000000..51a9c0c72f0 --- /dev/null +++ b/vortex-compute/src/filter/mask.rs @@ -0,0 +1,21 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use vortex_mask::Mask; + +use crate::filter::Filter; + +impl Filter for Mask { + fn filter(&self, mask: &Mask) -> Self { + assert_eq!(self.len(), mask.len()); + + match (self, mask) { + (Mask::AllTrue(_), _) => Mask::AllTrue(mask.true_count()), + (Mask::AllFalse(_), _) => Mask::AllFalse(mask.true_count()), + + (Mask::Values(_), Mask::AllTrue(_)) => self.clone(), + (Mask::Values(_), Mask::AllFalse(_)) => Self::new_true(0), + (Mask::Values(v1), Mask::Values(_)) => Mask::from(v1.bit_buffer().filter(mask)), + } + } +} diff --git a/vortex-compute/src/filter/mod.rs b/vortex-compute/src/filter/mod.rs new file mode 100644 index 00000000000..84ab375ef1c --- /dev/null +++ b/vortex-compute/src/filter/mod.rs @@ -0,0 +1,23 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Filter function. + +mod bitbuffer; +mod bool; +mod buffer; +mod mask; + +use vortex_mask::Mask; + +/// Function for filtering based on a selection mask. +pub trait Filter { + /// Filters the vector using the provided mask, returning a new value. + /// + /// The result value will have length equal to the true count of the provided mask. + /// + /// # Panics + /// + /// If the length of the mask does not equal the length of the value being filtered. + fn filter(&self, mask: &Mask) -> Self; +} diff --git a/vortex-compute/src/lib.rs b/vortex-compute/src/lib.rs new file mode 100644 index 00000000000..5c2b009ef03 --- /dev/null +++ b/vortex-compute/src/lib.rs @@ -0,0 +1,13 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! A collection of compute functions primarily for operating over Vortex vectors. + +#![deny(missing_docs)] +#![deny(clippy::missing_panics_doc)] +#![deny(clippy::missing_safety_doc)] + +#[cfg(feature = "filter")] +pub mod filter; +#[cfg(feature = "logical")] +pub mod logical; diff --git a/vortex-compute/src/logical/and.rs b/vortex-compute/src/logical/and.rs new file mode 100644 index 00000000000..1727c356a2d --- /dev/null +++ b/vortex-compute/src/logical/and.rs @@ -0,0 +1,20 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::ops::{BitAnd, BitOr}; + +use vortex_vector::{BoolVector, VectorOps}; + +use crate::logical::LogicalAnd; + +// TODO(ngates): should we try to into_mut and reuse the existing buffer? Let's benchmark. +impl LogicalAnd for &BoolVector { + type Output = BoolVector; + + fn and(self, other: &BoolVector) -> BoolVector { + BoolVector::new( + self.bits().bitand(other.bits()), + self.validity().bitor(other.validity()), + ) + } +} diff --git a/vortex-compute/src/logical/and_kleene.rs b/vortex-compute/src/logical/and_kleene.rs new file mode 100644 index 00000000000..e446cca5026 --- /dev/null +++ b/vortex-compute/src/logical/and_kleene.rs @@ -0,0 +1,95 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::ops::{BitAnd, BitOr, Not}; + +use vortex_buffer::BitBuffer; +use vortex_mask::Mask; +use vortex_vector::{BoolVector, VectorOps}; + +use crate::logical::LogicalAndKleene; + +impl LogicalAndKleene for &BoolVector { + type Output = BoolVector; + + fn and_kleene(self, rhs: Self) -> Self::Output { + match (self.validity(), rhs.validity()) { + (Mask::AllTrue(_), Mask::AllTrue(_)) => { + BoolVector::new(self.bits().bitand(rhs.bits()), Mask::new_true(self.len())) + } + (Mask::AllTrue(_), Mask::AllFalse(_)) => { + // self valid, rhs all null + // Result: false where self is false (valid), null where self is true + let result_bits = BitBuffer::new_unset(self.len()); + let validity = self.bits().not(); // valid where self is false + BoolVector::new(result_bits, Mask::from(validity)) + } + (Mask::AllFalse(_), Mask::AllTrue(_)) => { + // self all null, rhs valid + // Result: false where rhs is false (valid), null where rhs is true + let result_bits = BitBuffer::new_unset(self.len()); + let validity = rhs.bits().not(); // valid where rhs is false + BoolVector::new(result_bits, Mask::from(validity)) + } + (Mask::AllFalse(_), Mask::AllFalse(_)) => { + // All values are null + BoolVector::new( + BitBuffer::new_unset(self.len()), + Mask::new_false(self.len()), + ) + } + (Mask::Values(lv), Mask::AllTrue(_)) => { + // self partial validity, rhs all valid + // Result valid where self valid OR self is null but rhs is false + let result_bits = self.bits().bitand(rhs.bits()); + let validity = lv.bit_buffer().bitor(&rhs.bits().not()); + BoolVector::new(result_bits, Mask::from(validity)) + } + (Mask::AllTrue(_), Mask::Values(rv)) => { + // self all valid, rhs partial validity + // Result valid where rhs valid OR rhs is null but self is false + let result_bits = self.bits().bitand(rhs.bits()); + let validity = rv.bit_buffer().bitor(&self.bits().not()); + BoolVector::new(result_bits, Mask::from(validity)) + } + (Mask::Values(lv), Mask::AllFalse(_)) => { + // self partial validity, rhs all null + // Result: false where self is false (valid), null otherwise + let result_bits = BitBuffer::new_unset(self.len()); + let validity = lv.bit_buffer().bitand(&self.bits().not()); + BoolVector::new(result_bits, Mask::from(validity)) + } + (Mask::AllFalse(_), Mask::Values(rv)) => { + // self all null, rhs partial validity + // Result: false where rhs is false (valid), null otherwise + let result_bits = BitBuffer::new_unset(self.len()); + let validity = rv.bit_buffer().bitand(&rhs.bits().not()); + BoolVector::new(result_bits, Mask::from(validity)) + } + (Mask::Values(lv), Mask::Values(rv)) => { + // Both have partial validity + // Result is valid where: + // 1. Both are valid, OR + // 2. One is null but the other is false (and valid) + let result_bits = self.bits().bitand(rhs.bits()); + + let both_valid = lv.bit_buffer().bitand(rv.bit_buffer()); + let self_null_rhs_false = lv + .bit_buffer() + .not() + .bitand(rv.bit_buffer()) + .bitand(&rhs.bits().not()); + let rhs_null_self_false = rv + .bit_buffer() + .not() + .bitand(lv.bit_buffer()) + .bitand(&self.bits().not()); + + let validity = both_valid + .bitor(&self_null_rhs_false) + .bitor(&rhs_null_self_false); + BoolVector::new(result_bits, Mask::from(validity)) + } + } + } +} diff --git a/vortex-compute/src/logical/and_not.rs b/vortex-compute/src/logical/and_not.rs new file mode 100644 index 00000000000..bb3df9ff200 --- /dev/null +++ b/vortex-compute/src/logical/and_not.rs @@ -0,0 +1,20 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::ops::BitOr; + +use vortex_vector::{BoolVector, VectorOps}; + +use crate::logical::LogicalAndNot; + +// TODO(ngates): should we try to into_mut and reuse the existing buffer? Let's benchmark. +impl LogicalAndNot for &BoolVector { + type Output = BoolVector; + + fn and_not(self, other: &BoolVector) -> BoolVector { + BoolVector::new( + self.bits().bitand_not(other.bits()), + self.validity().bitor(other.validity()), + ) + } +} diff --git a/vortex-compute/src/logical/mod.rs b/vortex-compute/src/logical/mod.rs new file mode 100644 index 00000000000..023cda048f1 --- /dev/null +++ b/vortex-compute/src/logical/mod.rs @@ -0,0 +1,65 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Logical boolean functions. + +mod and; +mod and_kleene; +mod and_not; +mod not; +mod or; +mod or_kleene; + +/// Trait for performing logical AND operations. +pub trait LogicalAnd { + /// The resulting type after performing the logical AND operation. + type Output; + + /// Perform a logical AND operation between two values. + fn and(self, other: Rhs) -> Self::Output; +} + +/// Trait for performing logical AND KLEENE operations. +pub trait LogicalAndKleene { + /// The resulting type after performing the logical AND KLEENE operation. + type Output; + + /// Perform a logical AND operation between two values. + fn and_kleene(self, other: Rhs) -> Self::Output; +} + +/// Trait for performing logical AND NOT operations. +pub trait LogicalAndNot { + /// The resulting type after performing the logical AND NOT operation. + type Output; + + /// Perform a logical AND operation between two values. + fn and_not(self, other: Rhs) -> Self::Output; +} + +/// Trait for performing logical OR operations. +pub trait LogicalOr { + /// The resulting type after performing the logical AND operation. + type Output; + + /// Perform a logical OR operation between two values. + fn or(self, other: Rhs) -> Self::Output; +} + +/// Trait for performing logical OR KLEENE operations. +pub trait LogicalOrKleene { + /// The resulting type after performing the logical AND operation. + type Output; + + /// Perform a logical OR KLEENE operation between two values. + fn or_kleene(self, other: Rhs) -> Self::Output; +} + +/// Trait for performing logical NOT operations. +pub trait LogicalNot { + /// The resulting type after performing the logical AND NOT operation. + type Output; + + /// Perform a logical NOT operation. + fn not(self) -> Self::Output; +} diff --git a/vortex-compute/src/logical/not.rs b/vortex-compute/src/logical/not.rs new file mode 100644 index 00000000000..a702aa9b1c3 --- /dev/null +++ b/vortex-compute/src/logical/not.rs @@ -0,0 +1,39 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::ops::Not; +use vortex_vector::{BoolVector, BoolVectorMut, VectorOps}; + +use crate::logical::LogicalNot; + +impl LogicalNot for &BoolVector { + type Output = BoolVector; + + fn not(self) -> ::Output { + BoolVector::new(self.bits().not(), self.validity().clone()) + } +} + +impl LogicalNot for BoolVector { + type Output = BoolVector; + + fn not(self) -> ::Output { + // Attempt to re-use the underlying buffer if possible + let (bits, validity) = self.into_parts(); + let bits = match bits.try_into_mut() { + Ok(bits) => bits.not().freeze(), + Err(bits) => (&bits).not(), + }; + BoolVector::new(bits, validity) + } +} + +impl LogicalNot for BoolVectorMut { + type Output = BoolVectorMut; + + fn not(self) -> ::Output { + let (bits, validity) = self.into_parts(); + // SAFETY: we did not change the length of capacity + unsafe { BoolVectorMut::new_unchecked(bits.not(), validity) } + } +} diff --git a/vortex-compute/src/logical/or.rs b/vortex-compute/src/logical/or.rs new file mode 100644 index 00000000000..c2ecb52faca --- /dev/null +++ b/vortex-compute/src/logical/or.rs @@ -0,0 +1,20 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::ops::BitOr; + +use vortex_vector::{BoolVector, VectorOps}; + +use crate::logical::LogicalOr; + +// TODO(ngates): should we try to into_mut and reuse the existing buffer? Let's benchmark. +impl LogicalOr for &BoolVector { + type Output = BoolVector; + + fn or(self, other: &BoolVector) -> BoolVector { + BoolVector::new( + self.bits().bitor(other.bits()), + self.validity().bitor(other.validity()), + ) + } +} diff --git a/vortex-compute/src/logical/or_kleene.rs b/vortex-compute/src/logical/or_kleene.rs new file mode 100644 index 00000000000..3040bd51e5e --- /dev/null +++ b/vortex-compute/src/logical/or_kleene.rs @@ -0,0 +1,95 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::ops::{BitAnd, BitOr, Not}; + +use vortex_buffer::BitBuffer; +use vortex_mask::Mask; +use vortex_vector::{BoolVector, VectorOps}; + +use crate::logical::LogicalOrKleene; + +impl LogicalOrKleene for &BoolVector { + type Output = BoolVector; + + fn or_kleene(self, rhs: &BoolVector) -> BoolVector { + match (self.validity(), rhs.validity()) { + (Mask::AllTrue(_), Mask::AllTrue(_)) => { + BoolVector::new(self.bits().bitor(rhs.bits()), Mask::new_true(self.len())) + } + (Mask::AllTrue(_), Mask::AllFalse(_)) => { + // self valid, rhs all null + // Result: true where self is true (valid), null where self is false + let result_bits = BitBuffer::new_set(self.len()); + let validity = self.bits().clone(); // valid where self is true + BoolVector::new(result_bits, Mask::from(validity)) + } + (Mask::AllFalse(_), Mask::AllTrue(_)) => { + // self all null, rhs valid + // Result: true where rhs is true (valid), null where rhs is false + let result_bits = BitBuffer::new_set(self.len()); + let validity = rhs.bits().clone(); // valid where rhs is true + BoolVector::new(result_bits, Mask::from(validity)) + } + (Mask::AllFalse(_), Mask::AllFalse(_)) => { + // All values are null + BoolVector::new( + BitBuffer::new_unset(self.len()), + Mask::new_false(self.len()), + ) + } + (Mask::Values(lv), Mask::AllTrue(_)) => { + // self partial validity, rhs all valid + // Result valid where self valid OR self is null but rhs is true + let result_bits = self.bits().bitor(rhs.bits()); + let validity = lv.bit_buffer().bitor(rhs.bits()); + BoolVector::new(result_bits, Mask::from(validity)) + } + (Mask::AllTrue(_), Mask::Values(rv)) => { + // self all valid, rhs partial validity + // Result valid where rhs valid OR rhs is null but self is true + let result_bits = self.bits().bitor(rhs.bits()); + let validity = rv.bit_buffer().bitor(self.bits()); + BoolVector::new(result_bits, Mask::from(validity)) + } + (Mask::Values(lv), Mask::AllFalse(_)) => { + // self partial validity, rhs all null + // Result: true where self is true (valid), null otherwise + let result_bits = BitBuffer::new_set(self.len()); + let validity = lv.bit_buffer().bitand(self.bits()); + BoolVector::new(result_bits, Mask::from(validity)) + } + (Mask::AllFalse(_), Mask::Values(rv)) => { + // self all null, rhs partial validity + // Result: true where rhs is true (valid), null otherwise + let result_bits = BitBuffer::new_set(self.len()); + let validity = rv.bit_buffer().bitand(rhs.bits()); + BoolVector::new(result_bits, Mask::from(validity)) + } + (Mask::Values(lv), Mask::Values(rv)) => { + // Both have partial validity + // Result is valid where: + // 1. Both are valid, OR + // 2. One is null but the other is true (and valid) + let result_bits = self.bits().bitor(rhs.bits()); + + let both_valid = lv.bit_buffer().bitand(rv.bit_buffer()); + let self_null_rhs_true = lv + .bit_buffer() + .not() + .bitand(rv.bit_buffer()) + .bitand(rhs.bits()); + let rhs_null_self_true = rv + .bit_buffer() + .not() + .bitand(lv.bit_buffer()) + .bitand(self.bits()); + + let validity = both_valid + .bitor(&self_null_rhs_true) + .bitor(&rhs_null_self_true); + BoolVector::new(result_bits, Mask::from(validity)) + } + } + } +} diff --git a/vortex-vector/src/bool/vector.rs b/vortex-vector/src/bool/vector.rs index 5ddcc82d57d..15f5e1f9d36 100644 --- a/vortex-vector/src/bool/vector.rs +++ b/vortex-vector/src/bool/vector.rs @@ -38,6 +38,11 @@ impl BoolVector { Self { bits, validity } } + /// Decomposes the boolean vector into its constituent parts. + pub fn into_parts(self) -> (BitBuffer, Mask) { + (self.bits, self.validity) + } + /// Returns the bits buffer of the boolean vector. pub fn bits(&self) -> &BitBuffer { &self.bits diff --git a/vortex-vector/src/bool/vector_mut.rs b/vortex-vector/src/bool/vector_mut.rs index b9debb87b35..384b0d61782 100644 --- a/vortex-vector/src/bool/vector_mut.rs +++ b/vortex-vector/src/bool/vector_mut.rs @@ -71,6 +71,12 @@ pub struct BoolVectorMut { } impl BoolVectorMut { + /// Create a mutable vector from the given parts, without checking lengths or capacities. + pub unsafe fn new_unchecked(bits: BitBufferMut, validity: MaskMut) -> Self { + debug_assert_eq!(bits.len(), validity.len()); + Self { bits, validity } + } + /// Creates a new mutable boolean vector with the given `capacity`. pub fn with_capacity(capacity: usize) -> Self { Self { @@ -78,6 +84,11 @@ impl BoolVectorMut { validity: MaskMut::with_capacity(capacity), } } + + /// Returns the parts of the mutable vector. + pub fn into_parts(self) -> (BitBufferMut, MaskMut) { + (self.bits, self.validity) + } } impl FromIterator> for BoolVectorMut { From 78bc67ae63147de8f1292c3b2475dc73236ac16a Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 23 Oct 2025 15:15:43 -0400 Subject: [PATCH 2/6] Vortex Compute Signed-off-by: Nicholas Gates --- vortex-buffer/src/bit/buf_mut.rs | 7 ++++--- vortex-compute/src/logical/not.rs | 1 + vortex-vector/src/bool/vector_mut.rs | 6 ++++++ 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/vortex-buffer/src/bit/buf_mut.rs b/vortex-buffer/src/bit/buf_mut.rs index 52c9f2a01a7..bd37ef73508 100644 --- a/vortex-buffer/src/bit/buf_mut.rs +++ b/vortex-buffer/src/bit/buf_mut.rs @@ -1,12 +1,13 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use std::ops::Not; + use arrow_buffer::bit_chunk_iterator::BitChunks; use bitvec::view::BitView; -use std::ops::Not; use crate::bit::{get_bit_unchecked, ops, set_bit_unchecked, unset_bit_unchecked}; -use crate::{buffer_mut, BitBuffer, BufferMut, ByteBufferMut}; +use crate::{BitBuffer, BufferMut, ByteBufferMut, buffer_mut}; /// A mutable bitset buffer that allows random access to individual bits for set and get. /// @@ -518,7 +519,7 @@ impl FromIterator for BitBufferMut { #[cfg(test)] mod tests { use crate::bit::buf_mut::BitBufferMut; - use crate::{bitbuffer, bitbuffer_mut, buffer_mut, BufferMut}; + use crate::{BufferMut, bitbuffer, bitbuffer_mut, buffer_mut}; #[test] fn test_bits_mut() { diff --git a/vortex-compute/src/logical/not.rs b/vortex-compute/src/logical/not.rs index a702aa9b1c3..c07a7991a68 100644 --- a/vortex-compute/src/logical/not.rs +++ b/vortex-compute/src/logical/not.rs @@ -2,6 +2,7 @@ // SPDX-FileCopyrightText: Copyright the Vortex contributors use std::ops::Not; + use vortex_vector::{BoolVector, BoolVectorMut, VectorOps}; use crate::logical::LogicalNot; diff --git a/vortex-vector/src/bool/vector_mut.rs b/vortex-vector/src/bool/vector_mut.rs index 384b0d61782..3d4bab4cc33 100644 --- a/vortex-vector/src/bool/vector_mut.rs +++ b/vortex-vector/src/bool/vector_mut.rs @@ -72,6 +72,12 @@ pub struct BoolVectorMut { impl BoolVectorMut { /// Create a mutable vector from the given parts, without checking lengths or capacities. + /// + /// # SAFETY + /// + /// The caller must ensure both parts have the same length and capacity. Ideally they are + /// taken from `into_parts`, mutated in a way that doesn't re-allocate, and then passed back + /// to this function. pub unsafe fn new_unchecked(bits: BitBufferMut, validity: MaskMut) -> Self { debug_assert_eq!(bits.len(), validity.len()); Self { bits, validity } From b402ff10932c511bd48c94f112299e2676b0cd63 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 23 Oct 2025 15:53:55 -0400 Subject: [PATCH 3/6] Vortex Compute Signed-off-by: Nicholas Gates --- vortex-buffer/src/bit/macros.rs | 16 ++++---- vortex-compute/src/filter/bitbuffer.rs | 8 ++-- vortex-compute/src/filter/buffer.rs | 49 +++++++++++++++++++++++ vortex-compute/src/logical/and.rs | 40 ++++++++++++++++++- vortex-compute/src/logical/and_kleene.rs | 50 ++++++++++++++++++++++++ vortex-compute/src/logical/and_not.rs | 41 ++++++++++++++++++- vortex-compute/src/logical/not.rs | 26 ++++++++++++ vortex-compute/src/logical/or.rs | 40 ++++++++++++++++++- vortex-compute/src/logical/or_kleene.rs | 50 ++++++++++++++++++++++++ 9 files changed, 302 insertions(+), 18 deletions(-) diff --git a/vortex-buffer/src/bit/macros.rs b/vortex-buffer/src/bit/macros.rs index 2abda8c152f..b3a66c41a8d 100644 --- a/vortex-buffer/src/bit/macros.rs +++ b/vortex-buffer/src/bit/macros.rs @@ -17,16 +17,16 @@ macro_rules! bitbuffer { () => ( $crate::BitBuffer::empty() ); + // Match space-separated bit literals (0 or 1) + ($($bit:tt)+) => { + $crate::BitBuffer::from_iter([$( $crate::bitbuffer!(@bit $bit) ),+]) + }; ($elem:expr; $n:expr) => ( $crate::BitBuffer::full($elem, $n) ); ($($x:expr),+ $(,)?) => ( $crate::BitBuffer::from_iter([$($x),+]) ); - // Match space-separated bit literals (0 or 1) - ($($bit:tt)+) => { - $crate::BitBuffer::from_iter([$( $crate::bitbuffer!(@bit $bit) ),+]) - }; } /// A macro for constructing bit-buffers akin to `vec![..]`. @@ -45,14 +45,14 @@ macro_rules! bitbuffer_mut { () => ( $crate::BitBufferMut::empty() ); + // Match space-separated bit literals (0 or 1) + ($($bit:tt)+) => { + $crate::BitBufferMut::from_iter([$( $crate::bitbuffer_mut!(@bit $bit) ),+]) + }; ($elem:expr; $n:expr) => ( $crate::BitBufferMut::full($elem, $n) ); ($($x:expr),+ $(,)?) => ( $crate::BitBufferMut::from_iter([$($x),+]) ); - // Match space-separated bit literals (0 or 1) - ($($bit:tt)+) => { - $crate::BitBufferMut::from_iter([$( $crate::bitbuffer_mut!(@bit $bit) ),+]) - }; } diff --git a/vortex-compute/src/filter/bitbuffer.rs b/vortex-compute/src/filter/bitbuffer.rs index c6dc2a9d423..a480d744f4e 100644 --- a/vortex-compute/src/filter/bitbuffer.rs +++ b/vortex-compute/src/filter/bitbuffer.rs @@ -49,19 +49,19 @@ mod test { #[test] fn filter_bool_by_slice_test() { - let bits = bitbuffer![true, true, false]; + let bits = bitbuffer![1 1 0]; let filtered = filter_slices(&bits, 2, &[(0, 1), (2, 3)]); assert_eq!(2, filtered.len()); - assert_eq!(filtered, bitbuffer![true, false]) + assert_eq!(filtered, bitbuffer![1 0]) } #[test] fn filter_bool_by_index_test() { - let buf = bitbuffer![true, true, false]; + let buf = bitbuffer![1 1 0]; let filtered = filter_indices(&buf, &[0, 2]); assert_eq!(2, filtered.len()); - assert_eq!(bitbuffer![true, false], filtered) + assert_eq!(filtered, bitbuffer![1 0]) } } diff --git a/vortex-compute/src/filter/buffer.rs b/vortex-compute/src/filter/buffer.rs index 572f128bdca..b8eaa981ec1 100644 --- a/vortex-compute/src/filter/buffer.rs +++ b/vortex-compute/src/filter/buffer.rs @@ -36,3 +36,52 @@ fn filter_slices(values: &[T], output_len: usize, slices: &[(usize, usize)]) } out.freeze() } + +#[cfg(test)] +mod tests { + use vortex_buffer::buffer; + use vortex_mask::Mask; + + use super::*; + + #[test] + fn test_filter_buffer_by_indices() { + let buf = buffer![10u32, 20, 30, 40, 50]; + let mask = Mask::from_iter([true, false, true, false, true]); + + let result = buf.filter(&mask); + assert_eq!(result, buffer![10u32, 30, 50]); + } + + #[test] + fn test_filter_buffer_all_true() { + let buf = buffer![1u64, 2, 3]; + let mask = Mask::new_true(3); + + let result = buf.filter(&mask); + assert_eq!(result, buffer![1u64, 2, 3]); + } + + #[test] + fn test_filter_buffer_all_false() { + let buf = buffer![1i32, 2, 3, 4]; + let mask = Mask::new_false(4); + + let result = buf.filter(&mask); + assert!(result.is_empty()); + } + + #[test] + fn test_filter_indices_direct() { + let buf = buffer![100u32, 200, 300, 400]; + let result = filter_indices(buf.as_slice(), &[0, 2, 3]); + assert_eq!(result, buffer![100u32, 300, 400]); + } + + #[test] + fn test_filter_slices_direct() { + let buf = buffer![1u32, 2, 3, 4, 5]; + let result = filter_slices(buf.as_slice(), 3, &[(0, 2), (4, 5)]); + assert_eq!(result, buffer![1u32, 2, 5]); + } +} diff --git a/vortex-compute/src/logical/and.rs b/vortex-compute/src/logical/and.rs index 1727c356a2d..a5f1ec04e34 100644 --- a/vortex-compute/src/logical/and.rs +++ b/vortex-compute/src/logical/and.rs @@ -1,7 +1,7 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors -use std::ops::{BitAnd, BitOr}; +use std::ops::BitAnd; use vortex_vector::{BoolVector, VectorOps}; @@ -14,7 +14,43 @@ impl LogicalAnd for &BoolVector { fn and(self, other: &BoolVector) -> BoolVector { BoolVector::new( self.bits().bitand(other.bits()), - self.validity().bitor(other.validity()), + self.validity().bitand(other.validity()), ) } } + +impl LogicalAnd<&BoolVector> for BoolVector { + type Output = BoolVector; + + fn and(self, other: &BoolVector) -> BoolVector { + (&self).and(other) + } +} + +#[cfg(test)] +mod tests { + use vortex_buffer::bitbuffer; + use vortex_mask::Mask; + use vortex_vector::BoolVector; + + use super::*; + + #[test] + fn test_and_basic() { + let left = BoolVector::new(bitbuffer![1 1 0 0], Mask::new_true(4)); + let right = BoolVector::new(bitbuffer![1 0 1 0], Mask::new_true(4)); + + let result = left.and(&right); + assert_eq!(result.bits(), &bitbuffer![1 0 0 0]); + } + + #[test] + fn test_and_with_nulls() { + let left = BoolVector::new(bitbuffer![1 0], Mask::from(bitbuffer![1 0])); + let right = BoolVector::new(bitbuffer![1 1], Mask::new_true(2)); + + let result = left.and(&right); + // Validity is AND'd, so if either side is null, result is null + assert_eq!(result.validity(), &Mask::from(bitbuffer![1 0])); + } +} diff --git a/vortex-compute/src/logical/and_kleene.rs b/vortex-compute/src/logical/and_kleene.rs index e446cca5026..18b0a3fd508 100644 --- a/vortex-compute/src/logical/and_kleene.rs +++ b/vortex-compute/src/logical/and_kleene.rs @@ -93,3 +93,53 @@ impl LogicalAndKleene for &BoolVector { } } } + +impl LogicalAndKleene<&BoolVector> for BoolVector { + type Output = BoolVector; + + fn and_kleene(self, rhs: &BoolVector) -> Self::Output { + (&self).and_kleene(rhs) + } +} + +#[cfg(test)] +mod tests { + use vortex_buffer::bitbuffer; + use vortex_mask::Mask; + use vortex_vector::BoolVector; + + use super::*; + + #[test] + fn test_and_kleene_all_valid() { + // When both sides are all valid, behaves like regular AND + let left = BoolVector::new(bitbuffer![1 0 1], Mask::new_true(3)); + let right = BoolVector::new(bitbuffer![1 1 0], Mask::new_true(3)); + + let result = left.and_kleene(&right); + assert_eq!(result.bits(), &bitbuffer![1 0 0]); + assert_eq!(result.validity(), &Mask::new_true(3)); + } + + #[test] + fn test_and_kleene_all_null() { + // When both are null, result is all null + let left = BoolVector::new(bitbuffer![1 1], Mask::new_false(2)); + let right = BoolVector::new(bitbuffer![1 1], Mask::new_false(2)); + + let result = left.and_kleene(&right); + assert_eq!(result.validity(), &Mask::new_false(2)); + } + + #[test] + fn test_and_kleene_false_and_null() { + // false AND null = false (Kleene logic) + let left = BoolVector::new(bitbuffer![0], Mask::new_true(1)); + let right = BoolVector::new(bitbuffer![1], Mask::new_false(1)); + + let result = left.and_kleene(&right); + assert_eq!(result.bits(), &bitbuffer![0]); + // Result should be valid because false AND anything is false + assert_eq!(result.validity(), &Mask::new_true(1)); + } +} diff --git a/vortex-compute/src/logical/and_not.rs b/vortex-compute/src/logical/and_not.rs index bb3df9ff200..3e91b61be10 100644 --- a/vortex-compute/src/logical/and_not.rs +++ b/vortex-compute/src/logical/and_not.rs @@ -1,7 +1,7 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors -use std::ops::BitOr; +use std::ops::BitAnd; use vortex_vector::{BoolVector, VectorOps}; @@ -14,7 +14,44 @@ impl LogicalAndNot for &BoolVector { fn and_not(self, other: &BoolVector) -> BoolVector { BoolVector::new( self.bits().bitand_not(other.bits()), - self.validity().bitor(other.validity()), + self.validity().bitand(other.validity()), ) } } + +impl LogicalAndNot<&BoolVector> for BoolVector { + type Output = BoolVector; + + fn and_not(self, other: &BoolVector) -> BoolVector { + (&self).and_not(other) + } +} + +#[cfg(test)] +mod tests { + use vortex_buffer::bitbuffer; + use vortex_mask::Mask; + use vortex_vector::BoolVector; + + use super::*; + + #[test] + fn test_and_not_basic() { + // left AND (NOT right) + let left = BoolVector::new(bitbuffer![1 1 0 0], Mask::new_true(4)); + let right = BoolVector::new(bitbuffer![1 0 1 0], Mask::new_true(4)); + + let result = left.and_not(&right); + // 1 & !1 = 0, 1 & !0 = 1, 0 & !1 = 0, 0 & !0 = 0 + assert_eq!(result.bits(), &bitbuffer![0 1 0 0]); + } + + #[test] + fn test_and_not_all_true() { + let left = BoolVector::new(bitbuffer![1 1], Mask::new_true(2)); + let right = BoolVector::new(bitbuffer![1 1], Mask::new_true(2)); + + let result = left.and_not(&right); + assert_eq!(result.bits(), &bitbuffer![0 0]); + } +} diff --git a/vortex-compute/src/logical/not.rs b/vortex-compute/src/logical/not.rs index c07a7991a68..5ae937387f4 100644 --- a/vortex-compute/src/logical/not.rs +++ b/vortex-compute/src/logical/not.rs @@ -38,3 +38,29 @@ impl LogicalNot for BoolVectorMut { unsafe { BoolVectorMut::new_unchecked(bits.not(), validity) } } } + +#[cfg(test)] +mod tests { + use vortex_buffer::bitbuffer; + use vortex_mask::Mask; + use vortex_vector::BoolVector; + + use super::*; + + #[test] + fn test_not_basic() { + let vec = BoolVector::new(bitbuffer![1 0 1 0], Mask::new_true(4)); + + let result = vec.not(); + assert_eq!(result.bits(), &bitbuffer![0 1 0 1]); + assert_eq!(result.validity(), &Mask::new_true(4)); + } + + #[test] + fn test_not_owned() { + let vec = BoolVector::new(bitbuffer![1 1], Mask::new_true(2)); + + let result = vec.not(); + assert_eq!(result.bits(), &bitbuffer![0 0]); + } +} diff --git a/vortex-compute/src/logical/or.rs b/vortex-compute/src/logical/or.rs index c2ecb52faca..b88cc840c26 100644 --- a/vortex-compute/src/logical/or.rs +++ b/vortex-compute/src/logical/or.rs @@ -1,7 +1,7 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors -use std::ops::BitOr; +use std::ops::{BitAnd, BitOr}; use vortex_vector::{BoolVector, VectorOps}; @@ -14,7 +14,43 @@ impl LogicalOr for &BoolVector { fn or(self, other: &BoolVector) -> BoolVector { BoolVector::new( self.bits().bitor(other.bits()), - self.validity().bitor(other.validity()), + self.validity().bitand(other.validity()), ) } } + +impl LogicalOr<&BoolVector> for BoolVector { + type Output = BoolVector; + + fn or(self, other: &BoolVector) -> BoolVector { + (&self).or(other) + } +} + +#[cfg(test)] +mod tests { + use vortex_buffer::bitbuffer; + use vortex_mask::Mask; + use vortex_vector::BoolVector; + + use super::*; + + #[test] + fn test_or_basic() { + let left = BoolVector::new(bitbuffer![1 1 0 0], Mask::new_true(4)); + let right = BoolVector::new(bitbuffer![1 0 1 0], Mask::new_true(4)); + + let result = left.or(&right); + assert_eq!(result.bits(), &bitbuffer![1 1 1 0]); + } + + #[test] + fn test_or_with_nulls() { + let left = BoolVector::new(bitbuffer![0 1], Mask::from(bitbuffer![0 1])); + let right = BoolVector::new(bitbuffer![0 0], Mask::new_true(2)); + + let result = left.or(&right); + // Validity is AND'd, so if either side is null, result is null + assert_eq!(result.validity(), &Mask::from(bitbuffer![0 1])); + } +} diff --git a/vortex-compute/src/logical/or_kleene.rs b/vortex-compute/src/logical/or_kleene.rs index 3040bd51e5e..e077fe65ac0 100644 --- a/vortex-compute/src/logical/or_kleene.rs +++ b/vortex-compute/src/logical/or_kleene.rs @@ -93,3 +93,53 @@ impl LogicalOrKleene for &BoolVector { } } } + +impl LogicalOrKleene<&BoolVector> for BoolVector { + type Output = BoolVector; + + fn or_kleene(self, rhs: &BoolVector) -> BoolVector { + (&self).or_kleene(rhs) + } +} + +#[cfg(test)] +mod tests { + use vortex_buffer::bitbuffer; + use vortex_mask::Mask; + use vortex_vector::BoolVector; + + use super::*; + + #[test] + fn test_or_kleene_all_valid() { + // When both sides are all valid, behaves like regular OR + let left = BoolVector::new(bitbuffer![1 0 0], Mask::new_true(3)); + let right = BoolVector::new(bitbuffer![0 1 0], Mask::new_true(3)); + + let result = left.or_kleene(&right); + assert_eq!(result.bits(), &bitbuffer![1 1 0]); + assert_eq!(result.validity(), &Mask::new_true(3)); + } + + #[test] + fn test_or_kleene_all_null() { + // When both are null, result is all null + let left = BoolVector::new(bitbuffer![0 0], Mask::new_false(2)); + let right = BoolVector::new(bitbuffer![0 0], Mask::new_false(2)); + + let result = left.or_kleene(&right); + assert_eq!(result.validity(), &Mask::new_false(2)); + } + + #[test] + fn test_or_kleene_true_and_null() { + // true OR null = true (Kleene logic) + let left = BoolVector::new(bitbuffer![1], Mask::new_true(1)); + let right = BoolVector::new(bitbuffer![0], Mask::new_false(1)); + + let result = left.or_kleene(&right); + assert_eq!(result.bits(), &bitbuffer![1]); + // Result should be valid because true OR anything is true + assert_eq!(result.validity(), &Mask::new_true(1)); + } +} From 80737a3fb128a78907c186ef9072c3d212727151 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 23 Oct 2025 16:08:38 -0400 Subject: [PATCH 4/6] Vortex Compute Signed-off-by: Nicholas Gates --- vortex-buffer/src/bit/macros.rs | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/vortex-buffer/src/bit/macros.rs b/vortex-buffer/src/bit/macros.rs index b3a66c41a8d..9e5338b7f4e 100644 --- a/vortex-buffer/src/bit/macros.rs +++ b/vortex-buffer/src/bit/macros.rs @@ -18,9 +18,16 @@ macro_rules! bitbuffer { $crate::BitBuffer::empty() ); // Match space-separated bit literals (0 or 1) - ($($bit:tt)+) => { + (0) => { + $crate::BitBuffer::from_iter([false]) + }; + (1) => { + $crate::BitBuffer::from_iter([true]) + }; + ($($bit:tt )+) => { $crate::BitBuffer::from_iter([$( $crate::bitbuffer!(@bit $bit) ),+]) }; + ($elem:expr; $n:expr) => ( $crate::BitBuffer::full($elem, $n) ); @@ -46,13 +53,21 @@ macro_rules! bitbuffer_mut { $crate::BitBufferMut::empty() ); // Match space-separated bit literals (0 or 1) - ($($bit:tt)+) => { + (0) => { + $crate::BitBuffer::from_iter([false]) + }; + (1) => { + $crate::BitBuffer::from_iter([true]) + }; + ($($bit:tt )+) => { $crate::BitBufferMut::from_iter([$( $crate::bitbuffer_mut!(@bit $bit) ),+]) }; + ($elem:expr; $n:expr) => ( $crate::BitBufferMut::full($elem, $n) ); ($($x:expr),+ $(,)?) => ( $crate::BitBufferMut::from_iter([$($x),+]) ); + } From fb15158020db4e10c6f61050318904238270bbcf Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 23 Oct 2025 16:39:00 -0400 Subject: [PATCH 5/6] Vortex Compute Signed-off-by: Nicholas Gates --- vortex-buffer/src/bit/macros.rs | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/vortex-buffer/src/bit/macros.rs b/vortex-buffer/src/bit/macros.rs index 9e5338b7f4e..34b24ecc65e 100644 --- a/vortex-buffer/src/bit/macros.rs +++ b/vortex-buffer/src/bit/macros.rs @@ -17,23 +17,21 @@ macro_rules! bitbuffer { () => ( $crate::BitBuffer::empty() ); - // Match space-separated bit literals (0 or 1) (0) => { $crate::BitBuffer::from_iter([false]) }; (1) => { $crate::BitBuffer::from_iter([true]) }; - ($($bit:tt )+) => { - $crate::BitBuffer::from_iter([$( $crate::bitbuffer!(@bit $bit) ),+]) - }; - ($elem:expr; $n:expr) => ( $crate::BitBuffer::full($elem, $n) ); ($($x:expr),+ $(,)?) => ( $crate::BitBuffer::from_iter([$($x),+]) ); + ($($bit:expr )+) => { + $crate::BitBuffer::from_iter([$( $crate::bitbuffer!(@bit $bit) ),+]) + }; } /// A macro for constructing bit-buffers akin to `vec![..]`. @@ -52,22 +50,19 @@ macro_rules! bitbuffer_mut { () => ( $crate::BitBufferMut::empty() ); - // Match space-separated bit literals (0 or 1) (0) => { $crate::BitBuffer::from_iter([false]) }; (1) => { $crate::BitBuffer::from_iter([true]) }; - ($($bit:tt )+) => { - $crate::BitBufferMut::from_iter([$( $crate::bitbuffer_mut!(@bit $bit) ),+]) - }; - ($elem:expr; $n:expr) => ( $crate::BitBufferMut::full($elem, $n) ); ($($x:expr),+ $(,)?) => ( $crate::BitBufferMut::from_iter([$($x),+]) ); - + ($($bit:tt )+) => { + $crate::BitBufferMut::from_iter([$( $crate::bitbuffer_mut!(@bit $bit) ),+]) + }; } From 9eebdc59e8f6d24d60185bb4fadeff574528b55d Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 23 Oct 2025 16:44:53 -0400 Subject: [PATCH 6/6] Vortex Compute Signed-off-by: Nicholas Gates --- vortex-buffer/src/bit/macros.rs | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/vortex-buffer/src/bit/macros.rs b/vortex-buffer/src/bit/macros.rs index 34b24ecc65e..c186318440c 100644 --- a/vortex-buffer/src/bit/macros.rs +++ b/vortex-buffer/src/bit/macros.rs @@ -17,19 +17,23 @@ macro_rules! bitbuffer { () => ( $crate::BitBuffer::empty() ); + + // We capture single-element 0/1 cases to avoid ambiguity with the + // comma-separated expression case. (0) => { $crate::BitBuffer::from_iter([false]) }; (1) => { $crate::BitBuffer::from_iter([true]) }; + ($elem:expr; $n:expr) => ( $crate::BitBuffer::full($elem, $n) ); ($($x:expr),+ $(,)?) => ( $crate::BitBuffer::from_iter([$($x),+]) ); - ($($bit:expr )+) => { + ($($bit:tt)+) => { $crate::BitBuffer::from_iter([$( $crate::bitbuffer!(@bit $bit) ),+]) }; } @@ -50,19 +54,23 @@ macro_rules! bitbuffer_mut { () => ( $crate::BitBufferMut::empty() ); + + // We capture single-element 0/1 cases to avoid ambiguity with the + // comma-separated expression case. (0) => { $crate::BitBuffer::from_iter([false]) }; (1) => { $crate::BitBuffer::from_iter([true]) }; + ($elem:expr; $n:expr) => ( $crate::BitBufferMut::full($elem, $n) ); ($($x:expr),+ $(,)?) => ( $crate::BitBufferMut::from_iter([$($x),+]) ); - ($($bit:tt )+) => { + ($($bit:tt)+) => { $crate::BitBufferMut::from_iter([$( $crate::bitbuffer_mut!(@bit $bit) ),+]) }; }