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    /// Returns a [`BitChunks`] instance which can be used to iterate over
370    /// this buffer's bits in `u64` chunks
371    #[inline]
372    pub fn bit_chunks(&self) -> BitChunks<'_> {
373        BitChunks::new(self.values(), self.bit_offset, self.bit_len)
374    }
375
376    /// Returns the offset of this [`BooleanBuffer`] in bits (not bytes)
377    #[inline]
378    pub fn offset(&self) -> usize {
379        self.bit_offset
380    }
381
382    /// Returns the length of this [`BooleanBuffer`] in bits (not bytes)
383    #[inline]
384    pub fn len(&self) -> usize {
385        self.bit_len
386    }
387
388    /// Returns true if this [`BooleanBuffer`] is empty
389    #[inline]
390    pub fn is_empty(&self) -> bool {
391        self.bit_len == 0
392    }
393
394    /// Free up unused memory.
395    pub fn shrink_to_fit(&mut self) {
396        // TODO(emilk): we could shrink even more in the case where we are a small sub-slice of the full buffer
397        self.buffer.shrink_to_fit();
398    }
399
400    /// Returns the boolean value at index `i`.
401    ///
402    /// # Panics
403    ///
404    /// Panics if `i >= self.len()`
405    #[inline]
406    pub fn value(&self, idx: usize) -> bool {
407        assert!(idx < self.bit_len);
408        unsafe { self.value_unchecked(idx) }
409    }
410
411    /// Returns the boolean value at index `i`.
412    ///
413    /// # Safety
414    /// This doesn't check bounds, the caller must ensure that index < self.len()
415    #[inline]
416    pub unsafe fn value_unchecked(&self, i: usize) -> bool {
417        unsafe { bit_util::get_bit_raw(self.buffer.as_ptr(), i + self.bit_offset) }
418    }
419
420    /// Returns the packed values of this [`BooleanBuffer`] not including any offset
421    #[inline]
422    pub fn values(&self) -> &[u8] {
423        &self.buffer
424    }
425
426    /// Slices this [`BooleanBuffer`] by the provided `offset` and `length`
427    pub fn slice(&self, offset: usize, len: usize) -> Self {
428        assert!(
429            offset.saturating_add(len) <= self.bit_len,
430            "the length + offset of the sliced BooleanBuffer cannot exceed the existing length"
431        );
432        Self {
433            buffer: self.buffer.clone(),
434            bit_offset: self.bit_offset + offset,
435            bit_len: len,
436        }
437    }
438
439    /// Returns a [`Buffer`] containing the sliced contents of this [`BooleanBuffer`]
440    ///
441    /// Equivalent to `self.buffer.bit_slice(self.offset, self.len)`
442    pub fn sliced(&self) -> Buffer {
443        self.buffer.bit_slice(self.bit_offset, self.bit_len)
444    }
445
446    /// Returns true if this [`BooleanBuffer`] is equal to `other`, using pointer comparisons
447    /// to determine buffer equality. This is cheaper than `PartialEq::eq` but may
448    /// return false when the arrays are logically equal
449    pub fn ptr_eq(&self, other: &Self) -> bool {
450        self.buffer.as_ptr() == other.buffer.as_ptr()
451            && self.bit_offset == other.bit_offset
452            && self.bit_len == other.bit_len
453    }
454
455    /// Returns the inner [`Buffer`]
456    ///
457    /// Note: this does not account for offset and length of this [`BooleanBuffer`]
458    #[inline]
459    pub fn inner(&self) -> &Buffer {
460        &self.buffer
461    }
462
463    /// Returns the inner [`Buffer`], consuming self
464    ///
465    /// Note: this does not account for offset and length of this [`BooleanBuffer`]
466    pub fn into_inner(self) -> Buffer {
467        self.buffer
468    }
469
470    /// Returns an iterator over the bits in this [`BooleanBuffer`]
471    pub fn iter(&self) -> BitIterator<'_> {
472        self.into_iter()
473    }
474
475    /// Returns an iterator over the set bit positions in this [`BooleanBuffer`]
476    pub fn set_indices(&self) -> BitIndexIterator<'_> {
477        BitIndexIterator::new(self.values(), self.bit_offset, self.bit_len)
478    }
479
480    /// Returns a `u32` iterator over set bit positions without any usize->u32 conversion
481    pub fn set_indices_u32(&self) -> BitIndexU32Iterator<'_> {
482        BitIndexU32Iterator::new(self.values(), self.bit_offset, self.bit_len)
483    }
484
485    /// Returns a [`BitSliceIterator`] yielding contiguous ranges of set bits
486    pub fn set_slices(&self) -> BitSliceIterator<'_> {
487        BitSliceIterator::new(self.values(), self.bit_offset, self.bit_len)
488    }
489}
490
491impl Not for &BooleanBuffer {
492    type Output = BooleanBuffer;
493
494    fn not(self) -> Self::Output {
495        BooleanBuffer {
496            buffer: buffer_unary_not(&self.buffer, self.bit_offset, self.bit_len),
497            bit_offset: 0,
498            bit_len: self.bit_len,
499        }
500    }
501}
502
503impl BitAnd<&BooleanBuffer> for &BooleanBuffer {
504    type Output = BooleanBuffer;
505
506    fn bitand(self, rhs: &BooleanBuffer) -> Self::Output {
507        assert_eq!(self.bit_len, rhs.bit_len);
508        BooleanBuffer {
509            buffer: buffer_bin_and(
510                &self.buffer,
511                self.bit_offset,
512                &rhs.buffer,
513                rhs.bit_offset,
514                self.bit_len,
515            ),
516            bit_offset: 0,
517            bit_len: self.bit_len,
518        }
519    }
520}
521
522impl BitOr<&BooleanBuffer> for &BooleanBuffer {
523    type Output = BooleanBuffer;
524
525    fn bitor(self, rhs: &BooleanBuffer) -> Self::Output {
526        assert_eq!(self.bit_len, rhs.bit_len);
527        BooleanBuffer {
528            buffer: buffer_bin_or(
529                &self.buffer,
530                self.bit_offset,
531                &rhs.buffer,
532                rhs.bit_offset,
533                self.bit_len,
534            ),
535            bit_offset: 0,
536            bit_len: self.bit_len,
537        }
538    }
539}
540
541impl BitXor<&BooleanBuffer> for &BooleanBuffer {
542    type Output = BooleanBuffer;
543
544    fn bitxor(self, rhs: &BooleanBuffer) -> Self::Output {
545        assert_eq!(self.bit_len, rhs.bit_len);
546        BooleanBuffer {
547            buffer: buffer_bin_xor(
548                &self.buffer,
549                self.bit_offset,
550                &rhs.buffer,
551                rhs.bit_offset,
552                self.bit_len,
553            ),
554            bit_offset: 0,
555            bit_len: self.bit_len,
556        }
557    }
558}
559
560impl<'a> IntoIterator for &'a BooleanBuffer {
561    type Item = bool;
562    type IntoIter = BitIterator<'a>;
563
564    fn into_iter(self) -> Self::IntoIter {
565        BitIterator::new(self.values(), self.bit_offset, self.bit_len)
566    }
567}
568
569impl From<&[bool]> for BooleanBuffer {
570    fn from(value: &[bool]) -> Self {
571        let mut builder = BooleanBufferBuilder::new(value.len());
572        builder.append_slice(value);
573        builder.finish()
574    }
575}
576
577impl From<Vec<bool>> for BooleanBuffer {
578    fn from(value: Vec<bool>) -> Self {
579        value.as_slice().into()
580    }
581}
582
583impl FromIterator<bool> for BooleanBuffer {
584    fn from_iter<T: IntoIterator<Item = bool>>(iter: T) -> Self {
585        let iter = iter.into_iter();
586        let (hint, _) = iter.size_hint();
587        let mut builder = BooleanBufferBuilder::new(hint);
588        iter.for_each(|b| builder.append(b));
589        builder.finish()
590    }
591}
592
593#[cfg(test)]
594mod tests {
595    use super::*;
596
597    #[test]
598    fn test_boolean_new() {
599        let bytes = &[0, 1, 2, 3, 4];
600        let buf = Buffer::from(bytes);
601        let offset = 0;
602        let len = 24;
603
604        let boolean_buf = BooleanBuffer::new(buf.clone(), offset, len);
605        assert_eq!(bytes, boolean_buf.values());
606        assert_eq!(offset, boolean_buf.offset());
607        assert_eq!(len, boolean_buf.len());
608
609        assert_eq!(2, boolean_buf.count_set_bits());
610        assert_eq!(&buf, boolean_buf.inner());
611        assert_eq!(buf, boolean_buf.clone().into_inner());
612
613        assert!(!boolean_buf.is_empty())
614    }
615
616    #[test]
617    fn test_boolean_data_equality() {
618        let boolean_buf1 = BooleanBuffer::new(Buffer::from(&[0, 1, 4, 3, 5]), 0, 32);
619        let boolean_buf2 = BooleanBuffer::new(Buffer::from(&[0, 1, 4, 3, 5]), 0, 32);
620        assert_eq!(boolean_buf1, boolean_buf2);
621
622        // slice with same offset and same length should still preserve equality
623        let boolean_buf3 = boolean_buf1.slice(8, 16);
624        assert_ne!(boolean_buf1, boolean_buf3);
625        let boolean_buf4 = boolean_buf1.slice(0, 32);
626        assert_eq!(boolean_buf1, boolean_buf4);
627
628        // unequal because of different elements
629        let boolean_buf2 = BooleanBuffer::new(Buffer::from(&[0, 0, 2, 3, 4]), 0, 32);
630        assert_ne!(boolean_buf1, boolean_buf2);
631
632        // unequal because of different length
633        let boolean_buf2 = BooleanBuffer::new(Buffer::from(&[0, 1, 4, 3, 5]), 0, 24);
634        assert_ne!(boolean_buf1, boolean_buf2);
635
636        // ptr_eq
637        assert!(boolean_buf1.ptr_eq(&boolean_buf1));
638        assert!(boolean_buf2.ptr_eq(&boolean_buf2));
639        assert!(!boolean_buf1.ptr_eq(&boolean_buf2));
640    }
641
642    #[test]
643    fn test_boolean_slice() {
644        let bytes = &[0, 3, 2, 6, 2];
645        let boolean_buf1 = BooleanBuffer::new(Buffer::from(bytes), 0, 32);
646        let boolean_buf2 = BooleanBuffer::new(Buffer::from(bytes), 0, 32);
647
648        let boolean_slice1 = boolean_buf1.slice(16, 16);
649        let boolean_slice2 = boolean_buf2.slice(0, 16);
650        assert_eq!(boolean_slice1.values(), boolean_slice2.values());
651
652        assert_eq!(bytes, boolean_slice1.values());
653        assert_eq!(16, boolean_slice1.bit_offset);
654        assert_eq!(16, boolean_slice1.bit_len);
655
656        assert_eq!(bytes, boolean_slice2.values());
657        assert_eq!(0, boolean_slice2.bit_offset);
658        assert_eq!(16, boolean_slice2.bit_len);
659    }
660
661    #[test]
662    fn test_boolean_bitand() {
663        let offset = 0;
664        let len = 40;
665
666        let buf1 = Buffer::from(&[0, 1, 1, 0, 0]);
667        let boolean_buf1 = &BooleanBuffer::new(buf1, offset, len);
668
669        let buf2 = Buffer::from(&[0, 1, 1, 1, 0]);
670        let boolean_buf2 = &BooleanBuffer::new(buf2, offset, len);
671
672        let expected = BooleanBuffer::new(Buffer::from(&[0, 1, 1, 0, 0]), offset, len);
673        assert_eq!(boolean_buf1 & boolean_buf2, expected);
674    }
675
676    #[test]
677    fn test_boolean_bitor() {
678        let offset = 0;
679        let len = 40;
680
681        let buf1 = Buffer::from(&[0, 1, 1, 0, 0]);
682        let boolean_buf1 = &BooleanBuffer::new(buf1, offset, len);
683
684        let buf2 = Buffer::from(&[0, 1, 1, 1, 0]);
685        let boolean_buf2 = &BooleanBuffer::new(buf2, offset, len);
686
687        let expected = BooleanBuffer::new(Buffer::from(&[0, 1, 1, 1, 0]), offset, len);
688        assert_eq!(boolean_buf1 | boolean_buf2, expected);
689    }
690
691    #[test]
692    fn test_boolean_bitxor() {
693        let offset = 0;
694        let len = 40;
695
696        let buf1 = Buffer::from(&[0, 1, 1, 0, 0]);
697        let boolean_buf1 = &BooleanBuffer::new(buf1, offset, len);
698
699        let buf2 = Buffer::from(&[0, 1, 1, 1, 0]);
700        let boolean_buf2 = &BooleanBuffer::new(buf2, offset, len);
701
702        let expected = BooleanBuffer::new(Buffer::from(&[0, 0, 0, 1, 0]), offset, len);
703        assert_eq!(boolean_buf1 ^ boolean_buf2, expected);
704    }
705
706    #[test]
707    fn test_boolean_not() {
708        let offset = 0;
709        let len = 40;
710
711        let buf = Buffer::from(&[0, 1, 1, 0, 0]);
712        let boolean_buf = &BooleanBuffer::new(buf, offset, len);
713
714        let expected = BooleanBuffer::new(Buffer::from(&[255, 254, 254, 255, 255]), offset, len);
715        assert_eq!(!boolean_buf, expected);
716    }
717
718    #[test]
719    fn test_boolean_from_slice_bool() {
720        let v = [true, false, false];
721        let buf = BooleanBuffer::from(&v[..]);
722        assert_eq!(buf.offset(), 0);
723        assert_eq!(buf.len(), 3);
724        assert_eq!(buf.values().len(), 1);
725        assert!(buf.value(0));
726    }
727
728    #[test]
729    fn test_from_bitwise_unary_op() {
730        // Use 1024 boolean values so that at least some of the tests cover multiple u64 chunks and
731        // perfect alignment
732        let input_bools = (0..1024)
733            .map(|_| rand::random::<bool>())
734            .collect::<Vec<bool>>();
735        let input_buffer = BooleanBuffer::from(&input_bools[..]);
736
737        // Note ensure we test offsets over 100 to cover multiple u64 chunks
738        for offset in 0..1024 {
739            let result = BooleanBuffer::from_bitwise_unary_op(
740                input_buffer.values(),
741                offset,
742                input_buffer.len() - offset,
743                |a| !a,
744            );
745            let expected = input_bools[offset..]
746                .iter()
747                .map(|b| !*b)
748                .collect::<BooleanBuffer>();
749            assert_eq!(result, expected);
750        }
751
752        // Also test when the input doesn't cover the entire buffer
753        for offset in 0..512 {
754            let len = 512 - offset; // fixed length less than total
755            let result =
756                BooleanBuffer::from_bitwise_unary_op(input_buffer.values(), offset, len, |a| !a);
757            let expected = input_bools[offset..]
758                .iter()
759                .take(len)
760                .map(|b| !*b)
761                .collect::<BooleanBuffer>();
762            assert_eq!(result, expected);
763        }
764    }
765
766    #[test]
767    fn test_from_bitwise_binary_op() {
768        // pick random boolean inputs
769        let input_bools_left = (0..1024)
770            .map(|_| rand::random::<bool>())
771            .collect::<Vec<bool>>();
772        let input_bools_right = (0..1024)
773            .map(|_| rand::random::<bool>())
774            .collect::<Vec<bool>>();
775        let input_buffer_left = BooleanBuffer::from(&input_bools_left[..]);
776        let input_buffer_right = BooleanBuffer::from(&input_bools_right[..]);
777
778        for left_offset in 0..200 {
779            for right_offset in [0, 4, 5, 17, 33, 24, 45, 64, 65, 100, 200] {
780                for len_offset in [0, 1, 44, 100, 256, 300, 512] {
781                    let len = 1024 - len_offset - left_offset.max(right_offset); // ensure we don't go out of bounds
782                    // compute with AND
783                    let result = BooleanBuffer::from_bitwise_binary_op(
784                        input_buffer_left.values(),
785                        left_offset,
786                        input_buffer_right.values(),
787                        right_offset,
788                        len,
789                        |a, b| a & b,
790                    );
791                    // compute directly from bools
792                    let expected = input_bools_left[left_offset..]
793                        .iter()
794                        .zip(&input_bools_right[right_offset..])
795                        .take(len)
796                        .map(|(a, b)| *a & *b)
797                        .collect::<BooleanBuffer>();
798                    assert_eq!(result, expected);
799                }
800            }
801        }
802    }
803}