rluvaton commented on code in PR #8619: URL: https://github.com/apache/arrow-rs/pull/8619#discussion_r2440302646
########## arrow-buffer/src/buffer/mutable_ops.rs: ########## @@ -0,0 +1,1256 @@ +// 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 super::{Buffer, MutableBuffer}; +use crate::BooleanBufferBuilder; +use crate::bit_chunk_iterator::BitChunks; +use crate::util::bit_util; + +/// What can be used as the right-hand side (RHS) buffer in mutable operations. +/// +/// this is not mutated. +/// +/// # Implementation notes +/// +/// ## Why `pub(crate)`? +/// This is because we don't want this trait to expose the inner buffer to the public. +/// this is the trait implementor choice. +/// +pub(crate) trait BufferSupportedRhs { + fn as_slice(&self) -> &[u8]; +} + +impl BufferSupportedRhs for Buffer { + fn as_slice(&self) -> &[u8] { + self.as_slice() + } +} + +impl BufferSupportedRhs for MutableBuffer { + fn as_slice(&self) -> &[u8] { + self.as_slice() + } +} + +impl BufferSupportedRhs for BooleanBufferBuilder { + fn as_slice(&self) -> &[u8] { + self.as_slice() + } +} + +/// Trait that will be operated on as the left-hand side (LHS) buffer in mutable operations. +/// +/// This consumer of the trait must satisfies the following guarantees: +/// 1. It will not change the length of the buffer. +/// +/// # Implementation notes +/// +/// ## Why is this trait `pub(crate)`? +/// Because we don't wanna expose the inner mutable buffer to the public. +/// as this is the choice of the implementor of the trait and sometimes it is not desirable +/// (e.g. `BooleanBufferBuilder`). +/// +/// ## Why this trait is needed, can't we just use `MutableBuffer` directly? +/// Sometimes we don't want to expose the inner `MutableBuffer` +/// so it can't be misused. +/// +/// For example, [`BooleanBufferBuilder`] does not expose the inner `MutableBuffer` +/// as exposing it will allow the user to change the length of the buffer that will make the +/// `BooleanBufferBuilder` invalid. +/// +pub(crate) trait MutableOpsBufferSupportedLhs { + /// Get a mutable reference to the inner `MutableBuffer`. + /// + /// This is used to perform in-place operations on the buffer. + /// + /// the caller must ensure that the length of the buffer is not changed. + fn inner_mutable_buffer(&mut self) -> &mut MutableBuffer; +} + +impl MutableOpsBufferSupportedLhs for MutableBuffer { + fn inner_mutable_buffer(&mut self) -> &mut MutableBuffer { + self + } +} + +/// Apply a binary bitwise operation to two bit-packed buffers. +/// +/// This is the main entry point for binary operations. It handles both byte-aligned +/// and non-byte-aligned cases by delegating to specialized helper functions. +/// +/// # Arguments +/// +/// * `left` - The left mutable buffer to be modified in-place +/// * `left_offset_in_bits` - Starting bit offset in the left buffer +/// * `right` - The right buffer (as byte slice) +/// * `right_offset_in_bits` - Starting bit offset in the right buffer +/// * `len_in_bits` - Number of bits to process +/// * `op` - Binary operation to apply (e.g., `|a, b| a & b`) +/// +#[allow( + private_bounds, + reason = "MutableOpsBufferSupportedLhs and BufferSupportedRhs exposes the inner internals which is the implementor choice and we dont want to leak internals" +)] +pub fn mutable_bitwise_bin_op_helper<F>( + left: &mut impl MutableOpsBufferSupportedLhs, + left_offset_in_bits: usize, + right: &impl BufferSupportedRhs, + right_offset_in_bits: usize, + len_in_bits: usize, + mut op: F, +) where + F: FnMut(u64, u64) -> u64, +{ + if len_in_bits == 0 { + return; + } + + let mutable_buffer = left.inner_mutable_buffer(); + + let mutable_buffer_len = mutable_buffer.len(); + let mutable_buffer_cap = mutable_buffer.capacity(); + + // offset inside a byte + let left_bit_offset = left_offset_in_bits % 8; + + let is_mutable_buffer_byte_aligned = left_bit_offset == 0; + + if is_mutable_buffer_byte_aligned { + mutable_buffer_byte_aligned_bitwise_bin_op_helper( + mutable_buffer, + left_offset_in_bits, + right, + right_offset_in_bits, + len_in_bits, + op, + ); + } else { + // If we are not byte aligned, run `op` on the first few bits to reach byte alignment + let bits_to_next_byte = 8 - left_bit_offset; + + { + let right_byte_offset = right_offset_in_bits / 8; + + // Read the same amount of bits from the right buffer + let right_first_byte: u8 = read_up_to_byte_from_offset( + &right.as_slice()[right_byte_offset..], + bits_to_next_byte, + // Right bit offset + right_offset_in_bits % 8, + ); + + align_to_byte( + // Hope it gets inlined + &mut |left| op(left, right_first_byte as u64), + mutable_buffer, + left_offset_in_bits, + ); + } + + let left_offset_in_bits = left_offset_in_bits + bits_to_next_byte; + let right_offset_in_bits = right_offset_in_bits + bits_to_next_byte; + let len_in_bits = len_in_bits.saturating_sub(bits_to_next_byte); + + if len_in_bits == 0 { + // Making sure that our guarantee that the length and capacity of the mutable buffer + // will not change is upheld + assert_eq!( + mutable_buffer.len(), + mutable_buffer_len, + "The length of the mutable buffer must not change" + ); + assert_eq!( + mutable_buffer.capacity(), + mutable_buffer_cap, + "The capacity of the mutable buffer must not change" + ); + + return; + } + + // We are now byte aligned + mutable_buffer_byte_aligned_bitwise_bin_op_helper( + mutable_buffer, + left_offset_in_bits, + right, + right_offset_in_bits, + len_in_bits, + op, + ); + } + + // Making sure that our guarantee that the length and capacity of the mutable buffer + // will not change is upheld + assert_eq!( + mutable_buffer.len(), + mutable_buffer_len, + "The length of the mutable buffer must not change" + ); + assert_eq!( + mutable_buffer.capacity(), + mutable_buffer_cap, + "The capacity of the mutable buffer must not change" + ); +} + +/// Align to byte boundary by applying operation to bits before the next byte boundary. +/// +/// This function handles non-byte-aligned operations by processing bits from the current +/// position up to the next byte boundary, while preserving all other bits in the byte. +/// +/// # Arguments +/// +/// * `op` - Unary operation to apply +/// * `buffer` - The mutable buffer to modify +/// * `offset_in_bits` - Starting bit offset (not byte-aligned) +fn align_to_byte<F>(op: &mut F, buffer: &mut MutableBuffer, offset_in_bits: usize) +where + F: FnMut(u64) -> u64, +{ + let byte_offset = offset_in_bits / 8; + let bit_offset = offset_in_bits % 8; + + // 1. read the first byte from the buffer + let first_byte: u8 = buffer.as_slice()[byte_offset]; + + // 2. Shift byte by the bit offset, keeping only the relevant bits + let relevant_first_byte = first_byte >> bit_offset; + + // 3. run the op on the first byte only + let result_first_byte = op(relevant_first_byte as u64) as u8; + + // 4. Shift back the result to the original position + let result_first_byte = result_first_byte << bit_offset; + + // 5. Mask the bits that are outside the relevant bits in the byte + // so the bits until bit_offset are 1 and the rest are 0 + let mask_for_first_bit_offset = (1 << bit_offset) - 1; + + let result_first_byte = + (first_byte & mask_for_first_bit_offset) | (result_first_byte & !mask_for_first_bit_offset); + + // 6. write back the result to the buffer + buffer.as_slice_mut()[byte_offset] = result_first_byte; +} + +/// Read up to 8 bits from a byte slice starting at a given bit offset. +/// +/// This is similar to `get_8_bits_from_offset` but works with raw byte slices +/// and can read fewer than 8 bits. +/// +/// # Arguments +/// +/// * `slice` - The byte slice to read from +/// * `number_of_bits_to_read` - Number of bits to read (must be ≤ 8) +/// * `bit_offset` - Starting bit offset within the first byte +/// +/// # Returns +/// +/// A u8 containing the requested bits in the least significant positions +#[inline] +fn read_up_to_byte_from_offset( + slice: &[u8], + number_of_bits_to_read: usize, + bit_offset: usize, +) -> u8 { + assert!(number_of_bits_to_read <= 8); + assert_ne!(number_of_bits_to_read, 0); + assert_ne!(slice.len(), 0); + + let number_of_bytes_to_read = bit_util::ceil(number_of_bits_to_read + bit_offset, 8); + + // number of bytes to read + // might be one more than sizeof(u64) if the offset is in the middle of a byte + assert!(slice.len() >= number_of_bytes_to_read); + + let mut bits = slice[0] >> bit_offset; + for (i, &byte) in slice + .iter() + .take(number_of_bytes_to_read) + .enumerate() + .skip(1) + { + bits |= byte << (i * 8 - bit_offset); + } + + bits & ((1 << number_of_bits_to_read) - 1) +} + +/// Perform bitwise binary operation on byte-aligned buffers (i.e. not offsetting into a middle of a byte). +/// +/// This is the optimized path for byte-aligned operations. It processes data in +/// u64 chunks for maximum efficiency, then handles any remainder bits. +/// +/// # Arguments +/// +/// * `left` - The left mutable buffer (must be byte-aligned) +/// * `left_offset_in_bits` - Starting bit offset in the left buffer (must be multiple of 8) +/// * `right` - The right buffer as byte slice +/// * `right_offset_in_bits` - Starting bit offset in the right buffer +/// * `len_in_bits` - Number of bits to process +/// * `op` - Binary operation to apply +#[inline] +fn mutable_buffer_byte_aligned_bitwise_bin_op_helper<F>( + left: &mut MutableBuffer, + left_offset_in_bits: usize, + right: &impl BufferSupportedRhs, + right_offset_in_bits: usize, + len_in_bits: usize, + mut op: F, +) where + F: FnMut(u64, u64) -> u64, +{ + // Must not reach here if we not byte aligned + assert_eq!( + left_offset_in_bits % 8, + 0, + "left_offset_in_bits must be byte aligned" + ); + + // 1. Prepare the buffers + let (complete_u64_chunks, remainder_bytes) = + U64UnalignedSlice::split(left, left_offset_in_bits, len_in_bits); + + let right_chunks = BitChunks::new(right.as_slice(), right_offset_in_bits, len_in_bits); + assert_eq!( + bit_util::ceil(right_chunks.remainder_len(), 8), + remainder_bytes.len() + ); + + let right_chunks_iter = right_chunks.iter(); + assert_eq!(right_chunks_iter.len(), complete_u64_chunks.len()); + + // 2. Process complete u64 chunks + complete_u64_chunks.zip_modify(right_chunks_iter, &mut op); + + // Handle remainder bits if any + if right_chunks.remainder_len() > 0 { + handle_mutable_buffer_remainder( + &mut op, + remainder_bytes, + right_chunks.remainder_bits(), + right_chunks.remainder_len(), + ) + } +} + +/// Centralized structure to handle a mutable u8 slice as a mutable u64 pointer. +/// +/// Handle the following: +/// 1. the lifetime is correct +/// 2. we read/write within the bounds +/// 3. We read and write using unaligned +/// +/// This does not deallocate the underlying pointer when dropped +/// +/// This is the only place that uses unsafe code to read and write unaligned +/// +struct U64UnalignedSlice<'a> { + /// Pointer to the start of the u64 data + /// + /// We are using raw pointer as the data came from a u8 slice so we need to read and write unaligned Review Comment: Wouldn't it require copy? Or you mean https://doc.rust-lang.org/std/primitive.slice.html#method.align_to Which I used at first but removed as there is no guarantee that you wouldn't get the remainder at prefix instead of suffix -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
