arrow_buffer/util/
bit_util.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
18//! Utils for working with bits
19
20use crate::bit_chunk_iterator::BitChunks;
21
22/// Returns the nearest number that is `>=` than `num` and is a multiple of 64
23#[inline]
24pub fn round_upto_multiple_of_64(num: usize) -> usize {
25    num.checked_next_multiple_of(64)
26        .expect("failed to round upto multiple of 64")
27}
28
29/// Returns the nearest multiple of `factor` that is `>=` than `num`. Here `factor` must
30/// be a power of 2.
31pub fn round_upto_power_of_2(num: usize, factor: usize) -> usize {
32    debug_assert!(factor > 0 && factor.is_power_of_two());
33    num.checked_add(factor - 1)
34        .expect("failed to round to next highest power of 2")
35        & !(factor - 1)
36}
37
38/// Returns whether bit at position `i` in `data` is set or not
39#[inline]
40pub fn get_bit(data: &[u8], i: usize) -> bool {
41    data[i / 8] & (1 << (i % 8)) != 0
42}
43
44/// Returns whether bit at position `i` in `data` is set or not.
45///
46/// # Safety
47///
48/// Note this doesn't do any bound checking, for performance reason. The caller is
49/// responsible to guarantee that `i` is within bounds.
50#[inline]
51pub unsafe fn get_bit_raw(data: *const u8, i: usize) -> bool {
52    unsafe { (*data.add(i / 8) & (1 << (i % 8))) != 0 }
53}
54
55/// Sets bit at position `i` for `data` to 1
56#[inline]
57pub fn set_bit(data: &mut [u8], i: usize) {
58    data[i / 8] |= 1 << (i % 8);
59}
60
61/// Sets bit at position `i` for `data`
62///
63/// # Safety
64///
65/// Note this doesn't do any bound checking, for performance reason. The caller is
66/// responsible to guarantee that `i` is within bounds.
67#[inline]
68pub unsafe fn set_bit_raw(data: *mut u8, i: usize) {
69    unsafe {
70        *data.add(i / 8) |= 1 << (i % 8);
71    }
72}
73
74/// Sets bit at position `i` for `data` to 0
75#[inline]
76pub fn unset_bit(data: &mut [u8], i: usize) {
77    data[i / 8] &= !(1 << (i % 8));
78}
79
80/// Sets bit at position `i` for `data` to 0
81///
82/// # Safety
83///
84/// Note this doesn't do any bound checking, for performance reason. The caller is
85/// responsible to guarantee that `i` is within bounds.
86#[inline]
87pub unsafe fn unset_bit_raw(data: *mut u8, i: usize) {
88    unsafe {
89        *data.add(i / 8) &= !(1 << (i % 8));
90    }
91}
92
93/// Returns the ceil of `value`/`divisor`
94#[inline]
95pub fn ceil(value: usize, divisor: usize) -> usize {
96    value.div_ceil(divisor)
97}
98
99/// Read up to 8 bits from a byte slice starting at a given bit offset.
100///
101/// # Arguments
102///
103/// * `slice` - The byte slice to read from
104/// * `number_of_bits_to_read` - Number of bits to read (must be < 8)
105/// * `bit_offset` - Starting bit offset within the first byte (must be < 8)
106///
107/// # Returns
108///
109/// A `u8` containing the requested bits in the least significant positions
110///
111/// # Panics
112/// - Panics if `number_of_bits_to_read` is 0 or >= 8
113/// - Panics if `bit_offset` is >= 8
114/// - Panics if `slice` is empty or too small to read the requested bits
115///
116#[inline]
117pub(crate) fn read_up_to_byte_from_offset(
118    slice: &[u8],
119    number_of_bits_to_read: usize,
120    bit_offset: usize,
121) -> u8 {
122    assert!(number_of_bits_to_read < 8, "can read up to 8 bits only");
123    assert!(bit_offset < 8, "bit offset must be less than 8");
124    assert_ne!(
125        number_of_bits_to_read, 0,
126        "number of bits to read must be greater than 0"
127    );
128    assert_ne!(slice.len(), 0, "slice must not be empty");
129
130    let number_of_bytes_to_read = ceil(number_of_bits_to_read + bit_offset, 8);
131
132    // number of bytes to read
133    assert!(slice.len() >= number_of_bytes_to_read, "slice is too small");
134
135    let mut bits = slice[0] >> bit_offset;
136    for (i, &byte) in slice
137        .iter()
138        .take(number_of_bytes_to_read)
139        .enumerate()
140        .skip(1)
141    {
142        bits |= byte << (i * 8 - bit_offset);
143    }
144
145    bits & ((1 << number_of_bits_to_read) - 1)
146}
147
148/// Applies a bitwise operation relative to another bit-packed byte slice
149/// (right) in place
150///
151/// Note: applies the operation 64-bits (u64) at a time.
152///
153/// # Arguments
154///
155/// * `left` - The mutable buffer to be modified in-place
156/// * `offset_in_bits` - Starting bit offset in Self buffer
157/// * `right` - slice of bit-packed bytes in LSB order
158/// * `right_offset_in_bits` - Starting bit offset in the right buffer
159/// * `len_in_bits` - Number of bits to process
160/// * `op` - Binary operation to apply (e.g., `|a, b| a & b`). Applied a word at a time
161///
162/// # Example: Modify entire buffer
163/// ```
164/// # use arrow_buffer::MutableBuffer;
165/// # use arrow_buffer::bit_util::apply_bitwise_binary_op;
166/// let mut left = MutableBuffer::new(2);
167/// left.extend_from_slice(&[0b11110000u8, 0b00110011u8]);
168/// let right = &[0b10101010u8, 0b10101010u8];
169/// // apply bitwise AND between left and right buffers, updating left in place
170/// apply_bitwise_binary_op(left.as_slice_mut(), 0, right, 0, 16, |a, b| a & b);
171/// assert_eq!(left.as_slice(), &[0b10100000u8, 0b00100010u8]);
172/// ```
173///
174/// # Example: Modify buffer with offsets
175/// ```
176/// # use arrow_buffer::MutableBuffer;
177/// # use arrow_buffer::bit_util::apply_bitwise_binary_op;
178/// let mut left = MutableBuffer::new(2);
179/// left.extend_from_slice(&[0b00000000u8, 0b00000000u8]);
180/// let right = &[0b10110011u8, 0b11111110u8];
181/// // apply bitwise OR between left and right buffers,
182/// // Apply only 8 bits starting from bit offset 3 in left and bit offset 2 in right
183/// apply_bitwise_binary_op(left.as_slice_mut(), 3, right, 2, 8, |a, b| a | b);
184/// assert_eq!(left.as_slice(), &[0b01100000, 0b00000101u8]);
185/// ```
186///
187/// # Panics
188///
189/// If the offset or lengths exceed the buffer or slice size.
190pub fn apply_bitwise_binary_op<F>(
191    left: &mut [u8],
192    left_offset_in_bits: usize,
193    right: impl AsRef<[u8]>,
194    right_offset_in_bits: usize,
195    len_in_bits: usize,
196    mut op: F,
197) where
198    F: FnMut(u64, u64) -> u64,
199{
200    if len_in_bits == 0 {
201        return;
202    }
203
204    // offset inside a byte
205    let bit_offset = left_offset_in_bits % 8;
206
207    let is_mutable_buffer_byte_aligned = bit_offset == 0;
208
209    if is_mutable_buffer_byte_aligned {
210        byte_aligned_bitwise_bin_op_helper(
211            left,
212            left_offset_in_bits,
213            right,
214            right_offset_in_bits,
215            len_in_bits,
216            op,
217        );
218    } else {
219        // If we are not byte aligned, run `op` on the first few bits to reach byte alignment
220        let bits_to_next_byte = (8 - bit_offset)
221            // Minimum with the amount of bits we need to process
222            // to avoid reading out of bounds
223            .min(len_in_bits);
224
225        {
226            let right_byte_offset = right_offset_in_bits / 8;
227
228            // Read the same amount of bits from the right buffer
229            let right_first_byte: u8 = crate::util::bit_util::read_up_to_byte_from_offset(
230                &right.as_ref()[right_byte_offset..],
231                bits_to_next_byte,
232                // Right bit offset
233                right_offset_in_bits % 8,
234            );
235
236            align_to_byte(
237                left,
238                // Hope it gets inlined
239                &mut |left| op(left, right_first_byte as u64),
240                left_offset_in_bits,
241            );
242        }
243
244        let offset_in_bits = left_offset_in_bits + bits_to_next_byte;
245        let right_offset_in_bits = right_offset_in_bits + bits_to_next_byte;
246        let len_in_bits = len_in_bits.saturating_sub(bits_to_next_byte);
247
248        if len_in_bits == 0 {
249            return;
250        }
251
252        // We are now byte aligned
253        byte_aligned_bitwise_bin_op_helper(
254            left,
255            offset_in_bits,
256            right,
257            right_offset_in_bits,
258            len_in_bits,
259            op,
260        );
261    }
262}
263
264/// Apply a bitwise operation to a mutable buffer, updating it in place.
265///
266/// Note: applies the operation 64-bits (u64) at a time.
267///
268/// # Arguments
269///
270/// * `offset_in_bits` - Starting bit offset for the current buffer
271/// * `len_in_bits` - Number of bits to process
272/// * `op` - Unary operation to apply (e.g., `|a| !a`). Applied a word at a time
273///
274/// # Example: Modify entire buffer
275/// ```
276/// # use arrow_buffer::MutableBuffer;
277/// # use arrow_buffer::bit_util::apply_bitwise_unary_op;
278/// let mut buffer = MutableBuffer::new(2);
279/// buffer.extend_from_slice(&[0b11110000u8, 0b00110011u8]);
280/// // apply bitwise NOT to the buffer in place
281/// apply_bitwise_unary_op(buffer.as_slice_mut(), 0, 16, |a| !a);
282/// assert_eq!(buffer.as_slice(), &[0b00001111u8, 0b11001100u8]);
283/// ```
284///
285/// # Example: Modify buffer with offsets
286/// ```
287/// # use arrow_buffer::MutableBuffer;
288/// # use arrow_buffer::bit_util::apply_bitwise_unary_op;
289/// let mut buffer = MutableBuffer::new(2);
290/// buffer.extend_from_slice(&[0b00000000u8, 0b00000000u8]);
291/// // apply bitwise NOT to 8 bits starting from bit offset 3
292/// apply_bitwise_unary_op(buffer.as_slice_mut(), 3, 8, |a| !a);
293/// assert_eq!(buffer.as_slice(), &[0b11111000u8, 0b00000111u8]);
294/// ```
295///
296/// # Panics
297///
298/// If the offset and length exceed the buffer size.
299pub fn apply_bitwise_unary_op<F>(
300    buffer: &mut [u8],
301    offset_in_bits: usize,
302    len_in_bits: usize,
303    mut op: F,
304) where
305    F: FnMut(u64) -> u64,
306{
307    if len_in_bits == 0 {
308        return;
309    }
310
311    // offset inside a byte
312    let left_bit_offset = offset_in_bits % 8;
313
314    let is_mutable_buffer_byte_aligned = left_bit_offset == 0;
315
316    if is_mutable_buffer_byte_aligned {
317        byte_aligned_bitwise_unary_op_helper(buffer, offset_in_bits, len_in_bits, op);
318    } else {
319        align_to_byte(buffer, &mut op, offset_in_bits);
320
321        // If we are not byte aligned we will read the first few bits
322        let bits_to_next_byte = 8 - left_bit_offset;
323
324        let offset_in_bits = offset_in_bits + bits_to_next_byte;
325        let len_in_bits = len_in_bits.saturating_sub(bits_to_next_byte);
326
327        if len_in_bits == 0 {
328            return;
329        }
330
331        // We are now byte aligned
332        byte_aligned_bitwise_unary_op_helper(buffer, offset_in_bits, len_in_bits, op);
333    }
334}
335
336/// Perform bitwise binary operation on byte-aligned buffers (i.e. not offsetting into a middle of a byte).
337///
338/// This is the optimized path for byte-aligned operations. It processes data in
339/// u64 chunks for maximum efficiency, then handles any remainder bits.
340///
341/// # Arguments
342///
343/// * `left` - The left mutable buffer (must be byte-aligned)
344/// * `left_offset_in_bits` - Starting bit offset in the left buffer (must be multiple of 8)
345/// * `right` - The right buffer as byte slice
346/// * `right_offset_in_bits` - Starting bit offset in the right buffer
347/// * `len_in_bits` - Number of bits to process
348/// * `op` - Binary operation to apply
349#[inline]
350fn byte_aligned_bitwise_bin_op_helper<F>(
351    left: &mut [u8],
352    left_offset_in_bits: usize,
353    right: impl AsRef<[u8]>,
354    right_offset_in_bits: usize,
355    len_in_bits: usize,
356    mut op: F,
357) where
358    F: FnMut(u64, u64) -> u64,
359{
360    // Must not reach here if we not byte aligned
361    assert_eq!(
362        left_offset_in_bits % 8,
363        0,
364        "offset_in_bits must be byte aligned"
365    );
366
367    // 1. Prepare the buffers
368    let (complete_u64_chunks, remainder_bytes) =
369        U64UnalignedSlice::split(left, left_offset_in_bits, len_in_bits);
370
371    let right_chunks = BitChunks::new(right.as_ref(), right_offset_in_bits, len_in_bits);
372    assert_eq!(
373        self::ceil(right_chunks.remainder_len(), 8),
374        remainder_bytes.len()
375    );
376
377    let right_chunks_iter = right_chunks.iter();
378    assert_eq!(right_chunks_iter.len(), complete_u64_chunks.len());
379
380    // 2. Process complete u64 chunks
381    complete_u64_chunks.zip_modify(right_chunks_iter, &mut op);
382
383    // Handle remainder bits if any
384    if right_chunks.remainder_len() > 0 {
385        handle_mutable_buffer_remainder(
386            &mut op,
387            remainder_bytes,
388            right_chunks.remainder_bits(),
389            right_chunks.remainder_len(),
390        )
391    }
392}
393
394/// Perform bitwise unary operation on byte-aligned buffer.
395///
396/// This is the optimized path for byte-aligned unary operations. It processes data in
397/// u64 chunks for maximum efficiency, then handles any remainder bits.
398///
399/// # Arguments
400///
401/// * `buffer` - The mutable buffer (must be byte-aligned)
402/// * `offset_in_bits` - Starting bit offset (must be multiple of 8)
403/// * `len_in_bits` - Number of bits to process
404/// * `op` - Unary operation to apply (e.g., `|a| !a`)
405#[inline]
406fn byte_aligned_bitwise_unary_op_helper<F>(
407    buffer: &mut [u8],
408    offset_in_bits: usize,
409    len_in_bits: usize,
410    mut op: F,
411) where
412    F: FnMut(u64) -> u64,
413{
414    // Must not reach here if we not byte aligned
415    assert_eq!(offset_in_bits % 8, 0, "offset_in_bits must be byte aligned");
416
417    let remainder_len = len_in_bits % 64;
418
419    let (complete_u64_chunks, remainder_bytes) =
420        U64UnalignedSlice::split(buffer, offset_in_bits, len_in_bits);
421
422    assert_eq!(self::ceil(remainder_len, 8), remainder_bytes.len());
423
424    // 2. Process complete u64 chunks
425    complete_u64_chunks.apply_unary_op(&mut op);
426
427    // Handle remainder bits if any
428    if remainder_len > 0 {
429        handle_mutable_buffer_remainder_unary(&mut op, remainder_bytes, remainder_len)
430    }
431}
432
433/// Align to byte boundary by applying operation to bits before the next byte boundary.
434///
435/// This function handles non-byte-aligned operations by processing bits from the current
436/// position up to the next byte boundary, while preserving all other bits in the byte.
437///
438/// # Arguments
439///
440/// * `op` - Unary operation to apply
441/// * `buffer` - The mutable buffer to modify
442/// * `offset_in_bits` - Starting bit offset (not byte-aligned)
443fn align_to_byte<F>(buffer: &mut [u8], op: &mut F, offset_in_bits: usize)
444where
445    F: FnMut(u64) -> u64,
446{
447    let byte_offset = offset_in_bits / 8;
448    let bit_offset = offset_in_bits % 8;
449
450    // 1. read the first byte from the buffer
451    let first_byte: u8 = buffer[byte_offset];
452
453    // 2. Shift byte by the bit offset, keeping only the relevant bits
454    let relevant_first_byte = first_byte >> bit_offset;
455
456    // 3. run the op on the first byte only
457    let result_first_byte = op(relevant_first_byte as u64) as u8;
458
459    // 4. Shift back the result to the original position
460    let result_first_byte = result_first_byte << bit_offset;
461
462    // 5. Mask the bits that are outside the relevant bits in the byte
463    //    so the bits until bit_offset are 1 and the rest are 0
464    let mask_for_first_bit_offset = (1 << bit_offset) - 1;
465
466    let result_first_byte =
467        (first_byte & mask_for_first_bit_offset) | (result_first_byte & !mask_for_first_bit_offset);
468
469    // 6. write back the result to the buffer
470    buffer[byte_offset] = result_first_byte;
471}
472
473/// Centralized structure to handle a mutable u8 slice as a mutable u64 pointer.
474///
475/// Handle the following:
476/// 1. the lifetime is correct
477/// 2. we read/write within the bounds
478/// 3. We read and write using unaligned
479///
480/// This does not deallocate the underlying pointer when dropped
481///
482/// This is the only place that uses unsafe code to read and write unaligned
483///
484struct U64UnalignedSlice<'a> {
485    /// Pointer to the start of the u64 data
486    ///
487    /// We are using raw pointer as the data came from a u8 slice so we need to read and write unaligned
488    ptr: *mut u64,
489
490    /// Number of u64 elements
491    len: usize,
492
493    /// Marker to tie the lifetime of the pointer to the lifetime of the u8 slice
494    _marker: std::marker::PhantomData<&'a u8>,
495}
496
497impl<'a> U64UnalignedSlice<'a> {
498    /// Create a new [`U64UnalignedSlice`] from a `&mut [u8]` buffer
499    ///
500    /// return the [`U64UnalignedSlice`] and slice of bytes that are not part of the u64 chunks (guaranteed to be less than 8 bytes)
501    ///
502    fn split(
503        buffer: &'a mut [u8],
504        offset_in_bits: usize,
505        len_in_bits: usize,
506    ) -> (Self, &'a mut [u8]) {
507        // 1. Prepare the buffers
508        let left_buffer_mut: &mut [u8] = {
509            let last_offset = self::ceil(offset_in_bits + len_in_bits, 8);
510            assert!(last_offset <= buffer.len());
511
512            let byte_offset = offset_in_bits / 8;
513
514            &mut buffer[byte_offset..last_offset]
515        };
516
517        let number_of_u64_we_can_fit = len_in_bits / (u64::BITS as usize);
518
519        // 2. Split
520        let u64_len_in_bytes = number_of_u64_we_can_fit * size_of::<u64>();
521
522        assert!(u64_len_in_bytes <= left_buffer_mut.len());
523        let (bytes_for_u64, remainder) = left_buffer_mut.split_at_mut(u64_len_in_bytes);
524
525        let ptr = bytes_for_u64.as_mut_ptr() as *mut u64;
526
527        let this = Self {
528            ptr,
529            len: number_of_u64_we_can_fit,
530            _marker: std::marker::PhantomData,
531        };
532
533        (this, remainder)
534    }
535
536    fn len(&self) -> usize {
537        self.len
538    }
539
540    /// Modify the underlying u64 data in place using a binary operation
541    /// with another iterator.
542    fn zip_modify(
543        mut self,
544        mut zip_iter: impl ExactSizeIterator<Item = u64>,
545        mut map: impl FnMut(u64, u64) -> u64,
546    ) {
547        assert_eq!(self.len, zip_iter.len());
548
549        // In order to avoid advancing the pointer at the end of the loop which will
550        // make the last pointer invalid, we handle the first element outside the loop
551        // and then advance the pointer at the start of the loop
552        // making sure that the iterator is not empty
553        if let Some(right) = zip_iter.next() {
554            // SAFETY: We asserted that the iterator length and the current length are the same
555            // and the iterator is not empty, so the pointer is valid
556            unsafe {
557                self.apply_bin_op(right, &mut map);
558            }
559
560            // Because this consumes self we don't update the length
561        }
562
563        for right in zip_iter {
564            // Advance the pointer
565            //
566            // SAFETY: We asserted that the iterator length and the current length are the same
567            self.ptr = unsafe { self.ptr.add(1) };
568
569            // SAFETY: the pointer is valid as we are within the length
570            unsafe {
571                self.apply_bin_op(right, &mut map);
572            }
573
574            // Because this consumes self we don't update the length
575        }
576    }
577
578    /// Centralized function to correctly read the current u64 value and write back the result
579    ///
580    /// # SAFETY
581    /// the caller must ensure that the pointer is valid for reads and writes
582    ///
583    #[inline]
584    unsafe fn apply_bin_op(&mut self, right: u64, mut map: impl FnMut(u64, u64) -> u64) {
585        // SAFETY: The constructor ensures the pointer is valid,
586        // and as to all modifications in U64UnalignedSlice
587        let current_input = unsafe {
588            self.ptr
589                // Reading unaligned as we came from u8 slice
590                .read_unaligned()
591                // bit-packed buffers are stored starting with the least-significant byte first
592                // so when reading as u64 on a big-endian machine, the bytes need to be swapped
593                .to_le()
594        };
595
596        let combined = map(current_input, right);
597
598        // Write the result back
599        //
600        // The pointer came from mutable u8 slice so the pointer is valid for writes,
601        // and we need to write unaligned
602        unsafe { self.ptr.write_unaligned(combined) }
603    }
604
605    /// Modify the underlying u64 data in place using a unary operation.
606    fn apply_unary_op(mut self, mut map: impl FnMut(u64) -> u64) {
607        if self.len == 0 {
608            return;
609        }
610
611        // In order to avoid advancing the pointer at the end of the loop which will
612        // make the last pointer invalid, we handle the first element outside the loop
613        // and then advance the pointer at the start of the loop
614        // making sure that the iterator is not empty
615        unsafe {
616            // I hope the function get inlined and the compiler remove the dead right parameter
617            self.apply_bin_op(0, &mut |left, _| map(left));
618
619            // Because this consumes self we don't update the length
620        }
621
622        for _ in 1..self.len {
623            // Advance the pointer
624            //
625            // SAFETY: we only advance the pointer within the length and not beyond
626            self.ptr = unsafe { self.ptr.add(1) };
627
628            // SAFETY: the pointer is valid as we are within the length
629            unsafe {
630                // I hope the function get inlined and the compiler remove the dead right parameter
631                self.apply_bin_op(0, &mut |left, _| map(left));
632            }
633
634            // Because this consumes self we don't update the length
635        }
636    }
637}
638
639/// Handle remainder bits (< 64 bits) for binary operations.
640///
641/// This function processes the bits that don't form a complete u64 chunk,
642/// ensuring that bits outside the operation range are preserved.
643///
644/// # Arguments
645///
646/// * `op` - Binary operation to apply
647/// * `start_remainder_mut_slice` - slice to the start of remainder bytes
648///   the length must be equal to `ceil(remainder_len, 8)`
649/// * `right_remainder_bits` - Right operand bits
650/// * `remainder_len` - Number of remainder bits
651#[inline]
652fn handle_mutable_buffer_remainder<F>(
653    op: &mut F,
654    start_remainder_mut_slice: &mut [u8],
655    right_remainder_bits: u64,
656    remainder_len: usize,
657) where
658    F: FnMut(u64, u64) -> u64,
659{
660    // Only read from slice the number of remainder bits
661    let left_remainder_bits = get_remainder_bits(start_remainder_mut_slice, remainder_len);
662
663    // Apply the operation
664    let rem = op(left_remainder_bits, right_remainder_bits);
665
666    // Write only the relevant bits back the result to the mutable slice
667    set_remainder_bits(start_remainder_mut_slice, rem, remainder_len);
668}
669
670/// Write remainder bits back to buffer while preserving bits outside the range.
671///
672/// This function carefully updates only the specified bits, leaving all other
673/// bits in the affected bytes unchanged.
674///
675/// # Arguments
676///
677/// * `start_remainder_mut_slice` - the slice of bytes to write the remainder bits to,
678///   the length must be equal to `ceil(remainder_len, 8)`
679/// * `rem` - The result bits to write
680/// * `remainder_len` - Number of bits to write
681#[inline]
682fn set_remainder_bits(start_remainder_mut_slice: &mut [u8], rem: u64, remainder_len: usize) {
683    assert_ne!(
684        start_remainder_mut_slice.len(),
685        0,
686        "start_remainder_mut_slice must not be empty"
687    );
688    assert!(remainder_len < 64, "remainder_len must be less than 64");
689
690    // This assertion is to make sure that the last byte in the slice is the boundary byte
691    // (i.e., the byte that contains both remainder bits and bits outside the remainder)
692    assert_eq!(
693        start_remainder_mut_slice.len(),
694        self::ceil(remainder_len, 8),
695        "start_remainder_mut_slice length must be equal to ceil(remainder_len, 8)"
696    );
697
698    // Need to update the remainder bytes in the mutable buffer
699    // but not override the bits outside the remainder
700
701    // Update `rem` end with the current bytes in the mutable buffer
702    // to preserve the bits outside the remainder
703    let rem = {
704        // 1. Read the byte that we will override
705        //    we only read the last byte as we verified that start_remainder_mut_slice length is
706        //    equal to ceil(remainder_len, 8), which means the last byte is the boundary byte
707        //    containing both remainder bits and bits outside the remainder
708        let current = start_remainder_mut_slice
709            .last()
710            // Unwrap as we already validated the slice is not empty
711            .unwrap();
712
713        let current = *current as u64;
714
715        // Mask where the bits that are inside the remainder are 1
716        // and the bits outside the remainder are 0
717        let inside_remainder_mask = (1 << remainder_len) - 1;
718        // Mask where the bits that are outside the remainder are 1
719        // and the bits inside the remainder are 0
720        let outside_remainder_mask = !inside_remainder_mask;
721
722        // 2. Only keep the bits that are outside the remainder for the value from the mutable buffer
723        let current = current & outside_remainder_mask;
724
725        // 3. Only keep the bits that are inside the remainder for the value from the operation
726        let rem = rem & inside_remainder_mask;
727
728        // 4. Combine the two values
729        current | rem
730    };
731
732    // Write back the result to the mutable slice
733    {
734        let remainder_bytes = self::ceil(remainder_len, 8);
735
736        // we are counting starting from the least significant bit, so to_le_bytes should be correct
737        let rem = &rem.to_le_bytes()[0..remainder_bytes];
738
739        // this assumes that `[ToByteSlice]` can be copied directly
740        // without calling `to_byte_slice` for each element,
741        // which is correct for all ArrowNativeType implementations including u64.
742        let src = rem.as_ptr();
743        unsafe {
744            std::ptr::copy_nonoverlapping(
745                src,
746                start_remainder_mut_slice.as_mut_ptr(),
747                remainder_bytes,
748            )
749        };
750    }
751}
752
753/// Read remainder bits from a slice.
754///
755/// Reads the specified number of bits from slice and returns them as a u64.
756///
757/// # Arguments
758///
759/// * `remainder` - slice to the start of the bits
760/// * `remainder_len` - Number of bits to read (must be < 64)
761///
762/// # Returns
763///
764/// A u64 containing the bits in the least significant positions
765#[inline]
766fn get_remainder_bits(remainder: &[u8], remainder_len: usize) -> u64 {
767    assert!(remainder.len() < 64, "remainder_len must be less than 64");
768    assert_eq!(
769        remainder.len(),
770        self::ceil(remainder_len, 8),
771        "remainder and remainder len ceil must be the same"
772    );
773
774    let bits = remainder
775        .iter()
776        .enumerate()
777        .fold(0_u64, |acc, (index, &byte)| {
778            acc | (byte as u64) << (index * 8)
779        });
780
781    bits & ((1 << remainder_len) - 1)
782}
783
784/// Handle remainder bits (< 64 bits) for unary operations.
785///
786/// This function processes the bits that don't form a complete u64 chunk,
787/// ensuring that bits outside the operation range are preserved.
788///
789/// # Arguments
790///
791/// * `op` - Unary operation to apply
792/// * `start_remainder_mut` - Slice of bytes to write the remainder bits to
793/// * `remainder_len` - Number of remainder bits
794#[inline]
795fn handle_mutable_buffer_remainder_unary<F>(
796    op: &mut F,
797    start_remainder_mut: &mut [u8],
798    remainder_len: usize,
799) where
800    F: FnMut(u64) -> u64,
801{
802    // Only read from the slice the number of remainder bits
803    let left_remainder_bits = get_remainder_bits(start_remainder_mut, remainder_len);
804
805    // Apply the operation
806    let rem = op(left_remainder_bits);
807
808    // Write only the relevant bits back the result to the slice
809    set_remainder_bits(start_remainder_mut, rem, remainder_len);
810}
811
812#[cfg(test)]
813mod tests {
814    use std::collections::HashSet;
815
816    use super::*;
817    use crate::bit_iterator::BitIterator;
818    use crate::{BooleanBuffer, BooleanBufferBuilder, MutableBuffer};
819    use rand::rngs::StdRng;
820    use rand::{Rng, SeedableRng};
821
822    #[test]
823    fn test_round_upto_multiple_of_64() {
824        assert_eq!(0, round_upto_multiple_of_64(0));
825        assert_eq!(64, round_upto_multiple_of_64(1));
826        assert_eq!(64, round_upto_multiple_of_64(63));
827        assert_eq!(64, round_upto_multiple_of_64(64));
828        assert_eq!(128, round_upto_multiple_of_64(65));
829        assert_eq!(192, round_upto_multiple_of_64(129));
830    }
831
832    #[test]
833    #[should_panic(expected = "failed to round upto multiple of 64")]
834    fn test_round_upto_multiple_of_64_panic() {
835        let _ = round_upto_multiple_of_64(usize::MAX);
836    }
837
838    #[test]
839    #[should_panic(expected = "failed to round to next highest power of 2")]
840    fn test_round_upto_panic() {
841        let _ = round_upto_power_of_2(usize::MAX, 2);
842    }
843
844    #[test]
845    fn test_get_bit() {
846        // 00001101
847        assert!(get_bit(&[0b00001101], 0));
848        assert!(!get_bit(&[0b00001101], 1));
849        assert!(get_bit(&[0b00001101], 2));
850        assert!(get_bit(&[0b00001101], 3));
851
852        // 01001001 01010010
853        assert!(get_bit(&[0b01001001, 0b01010010], 0));
854        assert!(!get_bit(&[0b01001001, 0b01010010], 1));
855        assert!(!get_bit(&[0b01001001, 0b01010010], 2));
856        assert!(get_bit(&[0b01001001, 0b01010010], 3));
857        assert!(!get_bit(&[0b01001001, 0b01010010], 4));
858        assert!(!get_bit(&[0b01001001, 0b01010010], 5));
859        assert!(get_bit(&[0b01001001, 0b01010010], 6));
860        assert!(!get_bit(&[0b01001001, 0b01010010], 7));
861        assert!(!get_bit(&[0b01001001, 0b01010010], 8));
862        assert!(get_bit(&[0b01001001, 0b01010010], 9));
863        assert!(!get_bit(&[0b01001001, 0b01010010], 10));
864        assert!(!get_bit(&[0b01001001, 0b01010010], 11));
865        assert!(get_bit(&[0b01001001, 0b01010010], 12));
866        assert!(!get_bit(&[0b01001001, 0b01010010], 13));
867        assert!(get_bit(&[0b01001001, 0b01010010], 14));
868        assert!(!get_bit(&[0b01001001, 0b01010010], 15));
869    }
870
871    pub fn seedable_rng() -> StdRng {
872        StdRng::seed_from_u64(42)
873    }
874
875    #[test]
876    fn test_get_bit_raw() {
877        const NUM_BYTE: usize = 10;
878        let mut buf = [0; NUM_BYTE];
879        let mut expected = vec![];
880        let mut rng = seedable_rng();
881        for i in 0..8 * NUM_BYTE {
882            let b = rng.random_bool(0.5);
883            expected.push(b);
884            if b {
885                set_bit(&mut buf[..], i)
886            }
887        }
888
889        let raw_ptr = buf.as_ptr();
890        for (i, b) in expected.iter().enumerate() {
891            unsafe {
892                assert_eq!(*b, get_bit_raw(raw_ptr, i));
893            }
894        }
895    }
896
897    #[test]
898    fn test_set_bit() {
899        let mut b = [0b00000010];
900        set_bit(&mut b, 0);
901        assert_eq!([0b00000011], b);
902        set_bit(&mut b, 1);
903        assert_eq!([0b00000011], b);
904        set_bit(&mut b, 7);
905        assert_eq!([0b10000011], b);
906    }
907
908    #[test]
909    fn test_unset_bit() {
910        let mut b = [0b11111101];
911        unset_bit(&mut b, 0);
912        assert_eq!([0b11111100], b);
913        unset_bit(&mut b, 1);
914        assert_eq!([0b11111100], b);
915        unset_bit(&mut b, 7);
916        assert_eq!([0b01111100], b);
917    }
918
919    #[test]
920    fn test_set_bit_raw() {
921        const NUM_BYTE: usize = 10;
922        let mut buf = vec![0; NUM_BYTE];
923        let mut expected = vec![];
924        let mut rng = seedable_rng();
925        for i in 0..8 * NUM_BYTE {
926            let b = rng.random_bool(0.5);
927            expected.push(b);
928            if b {
929                unsafe {
930                    set_bit_raw(buf.as_mut_ptr(), i);
931                }
932            }
933        }
934
935        let raw_ptr = buf.as_ptr();
936        for (i, b) in expected.iter().enumerate() {
937            unsafe {
938                assert_eq!(*b, get_bit_raw(raw_ptr, i));
939            }
940        }
941    }
942
943    #[test]
944    fn test_unset_bit_raw() {
945        const NUM_BYTE: usize = 10;
946        let mut buf = vec![255; NUM_BYTE];
947        let mut expected = vec![];
948        let mut rng = seedable_rng();
949        for i in 0..8 * NUM_BYTE {
950            let b = rng.random_bool(0.5);
951            expected.push(b);
952            if !b {
953                unsafe {
954                    unset_bit_raw(buf.as_mut_ptr(), i);
955                }
956            }
957        }
958
959        let raw_ptr = buf.as_ptr();
960        for (i, b) in expected.iter().enumerate() {
961            unsafe {
962                assert_eq!(*b, get_bit_raw(raw_ptr, i));
963            }
964        }
965    }
966
967    #[test]
968    fn test_get_set_bit_roundtrip() {
969        const NUM_BYTES: usize = 10;
970        const NUM_SETS: usize = 10;
971
972        let mut buffer: [u8; NUM_BYTES * 8] = [0; NUM_BYTES * 8];
973        let mut v = HashSet::new();
974        let mut rng = seedable_rng();
975        for _ in 0..NUM_SETS {
976            let offset = rng.random_range(0..8 * NUM_BYTES);
977            v.insert(offset);
978            set_bit(&mut buffer[..], offset);
979        }
980        for i in 0..NUM_BYTES * 8 {
981            assert_eq!(v.contains(&i), get_bit(&buffer[..], i));
982        }
983    }
984
985    #[test]
986    fn test_ceil() {
987        assert_eq!(ceil(0, 1), 0);
988        assert_eq!(ceil(1, 1), 1);
989        assert_eq!(ceil(1, 2), 1);
990        assert_eq!(ceil(1, 8), 1);
991        assert_eq!(ceil(7, 8), 1);
992        assert_eq!(ceil(8, 8), 1);
993        assert_eq!(ceil(9, 8), 2);
994        assert_eq!(ceil(9, 9), 1);
995        assert_eq!(ceil(10000000000, 10), 1000000000);
996        assert_eq!(ceil(10, 10000000000), 1);
997        assert_eq!(ceil(10000000000, 1000000000), 10);
998    }
999
1000    #[test]
1001    fn test_read_up_to() {
1002        let all_ones = &[0b10111001, 0b10001100];
1003
1004        for (bit_offset, expected) in [
1005            (0, 0b00000001),
1006            (1, 0b00000000),
1007            (2, 0b00000000),
1008            (3, 0b00000001),
1009            (4, 0b00000001),
1010            (5, 0b00000001),
1011            (6, 0b00000000),
1012            (7, 0b00000001),
1013        ] {
1014            let result = read_up_to_byte_from_offset(all_ones, 1, bit_offset);
1015            assert_eq!(
1016                result, expected,
1017                "failed at bit_offset {bit_offset}. result, expected:\n{result:08b}\n{expected:08b}"
1018            );
1019        }
1020
1021        for (bit_offset, expected) in [
1022            (0, 0b00000001),
1023            (1, 0b00000000),
1024            (2, 0b00000010),
1025            (3, 0b00000011),
1026            (4, 0b00000011),
1027            (5, 0b00000001),
1028            (6, 0b00000010),
1029            (7, 0b00000001),
1030        ] {
1031            let result = read_up_to_byte_from_offset(all_ones, 2, bit_offset);
1032            assert_eq!(
1033                result, expected,
1034                "failed at bit_offset {bit_offset}. result, expected:\n{result:08b}\n{expected:08b}"
1035            );
1036        }
1037
1038        for (bit_offset, expected) in [
1039            (0, 0b00111001),
1040            (1, 0b00011100),
1041            (2, 0b00101110),
1042            (3, 0b00010111),
1043            (4, 0b00001011),
1044            (5, 0b00100101),
1045            (6, 0b00110010),
1046            (7, 0b00011001),
1047        ] {
1048            let result = read_up_to_byte_from_offset(all_ones, 6, bit_offset);
1049            assert_eq!(
1050                result, expected,
1051                "failed at bit_offset {bit_offset}. result, expected:\n{result:08b}\n{expected:08b}"
1052            );
1053        }
1054
1055        for (bit_offset, expected) in [
1056            (0, 0b00111001),
1057            (1, 0b01011100),
1058            (2, 0b00101110),
1059            (3, 0b00010111),
1060            (4, 0b01001011),
1061            (5, 0b01100101),
1062            (6, 0b00110010),
1063            (7, 0b00011001),
1064        ] {
1065            let result = read_up_to_byte_from_offset(all_ones, 7, bit_offset);
1066            assert_eq!(
1067                result, expected,
1068                "failed at bit_offset {bit_offset}. result, expected:\n{result:08b}\n{expected:08b}"
1069            );
1070        }
1071    }
1072
1073    /// Verifies that a unary operation applied to a buffer using u64 chunks
1074    /// is the same as applying the operation bit by bit.
1075    fn test_mutable_buffer_bin_op_helper<F, G>(
1076        left_data: &[bool],
1077        right_data: &[bool],
1078        left_offset_in_bits: usize,
1079        right_offset_in_bits: usize,
1080        len_in_bits: usize,
1081        op: F,
1082        mut expected_op: G,
1083    ) where
1084        F: FnMut(u64, u64) -> u64,
1085        G: FnMut(bool, bool) -> bool,
1086    {
1087        let mut left_buffer = BooleanBufferBuilder::new(len_in_bits);
1088        left_buffer.append_slice(left_data);
1089        let right_buffer = BooleanBuffer::from(right_data);
1090
1091        let expected: Vec<bool> = left_data
1092            .iter()
1093            .skip(left_offset_in_bits)
1094            .zip(right_data.iter().skip(right_offset_in_bits))
1095            .take(len_in_bits)
1096            .map(|(l, r)| expected_op(*l, *r))
1097            .collect();
1098
1099        apply_bitwise_binary_op(
1100            left_buffer.as_slice_mut(),
1101            left_offset_in_bits,
1102            right_buffer.inner(),
1103            right_offset_in_bits,
1104            len_in_bits,
1105            op,
1106        );
1107
1108        let result: Vec<bool> =
1109            BitIterator::new(left_buffer.as_slice(), left_offset_in_bits, len_in_bits).collect();
1110
1111        assert_eq!(
1112            result, expected,
1113            "Failed with left_offset={}, right_offset={}, len={}",
1114            left_offset_in_bits, right_offset_in_bits, len_in_bits
1115        );
1116    }
1117
1118    /// Verifies that a unary operation applied to a buffer using u64 chunks
1119    /// is the same as applying the operation bit by bit.
1120    fn test_mutable_buffer_unary_op_helper<F, G>(
1121        data: &[bool],
1122        offset_in_bits: usize,
1123        len_in_bits: usize,
1124        op: F,
1125        mut expected_op: G,
1126    ) where
1127        F: FnMut(u64) -> u64,
1128        G: FnMut(bool) -> bool,
1129    {
1130        let mut buffer = BooleanBufferBuilder::new(len_in_bits);
1131        buffer.append_slice(data);
1132
1133        let expected: Vec<bool> = data
1134            .iter()
1135            .skip(offset_in_bits)
1136            .take(len_in_bits)
1137            .map(|b| expected_op(*b))
1138            .collect();
1139
1140        apply_bitwise_unary_op(buffer.as_slice_mut(), offset_in_bits, len_in_bits, op);
1141
1142        let result: Vec<bool> =
1143            BitIterator::new(buffer.as_slice(), offset_in_bits, len_in_bits).collect();
1144
1145        assert_eq!(
1146            result, expected,
1147            "Failed with offset={}, len={}",
1148            offset_in_bits, len_in_bits
1149        );
1150    }
1151
1152    // Helper to create test data of specific length
1153    fn create_test_data(len: usize) -> (Vec<bool>, Vec<bool>) {
1154        let mut rng = rand::rng();
1155        let left: Vec<bool> = (0..len).map(|_| rng.random_bool(0.5)).collect();
1156        let right: Vec<bool> = (0..len).map(|_| rng.random_bool(0.5)).collect();
1157        (left, right)
1158    }
1159
1160    /// Test all binary operations (AND, OR, XOR) with the given parameters
1161    fn test_all_binary_ops(
1162        left_data: &[bool],
1163        right_data: &[bool],
1164        left_offset_in_bits: usize,
1165        right_offset_in_bits: usize,
1166        len_in_bits: usize,
1167    ) {
1168        // Test AND
1169        test_mutable_buffer_bin_op_helper(
1170            left_data,
1171            right_data,
1172            left_offset_in_bits,
1173            right_offset_in_bits,
1174            len_in_bits,
1175            |a, b| a & b,
1176            |a, b| a & b,
1177        );
1178
1179        // Test OR
1180        test_mutable_buffer_bin_op_helper(
1181            left_data,
1182            right_data,
1183            left_offset_in_bits,
1184            right_offset_in_bits,
1185            len_in_bits,
1186            |a, b| a | b,
1187            |a, b| a | b,
1188        );
1189
1190        // Test XOR
1191        test_mutable_buffer_bin_op_helper(
1192            left_data,
1193            right_data,
1194            left_offset_in_bits,
1195            right_offset_in_bits,
1196            len_in_bits,
1197            |a, b| a ^ b,
1198            |a, b| a ^ b,
1199        );
1200    }
1201
1202    // ===== Combined Binary Operation Tests =====
1203
1204    #[test]
1205    fn test_binary_ops_less_than_byte() {
1206        let (left, right) = create_test_data(4);
1207        test_all_binary_ops(&left, &right, 0, 0, 4);
1208    }
1209
1210    #[test]
1211    fn test_binary_ops_less_than_byte_across_boundary() {
1212        let (left, right) = create_test_data(16);
1213        test_all_binary_ops(&left, &right, 6, 6, 4);
1214    }
1215
1216    #[test]
1217    fn test_binary_ops_exactly_byte() {
1218        let (left, right) = create_test_data(16);
1219        test_all_binary_ops(&left, &right, 0, 0, 8);
1220    }
1221
1222    #[test]
1223    fn test_binary_ops_more_than_byte_less_than_u64() {
1224        let (left, right) = create_test_data(64);
1225        test_all_binary_ops(&left, &right, 0, 0, 32);
1226    }
1227
1228    #[test]
1229    fn test_binary_ops_exactly_u64() {
1230        let (left, right) = create_test_data(180);
1231        test_all_binary_ops(&left, &right, 0, 0, 64);
1232        test_all_binary_ops(&left, &right, 64, 9, 64);
1233        test_all_binary_ops(&left, &right, 8, 100, 64);
1234        test_all_binary_ops(&left, &right, 1, 15, 64);
1235        test_all_binary_ops(&left, &right, 12, 10, 64);
1236        test_all_binary_ops(&left, &right, 180 - 64, 2, 64);
1237    }
1238
1239    #[test]
1240    fn test_binary_ops_more_than_u64_not_multiple() {
1241        let (left, right) = create_test_data(200);
1242        test_all_binary_ops(&left, &right, 0, 0, 100);
1243    }
1244
1245    #[test]
1246    fn test_binary_ops_exactly_multiple_u64() {
1247        let (left, right) = create_test_data(256);
1248        test_all_binary_ops(&left, &right, 0, 0, 128);
1249    }
1250
1251    #[test]
1252    fn test_binary_ops_more_than_multiple_u64() {
1253        let (left, right) = create_test_data(300);
1254        test_all_binary_ops(&left, &right, 0, 0, 200);
1255    }
1256
1257    #[test]
1258    fn test_binary_ops_byte_aligned_no_remainder() {
1259        let (left, right) = create_test_data(200);
1260        test_all_binary_ops(&left, &right, 0, 0, 128);
1261    }
1262
1263    #[test]
1264    fn test_binary_ops_byte_aligned_with_remainder() {
1265        let (left, right) = create_test_data(200);
1266        test_all_binary_ops(&left, &right, 0, 0, 100);
1267    }
1268
1269    #[test]
1270    fn test_binary_ops_not_byte_aligned_no_remainder() {
1271        let (left, right) = create_test_data(200);
1272        test_all_binary_ops(&left, &right, 3, 3, 128);
1273    }
1274
1275    #[test]
1276    fn test_binary_ops_not_byte_aligned_with_remainder() {
1277        let (left, right) = create_test_data(200);
1278        test_all_binary_ops(&left, &right, 5, 5, 100);
1279    }
1280
1281    #[test]
1282    fn test_binary_ops_different_offsets() {
1283        let (left, right) = create_test_data(200);
1284        test_all_binary_ops(&left, &right, 3, 7, 50);
1285    }
1286
1287    #[test]
1288    fn test_binary_ops_offsets_greater_than_8_less_than_64() {
1289        let (left, right) = create_test_data(200);
1290        test_all_binary_ops(&left, &right, 13, 27, 100);
1291    }
1292
1293    // ===== NOT (Unary) Operation Tests =====
1294
1295    #[test]
1296    fn test_not_less_than_byte() {
1297        let data = vec![true, false, true, false];
1298        test_mutable_buffer_unary_op_helper(&data, 0, 4, |a| !a, |a| !a);
1299    }
1300
1301    #[test]
1302    fn test_not_less_than_byte_across_boundary() {
1303        let data: Vec<bool> = (0..16).map(|i| i % 2 == 0).collect();
1304        test_mutable_buffer_unary_op_helper(&data, 6, 4, |a| !a, |a| !a);
1305    }
1306
1307    #[test]
1308    fn test_not_exactly_byte() {
1309        let data: Vec<bool> = (0..16).map(|i| i % 2 == 0).collect();
1310        test_mutable_buffer_unary_op_helper(&data, 0, 8, |a| !a, |a| !a);
1311    }
1312
1313    #[test]
1314    fn test_not_more_than_byte_less_than_u64() {
1315        let data: Vec<bool> = (0..64).map(|i| i % 2 == 0).collect();
1316        test_mutable_buffer_unary_op_helper(&data, 0, 32, |a| !a, |a| !a);
1317    }
1318
1319    #[test]
1320    fn test_not_exactly_u64() {
1321        let data: Vec<bool> = (0..128).map(|i| i % 2 == 0).collect();
1322        test_mutable_buffer_unary_op_helper(&data, 0, 64, |a| !a, |a| !a);
1323    }
1324
1325    #[test]
1326    fn test_not_more_than_u64_not_multiple() {
1327        let data: Vec<bool> = (0..200).map(|i| i % 2 == 0).collect();
1328        test_mutable_buffer_unary_op_helper(&data, 0, 100, |a| !a, |a| !a);
1329    }
1330
1331    #[test]
1332    fn test_not_exactly_multiple_u64() {
1333        let data: Vec<bool> = (0..256).map(|i| i % 2 == 0).collect();
1334        test_mutable_buffer_unary_op_helper(&data, 0, 128, |a| !a, |a| !a);
1335    }
1336
1337    #[test]
1338    fn test_not_more_than_multiple_u64() {
1339        let data: Vec<bool> = (0..300).map(|i| i % 2 == 0).collect();
1340        test_mutable_buffer_unary_op_helper(&data, 0, 200, |a| !a, |a| !a);
1341    }
1342
1343    #[test]
1344    fn test_not_byte_aligned_no_remainder() {
1345        let data: Vec<bool> = (0..200).map(|i| i % 2 == 0).collect();
1346        test_mutable_buffer_unary_op_helper(&data, 0, 128, |a| !a, |a| !a);
1347    }
1348
1349    #[test]
1350    fn test_not_byte_aligned_with_remainder() {
1351        let data: Vec<bool> = (0..200).map(|i| i % 2 == 0).collect();
1352        test_mutable_buffer_unary_op_helper(&data, 0, 100, |a| !a, |a| !a);
1353    }
1354
1355    #[test]
1356    fn test_not_not_byte_aligned_no_remainder() {
1357        let data: Vec<bool> = (0..200).map(|i| i % 2 == 0).collect();
1358        test_mutable_buffer_unary_op_helper(&data, 3, 128, |a| !a, |a| !a);
1359    }
1360
1361    #[test]
1362    fn test_not_not_byte_aligned_with_remainder() {
1363        let data: Vec<bool> = (0..200).map(|i| i % 2 == 0).collect();
1364        test_mutable_buffer_unary_op_helper(&data, 5, 100, |a| !a, |a| !a);
1365    }
1366
1367    // ===== Edge Cases =====
1368
1369    #[test]
1370    fn test_empty_length() {
1371        let (left, right) = create_test_data(16);
1372        test_all_binary_ops(&left, &right, 0, 0, 0);
1373    }
1374
1375    #[test]
1376    fn test_single_bit() {
1377        let (left, right) = create_test_data(16);
1378        test_all_binary_ops(&left, &right, 0, 0, 1);
1379    }
1380
1381    #[test]
1382    fn test_single_bit_at_offset() {
1383        let (left, right) = create_test_data(16);
1384        test_all_binary_ops(&left, &right, 7, 7, 1);
1385    }
1386
1387    #[test]
1388    fn test_not_single_bit() {
1389        let data = vec![true, false, true, false];
1390        test_mutable_buffer_unary_op_helper(&data, 0, 1, |a| !a, |a| !a);
1391    }
1392
1393    #[test]
1394    fn test_not_empty_length() {
1395        let data = vec![true, false, true, false];
1396        test_mutable_buffer_unary_op_helper(&data, 0, 0, |a| !a, |a| !a);
1397    }
1398
1399    #[test]
1400    fn test_less_than_byte_unaligned_and_not_enough_bits() {
1401        let left_offset_in_bits = 2;
1402        let right_offset_in_bits = 4;
1403        let len_in_bits = 1;
1404
1405        // Single byte
1406        let right = (0..8).map(|i| (i / 2) % 2 == 0).collect::<Vec<_>>();
1407        // less than a byte
1408        let left = (0..3).map(|i| i % 2 == 0).collect::<Vec<_>>();
1409        test_all_binary_ops(
1410            &left,
1411            &right,
1412            left_offset_in_bits,
1413            right_offset_in_bits,
1414            len_in_bits,
1415        );
1416    }
1417
1418    #[test]
1419    fn test_bitwise_binary_op_offset_out_of_bounds() {
1420        let input = vec![0b10101010u8, 0b01010101u8];
1421        let mut buffer = MutableBuffer::new(2); // space for 16 bits
1422        buffer.extend_from_slice(&input); // only 2 bytes
1423        apply_bitwise_binary_op(
1424            buffer.as_slice_mut(),
1425            100, // exceeds buffer length, becomes a noop
1426            [0b11110000u8, 0b00001111u8],
1427            0,
1428            0,
1429            |a, b| a & b,
1430        );
1431        assert_eq!(buffer.as_slice(), &input);
1432    }
1433
1434    #[test]
1435    #[should_panic(expected = "assertion failed: last_offset <= buffer.len()")]
1436    fn test_bitwise_binary_op_length_out_of_bounds() {
1437        let mut buffer = MutableBuffer::new(2); // space for 16 bits
1438        buffer.extend_from_slice(&[0b10101010u8, 0b01010101u8]); // only 2 bytes
1439        apply_bitwise_binary_op(
1440            buffer.as_slice_mut(),
1441            0, // exceeds buffer length
1442            [0b11110000u8, 0b00001111u8],
1443            0,
1444            100,
1445            |a, b| a & b,
1446        );
1447        assert_eq!(buffer.as_slice(), &[0b10101010u8, 0b01010101u8]);
1448    }
1449
1450    #[test]
1451    #[should_panic(expected = "offset + len out of bounds")]
1452    fn test_bitwise_binary_op_right_len_out_of_bounds() {
1453        let mut buffer = MutableBuffer::new(2); // space for 16 bits
1454        buffer.extend_from_slice(&[0b10101010u8, 0b01010101u8]); // only 2 bytes
1455        apply_bitwise_binary_op(
1456            buffer.as_slice_mut(),
1457            0, // exceeds buffer length
1458            [0b11110000u8, 0b00001111u8],
1459            1000,
1460            16,
1461            |a, b| a & b,
1462        );
1463        assert_eq!(buffer.as_slice(), &[0b10101010u8, 0b01010101u8]);
1464    }
1465
1466    #[test]
1467    #[should_panic(expected = "the len is 2 but the index is 12")]
1468    fn test_bitwise_unary_op_offset_out_of_bounds() {
1469        let input = vec![0b10101010u8, 0b01010101u8];
1470        let mut buffer = MutableBuffer::new(2); // space for 16 bits
1471        buffer.extend_from_slice(&input); // only 2 bytes
1472        apply_bitwise_unary_op(
1473            buffer.as_slice_mut(),
1474            100, // exceeds buffer length, becomes a noop
1475            8,
1476            |a| !a,
1477        );
1478        assert_eq!(buffer.as_slice(), &input);
1479    }
1480
1481    #[test]
1482    #[should_panic(expected = "assertion failed: last_offset <= buffer.len()")]
1483    fn test_bitwise_unary_op_length_out_of_bounds2() {
1484        let input = vec![0b10101010u8, 0b01010101u8];
1485        let mut buffer = MutableBuffer::new(2); // space for 16 bits
1486        buffer.extend_from_slice(&input); // only 2 bytes
1487        apply_bitwise_unary_op(
1488            buffer.as_slice_mut(),
1489            3,   // start at bit 3, to exercise different path
1490            100, // exceeds buffer length
1491            |a| !a,
1492        );
1493        assert_eq!(buffer.as_slice(), &input);
1494    }
1495}