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    /// - [`apply_bitwise_unary_op`](bit_util::apply_bitwise_unary_op) for in-place unary bitwise operations
173    ///
174    /// # Example: Create new [`BooleanBuffer`] from bitwise `NOT` of an input [`Buffer`]
175    /// ```
176    /// # use arrow_buffer::BooleanBuffer;
177    /// let input = [0b11001100u8, 0b10111010u8]; // 2 bytes = 16 bits
178    /// // NOT of the first 12 bits
179    /// let result = BooleanBuffer::from_bitwise_unary_op(
180    ///  &input, 0, 12, |a| !a
181    /// );
182    /// assert_eq!(result.values(), &[0b00110011u8, 0b11110101u8]);
183    /// ```
184    pub fn from_bitwise_unary_op<F>(
185        src: impl AsRef<[u8]>,
186        offset_in_bits: usize,
187        len_in_bits: usize,
188        mut op: F,
189    ) -> Self
190    where
191        F: FnMut(u64) -> u64,
192    {
193        // try fast path for aligned input
194        if offset_in_bits & 0x7 == 0 {
195            // align to byte boundary
196            let aligned = &src.as_ref()[offset_in_bits / 8..];
197            if let Some(result) =
198                Self::try_from_aligned_bitwise_unary_op(aligned, len_in_bits, &mut op)
199            {
200                return result;
201            }
202        }
203
204        let chunks = BitChunks::new(src.as_ref(), offset_in_bits, len_in_bits);
205        let mut result = MutableBuffer::with_capacity(chunks.num_u64s() * 8);
206        for chunk in chunks.iter() {
207            // SAFETY: reserved enough capacity above, (exactly num_u64s()
208            // items) and we assume `BitChunks` correctly reports upper bound
209            unsafe {
210                result.push_unchecked(op(chunk));
211            }
212        }
213        if chunks.remainder_len() > 0 {
214            debug_assert!(result.capacity() >= result.len() + 8); // should not reallocate
215            // SAFETY: reserved enough capacity above, (exactly num_u64s()
216            // items) and we assume `BitChunks` correctly reports upper bound
217            unsafe {
218                result.push_unchecked(op(chunks.remainder_bits()));
219            }
220            // Just pushed one u64, which may have trailing zeros
221            result.truncate(chunks.num_bytes());
222        }
223
224        let buffer = Buffer::from(result);
225        BooleanBuffer {
226            buffer,
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    /// Returns the number of set bits in this buffer
258    pub fn count_set_bits(&self) -> usize {
259        self.buffer
260            .count_set_bits_offset(self.bit_offset, self.bit_len)
261    }
262
263    /// Returns a [`BitChunks`] instance which can be used to iterate over
264    /// this buffer's bits in `u64` chunks
265    #[inline]
266    pub fn bit_chunks(&self) -> BitChunks<'_> {
267        BitChunks::new(self.values(), self.bit_offset, self.bit_len)
268    }
269
270    /// Returns the offset of this [`BooleanBuffer`] in bits (not bytes)
271    #[inline]
272    pub fn offset(&self) -> usize {
273        self.bit_offset
274    }
275
276    /// Returns the length of this [`BooleanBuffer`] in bits (not bytes)
277    #[inline]
278    pub fn len(&self) -> usize {
279        self.bit_len
280    }
281
282    /// Returns true if this [`BooleanBuffer`] is empty
283    #[inline]
284    pub fn is_empty(&self) -> bool {
285        self.bit_len == 0
286    }
287
288    /// Free up unused memory.
289    pub fn shrink_to_fit(&mut self) {
290        // TODO(emilk): we could shrink even more in the case where we are a small sub-slice of the full buffer
291        self.buffer.shrink_to_fit();
292    }
293
294    /// Returns the boolean value at index `i`.
295    ///
296    /// # Panics
297    ///
298    /// Panics if `i >= self.len()`
299    #[inline]
300    pub fn value(&self, idx: usize) -> bool {
301        assert!(idx < self.bit_len);
302        unsafe { self.value_unchecked(idx) }
303    }
304
305    /// Returns the boolean value at index `i`.
306    ///
307    /// # Safety
308    /// This doesn't check bounds, the caller must ensure that index < self.len()
309    #[inline]
310    pub unsafe fn value_unchecked(&self, i: usize) -> bool {
311        unsafe { bit_util::get_bit_raw(self.buffer.as_ptr(), i + self.bit_offset) }
312    }
313
314    /// Returns the packed values of this [`BooleanBuffer`] not including any offset
315    #[inline]
316    pub fn values(&self) -> &[u8] {
317        &self.buffer
318    }
319
320    /// Slices this [`BooleanBuffer`] by the provided `offset` and `length`
321    pub fn slice(&self, offset: usize, len: usize) -> Self {
322        assert!(
323            offset.saturating_add(len) <= self.bit_len,
324            "the length + offset of the sliced BooleanBuffer cannot exceed the existing length"
325        );
326        Self {
327            buffer: self.buffer.clone(),
328            bit_offset: self.bit_offset + offset,
329            bit_len: len,
330        }
331    }
332
333    /// Returns a [`Buffer`] containing the sliced contents of this [`BooleanBuffer`]
334    ///
335    /// Equivalent to `self.buffer.bit_slice(self.offset, self.len)`
336    pub fn sliced(&self) -> Buffer {
337        self.buffer.bit_slice(self.bit_offset, self.bit_len)
338    }
339
340    /// Returns true if this [`BooleanBuffer`] is equal to `other`, using pointer comparisons
341    /// to determine buffer equality. This is cheaper than `PartialEq::eq` but may
342    /// return false when the arrays are logically equal
343    pub fn ptr_eq(&self, other: &Self) -> bool {
344        self.buffer.as_ptr() == other.buffer.as_ptr()
345            && self.bit_offset == other.bit_offset
346            && self.bit_len == other.bit_len
347    }
348
349    /// Returns the inner [`Buffer`]
350    ///
351    /// Note: this does not account for offset and length of this [`BooleanBuffer`]
352    #[inline]
353    pub fn inner(&self) -> &Buffer {
354        &self.buffer
355    }
356
357    /// Returns the inner [`Buffer`], consuming self
358    ///
359    /// Note: this does not account for offset and length of this [`BooleanBuffer`]
360    pub fn into_inner(self) -> Buffer {
361        self.buffer
362    }
363
364    /// Returns an iterator over the bits in this [`BooleanBuffer`]
365    pub fn iter(&self) -> BitIterator<'_> {
366        self.into_iter()
367    }
368
369    /// Returns an iterator over the set bit positions in this [`BooleanBuffer`]
370    pub fn set_indices(&self) -> BitIndexIterator<'_> {
371        BitIndexIterator::new(self.values(), self.bit_offset, self.bit_len)
372    }
373
374    /// Returns a `u32` iterator over set bit positions without any usize->u32 conversion
375    pub fn set_indices_u32(&self) -> BitIndexU32Iterator<'_> {
376        BitIndexU32Iterator::new(self.values(), self.bit_offset, self.bit_len)
377    }
378
379    /// Returns a [`BitSliceIterator`] yielding contiguous ranges of set bits
380    pub fn set_slices(&self) -> BitSliceIterator<'_> {
381        BitSliceIterator::new(self.values(), self.bit_offset, self.bit_len)
382    }
383}
384
385impl Not for &BooleanBuffer {
386    type Output = BooleanBuffer;
387
388    fn not(self) -> Self::Output {
389        BooleanBuffer {
390            buffer: buffer_unary_not(&self.buffer, self.bit_offset, self.bit_len),
391            bit_offset: 0,
392            bit_len: self.bit_len,
393        }
394    }
395}
396
397impl BitAnd<&BooleanBuffer> for &BooleanBuffer {
398    type Output = BooleanBuffer;
399
400    fn bitand(self, rhs: &BooleanBuffer) -> Self::Output {
401        assert_eq!(self.bit_len, rhs.bit_len);
402        BooleanBuffer {
403            buffer: buffer_bin_and(
404                &self.buffer,
405                self.bit_offset,
406                &rhs.buffer,
407                rhs.bit_offset,
408                self.bit_len,
409            ),
410            bit_offset: 0,
411            bit_len: self.bit_len,
412        }
413    }
414}
415
416impl BitOr<&BooleanBuffer> for &BooleanBuffer {
417    type Output = BooleanBuffer;
418
419    fn bitor(self, rhs: &BooleanBuffer) -> Self::Output {
420        assert_eq!(self.bit_len, rhs.bit_len);
421        BooleanBuffer {
422            buffer: buffer_bin_or(
423                &self.buffer,
424                self.bit_offset,
425                &rhs.buffer,
426                rhs.bit_offset,
427                self.bit_len,
428            ),
429            bit_offset: 0,
430            bit_len: self.bit_len,
431        }
432    }
433}
434
435impl BitXor<&BooleanBuffer> for &BooleanBuffer {
436    type Output = BooleanBuffer;
437
438    fn bitxor(self, rhs: &BooleanBuffer) -> Self::Output {
439        assert_eq!(self.bit_len, rhs.bit_len);
440        BooleanBuffer {
441            buffer: buffer_bin_xor(
442                &self.buffer,
443                self.bit_offset,
444                &rhs.buffer,
445                rhs.bit_offset,
446                self.bit_len,
447            ),
448            bit_offset: 0,
449            bit_len: self.bit_len,
450        }
451    }
452}
453
454impl<'a> IntoIterator for &'a BooleanBuffer {
455    type Item = bool;
456    type IntoIter = BitIterator<'a>;
457
458    fn into_iter(self) -> Self::IntoIter {
459        BitIterator::new(self.values(), self.bit_offset, self.bit_len)
460    }
461}
462
463impl From<&[bool]> for BooleanBuffer {
464    fn from(value: &[bool]) -> Self {
465        let mut builder = BooleanBufferBuilder::new(value.len());
466        builder.append_slice(value);
467        builder.finish()
468    }
469}
470
471impl From<Vec<bool>> for BooleanBuffer {
472    fn from(value: Vec<bool>) -> Self {
473        value.as_slice().into()
474    }
475}
476
477impl FromIterator<bool> for BooleanBuffer {
478    fn from_iter<T: IntoIterator<Item = bool>>(iter: T) -> Self {
479        let iter = iter.into_iter();
480        let (hint, _) = iter.size_hint();
481        let mut builder = BooleanBufferBuilder::new(hint);
482        iter.for_each(|b| builder.append(b));
483        builder.finish()
484    }
485}
486
487#[cfg(test)]
488mod tests {
489    use super::*;
490
491    #[test]
492    fn test_boolean_new() {
493        let bytes = &[0, 1, 2, 3, 4];
494        let buf = Buffer::from(bytes);
495        let offset = 0;
496        let len = 24;
497
498        let boolean_buf = BooleanBuffer::new(buf.clone(), offset, len);
499        assert_eq!(bytes, boolean_buf.values());
500        assert_eq!(offset, boolean_buf.offset());
501        assert_eq!(len, boolean_buf.len());
502
503        assert_eq!(2, boolean_buf.count_set_bits());
504        assert_eq!(&buf, boolean_buf.inner());
505        assert_eq!(buf, boolean_buf.clone().into_inner());
506
507        assert!(!boolean_buf.is_empty())
508    }
509
510    #[test]
511    fn test_boolean_data_equality() {
512        let boolean_buf1 = BooleanBuffer::new(Buffer::from(&[0, 1, 4, 3, 5]), 0, 32);
513        let boolean_buf2 = BooleanBuffer::new(Buffer::from(&[0, 1, 4, 3, 5]), 0, 32);
514        assert_eq!(boolean_buf1, boolean_buf2);
515
516        // slice with same offset and same length should still preserve equality
517        let boolean_buf3 = boolean_buf1.slice(8, 16);
518        assert_ne!(boolean_buf1, boolean_buf3);
519        let boolean_buf4 = boolean_buf1.slice(0, 32);
520        assert_eq!(boolean_buf1, boolean_buf4);
521
522        // unequal because of different elements
523        let boolean_buf2 = BooleanBuffer::new(Buffer::from(&[0, 0, 2, 3, 4]), 0, 32);
524        assert_ne!(boolean_buf1, boolean_buf2);
525
526        // unequal because of different length
527        let boolean_buf2 = BooleanBuffer::new(Buffer::from(&[0, 1, 4, 3, 5]), 0, 24);
528        assert_ne!(boolean_buf1, boolean_buf2);
529
530        // ptr_eq
531        assert!(boolean_buf1.ptr_eq(&boolean_buf1));
532        assert!(boolean_buf2.ptr_eq(&boolean_buf2));
533        assert!(!boolean_buf1.ptr_eq(&boolean_buf2));
534    }
535
536    #[test]
537    fn test_boolean_slice() {
538        let bytes = &[0, 3, 2, 6, 2];
539        let boolean_buf1 = BooleanBuffer::new(Buffer::from(bytes), 0, 32);
540        let boolean_buf2 = BooleanBuffer::new(Buffer::from(bytes), 0, 32);
541
542        let boolean_slice1 = boolean_buf1.slice(16, 16);
543        let boolean_slice2 = boolean_buf2.slice(0, 16);
544        assert_eq!(boolean_slice1.values(), boolean_slice2.values());
545
546        assert_eq!(bytes, boolean_slice1.values());
547        assert_eq!(16, boolean_slice1.bit_offset);
548        assert_eq!(16, boolean_slice1.bit_len);
549
550        assert_eq!(bytes, boolean_slice2.values());
551        assert_eq!(0, boolean_slice2.bit_offset);
552        assert_eq!(16, boolean_slice2.bit_len);
553    }
554
555    #[test]
556    fn test_boolean_bitand() {
557        let offset = 0;
558        let len = 40;
559
560        let buf1 = Buffer::from(&[0, 1, 1, 0, 0]);
561        let boolean_buf1 = &BooleanBuffer::new(buf1, offset, len);
562
563        let buf2 = Buffer::from(&[0, 1, 1, 1, 0]);
564        let boolean_buf2 = &BooleanBuffer::new(buf2, offset, len);
565
566        let expected = BooleanBuffer::new(Buffer::from(&[0, 1, 1, 0, 0]), offset, len);
567        assert_eq!(boolean_buf1 & boolean_buf2, expected);
568    }
569
570    #[test]
571    fn test_boolean_bitor() {
572        let offset = 0;
573        let len = 40;
574
575        let buf1 = Buffer::from(&[0, 1, 1, 0, 0]);
576        let boolean_buf1 = &BooleanBuffer::new(buf1, offset, len);
577
578        let buf2 = Buffer::from(&[0, 1, 1, 1, 0]);
579        let boolean_buf2 = &BooleanBuffer::new(buf2, offset, len);
580
581        let expected = BooleanBuffer::new(Buffer::from(&[0, 1, 1, 1, 0]), offset, len);
582        assert_eq!(boolean_buf1 | boolean_buf2, expected);
583    }
584
585    #[test]
586    fn test_boolean_bitxor() {
587        let offset = 0;
588        let len = 40;
589
590        let buf1 = Buffer::from(&[0, 1, 1, 0, 0]);
591        let boolean_buf1 = &BooleanBuffer::new(buf1, offset, len);
592
593        let buf2 = Buffer::from(&[0, 1, 1, 1, 0]);
594        let boolean_buf2 = &BooleanBuffer::new(buf2, offset, len);
595
596        let expected = BooleanBuffer::new(Buffer::from(&[0, 0, 0, 1, 0]), offset, len);
597        assert_eq!(boolean_buf1 ^ boolean_buf2, expected);
598    }
599
600    #[test]
601    fn test_boolean_not() {
602        let offset = 0;
603        let len = 40;
604
605        let buf = Buffer::from(&[0, 1, 1, 0, 0]);
606        let boolean_buf = &BooleanBuffer::new(buf, offset, len);
607
608        let expected = BooleanBuffer::new(Buffer::from(&[255, 254, 254, 255, 255]), offset, len);
609        assert_eq!(!boolean_buf, expected);
610    }
611
612    #[test]
613    fn test_boolean_from_slice_bool() {
614        let v = [true, false, false];
615        let buf = BooleanBuffer::from(&v[..]);
616        assert_eq!(buf.offset(), 0);
617        assert_eq!(buf.len(), 3);
618        assert_eq!(buf.values().len(), 1);
619        assert!(buf.value(0));
620    }
621
622    #[test]
623    fn test_from_bitwise_unary_op() {
624        // Use 1024 boolean values so that at least some of the tests cover multiple u64 chunks and
625        // perfect alignment
626        let input_bools = (0..1024)
627            .map(|_| rand::random::<bool>())
628            .collect::<Vec<bool>>();
629        let input_buffer = BooleanBuffer::from(&input_bools[..]);
630
631        // Note ensure we test offsets over 100 to cover multiple u64 chunks
632        for offset in 0..1024 {
633            let result = BooleanBuffer::from_bitwise_unary_op(
634                input_buffer.values(),
635                offset,
636                input_buffer.len() - offset,
637                |a| !a,
638            );
639            let expected = input_bools[offset..]
640                .iter()
641                .map(|b| !*b)
642                .collect::<BooleanBuffer>();
643            assert_eq!(result, expected);
644        }
645
646        // Also test when the input doesn't cover the entire buffer
647        for offset in 0..512 {
648            let len = 512 - offset; // fixed length less than total
649            let result =
650                BooleanBuffer::from_bitwise_unary_op(input_buffer.values(), offset, len, |a| !a);
651            let expected = input_bools[offset..]
652                .iter()
653                .take(len)
654                .map(|b| !*b)
655                .collect::<BooleanBuffer>();
656            assert_eq!(result, expected);
657        }
658    }
659}