Skip to main content

arrow_buffer/buffer/
boolean.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18use crate::bit_chunk_iterator::BitChunks;
19use crate::bit_iterator::{BitIndexIterator, BitIndexU32Iterator, BitIterator, BitSliceIterator};
20use crate::{
21    BooleanBufferBuilder, Buffer, MutableBuffer, bit_util, buffer_bin_and, buffer_bin_or,
22    buffer_bin_xor, buffer_unary_not,
23};
24
25use std::ops::{BitAnd, BitOr, BitXor, Not};
26
27/// A slice-able [`Buffer`] containing bit-packed booleans
28///
29/// This structure represents a sequence of boolean values packed into a
30/// byte-aligned [`Buffer`]. Both the offset and length are represented in bits.
31///
32/// # Layout
33///
34/// The values are represented as little endian bit-packed values, where the
35/// least significant bit of each byte represents the first boolean value and
36/// then proceeding to the most significant bit.
37///
38/// For example, the 10 bit bitmask `0b0111001101` has length 10, and is
39/// represented using 2 bytes with offset 0 like this:
40///
41/// ```text
42///        ┌─────────────────────────────────┐    ┌─────────────────────────────────┐
43///        │┌───┬───┬───┬───┬───┬───┬───┬───┐│    │┌───┬───┬───┬───┬───┬───┬───┬───┐│
44///        ││ 1 │ 0 │ 1 │ 1 │ 0 │ 0 │ 1 │ 1 ││    ││ 1 │ 0 │ ? │ ? │ ? │ ? │ ? │ ? ││
45///        │└───┴───┴───┴───┴───┴───┴───┴───┘│    │└───┴───┴───┴───┴───┴───┴───┴───┘│
46/// bit    └─────────────────────────────────┘    └─────────────────────────────────┘
47/// offset  0             Byte 0             7    0              Byte 1            7
48///
49///         length = 10 bits, offset = 0
50/// ```
51///
52/// The same bitmask with length 10 and offset 3 would be represented using 2
53/// bytes like this:
54///
55/// ```text
56///       ┌─────────────────────────────────┐    ┌─────────────────────────────────┐
57///       │┌───┬───┬───┬───┬───┬───┬───┬───┐│    │┌───┬───┬───┬───┬───┬───┬───┬───┐│
58///       ││ ? │ ? │ ? │ 1 │ 0 │ 1 │ 1 │ 0 ││    ││ 0 │ 1 │ 1 │ 1 │ 0 │ ? │ ? │ ? ││
59///       │└───┴───┴───┴───┴───┴───┴───┴───┘│    │└───┴───┴───┴───┴───┴───┴───┴───┘│
60/// bit   └─────────────────────────────────┘    └─────────────────────────────────┘
61/// offset 0             Byte 0             7    0              Byte 1            7
62///
63///        length = 10 bits, offset = 3
64/// ```
65///
66/// Note that the bits marked `?` are not logically part of the mask and may
67/// contain either `0` or `1`
68///
69/// # See Also
70/// * [`BooleanBufferBuilder`] for building [`BooleanBuffer`] instances
71/// * [`NullBuffer`] for representing null values in Arrow arrays
72///
73/// [`NullBuffer`]: crate::NullBuffer
74#[derive(Debug, Clone, Eq)]
75pub struct BooleanBuffer {
76    /// Underlying buffer (byte aligned)
77    buffer: Buffer,
78    /// Offset in bits (not bytes)
79    bit_offset: usize,
80    /// Length in bits (not bytes)
81    bit_len: usize,
82}
83
84impl PartialEq for BooleanBuffer {
85    fn eq(&self, other: &Self) -> bool {
86        if self.bit_len != other.bit_len {
87            return false;
88        }
89
90        let lhs = self.bit_chunks().iter_padded();
91        let rhs = other.bit_chunks().iter_padded();
92        lhs.zip(rhs).all(|(a, b)| a == b)
93    }
94}
95
96impl BooleanBuffer {
97    /// Create a new [`BooleanBuffer`] from a [`Buffer`], `bit_offset` offset and `bit_len` length
98    ///
99    /// # Panics
100    ///
101    /// This method will panic if `buffer` is not large enough
102    pub fn new(buffer: Buffer, bit_offset: usize, bit_len: usize) -> Self {
103        let total_len = bit_offset.saturating_add(bit_len);
104        let buffer_len = buffer.len();
105        let buffer_bit_len = buffer_len.saturating_mul(8);
106        assert!(
107            total_len <= buffer_bit_len,
108            "buffer not large enough (bit_offset: {bit_offset}, bit_len: {bit_len}, buffer_len: {buffer_len})"
109        );
110        Self {
111            buffer,
112            bit_offset,
113            bit_len,
114        }
115    }
116
117    /// Create a new [`BooleanBuffer`] of `length` bits (not bytes) where all values are `true`
118    pub fn new_set(length: usize) -> Self {
119        let mut builder = BooleanBufferBuilder::new(length);
120        builder.append_n(length, true);
121        builder.finish()
122    }
123
124    /// Create a new [`BooleanBuffer`] of `length` bits (not bytes) where all values are `false`
125    pub fn new_unset(length: usize) -> Self {
126        let buffer = MutableBuffer::new_null(length).into_buffer();
127        Self {
128            buffer,
129            bit_offset: 0,
130            bit_len: length,
131        }
132    }
133
134    /// Invokes `f` with indexes `0..len` collecting the boolean results into a new `BooleanBuffer`
135    pub fn collect_bool<F: FnMut(usize) -> bool>(len: usize, f: F) -> Self {
136        let buffer = MutableBuffer::collect_bool(len, f);
137        Self::new(buffer.into(), 0, len)
138    }
139
140    /// Create a new [`BooleanBuffer`] by copying the relevant bits from an
141    /// input buffer.
142    ///
143    /// # Notes:
144    /// * The new `BooleanBuffer` has zero offset, even if `offset_in_bits` is non-zero
145    ///
146    /// # Example: Create a new [`BooleanBuffer`] copying a bit slice from in input slice
147    /// ```
148    /// # use arrow_buffer::BooleanBuffer;
149    /// let input = [0b11001100u8, 0b10111010u8];
150    /// // // Copy bits 4..16 from input
151    /// let result = BooleanBuffer::from_bits(&input, 4, 12);
152    /// assert_eq!(result.values(), &[0b10101100u8, 0b00001011u8]);
153    pub fn from_bits(src: impl AsRef<[u8]>, offset_in_bits: usize, len_in_bits: usize) -> Self {
154        Self::from_bitwise_unary_op(src, offset_in_bits, len_in_bits, |a| a)
155    }
156
157    /// Create a new [`BooleanBuffer`] by applying the bitwise operation to `op`
158    /// to an input buffer.
159    ///
160    /// This function is faster than applying the operation bit by bit as
161    /// it processes input buffers in chunks of 64 bits (8 bytes) at a time
162    ///
163    /// # Notes:
164    /// * `op` takes a single `u64` inputs and produces one `u64` output.
165    /// * `op` must only apply bitwise operations
166    ///   on the relevant bits; the input `u64` may contain irrelevant bits
167    ///   and may be processed differently on different endian architectures.
168    /// * `op` may be called with input bits outside the requested range
169    /// * The output always has zero offset
170    ///
171    /// # See Also
172    /// - [`BooleanBuffer::from_bitwise_binary_op`] to create a new buffer from a binary operation
173    /// - [`apply_bitwise_unary_op`](bit_util::apply_bitwise_unary_op) for in-place unary bitwise operations
174    ///
175    /// # Example: Create new [`BooleanBuffer`] from bitwise `NOT` of a byte slice
176    /// ```
177    /// # use arrow_buffer::BooleanBuffer;
178    /// let input = [0b11001100u8, 0b10111010u8]; // 2 bytes = 16 bits
179    /// // NOT of the first 12 bits
180    /// let result = BooleanBuffer::from_bitwise_unary_op(
181    ///  &input, 0, 12, |a| !a
182    /// );
183    /// assert_eq!(result.values(), &[0b00110011u8, 0b11110101u8]);
184    /// ```
185    pub fn from_bitwise_unary_op<F>(
186        src: impl AsRef<[u8]>,
187        offset_in_bits: usize,
188        len_in_bits: usize,
189        mut op: F,
190    ) -> Self
191    where
192        F: FnMut(u64) -> u64,
193    {
194        // try fast path for aligned input
195        if offset_in_bits & 0x7 == 0 {
196            // align to byte boundary
197            let aligned = &src.as_ref()[offset_in_bits / 8..];
198            if let Some(result) =
199                Self::try_from_aligned_bitwise_unary_op(aligned, len_in_bits, &mut op)
200            {
201                return result;
202            }
203        }
204
205        let chunks = BitChunks::new(src.as_ref(), offset_in_bits, len_in_bits);
206        let mut result = MutableBuffer::with_capacity(chunks.num_u64s() * 8);
207        for chunk in chunks.iter() {
208            // SAFETY: reserved enough capacity above, (exactly num_u64s()
209            // items) and we assume `BitChunks` correctly reports upper bound
210            unsafe {
211                result.push_unchecked(op(chunk));
212            }
213        }
214        if chunks.remainder_len() > 0 {
215            debug_assert!(result.capacity() >= result.len() + 8); // should not reallocate
216            // SAFETY: reserved enough capacity above, (exactly num_u64s()
217            // items) and we assume `BitChunks` correctly reports upper bound
218            unsafe {
219                result.push_unchecked(op(chunks.remainder_bits()));
220            }
221            // Just pushed one u64, which may have trailing zeros
222            result.truncate(chunks.num_bytes());
223        }
224
225        BooleanBuffer {
226            buffer: Buffer::from(result),
227            bit_offset: 0,
228            bit_len: len_in_bits,
229        }
230    }
231
232    /// Fast path for [`Self::from_bitwise_unary_op`] when input is aligned to
233    /// 8-byte (64-bit) boundaries
234    ///
235    /// Returns None if the fast path cannot be taken
236    fn try_from_aligned_bitwise_unary_op<F>(
237        src: &[u8],
238        len_in_bits: usize,
239        op: &mut F,
240    ) -> Option<Self>
241    where
242        F: FnMut(u64) -> u64,
243    {
244        // Safety: all valid bytes are valid u64s
245        let (prefix, aligned_u6us, suffix) = unsafe { src.align_to::<u64>() };
246        if !(prefix.is_empty() && suffix.is_empty()) {
247            // Couldn't make this case any faster than the default path, see
248            // https://github.com/apache/arrow-rs/pull/8996/changes#r2620022082
249            return None;
250        }
251        // the buffer is word (64 bit) aligned, so use optimized Vec code.
252        let result_u64s: Vec<u64> = aligned_u6us.iter().map(|l| op(*l)).collect();
253        let buffer = Buffer::from(result_u64s);
254        Some(BooleanBuffer::new(buffer, 0, len_in_bits))
255    }
256
257    /// Create a new [`BooleanBuffer`] by applying the bitwise operation `op` to
258    /// the relevant bits from two input buffers.
259    ///
260    /// This function is faster than applying the operation bit by bit as
261    /// it processes input buffers in chunks of 64 bits (8 bytes) at a time
262    ///
263    /// # Notes:
264    /// See notes on [Self::from_bitwise_unary_op]
265    ///
266    /// # See Also
267    /// - [`BooleanBuffer::from_bitwise_unary_op`] for unary operations on a single input buffer.
268    /// - [`apply_bitwise_binary_op`](bit_util::apply_bitwise_binary_op) for in-place binary bitwise operations
269    ///
270    /// # Example: Create new [`BooleanBuffer`] from bitwise `AND` of two [`Buffer`]s
271    /// ```
272    /// # use arrow_buffer::{Buffer, BooleanBuffer};
273    /// let left = Buffer::from(vec![0b11001100u8, 0b10111010u8]); // 2 bytes = 16 bits
274    /// let right = Buffer::from(vec![0b10101010u8, 0b11011100u8, 0b11110000u8]); // 3 bytes = 24 bits
275    /// // AND of the first 12 bits
276    /// let result = BooleanBuffer::from_bitwise_binary_op(
277    ///   &left, 0, &right, 0, 12, |a, b| a & b
278    /// );
279    /// assert_eq!(result.inner().as_slice(), &[0b10001000u8, 0b00001000u8]);
280    /// ```
281    ///
282    /// # Example: Create new [`BooleanBuffer`] from bitwise `OR` of two byte slices
283    /// ```
284    /// # use arrow_buffer::BooleanBuffer;
285    /// let left = [0b11001100u8, 0b10111010u8];
286    /// let right = [0b10101010u8, 0b11011100u8];
287    /// // OR of bits 4..16 from left and bits 0..12 from right
288    /// let result = BooleanBuffer::from_bitwise_binary_op(
289    ///  &left, 4, &right, 0, 12, |a, b| a | b
290    /// );
291    /// assert_eq!(result.inner().as_slice(), &[0b10101110u8, 0b00001111u8]);
292    /// ```
293    pub fn from_bitwise_binary_op<F>(
294        left: impl AsRef<[u8]>,
295        left_offset_in_bits: usize,
296        right: impl AsRef<[u8]>,
297        right_offset_in_bits: usize,
298        len_in_bits: usize,
299        mut op: F,
300    ) -> Self
301    where
302        F: FnMut(u64, u64) -> u64,
303    {
304        let left = left.as_ref();
305        let right = right.as_ref();
306        // try fast path for aligned input
307        // If the underlying buffers are aligned to u64 we can apply the operation directly on the u64 slices
308        // to improve performance.
309        if left_offset_in_bits & 0x7 == 0 && right_offset_in_bits & 0x7 == 0 {
310            // align to byte boundary
311            let left = &left[left_offset_in_bits / 8..];
312            let right = &right[right_offset_in_bits / 8..];
313
314            unsafe {
315                let (left_prefix, left_u64s, left_suffix) = left.align_to::<u64>();
316                let (right_prefix, right_u64s, right_suffix) = right.align_to::<u64>();
317                // if there is no prefix or suffix, both buffers are aligned and
318                // we can do the operation directly on u64s.
319                // TODO: consider `slice::as_chunks` and `u64::from_le_bytes` when MSRV reaches 1.88.
320                // https://github.com/apache/arrow-rs/pull/9022#discussion_r2639949361
321                if left_prefix.is_empty()
322                    && right_prefix.is_empty()
323                    && left_suffix.is_empty()
324                    && right_suffix.is_empty()
325                {
326                    let result_u64s = left_u64s
327                        .iter()
328                        .zip(right_u64s.iter())
329                        .map(|(l, r)| op(*l, *r))
330                        .collect::<Vec<u64>>();
331                    return BooleanBuffer {
332                        buffer: Buffer::from(result_u64s),
333                        bit_offset: 0,
334                        bit_len: len_in_bits,
335                    };
336                }
337            }
338        }
339        let left_chunks = BitChunks::new(left, left_offset_in_bits, len_in_bits);
340        let right_chunks = BitChunks::new(right, right_offset_in_bits, len_in_bits);
341
342        let chunks = left_chunks
343            .iter()
344            .zip(right_chunks.iter())
345            .map(|(left, right)| op(left, right));
346        // Soundness: `BitChunks` is a `BitChunks` trusted length iterator which
347        // correctly reports its upper bound
348        let mut buffer = unsafe { MutableBuffer::from_trusted_len_iter(chunks) };
349
350        let remainder_bytes = bit_util::ceil(left_chunks.remainder_len(), 8);
351        let rem = op(left_chunks.remainder_bits(), right_chunks.remainder_bits());
352        // we are counting its starting from the least significant bit, to to_le_bytes should be correct
353        let rem = &rem.to_le_bytes()[0..remainder_bytes];
354        buffer.extend_from_slice(rem);
355
356        BooleanBuffer {
357            buffer: Buffer::from(buffer),
358            bit_offset: 0,
359            bit_len: len_in_bits,
360        }
361    }
362
363    /// Returns the number of set bits in this buffer
364    pub fn count_set_bits(&self) -> usize {
365        self.buffer
366            .count_set_bits_offset(self.bit_offset, self.bit_len)
367    }
368
369    /// Finds the position of the n-th set bit (1-based) starting from `start` index.
370    /// If fewer than `n` set bits are found, returns the length of the buffer.
371    pub fn find_nth_set_bit_position(&self, start: usize, n: usize) -> usize {
372        if n == 0 {
373            return start;
374        }
375
376        self.slice(start, self.bit_len - start)
377            .set_indices()
378            .nth(n - 1)
379            .map(|idx| start + idx + 1)
380            .unwrap_or(self.bit_len)
381    }
382
383    /// Returns a [`BitChunks`] instance which can be used to iterate over
384    /// this buffer's bits in `u64` chunks
385    #[inline]
386    pub fn bit_chunks(&self) -> BitChunks<'_> {
387        BitChunks::new(self.values(), self.bit_offset, self.bit_len)
388    }
389
390    /// Returns the offset of this [`BooleanBuffer`] in bits (not bytes)
391    #[inline]
392    pub fn offset(&self) -> usize {
393        self.bit_offset
394    }
395
396    /// Returns the length of this [`BooleanBuffer`] in bits (not bytes)
397    #[inline]
398    pub fn len(&self) -> usize {
399        self.bit_len
400    }
401
402    /// Returns true if this [`BooleanBuffer`] is empty
403    #[inline]
404    pub fn is_empty(&self) -> bool {
405        self.bit_len == 0
406    }
407
408    /// Free up unused memory.
409    pub fn shrink_to_fit(&mut self) {
410        // TODO(emilk): we could shrink even more in the case where we are a small sub-slice of the full buffer
411        self.buffer.shrink_to_fit();
412    }
413
414    /// Returns the boolean value at index `i`.
415    ///
416    /// # Panics
417    ///
418    /// Panics if `i >= self.len()`
419    #[inline]
420    pub fn value(&self, idx: usize) -> bool {
421        assert!(idx < self.bit_len);
422        unsafe { self.value_unchecked(idx) }
423    }
424
425    /// Returns the boolean value at index `i`.
426    ///
427    /// # Safety
428    /// This doesn't check bounds, the caller must ensure that index < self.len()
429    #[inline]
430    pub unsafe fn value_unchecked(&self, i: usize) -> bool {
431        unsafe { bit_util::get_bit_raw(self.buffer.as_ptr(), i + self.bit_offset) }
432    }
433
434    /// Returns the packed values of this [`BooleanBuffer`] not including any offset
435    #[inline]
436    pub fn values(&self) -> &[u8] {
437        &self.buffer
438    }
439
440    /// Slices this [`BooleanBuffer`] by the provided `offset` and `length`
441    pub fn slice(&self, offset: usize, len: usize) -> Self {
442        assert!(
443            offset.saturating_add(len) <= self.bit_len,
444            "the length + offset of the sliced BooleanBuffer cannot exceed the existing length"
445        );
446        Self {
447            buffer: self.buffer.clone(),
448            bit_offset: self.bit_offset + offset,
449            bit_len: len,
450        }
451    }
452
453    /// Returns a [`Buffer`] containing the sliced contents of this [`BooleanBuffer`]
454    ///
455    /// Equivalent to `self.buffer.bit_slice(self.offset, self.len)`
456    pub fn sliced(&self) -> Buffer {
457        self.buffer.bit_slice(self.bit_offset, self.bit_len)
458    }
459
460    /// Returns true if this [`BooleanBuffer`] is equal to `other`, using pointer comparisons
461    /// to determine buffer equality. This is cheaper than `PartialEq::eq` but may
462    /// return false when the arrays are logically equal
463    pub fn ptr_eq(&self, other: &Self) -> bool {
464        self.buffer.as_ptr() == other.buffer.as_ptr()
465            && self.bit_offset == other.bit_offset
466            && self.bit_len == other.bit_len
467    }
468
469    /// Returns the inner [`Buffer`]
470    ///
471    /// Note: this does not account for offset and length of this [`BooleanBuffer`]
472    #[inline]
473    pub fn inner(&self) -> &Buffer {
474        &self.buffer
475    }
476
477    /// Returns the inner [`Buffer`], consuming self
478    ///
479    /// Note: this does not account for offset and length of this [`BooleanBuffer`]
480    pub fn into_inner(self) -> Buffer {
481        self.buffer
482    }
483
484    /// Returns an iterator over the bits in this [`BooleanBuffer`]
485    pub fn iter(&self) -> BitIterator<'_> {
486        self.into_iter()
487    }
488
489    /// Returns an iterator over the set bit positions in this [`BooleanBuffer`]
490    pub fn set_indices(&self) -> BitIndexIterator<'_> {
491        BitIndexIterator::new(self.values(), self.bit_offset, self.bit_len)
492    }
493
494    /// Returns a `u32` iterator over set bit positions without any usize->u32 conversion
495    pub fn set_indices_u32(&self) -> BitIndexU32Iterator<'_> {
496        BitIndexU32Iterator::new(self.values(), self.bit_offset, self.bit_len)
497    }
498
499    /// Returns a [`BitSliceIterator`] yielding contiguous ranges of set bits
500    pub fn set_slices(&self) -> BitSliceIterator<'_> {
501        BitSliceIterator::new(self.values(), self.bit_offset, self.bit_len)
502    }
503}
504
505impl Not for &BooleanBuffer {
506    type Output = BooleanBuffer;
507
508    fn not(self) -> Self::Output {
509        BooleanBuffer {
510            buffer: buffer_unary_not(&self.buffer, self.bit_offset, self.bit_len),
511            bit_offset: 0,
512            bit_len: self.bit_len,
513        }
514    }
515}
516
517impl BitAnd<&BooleanBuffer> for &BooleanBuffer {
518    type Output = BooleanBuffer;
519
520    fn bitand(self, rhs: &BooleanBuffer) -> Self::Output {
521        assert_eq!(self.bit_len, rhs.bit_len);
522        BooleanBuffer {
523            buffer: buffer_bin_and(
524                &self.buffer,
525                self.bit_offset,
526                &rhs.buffer,
527                rhs.bit_offset,
528                self.bit_len,
529            ),
530            bit_offset: 0,
531            bit_len: self.bit_len,
532        }
533    }
534}
535
536impl BitOr<&BooleanBuffer> for &BooleanBuffer {
537    type Output = BooleanBuffer;
538
539    fn bitor(self, rhs: &BooleanBuffer) -> Self::Output {
540        assert_eq!(self.bit_len, rhs.bit_len);
541        BooleanBuffer {
542            buffer: buffer_bin_or(
543                &self.buffer,
544                self.bit_offset,
545                &rhs.buffer,
546                rhs.bit_offset,
547                self.bit_len,
548            ),
549            bit_offset: 0,
550            bit_len: self.bit_len,
551        }
552    }
553}
554
555impl BitXor<&BooleanBuffer> for &BooleanBuffer {
556    type Output = BooleanBuffer;
557
558    fn bitxor(self, rhs: &BooleanBuffer) -> Self::Output {
559        assert_eq!(self.bit_len, rhs.bit_len);
560        BooleanBuffer {
561            buffer: buffer_bin_xor(
562                &self.buffer,
563                self.bit_offset,
564                &rhs.buffer,
565                rhs.bit_offset,
566                self.bit_len,
567            ),
568            bit_offset: 0,
569            bit_len: self.bit_len,
570        }
571    }
572}
573
574impl<'a> IntoIterator for &'a BooleanBuffer {
575    type Item = bool;
576    type IntoIter = BitIterator<'a>;
577
578    fn into_iter(self) -> Self::IntoIter {
579        BitIterator::new(self.values(), self.bit_offset, self.bit_len)
580    }
581}
582
583impl From<&[bool]> for BooleanBuffer {
584    fn from(value: &[bool]) -> Self {
585        let mut builder = BooleanBufferBuilder::new(value.len());
586        builder.append_slice(value);
587        builder.finish()
588    }
589}
590
591impl From<Vec<bool>> for BooleanBuffer {
592    fn from(value: Vec<bool>) -> Self {
593        value.as_slice().into()
594    }
595}
596
597impl FromIterator<bool> for BooleanBuffer {
598    fn from_iter<T: IntoIterator<Item = bool>>(iter: T) -> Self {
599        let iter = iter.into_iter();
600        let (hint, _) = iter.size_hint();
601        let mut builder = BooleanBufferBuilder::new(hint);
602        iter.for_each(|b| builder.append(b));
603        builder.finish()
604    }
605}
606
607#[cfg(test)]
608mod tests {
609    use super::*;
610
611    #[test]
612    fn test_boolean_new() {
613        let bytes = &[0, 1, 2, 3, 4];
614        let buf = Buffer::from(bytes);
615        let offset = 0;
616        let len = 24;
617
618        let boolean_buf = BooleanBuffer::new(buf.clone(), offset, len);
619        assert_eq!(bytes, boolean_buf.values());
620        assert_eq!(offset, boolean_buf.offset());
621        assert_eq!(len, boolean_buf.len());
622
623        assert_eq!(2, boolean_buf.count_set_bits());
624        assert_eq!(&buf, boolean_buf.inner());
625        assert_eq!(buf, boolean_buf.clone().into_inner());
626
627        assert!(!boolean_buf.is_empty())
628    }
629
630    #[test]
631    fn test_boolean_data_equality() {
632        let boolean_buf1 = BooleanBuffer::new(Buffer::from(&[0, 1, 4, 3, 5]), 0, 32);
633        let boolean_buf2 = BooleanBuffer::new(Buffer::from(&[0, 1, 4, 3, 5]), 0, 32);
634        assert_eq!(boolean_buf1, boolean_buf2);
635
636        // slice with same offset and same length should still preserve equality
637        let boolean_buf3 = boolean_buf1.slice(8, 16);
638        assert_ne!(boolean_buf1, boolean_buf3);
639        let boolean_buf4 = boolean_buf1.slice(0, 32);
640        assert_eq!(boolean_buf1, boolean_buf4);
641
642        // unequal because of different elements
643        let boolean_buf2 = BooleanBuffer::new(Buffer::from(&[0, 0, 2, 3, 4]), 0, 32);
644        assert_ne!(boolean_buf1, boolean_buf2);
645
646        // unequal because of different length
647        let boolean_buf2 = BooleanBuffer::new(Buffer::from(&[0, 1, 4, 3, 5]), 0, 24);
648        assert_ne!(boolean_buf1, boolean_buf2);
649
650        // ptr_eq
651        assert!(boolean_buf1.ptr_eq(&boolean_buf1));
652        assert!(boolean_buf2.ptr_eq(&boolean_buf2));
653        assert!(!boolean_buf1.ptr_eq(&boolean_buf2));
654    }
655
656    #[test]
657    fn test_boolean_slice() {
658        let bytes = &[0, 3, 2, 6, 2];
659        let boolean_buf1 = BooleanBuffer::new(Buffer::from(bytes), 0, 32);
660        let boolean_buf2 = BooleanBuffer::new(Buffer::from(bytes), 0, 32);
661
662        let boolean_slice1 = boolean_buf1.slice(16, 16);
663        let boolean_slice2 = boolean_buf2.slice(0, 16);
664        assert_eq!(boolean_slice1.values(), boolean_slice2.values());
665
666        assert_eq!(bytes, boolean_slice1.values());
667        assert_eq!(16, boolean_slice1.bit_offset);
668        assert_eq!(16, boolean_slice1.bit_len);
669
670        assert_eq!(bytes, boolean_slice2.values());
671        assert_eq!(0, boolean_slice2.bit_offset);
672        assert_eq!(16, boolean_slice2.bit_len);
673    }
674
675    #[test]
676    fn test_boolean_bitand() {
677        let offset = 0;
678        let len = 40;
679
680        let buf1 = Buffer::from(&[0, 1, 1, 0, 0]);
681        let boolean_buf1 = &BooleanBuffer::new(buf1, offset, len);
682
683        let buf2 = Buffer::from(&[0, 1, 1, 1, 0]);
684        let boolean_buf2 = &BooleanBuffer::new(buf2, offset, len);
685
686        let expected = BooleanBuffer::new(Buffer::from(&[0, 1, 1, 0, 0]), offset, len);
687        assert_eq!(boolean_buf1 & boolean_buf2, expected);
688    }
689
690    #[test]
691    fn test_boolean_bitor() {
692        let offset = 0;
693        let len = 40;
694
695        let buf1 = Buffer::from(&[0, 1, 1, 0, 0]);
696        let boolean_buf1 = &BooleanBuffer::new(buf1, offset, len);
697
698        let buf2 = Buffer::from(&[0, 1, 1, 1, 0]);
699        let boolean_buf2 = &BooleanBuffer::new(buf2, offset, len);
700
701        let expected = BooleanBuffer::new(Buffer::from(&[0, 1, 1, 1, 0]), offset, len);
702        assert_eq!(boolean_buf1 | boolean_buf2, expected);
703    }
704
705    #[test]
706    fn test_boolean_bitxor() {
707        let offset = 0;
708        let len = 40;
709
710        let buf1 = Buffer::from(&[0, 1, 1, 0, 0]);
711        let boolean_buf1 = &BooleanBuffer::new(buf1, offset, len);
712
713        let buf2 = Buffer::from(&[0, 1, 1, 1, 0]);
714        let boolean_buf2 = &BooleanBuffer::new(buf2, offset, len);
715
716        let expected = BooleanBuffer::new(Buffer::from(&[0, 0, 0, 1, 0]), offset, len);
717        assert_eq!(boolean_buf1 ^ boolean_buf2, expected);
718    }
719
720    #[test]
721    fn test_boolean_not() {
722        let offset = 0;
723        let len = 40;
724
725        let buf = Buffer::from(&[0, 1, 1, 0, 0]);
726        let boolean_buf = &BooleanBuffer::new(buf, offset, len);
727
728        let expected = BooleanBuffer::new(Buffer::from(&[255, 254, 254, 255, 255]), offset, len);
729        assert_eq!(!boolean_buf, expected);
730    }
731
732    #[test]
733    fn test_boolean_from_slice_bool() {
734        let v = [true, false, false];
735        let buf = BooleanBuffer::from(&v[..]);
736        assert_eq!(buf.offset(), 0);
737        assert_eq!(buf.len(), 3);
738        assert_eq!(buf.values().len(), 1);
739        assert!(buf.value(0));
740    }
741
742    #[test]
743    fn test_from_bitwise_unary_op() {
744        // Use 1024 boolean values so that at least some of the tests cover multiple u64 chunks and
745        // perfect alignment
746        let input_bools = (0..1024)
747            .map(|_| rand::random::<bool>())
748            .collect::<Vec<bool>>();
749        let input_buffer = BooleanBuffer::from(&input_bools[..]);
750
751        // Note ensure we test offsets over 100 to cover multiple u64 chunks
752        for offset in 0..1024 {
753            let result = BooleanBuffer::from_bitwise_unary_op(
754                input_buffer.values(),
755                offset,
756                input_buffer.len() - offset,
757                |a| !a,
758            );
759            let expected = input_bools[offset..]
760                .iter()
761                .map(|b| !*b)
762                .collect::<BooleanBuffer>();
763            assert_eq!(result, expected);
764        }
765
766        // Also test when the input doesn't cover the entire buffer
767        for offset in 0..512 {
768            let len = 512 - offset; // fixed length less than total
769            let result =
770                BooleanBuffer::from_bitwise_unary_op(input_buffer.values(), offset, len, |a| !a);
771            let expected = input_bools[offset..]
772                .iter()
773                .take(len)
774                .map(|b| !*b)
775                .collect::<BooleanBuffer>();
776            assert_eq!(result, expected);
777        }
778    }
779
780    #[test]
781    fn test_from_bitwise_binary_op() {
782        // pick random boolean inputs
783        let input_bools_left = (0..1024)
784            .map(|_| rand::random::<bool>())
785            .collect::<Vec<bool>>();
786        let input_bools_right = (0..1024)
787            .map(|_| rand::random::<bool>())
788            .collect::<Vec<bool>>();
789        let input_buffer_left = BooleanBuffer::from(&input_bools_left[..]);
790        let input_buffer_right = BooleanBuffer::from(&input_bools_right[..]);
791
792        for left_offset in 0..200 {
793            for right_offset in [0, 4, 5, 17, 33, 24, 45, 64, 65, 100, 200] {
794                for len_offset in [0, 1, 44, 100, 256, 300, 512] {
795                    let len = 1024 - len_offset - left_offset.max(right_offset); // ensure we don't go out of bounds
796                    // compute with AND
797                    let result = BooleanBuffer::from_bitwise_binary_op(
798                        input_buffer_left.values(),
799                        left_offset,
800                        input_buffer_right.values(),
801                        right_offset,
802                        len,
803                        |a, b| a & b,
804                    );
805                    // compute directly from bools
806                    let expected = input_bools_left[left_offset..]
807                        .iter()
808                        .zip(&input_bools_right[right_offset..])
809                        .take(len)
810                        .map(|(a, b)| *a & *b)
811                        .collect::<BooleanBuffer>();
812                    assert_eq!(result, expected);
813                }
814            }
815        }
816    }
817
818    #[test]
819    fn test_extend_trusted_len_sets_byte_len() {
820        // Ensures extend_trusted_len keeps the underlying byte length in sync with bit length.
821        let mut builder = BooleanBufferBuilder::new(0);
822        let bools: Vec<_> = (0..10).map(|i| i % 2 == 0).collect();
823        unsafe { builder.extend_trusted_len(bools.into_iter()) };
824        assert_eq!(builder.as_slice().len(), bit_util::ceil(builder.len(), 8));
825    }
826
827    #[test]
828    fn test_extend_trusted_len_then_append() {
829        // Exercises append after extend_trusted_len to validate byte length and values.
830        let mut builder = BooleanBufferBuilder::new(0);
831        let bools: Vec<_> = (0..9).map(|i| i % 3 == 0).collect();
832        unsafe { builder.extend_trusted_len(bools.clone().into_iter()) };
833        builder.append(true);
834        assert_eq!(builder.as_slice().len(), bit_util::ceil(builder.len(), 8));
835        let finished = builder.finish();
836        for (i, v) in bools.into_iter().chain(std::iter::once(true)).enumerate() {
837            assert_eq!(finished.value(i), v, "at index {}", i);
838        }
839    }
840
841    #[test]
842    fn test_find_nth_set_bit_position() {
843        let bools = vec![true, false, true, true, false, true];
844        let buffer = BooleanBuffer::from(bools);
845
846        assert_eq!(buffer.clone().find_nth_set_bit_position(0, 1), 1);
847        assert_eq!(buffer.clone().find_nth_set_bit_position(0, 2), 3);
848        assert_eq!(buffer.clone().find_nth_set_bit_position(0, 3), 4);
849        assert_eq!(buffer.clone().find_nth_set_bit_position(0, 4), 6);
850        assert_eq!(buffer.clone().find_nth_set_bit_position(0, 5), 6);
851
852        assert_eq!(buffer.clone().find_nth_set_bit_position(1, 1), 3);
853        assert_eq!(buffer.clone().find_nth_set_bit_position(3, 1), 4);
854        assert_eq!(buffer.clone().find_nth_set_bit_position(3, 2), 6);
855    }
856
857    #[test]
858    fn test_find_nth_set_bit_position_large() {
859        let mut bools = vec![false; 1000];
860        bools[100] = true;
861        bools[500] = true;
862        bools[999] = true;
863        let buffer = BooleanBuffer::from(bools);
864
865        assert_eq!(buffer.clone().find_nth_set_bit_position(0, 1), 101);
866        assert_eq!(buffer.clone().find_nth_set_bit_position(0, 2), 501);
867        assert_eq!(buffer.clone().find_nth_set_bit_position(0, 3), 1000);
868        assert_eq!(buffer.clone().find_nth_set_bit_position(0, 4), 1000);
869
870        assert_eq!(buffer.clone().find_nth_set_bit_position(101, 1), 501);
871    }
872
873    #[test]
874    fn test_find_nth_set_bit_position_sliced() {
875        let bools = vec![false, true, false, true, true, false, true]; // [F, T, F, T, T, F, T]
876        let buffer = BooleanBuffer::from(bools);
877        let slice = buffer.slice(1, 6); // [T, F, T, T, F, T]
878
879        assert_eq!(slice.len(), 6);
880        // Logical indices: 0, 1, 2, 3, 4, 5
881        // Logical values: T, F, T, T, F, T
882
883        assert_eq!(slice.clone().find_nth_set_bit_position(0, 1), 1);
884        assert_eq!(slice.clone().find_nth_set_bit_position(0, 2), 3);
885        assert_eq!(slice.clone().find_nth_set_bit_position(0, 3), 4);
886        assert_eq!(slice.clone().find_nth_set_bit_position(0, 4), 6);
887    }
888
889    #[test]
890    fn test_find_nth_set_bit_position_all_set() {
891        let buffer = BooleanBuffer::new_set(100);
892        for i in 1..=100 {
893            assert_eq!(buffer.clone().find_nth_set_bit_position(0, i), i);
894        }
895        assert_eq!(buffer.clone().find_nth_set_bit_position(0, 101), 100);
896    }
897
898    #[test]
899    fn test_find_nth_set_bit_position_none_set() {
900        let buffer = BooleanBuffer::new_unset(100);
901        assert_eq!(buffer.clone().find_nth_set_bit_position(0, 1), 100);
902    }
903}