arrow_buffer/util/
bit_iterator.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//! Types for iterating over packed bitmasks
19
20use crate::bit_chunk_iterator::{UnalignedBitChunk, UnalignedBitChunkIterator};
21use crate::bit_util::{ceil, get_bit_raw};
22
23/// Iterator over the bits within a packed bitmask
24///
25/// To efficiently iterate over just the set bits see [`BitIndexIterator`] and [`BitSliceIterator`]
26#[derive(Clone)]
27pub struct BitIterator<'a> {
28    buffer: &'a [u8],
29    current_offset: usize,
30    end_offset: usize,
31}
32
33impl<'a> BitIterator<'a> {
34    /// Create a new [`BitIterator`] from the provided `buffer`,
35    /// and `offset` and `len` in bits
36    ///
37    /// # Panic
38    ///
39    /// Panics if `buffer` is too short for the provided offset and length
40    pub fn new(buffer: &'a [u8], offset: usize, len: usize) -> Self {
41        let end_offset = offset.checked_add(len).unwrap();
42        let required_len = ceil(end_offset, 8);
43        assert!(
44            buffer.len() >= required_len,
45            "BitIterator buffer too small, expected {required_len} got {}",
46            buffer.len()
47        );
48
49        Self {
50            buffer,
51            current_offset: offset,
52            end_offset,
53        }
54    }
55}
56
57impl Iterator for BitIterator<'_> {
58    type Item = bool;
59
60    fn next(&mut self) -> Option<Self::Item> {
61        if self.current_offset == self.end_offset {
62            return None;
63        }
64        // Safety:
65        // offsets in bounds
66        let v = unsafe { get_bit_raw(self.buffer.as_ptr(), self.current_offset) };
67        self.current_offset += 1;
68        Some(v)
69    }
70
71    fn size_hint(&self) -> (usize, Option<usize>) {
72        let remaining_bits = self.end_offset - self.current_offset;
73        (remaining_bits, Some(remaining_bits))
74    }
75
76    fn count(self) -> usize
77    where
78        Self: Sized,
79    {
80        self.len()
81    }
82
83    fn nth(&mut self, n: usize) -> Option<Self::Item> {
84        // Check if we can advance to the desired offset.
85        // When n is 0 it means we want the next() value
86        // and when n is 1 we want the next().next() value
87        // so adding n to the current offset and not n - 1
88        match self.current_offset.checked_add(n) {
89            // Yes, and still within bounds
90            Some(new_offset) if new_offset < self.end_offset => {
91                self.current_offset = new_offset;
92            }
93
94            // Either overflow or would exceed end_offset
95            _ => {
96                self.current_offset = self.end_offset;
97                return None;
98            }
99        }
100
101        self.next()
102    }
103
104    fn last(mut self) -> Option<Self::Item> {
105        // If already at the end, return None
106        if self.current_offset == self.end_offset {
107            return None;
108        }
109
110        // Go to the one before the last bit
111        self.current_offset = self.end_offset - 1;
112
113        // Return the last bit
114        self.next()
115    }
116
117    fn max(self) -> Option<Self::Item>
118    where
119        Self: Sized,
120        Self::Item: Ord,
121    {
122        if self.current_offset == self.end_offset {
123            return None;
124        }
125
126        // true is greater than false so we only need to check if there's any true bit
127        let mut bit_index_iter = BitIndexIterator::new(
128            self.buffer,
129            self.current_offset,
130            self.end_offset - self.current_offset,
131        );
132
133        if bit_index_iter.next().is_some() {
134            return Some(true);
135        }
136
137        // We know the iterator is not empty and there are no set bits so false is the max
138        Some(false)
139    }
140}
141
142impl ExactSizeIterator for BitIterator<'_> {}
143
144impl DoubleEndedIterator for BitIterator<'_> {
145    fn next_back(&mut self) -> Option<Self::Item> {
146        if self.current_offset == self.end_offset {
147            return None;
148        }
149        self.end_offset -= 1;
150        // Safety:
151        // offsets in bounds
152        let v = unsafe { get_bit_raw(self.buffer.as_ptr(), self.end_offset) };
153        Some(v)
154    }
155
156    fn nth_back(&mut self, n: usize) -> Option<Self::Item> {
157        // Check if we can advance to the desired offset.
158        // When n is 0 it means we want the next_back() value
159        // and when n is 1 we want the next_back().next_back() value
160        // so subtracting n to the current offset and not n - 1
161        match self.end_offset.checked_sub(n) {
162            // Yes, and still within bounds
163            Some(new_offset) if self.current_offset < new_offset => {
164                self.end_offset = new_offset;
165            }
166
167            // Either underflow or would exceed current_offset
168            _ => {
169                self.current_offset = self.end_offset;
170                return None;
171            }
172        }
173
174        self.next_back()
175    }
176}
177
178/// Iterator of contiguous ranges of set bits within a provided packed bitmask
179///
180/// Returns `(usize, usize)` each representing an interval where the corresponding
181/// bits in the provides mask are set
182///
183/// the first value is the start of the range (inclusive) and the second value is the end of the range (exclusive)
184///
185#[derive(Debug)]
186pub struct BitSliceIterator<'a> {
187    iter: UnalignedBitChunkIterator<'a>,
188    len: usize,
189    current_offset: i64,
190    current_chunk: u64,
191}
192
193impl<'a> BitSliceIterator<'a> {
194    /// Create a new [`BitSliceIterator`] from the provided `buffer`,
195    /// and `offset` and `len` in bits
196    pub fn new(buffer: &'a [u8], offset: usize, len: usize) -> Self {
197        let chunk = UnalignedBitChunk::new(buffer, offset, len);
198        let mut iter = chunk.iter();
199
200        let current_offset = -(chunk.lead_padding() as i64);
201        let current_chunk = iter.next().unwrap_or(0);
202
203        Self {
204            iter,
205            len,
206            current_offset,
207            current_chunk,
208        }
209    }
210
211    /// Returns `Some((chunk_offset, bit_offset))` for the next chunk that has at
212    /// least one bit set, or None if there is no such chunk.
213    ///
214    /// Where `chunk_offset` is the bit offset to the current `u64` chunk
215    /// and `bit_offset` is the offset of the first `1` bit in that chunk
216    fn advance_to_set_bit(&mut self) -> Option<(i64, u32)> {
217        loop {
218            if self.current_chunk != 0 {
219                // Find the index of the first 1
220                let bit_pos = self.current_chunk.trailing_zeros();
221                return Some((self.current_offset, bit_pos));
222            }
223
224            self.current_chunk = self.iter.next()?;
225            self.current_offset += 64;
226        }
227    }
228}
229
230impl Iterator for BitSliceIterator<'_> {
231    type Item = (usize, usize);
232
233    fn next(&mut self) -> Option<Self::Item> {
234        // Used as termination condition
235        if self.len == 0 {
236            return None;
237        }
238
239        let (start_chunk, start_bit) = self.advance_to_set_bit()?;
240
241        // Set bits up to start
242        self.current_chunk |= (1 << start_bit) - 1;
243
244        loop {
245            if self.current_chunk != u64::MAX {
246                // Find the index of the first 0
247                let end_bit = self.current_chunk.trailing_ones();
248
249                // Zero out up to end_bit
250                self.current_chunk &= !((1 << end_bit) - 1);
251
252                return Some((
253                    (start_chunk + start_bit as i64) as usize,
254                    (self.current_offset + end_bit as i64) as usize,
255                ));
256            }
257
258            match self.iter.next() {
259                Some(next) => {
260                    self.current_chunk = next;
261                    self.current_offset += 64;
262                }
263                None => {
264                    return Some((
265                        (start_chunk + start_bit as i64) as usize,
266                        std::mem::replace(&mut self.len, 0),
267                    ));
268                }
269            }
270        }
271    }
272}
273
274/// An iterator of `usize` whose index in a provided bitmask is true
275///
276/// This provides the best performance on most masks, apart from those which contain
277/// large runs and therefore favour [`BitSliceIterator`]
278#[derive(Debug)]
279pub struct BitIndexIterator<'a> {
280    current_chunk: u64,
281    chunk_offset: i64,
282    iter: UnalignedBitChunkIterator<'a>,
283}
284
285impl<'a> BitIndexIterator<'a> {
286    /// Create a new [`BitIndexIterator`] from the provide `buffer`,
287    /// and `offset` and `len` in bits
288    pub fn new(buffer: &'a [u8], offset: usize, len: usize) -> Self {
289        let chunks = UnalignedBitChunk::new(buffer, offset, len);
290        let mut iter = chunks.iter();
291
292        let current_chunk = iter.next().unwrap_or(0);
293        let chunk_offset = -(chunks.lead_padding() as i64);
294
295        Self {
296            current_chunk,
297            chunk_offset,
298            iter,
299        }
300    }
301}
302
303impl Iterator for BitIndexIterator<'_> {
304    type Item = usize;
305
306    #[inline]
307    fn next(&mut self) -> Option<Self::Item> {
308        loop {
309            if self.current_chunk != 0 {
310                let bit_pos = self.current_chunk.trailing_zeros();
311                self.current_chunk ^= 1 << bit_pos;
312                return Some((self.chunk_offset + bit_pos as i64) as usize);
313            }
314
315            self.current_chunk = self.iter.next()?;
316            self.chunk_offset += 64;
317        }
318    }
319}
320
321/// An iterator of u32 whose index in a provided bitmask is true
322/// Respects arbitrary offsets and slice lead/trail padding exactly like BitIndexIterator
323#[derive(Debug)]
324pub struct BitIndexU32Iterator<'a> {
325    curr: u64,
326    chunk_offset: i64,
327    iter: UnalignedBitChunkIterator<'a>,
328}
329
330impl<'a> BitIndexU32Iterator<'a> {
331    /// Create a new [BitIndexU32Iterator] from the provided buffer,
332    /// offset and len in bits.
333    pub fn new(buffer: &'a [u8], offset: usize, len: usize) -> Self {
334        // Build the aligned chunks (including prefix/suffix masked)
335        let chunks = UnalignedBitChunk::new(buffer, offset, len);
336        let mut iter = chunks.iter();
337
338        // First 64-bit word (masked for lead padding), or 0 if empty
339        let curr = iter.next().unwrap_or(0);
340        // Negative lead padding ensures the first bit in curr maps to index 0
341        let chunk_offset = -(chunks.lead_padding() as i64);
342
343        Self {
344            curr,
345            chunk_offset,
346            iter,
347        }
348    }
349}
350
351impl<'a> Iterator for BitIndexU32Iterator<'a> {
352    type Item = u32;
353
354    #[inline(always)]
355    fn next(&mut self) -> Option<u32> {
356        loop {
357            if self.curr != 0 {
358                // Position of least-significant set bit
359                let tz = self.curr.trailing_zeros();
360                // Clear that bit
361                self.curr &= self.curr - 1;
362                // Return global index = chunk_offset + tz
363                return Some((self.chunk_offset + tz as i64) as u32);
364            }
365            // Advance to next 64-bit chunk
366            match self.iter.next() {
367                Some(next_chunk) => {
368                    // Move offset forward by 64 bits
369                    self.chunk_offset += 64;
370                    self.curr = next_chunk;
371                }
372                None => return None,
373            }
374        }
375    }
376}
377
378/// Calls the provided closure for each index in the provided null mask that is set,
379/// using an adaptive strategy based on the null count
380///
381/// Ideally this would be encapsulated in an [`Iterator`] that would determine the optimal
382/// strategy up front, and then yield indexes based on this.
383///
384/// Unfortunately, external iteration based on the resulting [`Iterator`] would match the strategy
385/// variant on each call to [`Iterator::next`], and LLVM generally cannot eliminate this.
386///
387/// One solution to this might be internal iteration, e.g. [`Iterator::try_fold`], however,
388/// it is currently [not possible] to override this for custom iterators in stable Rust.
389///
390/// As such this is the next best option
391///
392/// [not possible]: https://github.com/rust-lang/rust/issues/69595
393#[inline]
394pub fn try_for_each_valid_idx<E, F: FnMut(usize) -> Result<(), E>>(
395    len: usize,
396    offset: usize,
397    null_count: usize,
398    nulls: Option<&[u8]>,
399    f: F,
400) -> Result<(), E> {
401    let valid_count = len - null_count;
402
403    if valid_count == len {
404        (0..len).try_for_each(f)
405    } else if null_count != len {
406        BitIndexIterator::new(nulls.unwrap(), offset, len).try_for_each(f)
407    } else {
408        Ok(())
409    }
410}
411
412// Note: further tests located in arrow_select::filter module
413
414#[cfg(test)]
415mod tests {
416    use super::*;
417    use crate::BooleanBuffer;
418    use rand::rngs::StdRng;
419    use rand::{Rng, SeedableRng};
420    use std::fmt::Debug;
421    use std::iter::Copied;
422    use std::slice::Iter;
423
424    #[test]
425    fn test_bit_iterator_size_hint() {
426        let mut b = BitIterator::new(&[0b00000011], 0, 2);
427        assert_eq!(
428            b.size_hint(),
429            (2, Some(2)),
430            "Expected size_hint to be (2, Some(2))"
431        );
432
433        b.next();
434        assert_eq!(
435            b.size_hint(),
436            (1, Some(1)),
437            "Expected size_hint to be (1, Some(1)) after one bit consumed"
438        );
439
440        b.next();
441        assert_eq!(
442            b.size_hint(),
443            (0, Some(0)),
444            "Expected size_hint to be (0, Some(0)) after all bits consumed"
445        );
446    }
447
448    #[test]
449    fn test_bit_iterator() {
450        let mask = &[0b00010010, 0b00100011, 0b00000101, 0b00010001, 0b10010011];
451        let actual: Vec<_> = BitIterator::new(mask, 0, 5).collect();
452        assert_eq!(actual, &[false, true, false, false, true]);
453
454        let actual: Vec<_> = BitIterator::new(mask, 4, 5).collect();
455        assert_eq!(actual, &[true, false, false, false, true]);
456
457        let actual: Vec<_> = BitIterator::new(mask, 12, 14).collect();
458        assert_eq!(
459            actual,
460            &[
461                false, true, false, false, true, false, true, false, false, false, false, false,
462                true, false
463            ]
464        );
465
466        assert_eq!(BitIterator::new(mask, 0, 0).count(), 0);
467        assert_eq!(BitIterator::new(mask, 40, 0).count(), 0);
468    }
469
470    #[test]
471    #[should_panic(expected = "BitIterator buffer too small, expected 3 got 2")]
472    fn test_bit_iterator_bounds() {
473        let mask = &[223, 23];
474        BitIterator::new(mask, 17, 0);
475    }
476
477    #[test]
478    fn test_bit_index_u32_iterator_basic() {
479        let mask = &[0b00010010, 0b00100011];
480
481        let result: Vec<u32> = BitIndexU32Iterator::new(mask, 0, 16).collect();
482        let expected: Vec<u32> = BitIndexIterator::new(mask, 0, 16)
483            .map(|i| i as u32)
484            .collect();
485        assert_eq!(result, expected);
486
487        let result: Vec<u32> = BitIndexU32Iterator::new(mask, 4, 8).collect();
488        let expected: Vec<u32> = BitIndexIterator::new(mask, 4, 8)
489            .map(|i| i as u32)
490            .collect();
491        assert_eq!(result, expected);
492
493        let result: Vec<u32> = BitIndexU32Iterator::new(mask, 10, 4).collect();
494        let expected: Vec<u32> = BitIndexIterator::new(mask, 10, 4)
495            .map(|i| i as u32)
496            .collect();
497        assert_eq!(result, expected);
498
499        let result: Vec<u32> = BitIndexU32Iterator::new(mask, 0, 0).collect();
500        let expected: Vec<u32> = BitIndexIterator::new(mask, 0, 0)
501            .map(|i| i as u32)
502            .collect();
503        assert_eq!(result, expected);
504    }
505
506    #[test]
507    fn test_bit_index_u32_iterator_all_set() {
508        let mask = &[0xFF, 0xFF];
509        let result: Vec<u32> = BitIndexU32Iterator::new(mask, 0, 16).collect();
510        let expected: Vec<u32> = BitIndexIterator::new(mask, 0, 16)
511            .map(|i| i as u32)
512            .collect();
513        assert_eq!(result, expected);
514    }
515
516    #[test]
517    fn test_bit_index_u32_iterator_none_set() {
518        let mask = &[0x00, 0x00];
519        let result: Vec<u32> = BitIndexU32Iterator::new(mask, 0, 16).collect();
520        let expected: Vec<u32> = BitIndexIterator::new(mask, 0, 16)
521            .map(|i| i as u32)
522            .collect();
523        assert_eq!(result, expected);
524    }
525
526    #[test]
527    fn test_bit_index_u32_cross_chunk() {
528        let mut buf = vec![0u8; 16];
529        for bit in 60..68 {
530            let byte = (bit / 8) as usize;
531            let bit_in_byte = bit % 8;
532            buf[byte] |= 1 << bit_in_byte;
533        }
534        let offset = 58;
535        let len = 10;
536
537        let result: Vec<u32> = BitIndexU32Iterator::new(&buf, offset, len).collect();
538        let expected: Vec<u32> = BitIndexIterator::new(&buf, offset, len)
539            .map(|i| i as u32)
540            .collect();
541        assert_eq!(result, expected);
542    }
543
544    #[test]
545    fn test_bit_index_u32_unaligned_offset() {
546        let mask = &[0b0110_1100, 0b1010_0000];
547        let offset = 2;
548        let len = 12;
549
550        let result: Vec<u32> = BitIndexU32Iterator::new(mask, offset, len).collect();
551        let expected: Vec<u32> = BitIndexIterator::new(mask, offset, len)
552            .map(|i| i as u32)
553            .collect();
554        assert_eq!(result, expected);
555    }
556
557    #[test]
558    fn test_bit_index_u32_long_all_set() {
559        let len = 200;
560        let num_bytes = len / 8 + if len % 8 != 0 { 1 } else { 0 };
561        let bytes = vec![0xFFu8; num_bytes];
562
563        let result: Vec<u32> = BitIndexU32Iterator::new(&bytes, 0, len).collect();
564        let expected: Vec<u32> = BitIndexIterator::new(&bytes, 0, len)
565            .map(|i| i as u32)
566            .collect();
567        assert_eq!(result, expected);
568    }
569
570    #[test]
571    fn test_bit_index_u32_none_set() {
572        let len = 50;
573        let num_bytes = len / 8 + if len % 8 != 0 { 1 } else { 0 };
574        let bytes = vec![0u8; num_bytes];
575
576        let result: Vec<u32> = BitIndexU32Iterator::new(&bytes, 0, len).collect();
577        let expected: Vec<u32> = BitIndexIterator::new(&bytes, 0, len)
578            .map(|i| i as u32)
579            .collect();
580        assert_eq!(result, expected);
581    }
582
583    trait SharedBetweenBitIteratorAndSliceIter:
584        ExactSizeIterator<Item = bool> + DoubleEndedIterator<Item = bool>
585    {
586    }
587    impl<T: ?Sized + ExactSizeIterator<Item = bool> + DoubleEndedIterator<Item = bool>>
588        SharedBetweenBitIteratorAndSliceIter for T
589    {
590    }
591
592    fn get_bit_iterator_cases() -> impl Iterator<Item = (BooleanBuffer, Vec<bool>)> {
593        let mut rng = StdRng::seed_from_u64(42);
594
595        [0, 1, 6, 8, 100, 164]
596            .map(|len| {
597                let source = (0..len).map(|_| rng.random_bool(0.5)).collect::<Vec<_>>();
598
599                (BooleanBuffer::from(source.as_slice()), source)
600            })
601            .into_iter()
602    }
603
604    fn setup_and_assert(
605        setup_iters: impl Fn(&mut dyn SharedBetweenBitIteratorAndSliceIter),
606        assert_fn: impl Fn(BitIterator, Copied<Iter<bool>>),
607    ) {
608        for (boolean_buffer, source) in get_bit_iterator_cases() {
609            // Not using `boolean_buffer.iter()` in case the implementation change to not call BitIterator internally
610            // in which case the test would not test what it intends to test
611            let mut actual = BitIterator::new(boolean_buffer.values(), 0, boolean_buffer.len());
612            let mut expected = source.iter().copied();
613
614            setup_iters(&mut actual);
615            setup_iters(&mut expected);
616
617            assert_fn(actual, expected);
618        }
619    }
620
621    /// Trait representing an operation on a BitIterator
622    /// that can be compared against a slice iterator
623    trait BitIteratorOp {
624        /// What the operation returns (e.g. Option<bool> for last/max, usize for count, etc)
625        type Output: PartialEq + Debug;
626
627        /// The name of the operation, used for error messages
628        const NAME: &'static str;
629
630        /// Get the value of the operation for the provided iterator
631        /// This will be either a BitIterator or a slice iterator to make sure they produce the same result
632        fn get_value<T: SharedBetweenBitIteratorAndSliceIter>(iter: T) -> Self::Output;
633    }
634
635    /// Helper function that will assert that the provided operation
636    /// produces the same result for both BitIterator and slice iterator
637    /// under various consumption patterns (e.g. some calls to next/next_back/consume_all/etc)
638    fn assert_bit_iterator_cases<O: BitIteratorOp>() {
639        setup_and_assert(
640            |_iter: &mut dyn SharedBetweenBitIteratorAndSliceIter| {},
641            |actual, expected| {
642                let current_iterator_values: Vec<bool> = expected.clone().collect();
643                assert_eq!(
644                    O::get_value(actual),
645                    O::get_value(expected),
646                    "Failed on op {} for new iter (left actual, right expected) ({current_iterator_values:?})",
647                    O::NAME
648                );
649            },
650        );
651
652        setup_and_assert(
653            |iter: &mut dyn SharedBetweenBitIteratorAndSliceIter| {
654                iter.next();
655            },
656            |actual, expected| {
657                let current_iterator_values: Vec<bool> = expected.clone().collect();
658
659                assert_eq!(
660                    O::get_value(actual),
661                    O::get_value(expected),
662                    "Failed on op {} for new iter after consuming 1 element from the start (left actual, right expected) ({current_iterator_values:?})",
663                    O::NAME
664                );
665            },
666        );
667
668        setup_and_assert(
669            |iter: &mut dyn SharedBetweenBitIteratorAndSliceIter| {
670                iter.next_back();
671            },
672            |actual, expected| {
673                let current_iterator_values: Vec<bool> = expected.clone().collect();
674
675                assert_eq!(
676                    O::get_value(actual),
677                    O::get_value(expected),
678                    "Failed on op {} for new iter after consuming 1 element from the end (left actual, right expected) ({current_iterator_values:?})",
679                    O::NAME
680                );
681            },
682        );
683
684        setup_and_assert(
685            |iter: &mut dyn SharedBetweenBitIteratorAndSliceIter| {
686                iter.next();
687                iter.next_back();
688            },
689            |actual, expected| {
690                let current_iterator_values: Vec<bool> = expected.clone().collect();
691
692                assert_eq!(
693                    O::get_value(actual),
694                    O::get_value(expected),
695                    "Failed on op {} for new iter after consuming 1 element from start and end (left actual, right expected) ({current_iterator_values:?})",
696                    O::NAME
697                );
698            },
699        );
700
701        setup_and_assert(
702            |iter: &mut dyn SharedBetweenBitIteratorAndSliceIter| {
703                while iter.len() > 1 {
704                    iter.next();
705                }
706            },
707            |actual, expected| {
708                let current_iterator_values: Vec<bool> = expected.clone().collect();
709
710                assert_eq!(
711                    O::get_value(actual),
712                    O::get_value(expected),
713                    "Failed on op {} for new iter after consuming all from the start but 1 (left actual, right expected) ({current_iterator_values:?})",
714                    O::NAME
715                );
716            },
717        );
718
719        setup_and_assert(
720            |iter: &mut dyn SharedBetweenBitIteratorAndSliceIter| {
721                while iter.len() > 1 {
722                    iter.next_back();
723                }
724            },
725            |actual, expected| {
726                let current_iterator_values: Vec<bool> = expected.clone().collect();
727
728                assert_eq!(
729                    O::get_value(actual),
730                    O::get_value(expected),
731                    "Failed on op {} for new iter after consuming all from the end but 1 (left actual, right expected) ({current_iterator_values:?})",
732                    O::NAME
733                );
734            },
735        );
736
737        setup_and_assert(
738            |iter: &mut dyn SharedBetweenBitIteratorAndSliceIter| {
739                while iter.next().is_some() {}
740            },
741            |actual, expected| {
742                let current_iterator_values: Vec<bool> = expected.clone().collect();
743
744                assert_eq!(
745                    O::get_value(actual),
746                    O::get_value(expected),
747                    "Failed on op {} for new iter after consuming all from the start (left actual, right expected) ({current_iterator_values:?})",
748                    O::NAME
749                );
750            },
751        );
752
753        setup_and_assert(
754            |iter: &mut dyn SharedBetweenBitIteratorAndSliceIter| {
755                while iter.next_back().is_some() {}
756            },
757            |actual, expected| {
758                let current_iterator_values: Vec<bool> = expected.clone().collect();
759
760                assert_eq!(
761                    O::get_value(actual),
762                    O::get_value(expected),
763                    "Failed on op {} for new iter after consuming all from the end (left actual, right expected) ({current_iterator_values:?})",
764                    O::NAME
765                );
766            },
767        );
768    }
769
770    #[test]
771    fn assert_bit_iterator_count() {
772        struct CountOp;
773
774        impl BitIteratorOp for CountOp {
775            type Output = usize;
776            const NAME: &'static str = "count";
777
778            fn get_value<T: SharedBetweenBitIteratorAndSliceIter>(iter: T) -> Self::Output {
779                iter.count()
780            }
781        }
782
783        assert_bit_iterator_cases::<CountOp>()
784    }
785
786    #[test]
787    fn assert_bit_iterator_last() {
788        struct LastOp;
789
790        impl BitIteratorOp for LastOp {
791            type Output = Option<bool>;
792            const NAME: &'static str = "last";
793
794            fn get_value<T: SharedBetweenBitIteratorAndSliceIter>(iter: T) -> Self::Output {
795                iter.last()
796            }
797        }
798
799        assert_bit_iterator_cases::<LastOp>()
800    }
801
802    #[test]
803    fn assert_bit_iterator_max() {
804        struct MaxOp;
805
806        impl BitIteratorOp for MaxOp {
807            type Output = Option<bool>;
808            const NAME: &'static str = "max";
809
810            fn get_value<T: SharedBetweenBitIteratorAndSliceIter>(iter: T) -> Self::Output {
811                iter.max()
812            }
813        }
814
815        assert_bit_iterator_cases::<MaxOp>()
816    }
817
818    #[test]
819    fn assert_bit_iterator_nth_0() {
820        struct NthOp<const BACK: bool>;
821
822        impl<const BACK: bool> BitIteratorOp for NthOp<BACK> {
823            type Output = Option<bool>;
824            const NAME: &'static str = if BACK { "nth_back(0)" } else { "nth(0)" };
825
826            fn get_value<T: SharedBetweenBitIteratorAndSliceIter>(mut iter: T) -> Self::Output {
827                if BACK { iter.nth_back(0) } else { iter.nth(0) }
828            }
829        }
830
831        assert_bit_iterator_cases::<NthOp<false>>();
832        assert_bit_iterator_cases::<NthOp<true>>();
833    }
834
835    #[test]
836    fn assert_bit_iterator_nth_1() {
837        struct NthOp<const BACK: bool>;
838
839        impl<const BACK: bool> BitIteratorOp for NthOp<BACK> {
840            type Output = Option<bool>;
841            const NAME: &'static str = if BACK { "nth_back(1)" } else { "nth(1)" };
842
843            fn get_value<T: SharedBetweenBitIteratorAndSliceIter>(mut iter: T) -> Self::Output {
844                if BACK { iter.nth_back(1) } else { iter.nth(1) }
845            }
846        }
847
848        assert_bit_iterator_cases::<NthOp<false>>();
849        assert_bit_iterator_cases::<NthOp<true>>();
850    }
851
852    #[test]
853    fn assert_bit_iterator_nth_after_end() {
854        struct NthOp<const BACK: bool>;
855
856        impl<const BACK: bool> BitIteratorOp for NthOp<BACK> {
857            type Output = Option<bool>;
858            const NAME: &'static str = if BACK {
859                "nth_back(iter.len() + 1)"
860            } else {
861                "nth(iter.len() + 1)"
862            };
863
864            fn get_value<T: SharedBetweenBitIteratorAndSliceIter>(mut iter: T) -> Self::Output {
865                if BACK {
866                    iter.nth_back(iter.len() + 1)
867                } else {
868                    iter.nth(iter.len() + 1)
869                }
870            }
871        }
872
873        assert_bit_iterator_cases::<NthOp<false>>();
874        assert_bit_iterator_cases::<NthOp<true>>();
875    }
876
877    #[test]
878    fn assert_bit_iterator_nth_len() {
879        struct NthOp<const BACK: bool>;
880
881        impl<const BACK: bool> BitIteratorOp for NthOp<BACK> {
882            type Output = Option<bool>;
883            const NAME: &'static str = if BACK {
884                "nth_back(iter.len())"
885            } else {
886                "nth(iter.len())"
887            };
888
889            fn get_value<T: SharedBetweenBitIteratorAndSliceIter>(mut iter: T) -> Self::Output {
890                if BACK {
891                    iter.nth_back(iter.len())
892                } else {
893                    iter.nth(iter.len())
894                }
895            }
896        }
897
898        assert_bit_iterator_cases::<NthOp<false>>();
899        assert_bit_iterator_cases::<NthOp<true>>();
900    }
901
902    #[test]
903    fn assert_bit_iterator_nth_last() {
904        struct NthOp<const BACK: bool>;
905
906        impl<const BACK: bool> BitIteratorOp for NthOp<BACK> {
907            type Output = Option<bool>;
908            const NAME: &'static str = if BACK {
909                "nth_back(iter.len().saturating_sub(1))"
910            } else {
911                "nth(iter.len().saturating_sub(1))"
912            };
913
914            fn get_value<T: SharedBetweenBitIteratorAndSliceIter>(mut iter: T) -> Self::Output {
915                if BACK {
916                    iter.nth_back(iter.len().saturating_sub(1))
917                } else {
918                    iter.nth(iter.len().saturating_sub(1))
919                }
920            }
921        }
922
923        assert_bit_iterator_cases::<NthOp<false>>();
924        assert_bit_iterator_cases::<NthOp<true>>();
925    }
926
927    #[test]
928    fn assert_bit_iterator_nth_and_reuse() {
929        setup_and_assert(
930            |_| {},
931            |actual, expected| {
932                {
933                    let mut actual = actual.clone();
934                    let mut expected = expected.clone();
935                    for _ in 0..expected.len() {
936                        #[allow(clippy::iter_nth_zero)]
937                        let actual_val = actual.nth(0);
938                        #[allow(clippy::iter_nth_zero)]
939                        let expected_val = expected.nth(0);
940                        assert_eq!(actual_val, expected_val, "Failed on nth(0)");
941                    }
942                }
943
944                {
945                    let mut actual = actual.clone();
946                    let mut expected = expected.clone();
947                    for _ in 0..expected.len() {
948                        let actual_val = actual.nth(1);
949                        let expected_val = expected.nth(1);
950                        assert_eq!(actual_val, expected_val, "Failed on nth(1)");
951                    }
952                }
953
954                {
955                    let mut actual = actual.clone();
956                    let mut expected = expected.clone();
957                    for _ in 0..expected.len() {
958                        let actual_val = actual.nth(2);
959                        let expected_val = expected.nth(2);
960                        assert_eq!(actual_val, expected_val, "Failed on nth(2)");
961                    }
962                }
963            },
964        );
965    }
966
967    #[test]
968    fn assert_bit_iterator_nth_back_and_reuse() {
969        setup_and_assert(
970            |_| {},
971            |actual, expected| {
972                {
973                    let mut actual = actual.clone();
974                    let mut expected = expected.clone();
975                    for _ in 0..expected.len() {
976                        #[allow(clippy::iter_nth_zero)]
977                        let actual_val = actual.nth_back(0);
978                        let expected_val = expected.nth_back(0);
979                        assert_eq!(actual_val, expected_val, "Failed on nth_back(0)");
980                    }
981                }
982
983                {
984                    let mut actual = actual.clone();
985                    let mut expected = expected.clone();
986                    for _ in 0..expected.len() {
987                        let actual_val = actual.nth_back(1);
988                        let expected_val = expected.nth_back(1);
989                        assert_eq!(actual_val, expected_val, "Failed on nth_back(1)");
990                    }
991                }
992
993                {
994                    let mut actual = actual.clone();
995                    let mut expected = expected.clone();
996                    for _ in 0..expected.len() {
997                        let actual_val = actual.nth_back(2);
998                        let expected_val = expected.nth_back(2);
999                        assert_eq!(actual_val, expected_val, "Failed on nth_back(2)");
1000                    }
1001                }
1002            },
1003        );
1004    }
1005}