Skip to content

Commit

Permalink
BooleanBufferBuilder::append_packed (#1038) (#1039)
Browse files Browse the repository at this point in the history
* BooleanBufferBuilder::append_packed (#1038)

* Update docstring

* Add packed_append_range

* Fix capacity

* Use set_bits from transform::util

* Add license

* Format
  • Loading branch information
tustvold authored Jan 11, 2022
1 parent 76fde56 commit e2130d9
Show file tree
Hide file tree
Showing 8 changed files with 312 additions and 184 deletions.
4 changes: 4 additions & 0 deletions arrow/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,10 @@ harness = false
name = "boolean_kernels"
harness = false

[[bench]]
name = "boolean_append_packed"
harness = false

[[bench]]
name = "arithmetic_kernels"
harness = false
Expand Down
55 changes: 55 additions & 0 deletions arrow/benches/boolean_append_packed.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

use arrow::array::BooleanBufferBuilder;
use criterion::{criterion_group, criterion_main, Criterion};
use rand::{thread_rng, Rng};

fn rand_bytes(len: usize) -> Vec<u8> {
let mut rng = thread_rng();
let mut buf = vec![0_u8; len];
rng.fill(buf.as_mut_slice());
buf
}

fn boolean_append_packed(c: &mut Criterion) {
let mut rng = thread_rng();
let source = rand_bytes(1024);
let ranges: Vec<_> = (0..100)
.into_iter()
.map(|_| {
let start: usize = rng.gen_range(0..1024 * 8);
let end: usize = rng.gen_range(start..1024 * 8);
start..end
})
.collect();

let total_bits: usize = ranges.iter().map(|x| x.end - x.start).sum();

c.bench_function("boolean_append_packed", |b| {
b.iter(|| {
let mut buffer = BooleanBufferBuilder::new(total_bits);
for range in &ranges {
buffer.append_packed_range(range.clone(), &source);
}
assert_eq!(buffer.len(), total_bits);
})
});
}

criterion_group!(benches, boolean_append_packed);
criterion_main!(benches);
57 changes: 57 additions & 0 deletions arrow/src/array/builder.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ use std::collections::HashMap;
use std::fmt;
use std::marker::PhantomData;
use std::mem;
use std::ops::Range;
use std::sync::Arc;

use crate::array::*;
Expand Down Expand Up @@ -398,6 +399,26 @@ impl BooleanBufferBuilder {
}
}

/// Append `range` bits from `to_set`
///
/// `to_set` is a slice of bits packed LSB-first into `[u8]`
///
/// # Panics
///
/// Panics if `to_set` does not contain `ceil(range.end / 8)` bytes
pub fn append_packed_range(&mut self, range: Range<usize>, to_set: &[u8]) {
let offset_write = self.len;
let len = range.end - range.start;
self.advance(len);
crate::util::bit_mask::set_bits(
self.buffer.as_slice_mut(),
to_set,
offset_write,
range.start,
len,
);
}

#[inline]
pub fn finish(&mut self) -> Buffer {
let buf = std::mem::replace(&mut self.buffer, MutableBuffer::new(0));
Expand Down Expand Up @@ -2843,6 +2864,42 @@ mod tests {
assert!(buffer.get_bit(11));
}

#[test]
fn test_bool_buffer_fuzz() {
use rand::prelude::*;

let mut buffer = BooleanBufferBuilder::new(12);
let mut all_bools = vec![];
let mut rng = rand::thread_rng();

let src_len = 32;
let (src, compacted_src) = {
let src: Vec<_> = std::iter::from_fn(|| Some(rng.next_u32() & 1 == 0))
.take(src_len)
.collect();

let mut compacted_src = BooleanBufferBuilder::new(src_len);
compacted_src.append_slice(&src);
(src, compacted_src.finish())
};

for _ in 0..100 {
let a = rng.next_u32() as usize % src_len;
let b = rng.next_u32() as usize % src_len;

let start = a.min(b);
let end = a.max(b);

buffer.append_packed_range(start..end, compacted_src.as_slice());
all_bools.extend_from_slice(&src[start..end]);
}

let mut compacted = BooleanBufferBuilder::new(all_bools.len());
compacted.append_slice(&all_bools);

assert_eq!(buffer.finish(), compacted.finish())
}

#[test]
fn test_boolean_array_builder_append_slice() {
let arr1 =
Expand Down
7 changes: 2 additions & 5 deletions arrow/src/array/transform/boolean.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,12 +15,9 @@
// specific language governing permissions and limitations
// under the License.

use super::{Extend, _MutableArrayData, utils::resize_for_bits};
use crate::array::ArrayData;

use super::{
Extend, _MutableArrayData,
utils::{resize_for_bits, set_bits},
};
use crate::util::bit_mask::set_bits;

pub(super) fn build_extend(array: &ArrayData) -> Extend {
let values = array.buffers()[0].as_slice();
Expand Down
2 changes: 1 addition & 1 deletion arrow/src/array/transform/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ fn build_extend_null_bits(array: &ArrayData, use_nulls: bool) -> ExtendNullBits
let bytes = bitmap.bits.as_slice();
Box::new(move |mutable, start, len| {
utils::resize_for_bits(&mut mutable.null_buffer, mutable.len + len);
mutable.null_count += utils::set_bits(
mutable.null_count += crate::util::bit_mask::set_bits(
mutable.null_buffer.as_slice_mut(),
bytes,
mutable.len,
Expand Down
179 changes: 1 addition & 178 deletions arrow/src/array/transform/utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,14 +15,7 @@
// specific language governing permissions and limitations
// under the License.

use crate::{
array::OffsetSizeTrait,
buffer::MutableBuffer,
util::{
bit_chunk_iterator::BitChunks,
bit_util::{self, ceil},
},
};
use crate::{array::OffsetSizeTrait, buffer::MutableBuffer, util::bit_util};

/// extends the `buffer` to be able to hold `len` bits, setting all bits of the new size to zero.
#[inline]
Expand All @@ -33,49 +26,6 @@ pub(super) fn resize_for_bits(buffer: &mut MutableBuffer, len: usize) {
}
}

/// sets all bits on `write_data` on the range `[offset_write..offset_write+len]` to be equal to the
/// bits on `data` on the range `[offset_read..offset_read+len]`
/// returns the number of `0` bits `data[offset_read..offset_read+len]`
pub(super) fn set_bits(
write_data: &mut [u8],
data: &[u8],
offset_write: usize,
offset_read: usize,
len: usize,
) -> usize {
let mut null_count = 0;

let mut bits_to_align = offset_write % 8;
if bits_to_align > 0 {
bits_to_align = std::cmp::min(len, 8 - bits_to_align);
}
let mut write_byte_index = ceil(offset_write + bits_to_align, 8);

// Set full bytes provided by bit chunk iterator (which iterates in 64 bits at a time)
let chunks = BitChunks::new(data, offset_read + bits_to_align, len - bits_to_align);
chunks.iter().for_each(|chunk| {
null_count += chunk.count_zeros();
chunk.to_le_bytes().iter().for_each(|b| {
write_data[write_byte_index] = *b;
write_byte_index += 1;
})
});

// Set individual bits both to align write_data to a byte offset and the remainder bits not covered by the bit chunk iterator
let remainder_offset = len - chunks.remainder_len();
(0..bits_to_align)
.chain(remainder_offset..len)
.for_each(|i| {
if bit_util::get_bit(data, offset_read + i) {
bit_util::set_bit(write_data, offset_write + i);
} else {
null_count += 1;
}
});

null_count as usize
}

pub(super) fn extend_offsets<T: OffsetSizeTrait>(
buffer: &mut MutableBuffer,
mut last_offset: T,
Expand Down Expand Up @@ -104,130 +54,3 @@ pub(super) unsafe fn get_last_offset<T: OffsetSizeTrait>(
debug_assert!(prefix.is_empty() && suffix.is_empty());
*offsets.get_unchecked(offsets.len() - 1)
}

#[cfg(test)]
mod tests {
use super::*;

#[test]
fn test_set_bits_aligned() {
let mut destination: Vec<u8> = vec![0, 0, 0, 0, 0, 0, 0, 0, 0, 0];
let source: &[u8] = &[
0b11100111, 0b10100101, 0b10011001, 0b11011011, 0b11101011, 0b11000011,
0b11100111, 0b10100101,
];

let destination_offset = 8;
let source_offset = 0;

let len = 64;

let expected_data: &[u8] = &[
0, 0b11100111, 0b10100101, 0b10011001, 0b11011011, 0b11101011, 0b11000011,
0b11100111, 0b10100101, 0,
];
let expected_null_count = 24;
let result = set_bits(
destination.as_mut_slice(),
source,
destination_offset,
source_offset,
len,
);

assert_eq!(destination, expected_data);
assert_eq!(result, expected_null_count);
}

#[test]
fn test_set_bits_unaligned_destination_start() {
let mut destination: Vec<u8> = vec![0, 0, 0, 0, 0, 0, 0, 0, 0, 0];
let source: &[u8] = &[
0b11100111, 0b10100101, 0b10011001, 0b11011011, 0b11101011, 0b11000011,
0b11100111, 0b10100101,
];

let destination_offset = 3;
let source_offset = 0;

let len = 64;

let expected_data: &[u8] = &[
0b00111000, 0b00101111, 0b11001101, 0b11011100, 0b01011110, 0b00011111,
0b00111110, 0b00101111, 0b00000101, 0b00000000,
];
let expected_null_count = 24;
let result = set_bits(
destination.as_mut_slice(),
source,
destination_offset,
source_offset,
len,
);

assert_eq!(destination, expected_data);
assert_eq!(result, expected_null_count);
}

#[test]
fn test_set_bits_unaligned_destination_end() {
let mut destination: Vec<u8> = vec![0, 0, 0, 0, 0, 0, 0, 0, 0, 0];
let source: &[u8] = &[
0b11100111, 0b10100101, 0b10011001, 0b11011011, 0b11101011, 0b11000011,
0b11100111, 0b10100101,
];

let destination_offset = 8;
let source_offset = 0;

let len = 62;

let expected_data: &[u8] = &[
0, 0b11100111, 0b10100101, 0b10011001, 0b11011011, 0b11101011, 0b11000011,
0b11100111, 0b00100101, 0,
];
let expected_null_count = 23;
let result = set_bits(
destination.as_mut_slice(),
source,
destination_offset,
source_offset,
len,
);

assert_eq!(destination, expected_data);
assert_eq!(result, expected_null_count);
}

#[test]
fn test_set_bits_unaligned() {
let mut destination: Vec<u8> = vec![0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0];
let source: &[u8] = &[
0b11100111, 0b10100101, 0b10011001, 0b11011011, 0b11101011, 0b11000011,
0b11100111, 0b10100101, 0b10011001, 0b11011011, 0b11101011, 0b11000011,
0b11100111, 0b10100101, 0b10011001, 0b11011011, 0b11101011, 0b11000011,
];

let destination_offset = 3;
let source_offset = 5;

let len = 95;

let expected_data: &[u8] = &[
0b01111000, 0b01101001, 0b11100110, 0b11110110, 0b11111010, 0b11110000,
0b01111001, 0b01101001, 0b11100110, 0b11110110, 0b11111010, 0b11110000,
0b00000001,
];
let expected_null_count = 35;
let result = set_bits(
destination.as_mut_slice(),
source,
destination_offset,
source_offset,
len,
);

assert_eq!(destination, expected_data);
assert_eq!(result, expected_null_count);
}
}
Loading

0 comments on commit e2130d9

Please sign in to comment.