From c0896443934865ee3aca9904fe9b18e932c13664 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Mon, 16 Sep 2024 13:40:28 +0000 Subject: [PATCH 01/31] feature: support fastlanes bitpacking for uint8 type --- protos/encodings.proto | 13 + rust/lance-encoding/Cargo.toml | 1 + rust/lance-encoding/benches/decoder.rs | 45 +- rust/lance-encoding/src/buffer.rs | 33 ++ rust/lance-encoding/src/encoder.rs | 9 + rust/lance-encoding/src/encodings/physical.rs | 17 + .../encodings/physical/bitpack_fastlanes.rs | 551 ++++++++++++++++++ rust/lance-encoding/src/format.rs | 21 +- 8 files changed, 686 insertions(+), 4 deletions(-) create mode 100644 rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs diff --git a/protos/encodings.proto b/protos/encodings.proto index 7b8af601a4..19cd814257 100644 --- a/protos/encodings.proto +++ b/protos/encodings.proto @@ -190,6 +190,18 @@ message Bitpacked { bool signed = 4; } +// Items are bitpacked in a buffer +message BitpackedForNonNeg { + // the number of bits used for a value in the buffer + uint64 compressed_bits_per_value = 1; + + // the number of bits of the uncompressed value. e.g. for a u32, this will be 32 + uint64 uncompressed_bits_per_value = 2; + + // The items in the list + Buffer buffer = 3; +} + // An array encoding for shredded structs that will never be null // // There is no actual data in this column. @@ -240,6 +252,7 @@ message ArrayEncoding { PackedStruct packed_struct = 9; Bitpacked bitpacked = 10; FixedSizeBinary fixed_size_binary = 11; + BitpackedForNonNeg bitpacked_for_non_neg = 12; } } diff --git a/rust/lance-encoding/Cargo.toml b/rust/lance-encoding/Cargo.toml index 35c3c336ff..42c412943f 100644 --- a/rust/lance-encoding/Cargo.toml +++ b/rust/lance-encoding/Cargo.toml @@ -37,6 +37,7 @@ snafu.workspace = true tokio.workspace = true tracing.workspace = true zstd.workspace = true +fastlanes = "0.1.5" [dev-dependencies] lance-testing.workspace = true diff --git a/rust/lance-encoding/benches/decoder.rs b/rust/lance-encoding/benches/decoder.rs index 21155f2453..ecbc785e2f 100644 --- a/rust/lance-encoding/benches/decoder.rs +++ b/rust/lance-encoding/benches/decoder.rs @@ -2,7 +2,7 @@ // SPDX-FileCopyrightText: Copyright The Lance Authors use std::{collections::HashMap, sync::Arc}; -use arrow_array::{RecordBatch, UInt32Array}; +use arrow_array::{RecordBatch, UInt32Array, UInt8Array}; use arrow_schema::{DataType, Field, Schema, TimeUnit}; use arrow_select::take::take; use criterion::{criterion_group, criterion_main, Criterion}; @@ -60,6 +60,47 @@ const ENCODING_OPTIONS: EncodingOptions = EncodingOptions { keep_original_array: true, }; +fn bench_decode2(c: &mut Criterion) { + let rt = tokio::runtime::Runtime::new().unwrap(); + let mut group = c.benchmark_group("decode_uint8"); + group.measurement_time(std::time::Duration::new(12, 0)); // 11.1 seconds + let array = UInt8Array::from(vec![5; 1024 * 1024 * 1024]); + let data = RecordBatch::try_new( + Arc::new(Schema::new(vec![Field::new( + "uint8", + DataType::UInt8, + false, + )])), + vec![Arc::new(array)], + ) + .unwrap(); + let lance_schema = + Arc::new(lance_core::datatypes::Schema::try_from(data.schema().as_ref()).unwrap()); + let input_bytes = data.get_array_memory_size(); + group.throughput(criterion::Throughput::Bytes(input_bytes as u64)); + let encoding_strategy = CoreFieldEncodingStrategy::default(); + let encoded = rt + .block_on(encode_batch( + &data, + lance_schema, + &encoding_strategy, + &ENCODING_OPTIONS, + )) + .unwrap(); + group.bench_function("uint8", |b| { + b.iter(|| { + let batch = rt + .block_on(lance_encoding::decoder::decode_batch( + &encoded, + &FilterExpression::no_filter(), + &DecoderMiddlewareChain::default(), + )) + .unwrap(); + assert_eq!(data.num_rows(), batch.num_rows()); + }) + }); +} + fn bench_decode(c: &mut Criterion) { let rt = tokio::runtime::Runtime::new().unwrap(); let mut group = c.benchmark_group("decode_primitive"); @@ -314,7 +355,7 @@ criterion_group!( name=benches; config = Criterion::default().significance_level(0.1).sample_size(10) .with_profiler(pprof::criterion::PProfProfiler::new(100, pprof::criterion::Output::Flamegraph(None))); - targets = bench_decode, bench_decode_fsl, bench_decode_str_with_dict_encoding, bench_decode_packed_struct, + targets = bench_decode2, bench_decode, bench_decode_fsl, bench_decode_str_with_dict_encoding, bench_decode_packed_struct, bench_decode_str_with_fixed_size_binary_encoding); // Non-linux version does not support pprof. diff --git a/rust/lance-encoding/src/buffer.rs b/rust/lance-encoding/src/buffer.rs index 3e17588d71..79aa0c0de5 100644 --- a/rust/lance-encoding/src/buffer.rs +++ b/rust/lance-encoding/src/buffer.rs @@ -203,6 +203,39 @@ impl LanceBuffer { Self::Borrowed(Buffer::from_vec(vec)) } + pub fn reinterpret_to_rust_native(&mut self) -> Result<&[T]> + where + T: Copy, // Ensure `T` can be copied (as needed for safely reinterpreting bytes) + { + // Step 1: Clone or borrow the internal buffer. + let buffer = self.borrow_and_clone(); + + // Step 2: Convert the borrowed data into a `Buffer` type. + let buffer = buffer.into_buffer(); + + // Step 3: Get the raw byte slice from the buffer. + let byte_slice = buffer.as_slice(); + + // Step 4: Safety check - ensure that the byte slice length is a multiple of `T`. + if byte_slice.len() % std::mem::size_of::() != 0 { + return Err(Error::Internal { + message: "Buffer size is not a multiple of the target type size".to_string(), + location: location!(), + }); + } + + // Step 5: Reinterpret the byte slice as a slice of `T`. + let typed_slice = unsafe { + std::slice::from_raw_parts( + byte_slice.as_ptr() as *const T, + byte_slice.len() / std::mem::size_of::(), + ) + }; + + // Step 6: Return the reinterpreted slice. + Ok(typed_slice) + } + /// Reinterprets a LanceBuffer into a Vec /// /// Unfortunately, there is no way to do this safely in Rust without a copy, even if diff --git a/rust/lance-encoding/src/encoder.rs b/rust/lance-encoding/src/encoder.rs index cd94c63682..549e9d7b3a 100644 --- a/rust/lance-encoding/src/encoder.rs +++ b/rust/lance-encoding/src/encoder.rs @@ -14,6 +14,8 @@ use snafu::{location, Location}; use crate::buffer::LanceBuffer; use crate::data::DataBlock; use crate::encodings::logical::r#struct::StructFieldEncoder; +use crate::encodings::physical::bitpack_fastlanes::compute_compressed_bit_width_for_non_neg; +use crate::encodings::physical::bitpack_fastlanes::BitpackedForNonNegArrayEncoder; use crate::encodings::physical::block_compress::CompressionScheme; use crate::encodings::physical::dictionary::AlreadyDictionaryEncoder; use crate::encodings::physical::fsst::FsstArrayEncoder; @@ -331,6 +333,13 @@ impl CoreArrayEncodingStrategy { Ok(Box::new(PackedStructEncoder::new(inner_encoders))) } + DataType::UInt8 => { + let compressed_bit_width = compute_compressed_bit_width_for_non_neg(arrays); + Ok(Box::new(BitpackedForNonNegArrayEncoder::new( + compressed_bit_width as usize, + data_type.clone(), + ))) + } _ => Ok(Box::new(BasicEncoder::new(Box::new( ValueEncoder::default(), )))), diff --git a/rust/lance-encoding/src/encodings/physical.rs b/rust/lance-encoding/src/encodings/physical.rs index 7340305843..8d61885969 100644 --- a/rust/lance-encoding/src/encodings/physical.rs +++ b/rust/lance-encoding/src/encodings/physical.rs @@ -19,6 +19,7 @@ pub mod basic; pub mod binary; pub mod bitmap; pub mod bitpack; +pub mod bitpack_fastlanes; pub mod block_compress; pub mod dictionary; pub mod fixed_size_binary; @@ -109,6 +110,19 @@ fn get_bitpacked_buffer_decoder( )) } +fn get_bitpacked_for_non_neg_buffer_decoder( + encoding: &pb::BitpackedForNonNeg, + buffers: &PageBuffers, +) -> Box { + let (buffer_offset, _buffer_size) = get_buffer(encoding.buffer.as_ref().unwrap(), buffers); + + Box::new(bitpack_fastlanes::BitpackedForNonNegScheduler::new( + encoding.compressed_bits_per_value, + encoding.uncompressed_bits_per_value, + buffer_offset, + )) +} + /// Convert a protobuf array encoding into a physical page scheduler pub fn decoder_from_array_encoding( encoding: &pb::ArrayEncoding, @@ -252,6 +266,9 @@ pub fn decoder_from_array_encoding( buffer_offset, )) } + pb::array_encoding::ArrayEncoding::BitpackedForNonNeg(bitpacked) => { + get_bitpacked_for_non_neg_buffer_decoder(bitpacked, buffers) + } // Currently there is no way to encode struct nullability and structs are encoded with a "header" column // (that has no data). We never actually decode that column and so this branch is never actually encountered. // diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs new file mode 100644 index 0000000000..6a7026e69d --- /dev/null +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -0,0 +1,551 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright The Lance Authors + +use std::sync::Arc; + +use arrow::datatypes::{ + ArrowPrimitiveType, Int16Type, Int32Type, Int64Type, Int8Type, UInt16Type, UInt32Type, + UInt64Type, UInt8Type, +}; +use arrow::util::bit_util::ceil; +use arrow_array::{cast::AsArray, Array, PrimitiveArray}; +use arrow_schema::DataType; +use bytes::Bytes; +use futures::future::{BoxFuture, FutureExt}; +use log::trace; +use num_traits::{AsPrimitive, PrimInt, ToPrimitive}; +use snafu::{location, Location}; + +use lance_arrow::DataTypeExt; +use lance_core::{Error, Result}; + +use crate::buffer::LanceBuffer; +use crate::data::{DataBlock, FixedWidthDataBlock}; +use crate::decoder::{PageScheduler, PrimitivePageDecoder}; +use crate::encoder::{ArrayEncoder, EncodedArray}; +use crate::format::ProtobufUtils; +use arrow::array::ArrayRef; +use fastlanes::BitPacking; + +// Compute the compressed_bit_width for a given array of unsigned integers +// the vortex approach is better, they compute all stastistics before encoding +pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { + // is it possible to get here? + if arrays.len() == 0 { + return 0; + } + + let res; + + match arrays[0].data_type() { + DataType::UInt8 | DataType::Int8 => { + let mut global_max: u8 = 0; + for array in arrays { + // at this point we know that the array doesn't contain any negative values + let primitive_array = array + .as_any() + .downcast_ref::>() + .unwrap(); + //println!("primitive_array: {:?}", primitive_array); + let array_max = arrow::compute::bit_or(primitive_array); + global_max = global_max.max(array_max.unwrap_or(0)); + } + let num_bits = + arrays[0].data_type().byte_width() as u64 * 8 - global_max.leading_zeros() as u64; + if num_bits == 0 { + res = 1; + } else { + res = num_bits; + } + } + DataType::UInt16 | DataType::Int16 => { + let mut global_max: u16 = 0; + for array in arrays { + let primitive_array = array + .as_any() + .downcast_ref::>() + .unwrap(); + let array_max = arrow::compute::bit_or(primitive_array).unwrap_or(0); + global_max = global_max.max(array_max); + } + let num_bits = + arrays[0].data_type().byte_width() as u64 * 8 - global_max.leading_zeros() as u64; + if num_bits == 0 { + res = 1; + } else { + res = num_bits; + } + } + DataType::UInt32 | DataType::Int32 => { + let mut global_max: u32 = 0; + for array in arrays { + let primitive_array = array + .as_any() + .downcast_ref::>() + .unwrap(); + let array_max = arrow::compute::bit_or(primitive_array).unwrap_or(0); + global_max = global_max.max(array_max); + } + let num_bits = + arrays[0].data_type().byte_width() as u64 * 8 - global_max.leading_zeros() as u64; + if num_bits == 0 { + res = 1; + } else { + res = num_bits; + } + } + DataType::UInt64 | DataType::Int64 => { + let mut global_max: u64 = 0; + for array in arrays { + let primitive_array = array + .as_any() + .downcast_ref::>() + .unwrap(); + let array_max = arrow::compute::bit_or(primitive_array).unwrap_or(0); + global_max = global_max.max(array_max); + } + let num_bits = + arrays[0].data_type().byte_width() as u64 * 8 - global_max.leading_zeros() as u64; + if num_bits == 0 { + res = 1; + } else { + res = num_bits; + } + } + _ => { + res = 8; + } + }; + res +} + +#[derive(Debug)] +pub struct BitpackedForNonNegArrayEncoder { + pub compressed_bit_width: usize, + pub original_data_type: DataType, +} + +impl BitpackedForNonNegArrayEncoder { + pub fn new(compressed_bit_width: usize, data_type: DataType) -> Self { + Self { + compressed_bit_width, + original_data_type: data_type, + } + } +} + +impl ArrayEncoder for BitpackedForNonNegArrayEncoder { + fn encode( + &self, + data: DataBlock, + _data_type: &DataType, + buffer_index: &mut u32, + ) -> Result { + let DataBlock::FixedWidth(mut unpacked) = data else { + return Err(Error::InvalidInput { + source: "Bitpacking only supports fixed width data blocks".into(), + location: location!(), + }); + }; + match _data_type { + DataType::UInt8 | DataType::Int8 => { + let num_chunks = (unpacked.num_values + 1023) / 1024; + let num_full_chunks = unpacked.num_values / 1024; + // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 + let packed_chunk_size = 1024 * self.compressed_bit_width / 8; + + let input = unpacked.data.reinterpret_to_rust_native::().unwrap(); + let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); + + // Loop over all but the last chunk. + (0..num_full_chunks).for_each(|i| { + let start_elem = i as usize * 1024 as usize; + + let output_len = output.len(); + unsafe { + output.set_len(output_len + packed_chunk_size); + BitPacking::unchecked_pack( + self.compressed_bit_width, + &input[start_elem as usize..][..1024], + &mut output[output_len..][..packed_chunk_size], + ); + }; + }); + + if num_chunks != num_full_chunks { + let last_chunk_elem_num = unpacked.num_values % 1024; + let mut last_chunk = vec![0u8; 1024]; + last_chunk[..last_chunk_elem_num as usize].clone_from_slice( + &input[unpacked.num_values as usize - last_chunk_elem_num as usize..], + ); + + let output_len = output.len(); + unsafe { + output.set_len(output.len() + packed_chunk_size); + BitPacking::unchecked_pack( + self.compressed_bit_width, + &last_chunk, + &mut output[output_len..][..packed_chunk_size], + ); + } + } + let bitpacked_for_non_neg_buffer_index = *buffer_index; + *buffer_index += 1; + + let encoding = ProtobufUtils::bitpacked_for_non_neg_encoding( + self.compressed_bit_width as u64, + _data_type.byte_width() as u64 * 8, + bitpacked_for_non_neg_buffer_index, + ); + let packed = DataBlock::FixedWidth(FixedWidthDataBlock { + bits_per_value: self.compressed_bit_width as u64, + data: LanceBuffer::Owned(output), + num_values: unpacked.num_values, + }); + + return Ok(EncodedArray { + data: packed, + encoding, + }); + } + _ => todo!(), + } + } +} + +#[derive(Debug)] +pub struct BitpackedForNonNegScheduler { + compressed_bit_width: u64, + uncompressed_bits_per_value: u64, + buffer_offset: u64, +} + +fn locate_chunk_start2(scheduler: &BitpackedForNonNegScheduler, relative_row_num: u64) -> u64 { + let elems_per_chunk = 1024; + let chunk_size = elems_per_chunk * scheduler.compressed_bit_width / 8; + relative_row_num / elems_per_chunk * chunk_size +} + +fn locate_chunk_end2(scheduler: &BitpackedForNonNegScheduler, relative_row_num: u64) -> u64 { + let elems_per_chunk: u64 = 1024; + let chunk_size = elems_per_chunk * scheduler.compressed_bit_width / 8; + relative_row_num / elems_per_chunk * chunk_size + chunk_size +} + +impl BitpackedForNonNegScheduler { + pub fn new( + compressed_bit_width: u64, + uncompressed_bits_per_value: u64, + buffer_offset: u64, + ) -> Self { + Self { + compressed_bit_width, + uncompressed_bits_per_value, + buffer_offset, + } + } +} + +impl PageScheduler for BitpackedForNonNegScheduler { + fn schedule_ranges( + &self, + ranges: &[std::ops::Range], + scheduler: &Arc, + top_level_row: u64, + ) -> BoxFuture<'static, Result>> { + // can we get here? + if ranges.is_empty() { + panic!("cannot schedule empty ranges"); + } + let mut byte_ranges = vec![]; + let mut bytes_idx_to_range_indices = vec![]; + let first_byte_range = std::ops::Range { + start: self.buffer_offset + locate_chunk_start2(self, ranges[0].start), + end: self.buffer_offset + locate_chunk_end2(self, ranges[0].end - 1), + }; // the ranges are half-open + byte_ranges.push(first_byte_range); + bytes_idx_to_range_indices.push(vec![ranges[0].clone()]); + for (i, range) in ranges.iter().enumerate().skip(1) { + let this_start = locate_chunk_start2(self, range.start); + let this_end = locate_chunk_end2(self, range.end - 1); + if this_start == locate_chunk_start2(self, ranges[i - 1].end - 1) { + byte_ranges.last_mut().unwrap().end = this_end; + bytes_idx_to_range_indices + .last_mut() + .unwrap() + .push(range.clone()); + } else { + byte_ranges.push(this_start..this_end); + bytes_idx_to_range_indices.push(vec![range.clone()]); + } + } + + trace!( + "Scheduling I/O for {} ranges spread across byte range {}..{}", + byte_ranges.len(), + byte_ranges[0].start, + byte_ranges.last().unwrap().end + ); + /* + println!( + "Scheduling I/O for {} ranges spread across byte range {}..{}", + byte_ranges.len(), + byte_ranges[0].start, + byte_ranges[byte_ranges.len() - 1].end + ); + println!("ranges: {:?}", ranges); + println!("byte_ranges: {:?}", byte_ranges); + */ + + let bytes = scheduler.submit_request(byte_ranges.clone(), top_level_row); + + // Clone the necessary data from `self` to move into the async block + let compressed_bit_width = self.compressed_bit_width; + let uncompressed_bits_per_value = self.uncompressed_bits_per_value; + let num_rows = ranges.iter().map(|range| range.end - range.start).sum(); + + async move { + let bytes = bytes.await?; + let decompressed_output = bitpacked_for_non_neg_decode( + compressed_bit_width, + uncompressed_bits_per_value, + &bytes, + &bytes_idx_to_range_indices, + num_rows, + ); + Ok(Box::new(BitpackedForNonNegPageDecoder { + uncompressed_bits_per_value, + decompressed_buf: decompressed_output, + }) as Box) + } + .boxed() + } +} + +#[derive(Debug)] +struct BitpackedForNonNegPageDecoder { + // number of bits in the uncompressed value. E.g. this will be 32 for DataType::UInt32 + uncompressed_bits_per_value: u64, + + decompressed_buf: Vec, +} + +impl PrimitivePageDecoder for BitpackedForNonNegPageDecoder { + fn decode(&self, rows_to_skip: u64, num_rows: u64) -> Result { + match self.uncompressed_bits_per_value { + 8 => { + // I did an extra copy here, not sure how to avoid it and whether it's safe to avoid it + let mut output = Vec::with_capacity(num_rows as usize); + output.extend_from_slice( + &self.decompressed_buf[rows_to_skip as usize..][..num_rows as usize], + ); + let res = Ok(DataBlock::FixedWidth(FixedWidthDataBlock { + data: LanceBuffer::from(output), + bits_per_value: self.uncompressed_bits_per_value, + num_values: num_rows, + })); + return res; + } + _ => panic!("Unsupported data type"), + } + } +} + +fn bitpacked_for_non_neg_decode( + compressed_bit_width: u64, + uncompressed_bits_per_value: u64, + data: &Vec, + bytes_idx_to_range_indices: &Vec>>, + num_rows: u64, +) -> Vec { + match uncompressed_bits_per_value { + 8 => { + let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); + let packed_chunk_size: usize = 1024 * compressed_bit_width as usize / 8; + let mut decompress_chunk_buf = vec![0_u8; 1024]; + for (i, bytes) in data.iter().enumerate() { + let mut j = 0; + let mut ranges_idx = 0; + let mut curr_range_start = bytes_idx_to_range_indices[i][0].start; + while j * packed_chunk_size < bytes.len() { + let chunk: &[u8] = &bytes[j * packed_chunk_size..][..packed_chunk_size]; + unsafe { + BitPacking::unchecked_unpack( + compressed_bit_width as usize, + chunk, + &mut decompress_chunk_buf[..1024], + ); + } + loop { + if curr_range_start + 1024 < bytes_idx_to_range_indices[i][ranges_idx].end { + let this_part_len = 1024 - curr_range_start % 1024; + decompressed.extend_from_slice( + &decompress_chunk_buf[curr_range_start as usize % 1024..], + ); + curr_range_start += this_part_len; + break; + } else { + let this_part_len = + bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; + decompressed.extend_from_slice( + &decompress_chunk_buf[curr_range_start as usize % 1024..] + [..this_part_len as usize], + ); + ranges_idx += 1; + if ranges_idx == bytes_idx_to_range_indices[i].len() { + break; + } + curr_range_start = bytes_idx_to_range_indices[i][ranges_idx].start; + } + } + j += 1; + } + } + decompressed + } + _ => panic!("Unsupported data type"), + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::array::{Int32Array, PrimitiveArray}; + use arrow::buffer::Buffer; + use arrow::datatypes::DataType; + use arrow::record_batch::RecordBatch; + use arrow::util::bit_util::ceil; + + #[test] + fn test_encode() { + // Prepare input data + } + use crate::decoder::decode_batch; + use crate::decoder::DecoderMiddlewareChain; + use crate::decoder::FilterExpression; + use crate::encoder::encode_batch; + use crate::encoder::CoreFieldEncodingStrategy; + use crate::encoder::EncodingOptions; + use arrow::array::UInt8Array; + use arrow::datatypes::Field; + use arrow::datatypes::Schema; + + #[test_log::test(tokio::test)] + async fn test_compute_compressed_bit_width_for_non_neg() {} + + use std::collections::HashMap; + + use lance_datagen::{ByteCount, RowCount}; + + use crate::testing::{check_round_trip_encoding_of_data, TestCases}; + + #[test_log::test(tokio::test)] + async fn test_bitpack_fastlanes() { + let values: Vec = vec![5; 1024]; + let array = UInt8Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![66; 1000]; + let array = UInt8Array::from(values); + let array: Arc = Arc::new(array); + + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![77; 2000]; + let array = UInt8Array::from(values); + let array: Arc = Arc::new(array); + + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![0; 10000]; + let array = UInt8Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![88; 10000]; + let array = UInt8Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_primitive::( + arrow::datatypes::UInt8Type::DATA_TYPE, + )) + .into_batch_rows(RowCount::from(1)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_primitive::( + arrow::datatypes::UInt8Type::DATA_TYPE, + )) + .into_batch_rows(RowCount::from(20)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_primitive::( + arrow::datatypes::UInt8Type::DATA_TYPE, + )) + .into_batch_rows(RowCount::from(50)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_primitive::( + arrow::datatypes::UInt8Type::DATA_TYPE, + )) + .into_batch_rows(RowCount::from(100)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_primitive::( + arrow::datatypes::UInt8Type::DATA_TYPE, + )) + .into_batch_rows(RowCount::from(1000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_primitive::( + arrow::datatypes::UInt8Type::DATA_TYPE, + )) + .into_batch_rows(RowCount::from(1024)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_primitive::( + arrow::datatypes::UInt8Type::DATA_TYPE, + )) + .into_batch_rows(RowCount::from(2000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_primitive::( + arrow::datatypes::UInt8Type::DATA_TYPE, + )) + .into_batch_rows(RowCount::from(3000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + } +} diff --git a/rust/lance-encoding/src/format.rs b/rust/lance-encoding/src/format.rs index ff16706768..117985751b 100644 --- a/rust/lance-encoding/src/format.rs +++ b/rust/lance-encoding/src/format.rs @@ -18,8 +18,8 @@ use pb::{ array_encoding::ArrayEncoding as ArrayEncodingEnum, buffer::BufferType, nullable::{AllNull, NoNull, Nullability, SomeNull}, - ArrayEncoding, Binary, Bitpacked, Dictionary, FixedSizeBinary, FixedSizeList, Flat, Fsst, - Nullable, PackedStruct, + ArrayEncoding, Binary, Bitpacked, BitpackedForNonNeg, Dictionary, FixedSizeBinary, + FixedSizeList, Flat, Fsst, Nullable, PackedStruct, }; use crate::encodings::physical::block_compress::CompressionScheme; @@ -98,6 +98,23 @@ impl ProtobufUtils { } } + pub fn bitpacked_for_non_neg_encoding( + compressed_bits_per_value: u64, + uncompressed_bits_per_value: u64, + buffer_index: u32, + ) -> ArrayEncoding { + ArrayEncoding { + array_encoding: Some(ArrayEncodingEnum::BitpackedForNonNeg(BitpackedForNonNeg { + compressed_bits_per_value, + buffer: Some(pb::Buffer { + buffer_index, + buffer_type: BufferType::Page as i32, + }), + uncompressed_bits_per_value, + })), + } + } + pub fn packed_struct( child_encodings: Vec, packed_buffer_index: u32, From a1e3cdf9bf950ac1436589110d5b3bb9199ee2f8 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Mon, 16 Sep 2024 13:54:37 +0000 Subject: [PATCH 02/31] minor fix --- rust/lance-encoding/benches/decoder.rs | 2 +- .../encodings/physical/bitpack_fastlanes.rs | 20 ++++++++----------- 2 files changed, 9 insertions(+), 13 deletions(-) diff --git a/rust/lance-encoding/benches/decoder.rs b/rust/lance-encoding/benches/decoder.rs index ecbc785e2f..cde9d2dcb8 100644 --- a/rust/lance-encoding/benches/decoder.rs +++ b/rust/lance-encoding/benches/decoder.rs @@ -63,7 +63,7 @@ const ENCODING_OPTIONS: EncodingOptions = EncodingOptions { fn bench_decode2(c: &mut Criterion) { let rt = tokio::runtime::Runtime::new().unwrap(); let mut group = c.benchmark_group("decode_uint8"); - group.measurement_time(std::time::Duration::new(12, 0)); // 11.1 seconds + group.measurement_time(std::time::Duration::new(12, 0)); let array = UInt8Array::from(vec![5; 1024 * 1024 * 1024]); let data = RecordBatch::try_new( Arc::new(Schema::new(vec![Field::new( diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index 6a7026e69d..4d968fe0fe 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -220,13 +220,13 @@ pub struct BitpackedForNonNegScheduler { buffer_offset: u64, } -fn locate_chunk_start2(scheduler: &BitpackedForNonNegScheduler, relative_row_num: u64) -> u64 { +fn locate_chunk_start(scheduler: &BitpackedForNonNegScheduler, relative_row_num: u64) -> u64 { let elems_per_chunk = 1024; let chunk_size = elems_per_chunk * scheduler.compressed_bit_width / 8; relative_row_num / elems_per_chunk * chunk_size } -fn locate_chunk_end2(scheduler: &BitpackedForNonNegScheduler, relative_row_num: u64) -> u64 { +fn locate_chunk_end(scheduler: &BitpackedForNonNegScheduler, relative_row_num: u64) -> u64 { let elems_per_chunk: u64 = 1024; let chunk_size = elems_per_chunk * scheduler.compressed_bit_width / 8; relative_row_num / elems_per_chunk * chunk_size + chunk_size @@ -260,15 +260,15 @@ impl PageScheduler for BitpackedForNonNegScheduler { let mut byte_ranges = vec![]; let mut bytes_idx_to_range_indices = vec![]; let first_byte_range = std::ops::Range { - start: self.buffer_offset + locate_chunk_start2(self, ranges[0].start), - end: self.buffer_offset + locate_chunk_end2(self, ranges[0].end - 1), + start: self.buffer_offset + locate_chunk_start(self, ranges[0].start), + end: self.buffer_offset + locate_chunk_end(self, ranges[0].end - 1), }; // the ranges are half-open byte_ranges.push(first_byte_range); bytes_idx_to_range_indices.push(vec![ranges[0].clone()]); for (i, range) in ranges.iter().enumerate().skip(1) { - let this_start = locate_chunk_start2(self, range.start); - let this_end = locate_chunk_end2(self, range.end - 1); - if this_start == locate_chunk_start2(self, ranges[i - 1].end - 1) { + let this_start = locate_chunk_start(self, range.start); + let this_end = locate_chunk_end(self, range.end - 1); + if this_start == locate_chunk_start(self, ranges[i - 1].end - 1) { byte_ranges.last_mut().unwrap().end = this_end; bytes_idx_to_range_indices .last_mut() @@ -299,7 +299,7 @@ impl PageScheduler for BitpackedForNonNegScheduler { let bytes = scheduler.submit_request(byte_ranges.clone(), top_level_row); - // Clone the necessary data from `self` to move into the async block + // copy the necessary data from `self` to move into the async block let compressed_bit_width = self.compressed_bit_width; let uncompressed_bits_per_value = self.uncompressed_bits_per_value; let num_rows = ranges.iter().map(|range| range.end - range.start).sum(); @@ -416,10 +416,6 @@ mod tests { use arrow::record_batch::RecordBatch; use arrow::util::bit_util::ceil; - #[test] - fn test_encode() { - // Prepare input data - } use crate::decoder::decode_batch; use crate::decoder::DecoderMiddlewareChain; use crate::decoder::FilterExpression; From 3f340a31e9073089d6720f755a17aa6632d4250d Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Tue, 17 Sep 2024 13:57:07 +0000 Subject: [PATCH 03/31] fix a bug, add self.buffer_offset in byte range --- .../src/encodings/physical/bitpack_fastlanes.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index 4d968fe0fe..986a502b40 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -113,6 +113,7 @@ pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { } } _ => { + // in dictionary encoding, they route it to here when array is utf8, don't know what we should do yet. res = 8; } }; @@ -269,13 +270,13 @@ impl PageScheduler for BitpackedForNonNegScheduler { let this_start = locate_chunk_start(self, range.start); let this_end = locate_chunk_end(self, range.end - 1); if this_start == locate_chunk_start(self, ranges[i - 1].end - 1) { - byte_ranges.last_mut().unwrap().end = this_end; + byte_ranges.last_mut().unwrap().end = self.buffer_offset + this_end; bytes_idx_to_range_indices .last_mut() .unwrap() .push(range.clone()); } else { - byte_ranges.push(this_start..this_end); + byte_ranges.push(self.buffer_offset + this_start..self.buffer_offset + this_end); bytes_idx_to_range_indices.push(vec![range.clone()]); } } From 9a6c489220a20b0c0e7238a9b9014a8b7119f065 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Tue, 17 Sep 2024 14:00:55 +0000 Subject: [PATCH 04/31] minor fix 2 --- rust/lance-encoding/src/buffer.rs | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/rust/lance-encoding/src/buffer.rs b/rust/lance-encoding/src/buffer.rs index 79aa0c0de5..483259911b 100644 --- a/rust/lance-encoding/src/buffer.rs +++ b/rust/lance-encoding/src/buffer.rs @@ -207,16 +207,14 @@ impl LanceBuffer { where T: Copy, // Ensure `T` can be copied (as needed for safely reinterpreting bytes) { - // Step 1: Clone or borrow the internal buffer. let buffer = self.borrow_and_clone(); - // Step 2: Convert the borrowed data into a `Buffer` type. let buffer = buffer.into_buffer(); - // Step 3: Get the raw byte slice from the buffer. + // Get the raw byte slice from the buffer. let byte_slice = buffer.as_slice(); - // Step 4: Safety check - ensure that the byte slice length is a multiple of `T`. + // Safety check - ensure that the byte slice length is a multiple of `T`. if byte_slice.len() % std::mem::size_of::() != 0 { return Err(Error::Internal { message: "Buffer size is not a multiple of the target type size".to_string(), @@ -224,7 +222,7 @@ impl LanceBuffer { }); } - // Step 5: Reinterpret the byte slice as a slice of `T`. + // Reinterpret the byte slice as a slice of `T`. let typed_slice = unsafe { std::slice::from_raw_parts( byte_slice.as_ptr() as *const T, @@ -232,7 +230,6 @@ impl LanceBuffer { ) }; - // Step 6: Return the reinterpreted slice. Ok(typed_slice) } From f55a44510147d953e52af2c9c618b479a3d96f37 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Wed, 18 Sep 2024 00:44:25 +0000 Subject: [PATCH 05/31] feat: add fastlanes bitpacking for other types --- rust/lance-encoding/Cargo.toml | 1 + rust/lance-encoding/benches/decoder.rs | 2 +- rust/lance-encoding/src/encoder.rs | 12 +- .../encodings/physical/bitpack_fastlanes.rs | 1493 +++++++++++++++-- 4 files changed, 1412 insertions(+), 96 deletions(-) diff --git a/rust/lance-encoding/Cargo.toml b/rust/lance-encoding/Cargo.toml index 42c412943f..56641e0e35 100644 --- a/rust/lance-encoding/Cargo.toml +++ b/rust/lance-encoding/Cargo.toml @@ -38,6 +38,7 @@ tokio.workspace = true tracing.workspace = true zstd.workspace = true fastlanes = "0.1.5" +bytemuck = "=1.18.0" [dev-dependencies] lance-testing.workspace = true diff --git a/rust/lance-encoding/benches/decoder.rs b/rust/lance-encoding/benches/decoder.rs index cde9d2dcb8..e8217a76cf 100644 --- a/rust/lance-encoding/benches/decoder.rs +++ b/rust/lance-encoding/benches/decoder.rs @@ -63,7 +63,7 @@ const ENCODING_OPTIONS: EncodingOptions = EncodingOptions { fn bench_decode2(c: &mut Criterion) { let rt = tokio::runtime::Runtime::new().unwrap(); let mut group = c.benchmark_group("decode_uint8"); - group.measurement_time(std::time::Duration::new(12, 0)); + group.measurement_time(std::time::Duration::new(12, 0)); let array = UInt8Array::from(vec![5; 1024 * 1024 * 1024]); let data = RecordBatch::try_new( Arc::new(Schema::new(vec![Field::new( diff --git a/rust/lance-encoding/src/encoder.rs b/rust/lance-encoding/src/encoder.rs index 549e9d7b3a..a92cfea9c2 100644 --- a/rust/lance-encoding/src/encoder.rs +++ b/rust/lance-encoding/src/encoder.rs @@ -333,7 +333,17 @@ impl CoreArrayEncodingStrategy { Ok(Box::new(PackedStructEncoder::new(inner_encoders))) } - DataType::UInt8 => { + DataType::UInt8 | DataType::UInt16 | DataType::UInt32 | DataType::UInt64 => { + let compressed_bit_width = compute_compressed_bit_width_for_non_neg(arrays); + Ok(Box::new(BitpackedForNonNegArrayEncoder::new( + compressed_bit_width as usize, + data_type.clone(), + ))) + } + + // for signed integers, I intend to make it a cascaded encoding, a sparse array for the negative values and very wide(bit-width) values, + // then a bitpacked array for the narrow(bit-width) values, I need `BitpackedForNeg` to be merged first + DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64 => { let compressed_bit_width = compute_compressed_bit_width_for_non_neg(arrays); Ok(Box::new(BitpackedForNonNegArrayEncoder::new( compressed_bit_width as usize, diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index 986a502b40..eebf302fae 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -1,19 +1,16 @@ -// SPDX-License-Identifier: Apache-2.0 +// spdx-license-identifier: apache-2.0 // SPDX-FileCopyrightText: Copyright The Lance Authors use std::sync::Arc; use arrow::datatypes::{ - ArrowPrimitiveType, Int16Type, Int32Type, Int64Type, Int8Type, UInt16Type, UInt32Type, - UInt64Type, UInt8Type, + Int16Type, Int32Type, Int64Type, Int8Type, UInt16Type, UInt32Type, UInt64Type, UInt8Type, }; -use arrow::util::bit_util::ceil; -use arrow_array::{cast::AsArray, Array, PrimitiveArray}; +use arrow_array::{Array, PrimitiveArray}; use arrow_schema::DataType; use bytes::Bytes; use futures::future::{BoxFuture, FutureExt}; use log::trace; -use num_traits::{AsPrimitive, PrimInt, ToPrimitive}; use snafu::{location, Location}; use lance_arrow::DataTypeExt; @@ -25,20 +22,22 @@ use crate::decoder::{PageScheduler, PrimitivePageDecoder}; use crate::encoder::{ArrayEncoder, EncodedArray}; use crate::format::ProtobufUtils; use arrow::array::ArrayRef; +use bytemuck::cast_slice; use fastlanes::BitPacking; // Compute the compressed_bit_width for a given array of unsigned integers // the vortex approach is better, they compute all stastistics before encoding +// todo: see how to use rust macro to rewrite this function pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { // is it possible to get here? - if arrays.len() == 0 { + if arrays.is_empty() { return 0; } let res; match arrays[0].data_type() { - DataType::UInt8 | DataType::Int8 => { + DataType::UInt8 => { let mut global_max: u8 = 0; for array in arrays { // at this point we know that the array doesn't contain any negative values @@ -52,13 +51,32 @@ pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { } let num_bits = arrays[0].data_type().byte_width() as u64 * 8 - global_max.leading_zeros() as u64; + // we will have constant encoding later if num_bits == 0 { res = 1; } else { res = num_bits; } } - DataType::UInt16 | DataType::Int16 => { + + DataType::Int8 => { + let mut global_max_width: u64 = 0; + for array in arrays { + let primitive_array = array + .as_any() + .downcast_ref::>() + .unwrap(); + let array_max_width = arrow::compute::bit_or(primitive_array).unwrap_or(0); + global_max_width = global_max_width.max(8 - array_max_width.leading_zeros() as u64); + } + if global_max_width == 0 { + res = 1; + } else { + res = global_max_width; + } + } + + DataType::UInt16 => { let mut global_max: u16 = 0; for array in arrays { let primitive_array = array @@ -76,7 +94,26 @@ pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { res = num_bits; } } - DataType::UInt32 | DataType::Int32 => { + + DataType::Int16 => { + let mut global_max_width: u64 = 0; + for array in arrays { + let primitive_array = array + .as_any() + .downcast_ref::>() + .unwrap(); + let array_max_width = arrow::compute::bit_or(primitive_array).unwrap_or(0); + global_max_width = + global_max_width.max(16 - array_max_width.leading_zeros() as u64); + } + if global_max_width == 0 { + res = 1; + } else { + res = global_max_width; + } + } + + DataType::UInt32 => { let mut global_max: u32 = 0; for array in arrays { let primitive_array = array @@ -94,7 +131,26 @@ pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { res = num_bits; } } - DataType::UInt64 | DataType::Int64 => { + + DataType::Int32 => { + let mut global_max_width: u64 = 0; + for array in arrays { + let primitive_array = array + .as_any() + .downcast_ref::>() + .unwrap(); + let array_max_width = arrow::compute::bit_or(primitive_array).unwrap_or(0); + global_max_width = + global_max_width.max(32 - array_max_width.leading_zeros() as u64); + } + if global_max_width == 0 { + res = 1; + } else { + res = global_max_width; + } + } + + DataType::UInt64 => { let mut global_max: u64 = 0; for array in arrays { let primitive_array = array @@ -112,6 +168,24 @@ pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { res = num_bits; } } + + DataType::Int64 => { + let mut global_max_width: u64 = 0; + for array in arrays { + let primitive_array = array + .as_any() + .downcast_ref::>() + .unwrap(); + let array_max_width = arrow::compute::bit_or(primitive_array).unwrap_or(0); + global_max_width = + global_max_width.max(64 - array_max_width.leading_zeros() as u64); + } + if global_max_width == 0 { + res = 1; + } else { + res = global_max_width; + } + } _ => { // in dictionary encoding, they route it to here when array is utf8, don't know what we should do yet. res = 8; @@ -153,21 +227,23 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { let num_chunks = (unpacked.num_values + 1023) / 1024; let num_full_chunks = unpacked.num_values / 1024; // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 - let packed_chunk_size = 1024 * self.compressed_bit_width / 8; + // the output type is the same as the input type + // 1024 * compressed_bit_width / 8 + let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); let input = unpacked.data.reinterpret_to_rust_native::().unwrap(); let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); // Loop over all but the last chunk. (0..num_full_chunks).for_each(|i| { - let start_elem = i as usize * 1024 as usize; + let start_elem = i as usize * 1024_usize; let output_len = output.len(); unsafe { output.set_len(output_len + packed_chunk_size); BitPacking::unchecked_pack( self.compressed_bit_width, - &input[start_elem as usize..][..1024], + &input[start_elem..][..1024], &mut output[output_len..][..packed_chunk_size], ); }; @@ -204,11 +280,199 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { num_values: unpacked.num_values, }); - return Ok(EncodedArray { + Ok(EncodedArray { + data: packed, + encoding, + }) + } + + DataType::UInt16 | DataType::Int16 => { + let num_chunks = (unpacked.num_values + 1023) / 1024; + let num_full_chunks = unpacked.num_values / 1024; + // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 + // the output type is the same as the input type + let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); + + let input = unpacked.data.reinterpret_to_rust_native::().unwrap(); + let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); + + // Loop over all but the last chunk. + (0..num_full_chunks).for_each(|i| { + let start_elem = i as usize * 1024_usize; + + let output_len = output.len(); + unsafe { + output.set_len(output_len + packed_chunk_size); + BitPacking::unchecked_pack( + self.compressed_bit_width, + &input[start_elem..][..1024], + &mut output[output_len..][..packed_chunk_size], + ); + }; + }); + + if num_chunks != num_full_chunks { + let last_chunk_elem_num = unpacked.num_values % 1024; + let mut last_chunk = vec![0u16; 1024]; + last_chunk[..last_chunk_elem_num as usize].clone_from_slice( + &input[unpacked.num_values as usize - last_chunk_elem_num as usize..], + ); + + let output_len = output.len(); + unsafe { + output.set_len(output.len() + packed_chunk_size); + BitPacking::unchecked_pack( + self.compressed_bit_width, + &last_chunk, + &mut output[output_len..][..packed_chunk_size], + ); + } + } + let bitpacked_for_non_neg_buffer_index = *buffer_index; + *buffer_index += 1; + + let encoding = ProtobufUtils::bitpacked_for_non_neg_encoding( + self.compressed_bit_width as u64, + _data_type.byte_width() as u64 * 8, + bitpacked_for_non_neg_buffer_index, + ); + let packed = DataBlock::FixedWidth(FixedWidthDataBlock { + bits_per_value: self.compressed_bit_width as u64, + data: LanceBuffer::reinterpret_vec(output).to_owned(), + num_values: unpacked.num_values, + }); + Ok(EncodedArray { + data: packed, + encoding, + }) + } + + DataType::UInt32 | DataType::Int32 => { + let num_chunks = (unpacked.num_values + 1023) / 1024; + let num_full_chunks = unpacked.num_values / 1024; + // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 + // the output type is the same as the input type + let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); + + let input_slice = unpacked.data.borrow_to_typed_slice::(); + let input = input_slice.as_ref(); + let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); + + // Loop over all but the last chunk. + (0..num_full_chunks).for_each(|i| { + let start_elem = i as usize * 1024_usize; + + let output_len = output.len(); + unsafe { + output.set_len(output_len + packed_chunk_size); + BitPacking::unchecked_pack( + self.compressed_bit_width, + &input[start_elem..][..1024], + &mut output[output_len..][..packed_chunk_size], + ); + }; + }); + + if num_chunks != num_full_chunks { + let last_chunk_elem_num = unpacked.num_values % 1024; + let mut last_chunk = vec![0u32; 1024]; + last_chunk[..last_chunk_elem_num as usize].clone_from_slice( + &input[unpacked.num_values as usize - last_chunk_elem_num as usize..], + ); + + let output_len = output.len(); + unsafe { + output.set_len(output.len() + packed_chunk_size); + BitPacking::unchecked_pack( + self.compressed_bit_width, + &last_chunk, + &mut output[output_len..][..packed_chunk_size], + ); + } + } + let bitpacked_for_non_neg_buffer_index = *buffer_index; + *buffer_index += 1; + + let encoding = ProtobufUtils::bitpacked_for_non_neg_encoding( + self.compressed_bit_width as u64, + _data_type.byte_width() as u64 * 8, + bitpacked_for_non_neg_buffer_index, + ); + let packed = DataBlock::FixedWidth(FixedWidthDataBlock { + bits_per_value: self.compressed_bit_width as u64, + data: LanceBuffer::reinterpret_vec(output).to_owned(), + num_values: unpacked.num_values, + }); + + Ok(EncodedArray { data: packed, encoding, + }) + } + + DataType::UInt64 | DataType::Int64 => { + let num_chunks = (unpacked.num_values + 1023) / 1024; + let num_full_chunks = unpacked.num_values / 1024; + // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 + // the output type is the same as the input type + let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); + + let input_slice = unpacked.data.borrow_to_typed_slice::(); + let input = input_slice.as_ref(); + let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); + + // Loop over all but the last chunk. + (0..num_full_chunks).for_each(|i| { + let start_elem = i as usize * 1024_usize; + + let output_len = output.len(); + unsafe { + output.set_len(output_len + packed_chunk_size); + BitPacking::unchecked_pack( + self.compressed_bit_width, + &input[start_elem..][..1024], + &mut output[output_len..][..packed_chunk_size], + ); + }; + }); + + if num_chunks != num_full_chunks { + let last_chunk_elem_num = unpacked.num_values % 1024; + let mut last_chunk = vec![0u64; 1024]; + last_chunk[..last_chunk_elem_num as usize].clone_from_slice( + &input[unpacked.num_values as usize - last_chunk_elem_num as usize..], + ); + + let output_len = output.len(); + unsafe { + output.set_len(output.len() + packed_chunk_size); + BitPacking::unchecked_pack( + self.compressed_bit_width, + &last_chunk, + &mut output[output_len..][..packed_chunk_size], + ); + } + } + let bitpacked_for_non_neg_buffer_index = *buffer_index; + *buffer_index += 1; + + let encoding = ProtobufUtils::bitpacked_for_non_neg_encoding( + self.compressed_bit_width as u64, + _data_type.byte_width() as u64 * 8, + bitpacked_for_non_neg_buffer_index, + ); + let packed = DataBlock::FixedWidth(FixedWidthDataBlock { + bits_per_value: self.compressed_bit_width as u64, + data: LanceBuffer::reinterpret_vec(output).to_owned(), + num_values: unpacked.num_values, }); + + Ok(EncodedArray { + data: packed, + encoding, + }) } + _ => todo!(), } } @@ -328,7 +592,7 @@ struct BitpackedForNonNegPageDecoder { // number of bits in the uncompressed value. E.g. this will be 32 for DataType::UInt32 uncompressed_bits_per_value: u64, - decompressed_buf: Vec, + decompressed_buf: LanceBuffer, } impl PrimitivePageDecoder for BitpackedForNonNegPageDecoder { @@ -340,14 +604,75 @@ impl PrimitivePageDecoder for BitpackedForNonNegPageDecoder { output.extend_from_slice( &self.decompressed_buf[rows_to_skip as usize..][..num_rows as usize], ); - let res = Ok(DataBlock::FixedWidth(FixedWidthDataBlock { + Ok(DataBlock::FixedWidth(FixedWidthDataBlock { data: LanceBuffer::from(output), bits_per_value: self.uncompressed_bits_per_value, num_values: num_rows, - })); - return res; + })) + } + + 16 => { + // I did an extra copy here, not sure how to avoid it and whether it's safe to avoid it + let mut output: Vec = Vec::with_capacity(num_rows as usize); + unsafe { + output.set_len(num_rows as usize); + std::ptr::copy_nonoverlapping( + self.decompressed_buf + .as_ptr() + .add(2 * rows_to_skip as usize), + output.as_ptr() as *mut u8, + num_rows as usize * 2, + ); + } + Ok(DataBlock::FixedWidth(FixedWidthDataBlock { + data: LanceBuffer::reinterpret_vec(output).to_owned(), + bits_per_value: self.uncompressed_bits_per_value, + num_values: num_rows, + })) + } + + 32 => { + // I did an extra copy here, not sure how to avoid it and whether it's safe to avoid it + let mut output: Vec = Vec::with_capacity(num_rows as usize); + unsafe { + output.set_len(num_rows as usize); + std::ptr::copy_nonoverlapping( + self.decompressed_buf + .as_ptr() + .add(4 * rows_to_skip as usize), + output.as_ptr() as *mut u8, + num_rows as usize * 4, + ); + } + Ok(DataBlock::FixedWidth(FixedWidthDataBlock { + data: LanceBuffer::reinterpret_vec(output).to_owned(), + bits_per_value: self.uncompressed_bits_per_value, + num_values: num_rows, + })) + } + + 64 => { + // I did an extra copy here, not sure how to avoid it and whether it's safe to avoid it + let mut output: Vec = Vec::with_capacity(num_rows as usize); + unsafe { + output.set_len(num_rows as usize); + std::ptr::copy_nonoverlapping( + self.decompressed_buf + .as_ptr() + .add(8 * rows_to_skip as usize), + output.as_ptr() as *mut u8, + num_rows as usize * 8, + ); + } + Ok(DataBlock::FixedWidth(FixedWidthDataBlock { + data: LanceBuffer::reinterpret_vec(output).to_owned(), + bits_per_value: self.uncompressed_bits_per_value, + num_values: num_rows, + })) + } + _ => { + panic!("Unsupported data type"); } - _ => panic!("Unsupported data type"), } } } @@ -355,10 +680,10 @@ impl PrimitivePageDecoder for BitpackedForNonNegPageDecoder { fn bitpacked_for_non_neg_decode( compressed_bit_width: u64, uncompressed_bits_per_value: u64, - data: &Vec, - bytes_idx_to_range_indices: &Vec>>, + data: &[Bytes], + bytes_idx_to_range_indices: &[Vec>], num_rows: u64, -) -> Vec { +) -> LanceBuffer { match uncompressed_bits_per_value { 8 => { let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); @@ -402,72 +727,777 @@ fn bitpacked_for_non_neg_decode( j += 1; } } - decompressed + LanceBuffer::Owned(decompressed) } - _ => panic!("Unsupported data type"), - } -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow::array::{Int32Array, PrimitiveArray}; - use arrow::buffer::Buffer; - use arrow::datatypes::DataType; - use arrow::record_batch::RecordBatch; - use arrow::util::bit_util::ceil; - - use crate::decoder::decode_batch; - use crate::decoder::DecoderMiddlewareChain; - use crate::decoder::FilterExpression; - use crate::encoder::encode_batch; - use crate::encoder::CoreFieldEncodingStrategy; - use crate::encoder::EncodingOptions; - use arrow::array::UInt8Array; - use arrow::datatypes::Field; - use arrow::datatypes::Schema; - - #[test_log::test(tokio::test)] - async fn test_compute_compressed_bit_width_for_non_neg() {} - - use std::collections::HashMap; - - use lance_datagen::{ByteCount, RowCount}; - - use crate::testing::{check_round_trip_encoding_of_data, TestCases}; - - #[test_log::test(tokio::test)] - async fn test_bitpack_fastlanes() { - let values: Vec = vec![5; 1024]; - let array = UInt8Array::from(values); - let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; - - let values: Vec = vec![66; 1000]; - let array = UInt8Array::from(values); - let array: Arc = Arc::new(array); - - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; - let values: Vec = vec![77; 2000]; - let array = UInt8Array::from(values); - let array: Arc = Arc::new(array); - - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; - - let values: Vec = vec![0; 10000]; - let array = UInt8Array::from(values); + 16 => { + let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); + let packed_chunk_size_in_byte: usize = 1024 * compressed_bit_width as usize / 8; + let mut decompress_chunk_buf = vec![0_u16; 1024]; + for (i, bytes) in data.iter().enumerate() { + let mut j = 0; + let mut ranges_idx = 0; + let mut curr_range_start = bytes_idx_to_range_indices[i][0].start; + while j * packed_chunk_size_in_byte < bytes.len() { + let chunk_in_u8: &[u8] = + &bytes[j * packed_chunk_size_in_byte..][..packed_chunk_size_in_byte]; + let chunk = cast_slice(chunk_in_u8); + unsafe { + BitPacking::unchecked_unpack( + compressed_bit_width as usize, + chunk, + &mut decompress_chunk_buf, + ); + } + loop { + if curr_range_start + 1024 < bytes_idx_to_range_indices[i][ranges_idx].end { + let this_part_len = 1024 - curr_range_start % 1024; + decompressed.extend_from_slice( + &decompress_chunk_buf[curr_range_start as usize % 1024..], + ); + curr_range_start += this_part_len; + break; + } else { + let this_part_len = + bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; + decompressed.extend_from_slice( + &decompress_chunk_buf[curr_range_start as usize % 1024..] + [..this_part_len as usize], + ); + ranges_idx += 1; + if ranges_idx == bytes_idx_to_range_indices[i].len() { + break; + } + curr_range_start = bytes_idx_to_range_indices[i][ranges_idx].start; + } + } + j += 1; + } + } + LanceBuffer::reinterpret_vec(decompressed).to_owned() + } + + 32 => { + let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); + let packed_chunk_size_in_byte: usize = 1024 * compressed_bit_width as usize / 8; + let mut decompress_chunk_buf = vec![0_u32; 1024]; + for (i, bytes) in data.iter().enumerate() { + let mut j = 0; + let mut ranges_idx = 0; + let mut curr_range_start = bytes_idx_to_range_indices[i][0].start; + while j * packed_chunk_size_in_byte < bytes.len() { + let chunk_in_u8: &[u8] = + &bytes[j * packed_chunk_size_in_byte..][..packed_chunk_size_in_byte]; + let chunk = cast_slice(chunk_in_u8); + unsafe { + BitPacking::unchecked_unpack( + compressed_bit_width as usize, + chunk, + &mut decompress_chunk_buf, + ); + } + loop { + if curr_range_start + 1024 < bytes_idx_to_range_indices[i][ranges_idx].end { + let this_part_len = 1024 - curr_range_start % 1024; + decompressed.extend_from_slice( + &decompress_chunk_buf[curr_range_start as usize % 1024..], + ); + curr_range_start += this_part_len; + break; + } else { + let this_part_len = + bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; + decompressed.extend_from_slice( + &decompress_chunk_buf[curr_range_start as usize % 1024..] + [..this_part_len as usize], + ); + ranges_idx += 1; + if ranges_idx == bytes_idx_to_range_indices[i].len() { + break; + } + curr_range_start = bytes_idx_to_range_indices[i][ranges_idx].start; + } + } + j += 1; + } + } + LanceBuffer::reinterpret_vec(decompressed).to_owned() + } + + 64 => { + let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); + let packed_chunk_size_in_byte: usize = 1024 * compressed_bit_width as usize / 8; + let mut decompress_chunk_buf = vec![0_u64; 1024]; + for (i, bytes) in data.iter().enumerate() { + let mut j = 0; + let mut ranges_idx = 0; + let mut curr_range_start = bytes_idx_to_range_indices[i][0].start; + while j * packed_chunk_size_in_byte < bytes.len() { + let chunk_in_u8: &[u8] = + &bytes[j * packed_chunk_size_in_byte..][..packed_chunk_size_in_byte]; + let chunk = cast_slice(chunk_in_u8); + unsafe { + BitPacking::unchecked_unpack( + compressed_bit_width as usize, + chunk, + &mut decompress_chunk_buf, + ); + } + loop { + if curr_range_start + 1024 < bytes_idx_to_range_indices[i][ranges_idx].end { + let this_part_len = 1024 - curr_range_start % 1024; + decompressed.extend_from_slice( + &decompress_chunk_buf[curr_range_start as usize % 1024..], + ); + curr_range_start += this_part_len; + break; + } else { + let this_part_len = + bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; + decompressed.extend_from_slice( + &decompress_chunk_buf[curr_range_start as usize % 1024..] + [..this_part_len as usize], + ); + ranges_idx += 1; + if ranges_idx == bytes_idx_to_range_indices[i].len() { + break; + } + curr_range_start = bytes_idx_to_range_indices[i][ranges_idx].start; + } + } + j += 1; + } + } + LanceBuffer::reinterpret_vec(decompressed).to_owned() + } + _ => panic!("Unsupported data type"), + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::array::{ + Int16Array, Int32Array, Int64Array, Int8Array, UInt16Array, UInt32Array, UInt64Array, + UInt8Array, + }; + use arrow::datatypes::DataType; + + #[test_log::test(tokio::test)] + async fn test_compute_compressed_bit_width_for_non_neg() {} + + use std::collections::HashMap; + + use lance_datagen::RowCount; + + use crate::testing::{check_round_trip_encoding_of_data, TestCases}; + + #[test_log::test(tokio::test)] + async fn test_bitpack_fastlanes_u8() { + let values: Vec = vec![5; 1024]; + let array = UInt8Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![66; 1000]; + let array = UInt8Array::from(values); + let array: Arc = Arc::new(array); + + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![77; 2000]; + let array = UInt8Array::from(values); + let array: Arc = Arc::new(array); + + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![0; 10000]; + let array = UInt8Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![88; 10000]; + let array = UInt8Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) + .into_batch_rows(RowCount::from(1)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) + .into_batch_rows(RowCount::from(20)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) + .into_batch_rows(RowCount::from(50)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) + .into_batch_rows(RowCount::from(100)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) + .into_batch_rows(RowCount::from(1000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) + .into_batch_rows(RowCount::from(1024)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) + .into_batch_rows(RowCount::from(2000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) + .into_batch_rows(RowCount::from(3000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + } + + #[test_log::test(tokio::test)] + async fn test_bitpack_fastlanes_u16() { + let values: Vec = vec![5; 1024]; + let array = UInt16Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![66; 1000]; + let array = UInt16Array::from(values); + let array: Arc = Arc::new(array); + + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![77; 2000]; + let array = UInt16Array::from(values); + let array: Arc = Arc::new(array); + + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![0; 10000]; + let array = UInt16Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![88; 10000]; + let array = UInt16Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![300; 100]; + let array = UInt16Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![800; 100]; + let array = UInt16Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) + .into_batch_rows(RowCount::from(1)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) + .into_batch_rows(RowCount::from(20)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) + .into_batch_rows(RowCount::from(100)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) + .into_batch_rows(RowCount::from(1000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) + .into_batch_rows(RowCount::from(1024)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) + .into_batch_rows(RowCount::from(2000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) + .into_batch_rows(RowCount::from(3000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + } + + #[test_log::test(tokio::test)] + async fn test_bitpack_fastlanes_u32() { + let values: Vec = vec![5; 1024]; + let array = UInt32Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![7; 2000]; + let array = UInt32Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![66; 1000]; + let array = UInt32Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![666; 1000]; + let array = UInt32Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![77; 2000]; + let array = UInt32Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![0; 10000]; + let array = UInt32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![1; 10000]; + let array = UInt32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![88; 10000]; + let array = UInt32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![300; 100]; + let array = UInt32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![3000; 100]; + let array = UInt32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![800; 100]; + let array = UInt32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![8000; 100]; + let array = UInt32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![65536; 100]; + let array = UInt32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![655360; 100]; + let array = UInt32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) + .into_batch_rows(RowCount::from(1)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) + .into_batch_rows(RowCount::from(20)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) + .into_batch_rows(RowCount::from(50)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) + .into_batch_rows(RowCount::from(100)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) + .into_batch_rows(RowCount::from(1000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) + .into_batch_rows(RowCount::from(1024)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) + .into_batch_rows(RowCount::from(2000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) + .into_batch_rows(RowCount::from(3000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + } + + #[test_log::test(tokio::test)] + async fn test_bitpack_fastlanes_u64() { + let values: Vec = vec![5; 1024]; + let array = UInt64Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![7; 2000]; + let array = UInt64Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![66; 1000]; + let array = UInt64Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![666; 1000]; + let array = UInt64Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![77; 2000]; + let array = UInt64Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![0; 10000]; + let array = UInt64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![1; 10000]; + let array = UInt64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![88; 10000]; + let array = UInt64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![300; 100]; + let array = UInt64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![3000; 100]; + let array = UInt64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![800; 100]; + let array = UInt64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![8000; 100]; + let array = UInt64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![65536; 100]; + let array = UInt64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![655360; 100]; + let array = UInt64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) + .into_batch_rows(RowCount::from(1)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) + .into_batch_rows(RowCount::from(20)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) + .into_batch_rows(RowCount::from(50)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) + .into_batch_rows(RowCount::from(100)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) + .into_batch_rows(RowCount::from(1000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) + .into_batch_rows(RowCount::from(1024)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) + .into_batch_rows(RowCount::from(2000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) + .into_batch_rows(RowCount::from(3000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + } + + #[test_log::test(tokio::test)] + async fn test_bitpack_fastlanes_i8() { + let values: Vec = vec![-5; 1024]; + let array = Int8Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![66; 1000]; + let array = Int8Array::from(values); + let array: Arc = Arc::new(array); + + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![77; 2000]; + let array = Int8Array::from(values); + let array: Arc = Arc::new(array); + + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![0; 10000]; + let array = Int8Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![88; 10000]; + let array = Int8Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-88; 10000]; + let array = Int8Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) + .into_batch_rows(RowCount::from(1)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) + .into_batch_rows(RowCount::from(20)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) + .into_batch_rows(RowCount::from(50)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) + .into_batch_rows(RowCount::from(100)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) + .into_batch_rows(RowCount::from(1000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) + .into_batch_rows(RowCount::from(1024)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) + .into_batch_rows(RowCount::from(2000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) + .into_batch_rows(RowCount::from(3000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + } + + #[test_log::test(tokio::test)] + async fn test_bitpack_fastlanes_i16() { + let values: Vec = vec![-5; 1024]; + let array = Int16Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + /* + let values: Vec = vec![66; 1000]; + let array = Int16Array::from(values); + let array: Arc = Arc::new(array); + + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![77; 2000]; + let array = Int16Array::from(values); + let array: Arc = Arc::new(array); + + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![0; 10000]; + let array = Int16Array::from(values); let arr = Arc::new(array) as ArrayRef; check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; - let values: Vec = vec![88; 10000]; - let array = UInt8Array::from(values); + let values: Vec = vec![88; 10000]; + let array = Int16Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![300; 100]; + let array = Int16Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![800; 100]; + let array = Int16Array::from(values); let arr = Arc::new(array) as ArrayRef; check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_primitive::( - arrow::datatypes::UInt8Type::DATA_TYPE, + .anon_col(lance_datagen::array::rand_type( + &DataType::Int16, )) .into_batch_rows(RowCount::from(1)) .unwrap() @@ -476,8 +1506,8 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_primitive::( - arrow::datatypes::UInt8Type::DATA_TYPE, + .anon_col(lance_datagen::array::rand_type( + &DataType::Int16, )) .into_batch_rows(RowCount::from(20)) .unwrap() @@ -486,8 +1516,8 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_primitive::( - arrow::datatypes::UInt8Type::DATA_TYPE, + .anon_col(lance_datagen::array::rand_type( + &DataType::Int16, )) .into_batch_rows(RowCount::from(50)) .unwrap() @@ -496,8 +1526,8 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_primitive::( - arrow::datatypes::UInt8Type::DATA_TYPE, + .anon_col(lance_datagen::array::rand_type( + &DataType::Int16, )) .into_batch_rows(RowCount::from(100)) .unwrap() @@ -506,8 +1536,8 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_primitive::( - arrow::datatypes::UInt8Type::DATA_TYPE, + .anon_col(lance_datagen::array::rand_type( + &DataType::Int16, )) .into_batch_rows(RowCount::from(1000)) .unwrap() @@ -516,8 +1546,8 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_primitive::( - arrow::datatypes::UInt8Type::DATA_TYPE, + .anon_col(lance_datagen::array::rand_type( + &DataType::Int16, )) .into_batch_rows(RowCount::from(1024)) .unwrap() @@ -526,8 +1556,8 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_primitive::( - arrow::datatypes::UInt8Type::DATA_TYPE, + .anon_col(lance_datagen::array::rand_type( + &DataType::Int16, )) .into_batch_rows(RowCount::from(2000)) .unwrap() @@ -536,13 +1566,288 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_primitive::( - arrow::datatypes::UInt8Type::DATA_TYPE, + .anon_col(lance_datagen::array::rand_type( + &DataType::Int16, )) .into_batch_rows(RowCount::from(3000)) .unwrap() .column(0) .clone(); check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + */ + } + + #[test_log::test(tokio::test)] + async fn test_bitpack_fastlanes_i32() { + let values: Vec = vec![-5; 1024]; + let array = Int32Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![66; 1000]; + let array = Int32Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-66; 1000]; + let array = Int32Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![77; 2000]; + let array = Int32Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-77; 2000]; + let array = Int32Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![0; 10000]; + let array = Int32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![88; 10000]; + let array = Int32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-88; 10000]; + let array = Int32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![300; 100]; + let array = Int32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-300; 100]; + let array = Int32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![800; 100]; + let array = Int32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-800; 100]; + let array = Int32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![65536; 100]; + let array = Int32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-65536; 100]; + let array = Int32Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) + .into_batch_rows(RowCount::from(1)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) + .into_batch_rows(RowCount::from(20)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) + .into_batch_rows(RowCount::from(50)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) + .into_batch_rows(RowCount::from(100)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) + .into_batch_rows(RowCount::from(1000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) + .into_batch_rows(RowCount::from(1024)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) + .into_batch_rows(RowCount::from(2000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) + .into_batch_rows(RowCount::from(3000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + } + + #[test_log::test(tokio::test)] + async fn test_bitpack_fastlanes_i64() { + let values: Vec = vec![-5; 1024]; + let array = Int64Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![66; 1000]; + let array = Int64Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-66; 1000]; + let array = Int64Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![77; 2000]; + let array = Int64Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-77; 2000]; + let array = Int64Array::from(values); + let array: Arc = Arc::new(array); + check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![0; 10000]; + let array = Int64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![88; 10000]; + let array = Int64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-88; 10000]; + let array = Int64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![300; 100]; + let array = Int64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-300; 100]; + let array = Int64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![800; 100]; + let array = Int64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-800; 100]; + let array = Int64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![65536; 100]; + let array = Int64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let values: Vec = vec![-65536; 100]; + let array = Int64Array::from(values); + let arr = Arc::new(array) as ArrayRef; + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) + .into_batch_rows(RowCount::from(1)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) + .into_batch_rows(RowCount::from(20)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) + .into_batch_rows(RowCount::from(50)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) + .into_batch_rows(RowCount::from(100)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) + .into_batch_rows(RowCount::from(1000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) + .into_batch_rows(RowCount::from(1024)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) + .into_batch_rows(RowCount::from(2000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + + let arr = lance_datagen::gen() + .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) + .into_batch_rows(RowCount::from(3000)) + .unwrap() + .column(0) + .clone(); + check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; } } From 7c21438b611392668df0906dec57c159edd90294 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Wed, 18 Sep 2024 19:03:19 +0000 Subject: [PATCH 06/31] address initial PR comments --- rust/lance-encoding/src/buffer.rs | 30 ----------------- rust/lance-encoding/src/encoder.rs | 32 +++++++++++++------ .../encodings/physical/bitpack_fastlanes.rs | 27 +++++----------- 3 files changed, 30 insertions(+), 59 deletions(-) diff --git a/rust/lance-encoding/src/buffer.rs b/rust/lance-encoding/src/buffer.rs index 483259911b..3e17588d71 100644 --- a/rust/lance-encoding/src/buffer.rs +++ b/rust/lance-encoding/src/buffer.rs @@ -203,36 +203,6 @@ impl LanceBuffer { Self::Borrowed(Buffer::from_vec(vec)) } - pub fn reinterpret_to_rust_native(&mut self) -> Result<&[T]> - where - T: Copy, // Ensure `T` can be copied (as needed for safely reinterpreting bytes) - { - let buffer = self.borrow_and_clone(); - - let buffer = buffer.into_buffer(); - - // Get the raw byte slice from the buffer. - let byte_slice = buffer.as_slice(); - - // Safety check - ensure that the byte slice length is a multiple of `T`. - if byte_slice.len() % std::mem::size_of::() != 0 { - return Err(Error::Internal { - message: "Buffer size is not a multiple of the target type size".to_string(), - location: location!(), - }); - } - - // Reinterpret the byte slice as a slice of `T`. - let typed_slice = unsafe { - std::slice::from_raw_parts( - byte_slice.as_ptr() as *const T, - byte_slice.len() / std::mem::size_of::(), - ) - }; - - Ok(typed_slice) - } - /// Reinterprets a LanceBuffer into a Vec /// /// Unfortunately, there is no way to do this safely in Rust without a copy, even if diff --git a/rust/lance-encoding/src/encoder.rs b/rust/lance-encoding/src/encoder.rs index a92cfea9c2..0b4bb01e4a 100644 --- a/rust/lance-encoding/src/encoder.rs +++ b/rust/lance-encoding/src/encoder.rs @@ -334,21 +334,33 @@ impl CoreArrayEncodingStrategy { Ok(Box::new(PackedStructEncoder::new(inner_encoders))) } DataType::UInt8 | DataType::UInt16 | DataType::UInt32 | DataType::UInt64 => { - let compressed_bit_width = compute_compressed_bit_width_for_non_neg(arrays); - Ok(Box::new(BitpackedForNonNegArrayEncoder::new( - compressed_bit_width as usize, - data_type.clone(), - ))) + if version >= LanceFileVersion::V2_1 { + let compressed_bit_width = compute_compressed_bit_width_for_non_neg(arrays); + Ok(Box::new(BitpackedForNonNegArrayEncoder::new( + compressed_bit_width as usize, + data_type.clone(), + ))) + } else { + Ok(Box::new(BasicEncoder::new(Box::new( + ValueEncoder::default(), + )))) + } } // for signed integers, I intend to make it a cascaded encoding, a sparse array for the negative values and very wide(bit-width) values, // then a bitpacked array for the narrow(bit-width) values, I need `BitpackedForNeg` to be merged first DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64 => { - let compressed_bit_width = compute_compressed_bit_width_for_non_neg(arrays); - Ok(Box::new(BitpackedForNonNegArrayEncoder::new( - compressed_bit_width as usize, - data_type.clone(), - ))) + if version >= LanceFileVersion::V2_1 { + let compressed_bit_width = compute_compressed_bit_width_for_non_neg(arrays); + Ok(Box::new(BitpackedForNonNegArrayEncoder::new( + compressed_bit_width as usize, + data_type.clone(), + ))) + } else { + Ok(Box::new(BasicEncoder::new(Box::new( + ValueEncoder::default(), + )))) + } } _ => Ok(Box::new(BasicEncoder::new(Box::new( ValueEncoder::default(), diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index eebf302fae..a1aebe8879 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -25,8 +25,8 @@ use arrow::array::ArrayRef; use bytemuck::cast_slice; use fastlanes::BitPacking; -// Compute the compressed_bit_width for a given array of unsigned integers -// the vortex approach is better, they compute all stastistics before encoding +// Compute the compressed_bit_width for a given array of integers +// todo: compute all statistics before encoding // todo: see how to use rust macro to rewrite this function pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { // is it possible to get here? @@ -40,12 +40,10 @@ pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { DataType::UInt8 => { let mut global_max: u8 = 0; for array in arrays { - // at this point we know that the array doesn't contain any negative values let primitive_array = array .as_any() .downcast_ref::>() .unwrap(); - //println!("primitive_array: {:?}", primitive_array); let array_max = arrow::compute::bit_or(primitive_array); global_max = global_max.max(array_max.unwrap_or(0)); } @@ -231,7 +229,9 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { // 1024 * compressed_bit_width / 8 let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); - let input = unpacked.data.reinterpret_to_rust_native::().unwrap(); + let input_slice = unpacked.data.borrow_to_typed_slice::(); + let input = input_slice.as_ref(); + let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); // Loop over all but the last chunk. @@ -293,7 +293,8 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { // the output type is the same as the input type let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); - let input = unpacked.data.reinterpret_to_rust_native::().unwrap(); + let input_slice = unpacked.data.borrow_to_typed_slice::(); + let input = input_slice.as_ref(); let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); // Loop over all but the last chunk. @@ -551,16 +552,6 @@ impl PageScheduler for BitpackedForNonNegScheduler { byte_ranges[0].start, byte_ranges.last().unwrap().end ); - /* - println!( - "Scheduling I/O for {} ranges spread across byte range {}..{}", - byte_ranges.len(), - byte_ranges[0].start, - byte_ranges[byte_ranges.len() - 1].end - ); - println!("ranges: {:?}", ranges); - println!("byte_ranges: {:?}", byte_ranges); - */ let bytes = scheduler.submit_request(byte_ranges.clone(), top_level_row); @@ -870,7 +861,7 @@ fn bitpacked_for_non_neg_decode( } LanceBuffer::reinterpret_vec(decompressed).to_owned() } - _ => panic!("Unsupported data type"), + _ => unreachable!("bitpacked_for_non_neg_decode only supports 8, 16, 32, 64 uncompressed_bits_per_value"), } } @@ -1462,7 +1453,6 @@ mod tests { let array: Arc = Arc::new(array); check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; - /* let values: Vec = vec![66; 1000]; let array = Int16Array::from(values); let array: Arc = Arc::new(array); @@ -1574,7 +1564,6 @@ mod tests { .column(0) .clone(); check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; - */ } #[test_log::test(tokio::test)] From f0bd3a8202efe5b31d95fbb541dc5759a8e0b242 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Wed, 18 Sep 2024 20:01:18 +0000 Subject: [PATCH 07/31] fix lint --- .../encodings/physical/bitpack_fastlanes.rs | 40 ++++++------------- 1 file changed, 13 insertions(+), 27 deletions(-) diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index a1aebe8879..ccf1b18a24 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -1,4 +1,4 @@ -// spdx-license-identifier: apache-2.0 +// SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright The Lance Authors use std::sync::Arc; @@ -26,7 +26,7 @@ use bytemuck::cast_slice; use fastlanes::BitPacking; // Compute the compressed_bit_width for a given array of integers -// todo: compute all statistics before encoding +// todo: compute all statistics before encoding // todo: see how to use rust macro to rewrite this function pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { // is it possible to get here? @@ -861,7 +861,9 @@ fn bitpacked_for_non_neg_decode( } LanceBuffer::reinterpret_vec(decompressed).to_owned() } - _ => unreachable!("bitpacked_for_non_neg_decode only supports 8, 16, 32, 64 uncompressed_bits_per_value"), + _ => unreachable!( + "bitpacked_for_non_neg_decode only supports 8, 16, 32, 64 uncompressed_bits_per_value" + ), } } @@ -1486,9 +1488,7 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_type( - &DataType::Int16, - )) + .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) .into_batch_rows(RowCount::from(1)) .unwrap() .column(0) @@ -1496,9 +1496,7 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_type( - &DataType::Int16, - )) + .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) .into_batch_rows(RowCount::from(20)) .unwrap() .column(0) @@ -1506,9 +1504,7 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_type( - &DataType::Int16, - )) + .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) .into_batch_rows(RowCount::from(50)) .unwrap() .column(0) @@ -1516,9 +1512,7 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_type( - &DataType::Int16, - )) + .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) .into_batch_rows(RowCount::from(100)) .unwrap() .column(0) @@ -1526,9 +1520,7 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_type( - &DataType::Int16, - )) + .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) .into_batch_rows(RowCount::from(1000)) .unwrap() .column(0) @@ -1536,9 +1528,7 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_type( - &DataType::Int16, - )) + .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) .into_batch_rows(RowCount::from(1024)) .unwrap() .column(0) @@ -1546,9 +1536,7 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_type( - &DataType::Int16, - )) + .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) .into_batch_rows(RowCount::from(2000)) .unwrap() .column(0) @@ -1556,9 +1544,7 @@ mod tests { check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; let arr = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_type( - &DataType::Int16, - )) + .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) .into_batch_rows(RowCount::from(3000)) .unwrap() .column(0) From ce0f798597aa63ec24ebb4ab87f6f53f76685028 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Wed, 18 Sep 2024 21:10:08 +0000 Subject: [PATCH 08/31] return a slice of LanceBuffer in `decode` --- rust/lance-encoding/src/buffer.rs | 23 ++++ .../encodings/physical/bitpack_fastlanes.rs | 113 ++++++------------ 2 files changed, 59 insertions(+), 77 deletions(-) diff --git a/rust/lance-encoding/src/buffer.rs b/rust/lance-encoding/src/buffer.rs index 3e17588d71..d7b15bdc71 100644 --- a/rust/lance-encoding/src/buffer.rs +++ b/rust/lance-encoding/src/buffer.rs @@ -283,6 +283,29 @@ impl LanceBuffer { pub fn copy_array(array: [u8; N]) -> Self { Self::Owned(Vec::from(array)) } + + /// Returns a new [LanceBuffer] that is a slice of this buffer starting at `offset`, + /// with `length` bytes. + /// Doing so allows the same memory region to be shared between lance buffers. + /// # Panics + /// Panics if `(offset + length)` is larger than the existing length. + pub fn slice_with_length(&self, offset: usize, length: usize) -> Self { + let original_buffer_len = match self { + Self::Borrowed(buffer) => buffer.len(), + Self::Owned(buffer) => buffer.len(), + }; + assert!( + offset.saturating_add(length) <= original_buffer_len, + "the offset of the new Buffer cannot exceed the existing length" + ); + match self { + Self::Borrowed(buffer) => Self::Borrowed(buffer.slice_with_length(offset, length)), + // A copy happened during `Buffer::from_slice_ref` + Self::Owned(buffer) => { + Self::Borrowed(Buffer::from_slice_ref(&buffer[offset..offset + length])) + } + } + } } impl AsRef<[u8]> for LanceBuffer { diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index ccf1b18a24..db324f9ada 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -276,7 +276,7 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { ); let packed = DataBlock::FixedWidth(FixedWidthDataBlock { bits_per_value: self.compressed_bit_width as u64, - data: LanceBuffer::Owned(output), + data: LanceBuffer::reinterpret_vec(output), num_values: unpacked.num_values, }); @@ -339,7 +339,7 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { ); let packed = DataBlock::FixedWidth(FixedWidthDataBlock { bits_per_value: self.compressed_bit_width as u64, - data: LanceBuffer::reinterpret_vec(output).to_owned(), + data: LanceBuffer::reinterpret_vec(output), num_values: unpacked.num_values, }); Ok(EncodedArray { @@ -401,7 +401,7 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { ); let packed = DataBlock::FixedWidth(FixedWidthDataBlock { bits_per_value: self.compressed_bit_width as u64, - data: LanceBuffer::reinterpret_vec(output).to_owned(), + data: LanceBuffer::reinterpret_vec(output), num_values: unpacked.num_values, }); @@ -464,7 +464,7 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { ); let packed = DataBlock::FixedWidth(FixedWidthDataBlock { bits_per_value: self.compressed_bit_width as u64, - data: LanceBuffer::reinterpret_vec(output).to_owned(), + data: LanceBuffer::reinterpret_vec(output), num_values: unpacked.num_values, }); @@ -589,80 +589,39 @@ struct BitpackedForNonNegPageDecoder { impl PrimitivePageDecoder for BitpackedForNonNegPageDecoder { fn decode(&self, rows_to_skip: u64, num_rows: u64) -> Result { match self.uncompressed_bits_per_value { - 8 => { - // I did an extra copy here, not sure how to avoid it and whether it's safe to avoid it - let mut output = Vec::with_capacity(num_rows as usize); - output.extend_from_slice( - &self.decompressed_buf[rows_to_skip as usize..][..num_rows as usize], - ); - Ok(DataBlock::FixedWidth(FixedWidthDataBlock { - data: LanceBuffer::from(output), - bits_per_value: self.uncompressed_bits_per_value, - num_values: num_rows, - })) - } - - 16 => { - // I did an extra copy here, not sure how to avoid it and whether it's safe to avoid it - let mut output: Vec = Vec::with_capacity(num_rows as usize); - unsafe { - output.set_len(num_rows as usize); - std::ptr::copy_nonoverlapping( - self.decompressed_buf - .as_ptr() - .add(2 * rows_to_skip as usize), - output.as_ptr() as *mut u8, - num_rows as usize * 2, - ); - } - Ok(DataBlock::FixedWidth(FixedWidthDataBlock { - data: LanceBuffer::reinterpret_vec(output).to_owned(), - bits_per_value: self.uncompressed_bits_per_value, - num_values: num_rows, - })) - } - - 32 => { - // I did an extra copy here, not sure how to avoid it and whether it's safe to avoid it - let mut output: Vec = Vec::with_capacity(num_rows as usize); - unsafe { - output.set_len(num_rows as usize); - std::ptr::copy_nonoverlapping( - self.decompressed_buf - .as_ptr() - .add(4 * rows_to_skip as usize), - output.as_ptr() as *mut u8, - num_rows as usize * 4, - ); - } - Ok(DataBlock::FixedWidth(FixedWidthDataBlock { - data: LanceBuffer::reinterpret_vec(output).to_owned(), - bits_per_value: self.uncompressed_bits_per_value, - num_values: num_rows, - })) - } - - 64 => { - // I did an extra copy here, not sure how to avoid it and whether it's safe to avoid it - let mut output: Vec = Vec::with_capacity(num_rows as usize); - unsafe { - output.set_len(num_rows as usize); - std::ptr::copy_nonoverlapping( - self.decompressed_buf - .as_ptr() - .add(8 * rows_to_skip as usize), - output.as_ptr() as *mut u8, - num_rows as usize * 8, - ); - } - Ok(DataBlock::FixedWidth(FixedWidthDataBlock { - data: LanceBuffer::reinterpret_vec(output).to_owned(), - bits_per_value: self.uncompressed_bits_per_value, - num_values: num_rows, - })) - } + 8 => Ok(DataBlock::FixedWidth(FixedWidthDataBlock { + data: self + .decompressed_buf + .slice_with_length(rows_to_skip as usize, num_rows as usize), + bits_per_value: self.uncompressed_bits_per_value, + num_values: num_rows, + })), + + 16 => Ok(DataBlock::FixedWidth(FixedWidthDataBlock { + data: self + .decompressed_buf + .slice_with_length((rows_to_skip * 2) as usize, (num_rows * 2) as usize), + bits_per_value: self.uncompressed_bits_per_value, + num_values: num_rows, + })), + + 32 => Ok(DataBlock::FixedWidth(FixedWidthDataBlock { + data: self + .decompressed_buf + .slice_with_length((rows_to_skip * 4) as usize, (num_rows * 4) as usize), + bits_per_value: self.uncompressed_bits_per_value, + num_values: num_rows, + })), + + 64 => Ok(DataBlock::FixedWidth(FixedWidthDataBlock { + data: self + .decompressed_buf + .slice_with_length((rows_to_skip * 8) as usize, (num_rows * 8) as usize), + bits_per_value: self.uncompressed_bits_per_value, + num_values: num_rows, + })), _ => { - panic!("Unsupported data type"); + unreachable!("bitpacked_for_non_neg_decode only supports 8, 16, 32, 64 uncompressed_bits_per_value") } } } From fb9ede24ee2165d23abbd8c71f1431ad2c9063d5 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Thu, 19 Sep 2024 15:00:05 +0000 Subject: [PATCH 09/31] use `elems_per_chunk` constant to represent 1024, delete `bench_decode2`function --- rust/lance-encoding/benches/decoder.rs | 11 +- .../encodings/physical/bitpack_fastlanes.rs | 177 +++++++++--------- 2 files changed, 94 insertions(+), 94 deletions(-) diff --git a/rust/lance-encoding/benches/decoder.rs b/rust/lance-encoding/benches/decoder.rs index e8217a76cf..3332050cb1 100644 --- a/rust/lance-encoding/benches/decoder.rs +++ b/rust/lance-encoding/benches/decoder.rs @@ -2,7 +2,7 @@ // SPDX-FileCopyrightText: Copyright The Lance Authors use std::{collections::HashMap, sync::Arc}; -use arrow_array::{RecordBatch, UInt32Array, UInt8Array}; +use arrow_array::{RecordBatch, UInt32Array}; use arrow_schema::{DataType, Field, Schema, TimeUnit}; use arrow_select::take::take; use criterion::{criterion_group, criterion_main, Criterion}; @@ -60,6 +60,7 @@ const ENCODING_OPTIONS: EncodingOptions = EncodingOptions { keep_original_array: true, }; +/* fn bench_decode2(c: &mut Criterion) { let rt = tokio::runtime::Runtime::new().unwrap(); let mut group = c.benchmark_group("decode_uint8"); @@ -100,14 +101,15 @@ fn bench_decode2(c: &mut Criterion) { }) }); } +*/ fn bench_decode(c: &mut Criterion) { let rt = tokio::runtime::Runtime::new().unwrap(); let mut group = c.benchmark_group("decode_primitive"); for data_type in PRIMITIVE_TYPES { let data = lance_datagen::gen() - .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) - .into_batch_rows(lance_datagen::RowCount::from(1024 * 1024)) + .anon_col(lance_datagen::array::rand_type(data_type)) + .into_batch_rows(lance_datagen::RowCount::from(1024 * 1024 * 1024)) .unwrap(); let lance_schema = Arc::new(lance_core::datatypes::Schema::try_from(data.schema().as_ref()).unwrap()); @@ -137,6 +139,7 @@ fn bench_decode(c: &mut Criterion) { }); } } + fn bench_decode_fsl(c: &mut Criterion) { let rt = tokio::runtime::Runtime::new().unwrap(); let mut group = c.benchmark_group("decode_primitive_fsl"); @@ -355,7 +358,7 @@ criterion_group!( name=benches; config = Criterion::default().significance_level(0.1).sample_size(10) .with_profiler(pprof::criterion::PProfProfiler::new(100, pprof::criterion::Output::Flamegraph(None))); - targets = bench_decode2, bench_decode, bench_decode_fsl, bench_decode_str_with_dict_encoding, bench_decode_packed_struct, + targets = bench_decode, bench_decode_fsl, bench_decode_str_with_dict_encoding, bench_decode_packed_struct, bench_decode_str_with_fixed_size_binary_encoding); // Non-linux version does not support pprof. diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index db324f9ada..44ecbd2191 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -25,6 +25,8 @@ use arrow::array::ArrayRef; use bytemuck::cast_slice; use fastlanes::BitPacking; +const ELEMS_PER_CHUNK: u64 = 1024; + // Compute the compressed_bit_width for a given array of integers // todo: compute all statistics before encoding // todo: see how to use rust macro to rewrite this function @@ -222,8 +224,8 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { }; match _data_type { DataType::UInt8 | DataType::Int8 => { - let num_chunks = (unpacked.num_values + 1023) / 1024; - let num_full_chunks = unpacked.num_values / 1024; + let num_chunks = (unpacked.num_values + ELEMS_PER_CHUNK - 1) / ELEMS_PER_CHUNK; + let num_full_chunks = unpacked.num_values / ELEMS_PER_CHUNK; // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 // the output type is the same as the input type // 1024 * compressed_bit_width / 8 @@ -236,22 +238,22 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { // Loop over all but the last chunk. (0..num_full_chunks).for_each(|i| { - let start_elem = i as usize * 1024_usize; + let start_elem = (i * ELEMS_PER_CHUNK) as usize; let output_len = output.len(); unsafe { output.set_len(output_len + packed_chunk_size); BitPacking::unchecked_pack( self.compressed_bit_width, - &input[start_elem..][..1024], + &input[start_elem..][..ELEMS_PER_CHUNK as usize], &mut output[output_len..][..packed_chunk_size], ); }; }); if num_chunks != num_full_chunks { - let last_chunk_elem_num = unpacked.num_values % 1024; - let mut last_chunk = vec![0u8; 1024]; + let last_chunk_elem_num = unpacked.num_values % ELEMS_PER_CHUNK; + let mut last_chunk = vec![0u8; ELEMS_PER_CHUNK as usize]; last_chunk[..last_chunk_elem_num as usize].clone_from_slice( &input[unpacked.num_values as usize - last_chunk_elem_num as usize..], ); @@ -287,34 +289,36 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { } DataType::UInt16 | DataType::Int16 => { - let num_chunks = (unpacked.num_values + 1023) / 1024; - let num_full_chunks = unpacked.num_values / 1024; + let num_chunks = (unpacked.num_values + ELEMS_PER_CHUNK - 1) / ELEMS_PER_CHUNK; + let num_full_chunks = unpacked.num_values / ELEMS_PER_CHUNK; // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 // the output type is the same as the input type + // 1024 * compressed_bit_width / 8 let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); let input_slice = unpacked.data.borrow_to_typed_slice::(); let input = input_slice.as_ref(); + let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); // Loop over all but the last chunk. (0..num_full_chunks).for_each(|i| { - let start_elem = i as usize * 1024_usize; + let start_elem = (i * ELEMS_PER_CHUNK) as usize; let output_len = output.len(); unsafe { output.set_len(output_len + packed_chunk_size); BitPacking::unchecked_pack( self.compressed_bit_width, - &input[start_elem..][..1024], + &input[start_elem..][..ELEMS_PER_CHUNK as usize], &mut output[output_len..][..packed_chunk_size], ); }; }); if num_chunks != num_full_chunks { - let last_chunk_elem_num = unpacked.num_values % 1024; - let mut last_chunk = vec![0u16; 1024]; + let last_chunk_elem_num = unpacked.num_values % ELEMS_PER_CHUNK; + let mut last_chunk = vec![0u16; ELEMS_PER_CHUNK as usize]; last_chunk[..last_chunk_elem_num as usize].clone_from_slice( &input[unpacked.num_values as usize - last_chunk_elem_num as usize..], ); @@ -342,6 +346,7 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { data: LanceBuffer::reinterpret_vec(output), num_values: unpacked.num_values, }); + Ok(EncodedArray { data: packed, encoding, @@ -349,34 +354,36 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { } DataType::UInt32 | DataType::Int32 => { - let num_chunks = (unpacked.num_values + 1023) / 1024; - let num_full_chunks = unpacked.num_values / 1024; + let num_chunks = (unpacked.num_values + ELEMS_PER_CHUNK - 1) / ELEMS_PER_CHUNK; + let num_full_chunks = unpacked.num_values / ELEMS_PER_CHUNK; // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 // the output type is the same as the input type + // 1024 * compressed_bit_width / 8 let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); let input_slice = unpacked.data.borrow_to_typed_slice::(); let input = input_slice.as_ref(); + let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); // Loop over all but the last chunk. (0..num_full_chunks).for_each(|i| { - let start_elem = i as usize * 1024_usize; + let start_elem = (i * ELEMS_PER_CHUNK) as usize; let output_len = output.len(); unsafe { output.set_len(output_len + packed_chunk_size); BitPacking::unchecked_pack( self.compressed_bit_width, - &input[start_elem..][..1024], + &input[start_elem..][..ELEMS_PER_CHUNK as usize], &mut output[output_len..][..packed_chunk_size], ); }; }); if num_chunks != num_full_chunks { - let last_chunk_elem_num = unpacked.num_values % 1024; - let mut last_chunk = vec![0u32; 1024]; + let last_chunk_elem_num = unpacked.num_values % ELEMS_PER_CHUNK; + let mut last_chunk = vec![0u32; ELEMS_PER_CHUNK as usize]; last_chunk[..last_chunk_elem_num as usize].clone_from_slice( &input[unpacked.num_values as usize - last_chunk_elem_num as usize..], ); @@ -412,34 +419,36 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { } DataType::UInt64 | DataType::Int64 => { - let num_chunks = (unpacked.num_values + 1023) / 1024; - let num_full_chunks = unpacked.num_values / 1024; + let num_chunks = (unpacked.num_values + ELEMS_PER_CHUNK - 1) / ELEMS_PER_CHUNK; + let num_full_chunks = unpacked.num_values / ELEMS_PER_CHUNK; // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 // the output type is the same as the input type + // 1024 * compressed_bit_width / 8 let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); let input_slice = unpacked.data.borrow_to_typed_slice::(); let input = input_slice.as_ref(); + let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); // Loop over all but the last chunk. (0..num_full_chunks).for_each(|i| { - let start_elem = i as usize * 1024_usize; + let start_elem = (i * ELEMS_PER_CHUNK) as usize; let output_len = output.len(); unsafe { output.set_len(output_len + packed_chunk_size); BitPacking::unchecked_pack( self.compressed_bit_width, - &input[start_elem..][..1024], + &input[start_elem..][..ELEMS_PER_CHUNK as usize], &mut output[output_len..][..packed_chunk_size], ); }; }); if num_chunks != num_full_chunks { - let last_chunk_elem_num = unpacked.num_values % 1024; - let mut last_chunk = vec![0u64; 1024]; + let last_chunk_elem_num = unpacked.num_values % ELEMS_PER_CHUNK; + let mut last_chunk = vec![0u64; ELEMS_PER_CHUNK as usize]; last_chunk[..last_chunk_elem_num as usize].clone_from_slice( &input[unpacked.num_values as usize - last_chunk_elem_num as usize..], ); @@ -487,15 +496,13 @@ pub struct BitpackedForNonNegScheduler { } fn locate_chunk_start(scheduler: &BitpackedForNonNegScheduler, relative_row_num: u64) -> u64 { - let elems_per_chunk = 1024; - let chunk_size = elems_per_chunk * scheduler.compressed_bit_width / 8; - relative_row_num / elems_per_chunk * chunk_size + let chunk_size = ELEMS_PER_CHUNK * scheduler.compressed_bit_width / 8; + relative_row_num / ELEMS_PER_CHUNK * chunk_size } fn locate_chunk_end(scheduler: &BitpackedForNonNegScheduler, relative_row_num: u64) -> u64 { - let elems_per_chunk: u64 = 1024; - let chunk_size = elems_per_chunk * scheduler.compressed_bit_width / 8; - relative_row_num / elems_per_chunk * chunk_size + chunk_size + let chunk_size = ELEMS_PER_CHUNK * scheduler.compressed_bit_width / 8; + relative_row_num / ELEMS_PER_CHUNK * chunk_size + chunk_size } impl BitpackedForNonNegScheduler { @@ -524,6 +531,8 @@ impl PageScheduler for BitpackedForNonNegScheduler { panic!("cannot schedule empty ranges"); } let mut byte_ranges = vec![]; + + // map one bytes to multiple ranges, one bytes has at least one range corresponding to it let mut bytes_idx_to_range_indices = vec![]; let first_byte_range = std::ops::Range { start: self.buffer_offset + locate_chunk_start(self, ranges[0].start), @@ -531,9 +540,13 @@ impl PageScheduler for BitpackedForNonNegScheduler { }; // the ranges are half-open byte_ranges.push(first_byte_range); bytes_idx_to_range_indices.push(vec![ranges[0].clone()]); + for (i, range) in ranges.iter().enumerate().skip(1) { let this_start = locate_chunk_start(self, range.start); let this_end = locate_chunk_end(self, range.end - 1); + + // when the current range start is in the same chunk as the previous range's end, we colaesce this two bytes ranges + // when the current range start is not in the same chunk as the previous range's end, we create a new bytes range if this_start == locate_chunk_start(self, ranges[i - 1].end - 1) { byte_ranges.last_mut().unwrap().end = self.buffer_offset + this_end; bytes_idx_to_range_indices @@ -588,42 +601,23 @@ struct BitpackedForNonNegPageDecoder { impl PrimitivePageDecoder for BitpackedForNonNegPageDecoder { fn decode(&self, rows_to_skip: u64, num_rows: u64) -> Result { - match self.uncompressed_bits_per_value { - 8 => Ok(DataBlock::FixedWidth(FixedWidthDataBlock { - data: self - .decompressed_buf - .slice_with_length(rows_to_skip as usize, num_rows as usize), - bits_per_value: self.uncompressed_bits_per_value, - num_values: num_rows, - })), - - 16 => Ok(DataBlock::FixedWidth(FixedWidthDataBlock { - data: self - .decompressed_buf - .slice_with_length((rows_to_skip * 2) as usize, (num_rows * 2) as usize), - bits_per_value: self.uncompressed_bits_per_value, - num_values: num_rows, - })), - - 32 => Ok(DataBlock::FixedWidth(FixedWidthDataBlock { - data: self - .decompressed_buf - .slice_with_length((rows_to_skip * 4) as usize, (num_rows * 4) as usize), - bits_per_value: self.uncompressed_bits_per_value, - num_values: num_rows, - })), - - 64 => Ok(DataBlock::FixedWidth(FixedWidthDataBlock { - data: self - .decompressed_buf - .slice_with_length((rows_to_skip * 8) as usize, (num_rows * 8) as usize), - bits_per_value: self.uncompressed_bits_per_value, - num_values: num_rows, - })), - _ => { - unreachable!("bitpacked_for_non_neg_decode only supports 8, 16, 32, 64 uncompressed_bits_per_value") - } + if ![8, 16, 32, 64].contains(&self.uncompressed_bits_per_value) { + return Err(Error::InvalidInput { + source: "BitpackedForNonNegPageDecoder should only has uncompressed_bits_per_value of 8, 16, 32, or 64".into(), + location: location!(), + }); } + + let elem_size_in_bytes = self.uncompressed_bits_per_value / 8; + + Ok(DataBlock::FixedWidth(FixedWidthDataBlock { + data: self.decompressed_buf.slice_with_length( + (rows_to_skip * elem_size_in_bytes) as usize, + (num_rows * elem_size_in_bytes) as usize, + ), + bits_per_value: self.uncompressed_bits_per_value, + num_values: num_rows, + })) } } @@ -637,8 +631,8 @@ fn bitpacked_for_non_neg_decode( match uncompressed_bits_per_value { 8 => { let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); - let packed_chunk_size: usize = 1024 * compressed_bit_width as usize / 8; - let mut decompress_chunk_buf = vec![0_u8; 1024]; + let packed_chunk_size: usize = ELEMS_PER_CHUNK as usize * compressed_bit_width as usize / 8; + let mut decompress_chunk_buf = vec![0_u8; ELEMS_PER_CHUNK as usize]; for (i, bytes) in data.iter().enumerate() { let mut j = 0; let mut ranges_idx = 0; @@ -649,14 +643,14 @@ fn bitpacked_for_non_neg_decode( BitPacking::unchecked_unpack( compressed_bit_width as usize, chunk, - &mut decompress_chunk_buf[..1024], + &mut decompress_chunk_buf[..ELEMS_PER_CHUNK as usize], ); } loop { - if curr_range_start + 1024 < bytes_idx_to_range_indices[i][ranges_idx].end { - let this_part_len = 1024 - curr_range_start % 1024; + if curr_range_start + ELEMS_PER_CHUNK < bytes_idx_to_range_indices[i][ranges_idx].end { + let this_part_len = ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; decompressed.extend_from_slice( - &decompress_chunk_buf[curr_range_start as usize % 1024..], + &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..], ); curr_range_start += this_part_len; break; @@ -664,7 +658,7 @@ fn bitpacked_for_non_neg_decode( let this_part_len = bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; decompressed.extend_from_slice( - &decompress_chunk_buf[curr_range_start as usize % 1024..] + &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..] [..this_part_len as usize], ); ranges_idx += 1; @@ -682,8 +676,8 @@ fn bitpacked_for_non_neg_decode( 16 => { let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); - let packed_chunk_size_in_byte: usize = 1024 * compressed_bit_width as usize / 8; - let mut decompress_chunk_buf = vec![0_u16; 1024]; + let packed_chunk_size_in_byte: usize = (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; + let mut decompress_chunk_buf = vec![0_u16; ELEMS_PER_CHUNK as usize]; for (i, bytes) in data.iter().enumerate() { let mut j = 0; let mut ranges_idx = 0; @@ -700,18 +694,21 @@ fn bitpacked_for_non_neg_decode( ); } loop { - if curr_range_start + 1024 < bytes_idx_to_range_indices[i][ranges_idx].end { - let this_part_len = 1024 - curr_range_start % 1024; + if curr_range_start + ELEMS_PER_CHUNK < bytes_idx_to_range_indices[i][ranges_idx].end { + let this_part_len = ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; decompressed.extend_from_slice( - &decompress_chunk_buf[curr_range_start as usize % 1024..], + &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..], ); curr_range_start += this_part_len; + + // when `curr_range_start + 1024 < bytes_idx_to_range_indices[i][ranges_idx].end`, + // we know this chunk has only data of this range break; } else { let this_part_len = bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; decompressed.extend_from_slice( - &decompress_chunk_buf[curr_range_start as usize % 1024..] + &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..] [..this_part_len as usize], ); ranges_idx += 1; @@ -729,8 +726,8 @@ fn bitpacked_for_non_neg_decode( 32 => { let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); - let packed_chunk_size_in_byte: usize = 1024 * compressed_bit_width as usize / 8; - let mut decompress_chunk_buf = vec![0_u32; 1024]; + let packed_chunk_size_in_byte: usize = (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; + let mut decompress_chunk_buf = vec![0_u32; ELEMS_PER_CHUNK as usize]; for (i, bytes) in data.iter().enumerate() { let mut j = 0; let mut ranges_idx = 0; @@ -747,10 +744,10 @@ fn bitpacked_for_non_neg_decode( ); } loop { - if curr_range_start + 1024 < bytes_idx_to_range_indices[i][ranges_idx].end { - let this_part_len = 1024 - curr_range_start % 1024; + if curr_range_start + ELEMS_PER_CHUNK < bytes_idx_to_range_indices[i][ranges_idx].end { + let this_part_len = ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; decompressed.extend_from_slice( - &decompress_chunk_buf[curr_range_start as usize % 1024..], + &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..], ); curr_range_start += this_part_len; break; @@ -758,7 +755,7 @@ fn bitpacked_for_non_neg_decode( let this_part_len = bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; decompressed.extend_from_slice( - &decompress_chunk_buf[curr_range_start as usize % 1024..] + &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..] [..this_part_len as usize], ); ranges_idx += 1; @@ -776,8 +773,8 @@ fn bitpacked_for_non_neg_decode( 64 => { let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); - let packed_chunk_size_in_byte: usize = 1024 * compressed_bit_width as usize / 8; - let mut decompress_chunk_buf = vec![0_u64; 1024]; + let packed_chunk_size_in_byte: usize = (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; + let mut decompress_chunk_buf = vec![0_u64; ELEMS_PER_CHUNK as usize]; for (i, bytes) in data.iter().enumerate() { let mut j = 0; let mut ranges_idx = 0; @@ -794,10 +791,10 @@ fn bitpacked_for_non_neg_decode( ); } loop { - if curr_range_start + 1024 < bytes_idx_to_range_indices[i][ranges_idx].end { - let this_part_len = 1024 - curr_range_start % 1024; + if curr_range_start + ELEMS_PER_CHUNK < bytes_idx_to_range_indices[i][ranges_idx].end { + let this_part_len = ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; decompressed.extend_from_slice( - &decompress_chunk_buf[curr_range_start as usize % 1024..], + &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..], ); curr_range_start += this_part_len; break; @@ -805,7 +802,7 @@ fn bitpacked_for_non_neg_decode( let this_part_len = bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; decompressed.extend_from_slice( - &decompress_chunk_buf[curr_range_start as usize % 1024..] + &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..] [..this_part_len as usize], ); ranges_idx += 1; From 3ad773c9ac6ed93f5c2212c14c72d730b192cd5c Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Thu, 19 Sep 2024 15:29:35 +0000 Subject: [PATCH 10/31] use macro in encode method --- rust/lance-encoding/benches/decoder.rs | 45 +-- .../encodings/physical/bitpack_fastlanes.rs | 335 ++++-------------- 2 files changed, 75 insertions(+), 305 deletions(-) diff --git a/rust/lance-encoding/benches/decoder.rs b/rust/lance-encoding/benches/decoder.rs index 3332050cb1..cdc13479e4 100644 --- a/rust/lance-encoding/benches/decoder.rs +++ b/rust/lance-encoding/benches/decoder.rs @@ -60,56 +60,13 @@ const ENCODING_OPTIONS: EncodingOptions = EncodingOptions { keep_original_array: true, }; -/* -fn bench_decode2(c: &mut Criterion) { - let rt = tokio::runtime::Runtime::new().unwrap(); - let mut group = c.benchmark_group("decode_uint8"); - group.measurement_time(std::time::Duration::new(12, 0)); - let array = UInt8Array::from(vec![5; 1024 * 1024 * 1024]); - let data = RecordBatch::try_new( - Arc::new(Schema::new(vec![Field::new( - "uint8", - DataType::UInt8, - false, - )])), - vec![Arc::new(array)], - ) - .unwrap(); - let lance_schema = - Arc::new(lance_core::datatypes::Schema::try_from(data.schema().as_ref()).unwrap()); - let input_bytes = data.get_array_memory_size(); - group.throughput(criterion::Throughput::Bytes(input_bytes as u64)); - let encoding_strategy = CoreFieldEncodingStrategy::default(); - let encoded = rt - .block_on(encode_batch( - &data, - lance_schema, - &encoding_strategy, - &ENCODING_OPTIONS, - )) - .unwrap(); - group.bench_function("uint8", |b| { - b.iter(|| { - let batch = rt - .block_on(lance_encoding::decoder::decode_batch( - &encoded, - &FilterExpression::no_filter(), - &DecoderMiddlewareChain::default(), - )) - .unwrap(); - assert_eq!(data.num_rows(), batch.num_rows()); - }) - }); -} -*/ - fn bench_decode(c: &mut Criterion) { let rt = tokio::runtime::Runtime::new().unwrap(); let mut group = c.benchmark_group("decode_primitive"); for data_type in PRIMITIVE_TYPES { let data = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(data_type)) - .into_batch_rows(lance_datagen::RowCount::from(1024 * 1024 * 1024)) + .into_batch_rows(lance_datagen::RowCount::from(1024 * 1024)) .unwrap(); let lance_schema = Arc::new(lance_core::datatypes::Schema::try_from(data.schema().as_ref()).unwrap()); diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index 44ecbd2191..cc81714ccc 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -194,6 +194,74 @@ pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { res } +macro_rules! encode_fixed_width { + ($self:expr, $unpacked:expr, $data_type:ty, $buffer_index:expr) => {{ + let num_chunks = ($unpacked.num_values + ELEMS_PER_CHUNK - 1) / ELEMS_PER_CHUNK; + let num_full_chunks = $unpacked.num_values / ELEMS_PER_CHUNK; + let uncompressed_bit_width = std::mem::size_of::<$data_type>() as u64 * 8; + + // the output vector type is the same as the input type, for example, when input is u16, output is Vec + let packed_chunk_size = 1024 * $self.compressed_bit_width as usize / uncompressed_bit_width as usize; + + let input_slice = $unpacked.data.borrow_to_typed_slice::<$data_type>(); + let input = input_slice.as_ref(); + + let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); + + // Loop over all but the last chunk. + (0..num_full_chunks).for_each(|i| { + let start_elem = (i * ELEMS_PER_CHUNK) as usize; + + let output_len = output.len(); + unsafe { + output.set_len(output_len + packed_chunk_size); + BitPacking::unchecked_pack( + $self.compressed_bit_width, + &input[start_elem..][..ELEMS_PER_CHUNK as usize], + &mut output[output_len..][..packed_chunk_size], + ); + } + }); + + if num_chunks != num_full_chunks { + let last_chunk_elem_num = $unpacked.num_values % ELEMS_PER_CHUNK; + let mut last_chunk = vec![0 as $data_type; ELEMS_PER_CHUNK as usize]; + last_chunk[..last_chunk_elem_num as usize].clone_from_slice( + &input[$unpacked.num_values as usize - last_chunk_elem_num as usize..], + ); + + let output_len = output.len(); + unsafe { + output.set_len(output_len + packed_chunk_size); + BitPacking::unchecked_pack( + $self.compressed_bit_width, + &last_chunk, + &mut output[output_len..][..packed_chunk_size], + ); + } + } + + let bitpacked_for_non_neg_buffer_index = *$buffer_index; + *$buffer_index += 1; + + let encoding = ProtobufUtils::bitpacked_for_non_neg_encoding( + $self.compressed_bit_width as u64, + uncompressed_bit_width, + bitpacked_for_non_neg_buffer_index, + ); + let packed = DataBlock::FixedWidth(FixedWidthDataBlock { + bits_per_value: $self.compressed_bit_width as u64, + data: LanceBuffer::reinterpret_vec(output), + num_values: $unpacked.num_values, + }); + + Ok(EncodedArray { + data: packed, + encoding, + }) + }}; +} + #[derive(Debug)] pub struct BitpackedForNonNegArrayEncoder { pub compressed_bit_width: usize, @@ -222,268 +290,13 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { location: location!(), }); }; + match _data_type { - DataType::UInt8 | DataType::Int8 => { - let num_chunks = (unpacked.num_values + ELEMS_PER_CHUNK - 1) / ELEMS_PER_CHUNK; - let num_full_chunks = unpacked.num_values / ELEMS_PER_CHUNK; - // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 - // the output type is the same as the input type - // 1024 * compressed_bit_width / 8 - let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); - - let input_slice = unpacked.data.borrow_to_typed_slice::(); - let input = input_slice.as_ref(); - - let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); - - // Loop over all but the last chunk. - (0..num_full_chunks).for_each(|i| { - let start_elem = (i * ELEMS_PER_CHUNK) as usize; - - let output_len = output.len(); - unsafe { - output.set_len(output_len + packed_chunk_size); - BitPacking::unchecked_pack( - self.compressed_bit_width, - &input[start_elem..][..ELEMS_PER_CHUNK as usize], - &mut output[output_len..][..packed_chunk_size], - ); - }; - }); - - if num_chunks != num_full_chunks { - let last_chunk_elem_num = unpacked.num_values % ELEMS_PER_CHUNK; - let mut last_chunk = vec![0u8; ELEMS_PER_CHUNK as usize]; - last_chunk[..last_chunk_elem_num as usize].clone_from_slice( - &input[unpacked.num_values as usize - last_chunk_elem_num as usize..], - ); - - let output_len = output.len(); - unsafe { - output.set_len(output.len() + packed_chunk_size); - BitPacking::unchecked_pack( - self.compressed_bit_width, - &last_chunk, - &mut output[output_len..][..packed_chunk_size], - ); - } - } - let bitpacked_for_non_neg_buffer_index = *buffer_index; - *buffer_index += 1; - - let encoding = ProtobufUtils::bitpacked_for_non_neg_encoding( - self.compressed_bit_width as u64, - _data_type.byte_width() as u64 * 8, - bitpacked_for_non_neg_buffer_index, - ); - let packed = DataBlock::FixedWidth(FixedWidthDataBlock { - bits_per_value: self.compressed_bit_width as u64, - data: LanceBuffer::reinterpret_vec(output), - num_values: unpacked.num_values, - }); - - Ok(EncodedArray { - data: packed, - encoding, - }) - } - - DataType::UInt16 | DataType::Int16 => { - let num_chunks = (unpacked.num_values + ELEMS_PER_CHUNK - 1) / ELEMS_PER_CHUNK; - let num_full_chunks = unpacked.num_values / ELEMS_PER_CHUNK; - // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 - // the output type is the same as the input type - // 1024 * compressed_bit_width / 8 - let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); - - let input_slice = unpacked.data.borrow_to_typed_slice::(); - let input = input_slice.as_ref(); - - let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); - - // Loop over all but the last chunk. - (0..num_full_chunks).for_each(|i| { - let start_elem = (i * ELEMS_PER_CHUNK) as usize; - - let output_len = output.len(); - unsafe { - output.set_len(output_len + packed_chunk_size); - BitPacking::unchecked_pack( - self.compressed_bit_width, - &input[start_elem..][..ELEMS_PER_CHUNK as usize], - &mut output[output_len..][..packed_chunk_size], - ); - }; - }); - - if num_chunks != num_full_chunks { - let last_chunk_elem_num = unpacked.num_values % ELEMS_PER_CHUNK; - let mut last_chunk = vec![0u16; ELEMS_PER_CHUNK as usize]; - last_chunk[..last_chunk_elem_num as usize].clone_from_slice( - &input[unpacked.num_values as usize - last_chunk_elem_num as usize..], - ); - - let output_len = output.len(); - unsafe { - output.set_len(output.len() + packed_chunk_size); - BitPacking::unchecked_pack( - self.compressed_bit_width, - &last_chunk, - &mut output[output_len..][..packed_chunk_size], - ); - } - } - let bitpacked_for_non_neg_buffer_index = *buffer_index; - *buffer_index += 1; - - let encoding = ProtobufUtils::bitpacked_for_non_neg_encoding( - self.compressed_bit_width as u64, - _data_type.byte_width() as u64 * 8, - bitpacked_for_non_neg_buffer_index, - ); - let packed = DataBlock::FixedWidth(FixedWidthDataBlock { - bits_per_value: self.compressed_bit_width as u64, - data: LanceBuffer::reinterpret_vec(output), - num_values: unpacked.num_values, - }); - - Ok(EncodedArray { - data: packed, - encoding, - }) - } - - DataType::UInt32 | DataType::Int32 => { - let num_chunks = (unpacked.num_values + ELEMS_PER_CHUNK - 1) / ELEMS_PER_CHUNK; - let num_full_chunks = unpacked.num_values / ELEMS_PER_CHUNK; - // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 - // the output type is the same as the input type - // 1024 * compressed_bit_width / 8 - let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); - - let input_slice = unpacked.data.borrow_to_typed_slice::(); - let input = input_slice.as_ref(); - - let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); - - // Loop over all but the last chunk. - (0..num_full_chunks).for_each(|i| { - let start_elem = (i * ELEMS_PER_CHUNK) as usize; - - let output_len = output.len(); - unsafe { - output.set_len(output_len + packed_chunk_size); - BitPacking::unchecked_pack( - self.compressed_bit_width, - &input[start_elem..][..ELEMS_PER_CHUNK as usize], - &mut output[output_len..][..packed_chunk_size], - ); - }; - }); - - if num_chunks != num_full_chunks { - let last_chunk_elem_num = unpacked.num_values % ELEMS_PER_CHUNK; - let mut last_chunk = vec![0u32; ELEMS_PER_CHUNK as usize]; - last_chunk[..last_chunk_elem_num as usize].clone_from_slice( - &input[unpacked.num_values as usize - last_chunk_elem_num as usize..], - ); - - let output_len = output.len(); - unsafe { - output.set_len(output.len() + packed_chunk_size); - BitPacking::unchecked_pack( - self.compressed_bit_width, - &last_chunk, - &mut output[output_len..][..packed_chunk_size], - ); - } - } - let bitpacked_for_non_neg_buffer_index = *buffer_index; - *buffer_index += 1; - - let encoding = ProtobufUtils::bitpacked_for_non_neg_encoding( - self.compressed_bit_width as u64, - _data_type.byte_width() as u64 * 8, - bitpacked_for_non_neg_buffer_index, - ); - let packed = DataBlock::FixedWidth(FixedWidthDataBlock { - bits_per_value: self.compressed_bit_width as u64, - data: LanceBuffer::reinterpret_vec(output), - num_values: unpacked.num_values, - }); - - Ok(EncodedArray { - data: packed, - encoding, - }) - } - - DataType::UInt64 | DataType::Int64 => { - let num_chunks = (unpacked.num_values + ELEMS_PER_CHUNK - 1) / ELEMS_PER_CHUNK; - let num_full_chunks = unpacked.num_values / ELEMS_PER_CHUNK; - // there is no ceiling needed to calculate the size of the packed chunk because 1024 has divisor 8 - // the output type is the same as the input type - // 1024 * compressed_bit_width / 8 - let packed_chunk_size = 128 * self.compressed_bit_width / _data_type.byte_width(); - - let input_slice = unpacked.data.borrow_to_typed_slice::(); - let input = input_slice.as_ref(); - - let mut output = Vec::with_capacity(num_chunks as usize * packed_chunk_size); - - // Loop over all but the last chunk. - (0..num_full_chunks).for_each(|i| { - let start_elem = (i * ELEMS_PER_CHUNK) as usize; - - let output_len = output.len(); - unsafe { - output.set_len(output_len + packed_chunk_size); - BitPacking::unchecked_pack( - self.compressed_bit_width, - &input[start_elem..][..ELEMS_PER_CHUNK as usize], - &mut output[output_len..][..packed_chunk_size], - ); - }; - }); - - if num_chunks != num_full_chunks { - let last_chunk_elem_num = unpacked.num_values % ELEMS_PER_CHUNK; - let mut last_chunk = vec![0u64; ELEMS_PER_CHUNK as usize]; - last_chunk[..last_chunk_elem_num as usize].clone_from_slice( - &input[unpacked.num_values as usize - last_chunk_elem_num as usize..], - ); - - let output_len = output.len(); - unsafe { - output.set_len(output.len() + packed_chunk_size); - BitPacking::unchecked_pack( - self.compressed_bit_width, - &last_chunk, - &mut output[output_len..][..packed_chunk_size], - ); - } - } - let bitpacked_for_non_neg_buffer_index = *buffer_index; - *buffer_index += 1; - - let encoding = ProtobufUtils::bitpacked_for_non_neg_encoding( - self.compressed_bit_width as u64, - _data_type.byte_width() as u64 * 8, - bitpacked_for_non_neg_buffer_index, - ); - let packed = DataBlock::FixedWidth(FixedWidthDataBlock { - bits_per_value: self.compressed_bit_width as u64, - data: LanceBuffer::reinterpret_vec(output), - num_values: unpacked.num_values, - }); - - Ok(EncodedArray { - data: packed, - encoding, - }) - } - - _ => todo!(), + DataType::UInt8 | DataType::Int8 => encode_fixed_width!(self, unpacked, u8, buffer_index), + DataType::UInt16 | DataType::Int16 => encode_fixed_width!(self, unpacked, u16, buffer_index), + DataType::UInt32 | DataType::Int32 => encode_fixed_width!(self, unpacked, u32, buffer_index), + DataType::UInt64 | DataType::Int64 => encode_fixed_width!(self, unpacked, u64, buffer_index), + _ => unreachable!("BitpackedForNonNegArrayEncoder only supports data types of UInt8, Int8, UInt16, Int16, UInt32, Int32, UInt64, Int64"), } } } From 403e89df1efd76f08225270a194612b4fbf5686a Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Wed, 18 Sep 2024 12:37:49 -0700 Subject: [PATCH 11/31] Don't pass strings to the choose_array_encoder method when choosing an encoder for dict indices --- rust/lance-encoding/src/encoder.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/rust/lance-encoding/src/encoder.rs b/rust/lance-encoding/src/encoder.rs index 0b4bb01e4a..12c1503b11 100644 --- a/rust/lance-encoding/src/encoder.rs +++ b/rust/lance-encoding/src/encoder.rs @@ -3,7 +3,7 @@ use std::{collections::HashMap, env, sync::Arc}; use arrow::array::AsArray; -use arrow_array::{Array, ArrayRef, RecordBatch}; +use arrow_array::{Array, ArrayRef, RecordBatch, UInt8Array}; use arrow_schema::DataType; use bytes::{Bytes, BytesMut}; use futures::future::BoxFuture; @@ -269,7 +269,11 @@ impl CoreArrayEncodingStrategy { DataType::Utf8 | DataType::LargeUtf8 | DataType::Binary | DataType::LargeBinary => { if use_dict_encoding { let dict_indices_encoder = Self::choose_array_encoder( - arrays, + // We need to pass arrays to this method to figure out what kind of compression to + // use but we haven't actually calculated the indices yet. For now, we just assume + // worst case and use the full range. In the future maybe we can pass in statistics + // instead of the actual data + &[Arc::new(UInt8Array::from_iter_values(0_u8..255_u8))], &DataType::UInt8, data_size, false, From 0ae836200a53c3350003523c50dd606a837ef6d6 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Fri, 20 Sep 2024 22:19:44 +0000 Subject: [PATCH 12/31] fix a bug in `bitpacked_for_non_neg_decode` --- .../encodings/physical/bitpack_fastlanes.rs | 40 ++++++++++--------- 1 file changed, 22 insertions(+), 18 deletions(-) diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index cc81714ccc..39d865c1f1 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -281,7 +281,7 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { fn encode( &self, data: DataBlock, - _data_type: &DataType, + data_type: &DataType, buffer_index: &mut u32, ) -> Result { let DataBlock::FixedWidth(mut unpacked) = data else { @@ -291,7 +291,7 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { }); }; - match _data_type { + match data_type { DataType::UInt8 | DataType::Int8 => encode_fixed_width!(self, unpacked, u8, buffer_index), DataType::UInt16 | DataType::Int16 => encode_fixed_width!(self, unpacked, u16, buffer_index), DataType::UInt32 | DataType::Int32 => encode_fixed_width!(self, unpacked, u32, buffer_index), @@ -542,12 +542,12 @@ fn bitpacked_for_non_neg_decode( let packed_chunk_size_in_byte: usize = (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; let mut decompress_chunk_buf = vec![0_u32; ELEMS_PER_CHUNK as usize]; for (i, bytes) in data.iter().enumerate() { - let mut j = 0; let mut ranges_idx = 0; let mut curr_range_start = bytes_idx_to_range_indices[i][0].start; - while j * packed_chunk_size_in_byte < bytes.len() { - let chunk_in_u8: &[u8] = - &bytes[j * packed_chunk_size_in_byte..][..packed_chunk_size_in_byte]; + let mut chunk_num = 0; + while chunk_num * packed_chunk_size_in_byte < bytes.len() { + let chunk_in_u8: &[u8] = &bytes[chunk_num * packed_chunk_size_in_byte..][..packed_chunk_size_in_byte]; + chunk_num += 1; let chunk = cast_slice(chunk_in_u8); unsafe { BitPacking::unchecked_unpack( @@ -557,28 +557,32 @@ fn bitpacked_for_non_neg_decode( ); } loop { - if curr_range_start + ELEMS_PER_CHUNK < bytes_idx_to_range_indices[i][ranges_idx].end { - let this_part_len = ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; + // case 1: all the elements after (curr_range_start % ELEMS_PER_CHUNK) inside this chunk is needed. + let elems_after_curr_range_start_in_this_chunk = ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; + if curr_range_start + elems_after_curr_range_start_in_this_chunk <= bytes_idx_to_range_indices[i][ranges_idx].end { decompressed.extend_from_slice( - &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..], + &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..], ); - curr_range_start += this_part_len; + curr_range_start += elems_after_curr_range_start_in_this_chunk; break; } else { - let this_part_len = - bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; + // case 2: only part of the elements after (curr_range_start % ELEMS_PER_CHUNK) inside this chunk is needed. + let elems_this_range_needed_in_this_chunk = (bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start).min(ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK); decompressed.extend_from_slice( &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..] - [..this_part_len as usize], + [..elems_this_range_needed_in_this_chunk as usize], ); - ranges_idx += 1; - if ranges_idx == bytes_idx_to_range_indices[i].len() { - break; + if curr_range_start + elems_this_range_needed_in_this_chunk == bytes_idx_to_range_indices[i][ranges_idx].end { + ranges_idx += 1; + if ranges_idx == bytes_idx_to_range_indices[i].len() { + break; + } + curr_range_start = bytes_idx_to_range_indices[i][ranges_idx].start; + } else { + curr_range_start += elems_this_range_needed_in_this_chunk; } - curr_range_start = bytes_idx_to_range_indices[i][ranges_idx].start; } } - j += 1; } } LanceBuffer::reinterpret_vec(decompressed).to_owned() From 23e261c2d8885a71416162dd4cdb79b888c1d27f Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Fri, 20 Sep 2024 22:33:15 +0000 Subject: [PATCH 13/31] add stable rust fastlanes --- .../compression-algo/fastlanes.rs | 865 ++++++++++++++++++ 1 file changed, 865 insertions(+) create mode 100644 rust/lance-encoding/compression-algo/fastlanes.rs diff --git a/rust/lance-encoding/compression-algo/fastlanes.rs b/rust/lance-encoding/compression-algo/fastlanes.rs new file mode 100644 index 0000000000..a514fadbe6 --- /dev/null +++ b/rust/lance-encoding/compression-algo/fastlanes.rs @@ -0,0 +1,865 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright The Lance Authors + +// This file is a modification of the `fastlanes` crate: https://github.com/spiraldb/fastlanes +// It is modified to allow a rust stable build + +use arrayref::{array_mut_ref, array_ref}; +use core::mem::size_of; +use paste::paste; + +pub const FL_ORDER: [usize; 8] = [0, 4, 2, 6, 1, 5, 3, 7]; + + +pub trait FastLanes: Sized + Copy { + const T: usize = size_of::() * 8; + const LANES: usize = 1024 / Self::T; +} + +// Implement the trait for basic unsigned integer types +impl FastLanes for u8 {} +impl FastLanes for u16 {} +impl FastLanes for u32 {} +impl FastLanes for u64 {} + +#[macro_export] +macro_rules! iterate { + ($T:ty, $lane: expr, | $_1:tt $idx:ident | $($body:tt)*) => { + macro_rules! __kernel__ {( $_1 $idx:ident ) => ( $($body)* )} + { + use $crate::{seq_t, FL_ORDER}; + use paste::paste; + + #[inline(always)] + fn index(row: usize, lane: usize) -> usize { + let o = row / 8; + let s = row % 8; + (FL_ORDER[o] * 16) + (s * 128) + lane + } + + paste!(seq_t!(row in $T { + let idx = index(row, $lane); + __kernel__!(idx); + })); + } + } +} + +#[macro_export] +macro_rules! pack { + ($T:ty, $W:expr, $packed:expr, $lane:expr, | $_1:tt $idx:ident | $($body:tt)*) => { + macro_rules! __kernel__ {( $_1 $idx:ident ) => ( $($body)* )} + { + use $crate::{seq_t, FL_ORDER}; + use paste::paste; + + // The number of bits of T. + const T: usize = <$T>::T; + + #[inline(always)] + fn index(row: usize, lane: usize) -> usize { + let o = row / 8; + let s = row % 8; + (FL_ORDER[o] * 16) + (s * 128) + lane + } + + if $W == 0 { + // Nothing to do if W is 0, since the packed array is zero bytes. + } else if $W == T { + // Special case for W=T, we can just copy the input value directly to the packed value. + paste!(seq_t!(row in $T { + let idx = index(row, $lane); + $packed[<$T>::LANES * row + $lane] = __kernel__!(idx); + })); + } else { + // A mask of W bits. + let mask: $T = (1 << $W) - 1; + + // First we loop over each lane in the virtual 1024 bit word. + let mut tmp: $T = 0; + + // Loop over each of the rows of the lane. + // Inlining this loop means all branches are known at compile time and + // the code is auto-vectorized for SIMD execution. + paste!(seq_t!(row in $T { + let idx = index(row, $lane); + let src = __kernel__!(idx); + let src = src & mask; + + // Shift the src bits into their position in the tmp output variable. + if row == 0 { + tmp = src; + } else { + tmp |= src << (row * $W) % T; + } + + // If the next packed position is after our current one, then we have filled + // the current output and we can write the packed value. + let curr_word: usize = (row * $W) / T; + let next_word: usize = ((row + 1) * $W) / T; + + #[allow(unused_assignments)] + if next_word > curr_word { + $packed[<$T>::LANES * curr_word + $lane] = tmp; + let remaining_bits: usize = ((row + 1) * $W) % T; + // Keep the remaining bits for the next packed value. + tmp = src >> $W - remaining_bits; + } + })); + } + } + }; +} + +#[macro_export] +macro_rules! unpack { + ($T:ty, $W:expr, $packed:expr, $lane:expr, | $_1:tt $idx:ident, $_2:tt $elem:ident | $($body:tt)*) => { + macro_rules! __kernel__ {( $_1 $idx:ident, $_2 $elem:ident ) => ( $($body)* )} + { + use $crate::{seq_t, FL_ORDER}; + use paste::paste; + + // The number of bits of T. + const T: usize = <$T>::T; + + #[inline(always)] + fn index(row: usize, lane: usize) -> usize { + let o = row / 8; + let s = row % 8; + (FL_ORDER[o] * 16) + (s * 128) + lane + } + + if $W == 0 { + // Special case for W=0, we just need to zero the output. + // We'll still respect the iteration order in case the kernel has side effects. + paste!(seq_t!(row in $T { + let idx = index(row, $lane); + let zero: $T = 0; + __kernel__!(idx, zero); + })); + } else if $W == T { + // Special case for W=T, we can just copy the packed value directly to the output. + paste!(seq_t!(row in $T { + let idx = index(row, $lane); + let src = $packed[<$T>::LANES * row + $lane]; + __kernel__!(idx, src); + })); + } else { + #[inline] + fn mask(width: usize) -> $T { + if width == T { <$T>::MAX } else { (1 << (width % T)) - 1 } + } + + let mut src: $T = $packed[$lane]; + let mut tmp: $T; + + paste!(seq_t!(row in $T { + // Figure out the packed positions + let curr_word: usize = (row * $W) / T; + let next_word = ((row + 1) * $W) / T; + + let shift = (row * $W) % T; + + if next_word > curr_word { + // Consume some bits from the curr packed input, the remainder are in the next + // packed input value + let remaining_bits = ((row + 1) * $W) % T; + let current_bits = $W - remaining_bits; + tmp = (src >> shift) & mask(current_bits); + + if next_word < $W { + // Load the next packed value + src = $packed[<$T>::LANES * next_word + $lane]; + // Consume the remaining bits from the next input value. + tmp |= (src & mask(remaining_bits)) << current_bits; + } + } else { + // Otherwise, just grab W bits from the src value + tmp = (src >> shift) & mask($W); + } + + // Write out the unpacked value + let idx = index(row, $lane); + __kernel__!(idx, tmp); + })); + } + } + }; +} + +// Macro for repeating a code block bit_size_of:: times. +#[macro_export] +macro_rules! seq_t { + ($ident:ident in u8 $body:tt) => {seq_macro::seq!($ident in 0..8 $body)}; + ($ident:ident in u16 $body:tt) => {seq_macro::seq!($ident in 0..16 $body)}; + ($ident:ident in u32 $body:tt) => {seq_macro::seq!($ident in 0..32 $body)}; + ($ident:ident in u64 $body:tt) => {seq_macro::seq!($ident in 0..64 $body)}; +} + +/// `BitPack` into a compile-time known bit-width. +pub trait BitPacking: FastLanes { + /// Packs 1024 elements into `W` bits each, where `W` is runtime-known instead of + /// compile-time known. + /// + /// # Safety + /// The input slice must be of exactly length 1024. The output slice must be of length + /// `1024 * W / T`, where `T` is the bit-width of Self and `W` is the packed width. + /// These lengths are checked only with `debug_assert` (i.e., not checked on release builds). + pub unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]); + + /// Unpacks 1024 elements from `W` bits each, where `W` is runtime-known instead of + /// compile-time known. + /// + /// # Safety + /// The input slice must be of length `1024 * W / T`, where `T` is the bit-width of Self and `W` + /// is the packed width. The output slice must be of exactly length 1024. + /// These lengths are checked only with `debug_assert` (i.e., not checked on release builds). + pub unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]); +} + +impl BitPacking for u8 { + unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); + debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => pack_8_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u8::T]), + 2 => pack_8_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u8::T]), + 3 => pack_8_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u8::T]), + 4 => pack_8_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u8::T]), + 5 => pack_8_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u8::T]), + 6 => pack_8_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u8::T]), + 7 => pack_8_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u8::T]), + 8 => pack_8_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u8::T]), + + _ => unreachable!("Unsupported width: {}", width) + } + } + + unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); + debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => unpack_8_1(array_ref![input, 0, 1024 * 1 / u8::T], array_mut_ref![output, 0, 1024]), + 2 => unpack_8_2(array_ref![input, 0, 1024 * 2 / u8::T], array_mut_ref![output, 0, 1024]), + 3 => unpack_8_3(array_ref![input, 0, 1024 * 3 / u8::T], array_mut_ref![output, 0, 1024]), + 4 => unpack_8_4(array_ref![input, 0, 1024 * 4 / u8::T], array_mut_ref![output, 0, 1024]), + 5 => unpack_8_5(array_ref![input, 0, 1024 * 5 / u8::T], array_mut_ref![output, 0, 1024]), + 6 => unpack_8_6(array_ref![input, 0, 1024 * 6 / u8::T], array_mut_ref![output, 0, 1024]), + 7 => unpack_8_7(array_ref![input, 0, 1024 * 7 / u8::T], array_mut_ref![output, 0, 1024]), + 8 => unpack_8_8(array_ref![input, 0, 1024 * 8 / u8::T], array_mut_ref![output, 0, 1024]), + + _ => unreachable!("Unsupported width: {}", width) + } + } +} + +impl BitPacking for u16 { + unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); + debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => pack_16_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u16::T]), + 2 => pack_16_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u16::T]), + 3 => pack_16_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u16::T]), + 4 => pack_16_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u16::T]), + 5 => pack_16_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u16::T]), + 6 => pack_16_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u16::T]), + 7 => pack_16_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u16::T]), + 8 => pack_16_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u16::T]), + 9 => pack_16_9(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 9 / u16::T]), + + 10 => pack_16_10(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 10 / u16::T]), + 11 => pack_16_11(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 11 / u16::T]), + 12 => pack_16_12(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 12 / u16::T]), + 13 => pack_16_13(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 13 / u16::T]), + 14 => pack_16_14(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 14 / u16::T]), + 15 => pack_16_15(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 15 / u16::T]), + 16 => pack_16_16(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 16 / u16::T]), + + _ => unreachable!("Unsupported width: {}", width) + } + } + + unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); + debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => unpack_16_1(array_ref![input, 0, 1024 * 1 / u16::T], array_mut_ref![output, 0, 1024]), + 2 => unpack_16_2(array_ref![input, 0, 1024 * 2 / u16::T], array_mut_ref![output, 0, 1024]), + 3 => unpack_16_3(array_ref![input, 0, 1024 * 3 / u16::T], array_mut_ref![output, 0, 1024]), + 4 => unpack_16_4(array_ref![input, 0, 1024 * 4 / u16::T], array_mut_ref![output, 0, 1024]), + 5 => unpack_16_5(array_ref![input, 0, 1024 * 5 / u16::T], array_mut_ref![output, 0, 1024]), + 6 => unpack_16_6(array_ref![input, 0, 1024 * 6 / u16::T], array_mut_ref![output, 0, 1024]), + 7 => unpack_16_7(array_ref![input, 0, 1024 * 7 / u16::T], array_mut_ref![output, 0, 1024]), + 8 => unpack_16_8(array_ref![input, 0, 1024 * 8 / u16::T], array_mut_ref![output, 0, 1024]), + 9 => unpack_16_9(array_ref![input, 0, 1024 * 9 / u16::T], array_mut_ref![output, 0, 1024]), + + 10 => unpack_16_10(array_ref![input, 0, 1024 * 10 / u16::T], array_mut_ref![output, 0, 1024]), + 11 => unpack_16_11(array_ref![input, 0, 1024 * 11 / u16::T], array_mut_ref![output, 0, 1024]), + 12 => unpack_16_12(array_ref![input, 0, 1024 * 12 / u16::T], array_mut_ref![output, 0, 1024]), + 13 => unpack_16_13(array_ref![input, 0, 1024 * 13 / u16::T], array_mut_ref![output, 0, 1024]), + 14 => unpack_16_14(array_ref![input, 0, 1024 * 14 / u16::T], array_mut_ref![output, 0, 1024]), + 15 => unpack_16_15(array_ref![input, 0, 1024 * 15 / u16::T], array_mut_ref![output, 0, 1024]), + 16 => unpack_16_16(array_ref![input, 0, 1024 * 16 / u16::T], array_mut_ref![output, 0, 1024]), + + _ => unreachable!("Unsupported width: {}", width) + } + } +} + +impl BitPacking for u32 { + unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); + debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => pack_32_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u32::T]), + 2 => pack_32_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u32::T]), + 3 => pack_32_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u32::T]), + 4 => pack_32_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u32::T]), + 5 => pack_32_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u32::T]), + 6 => pack_32_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u32::T]), + 7 => pack_32_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u32::T]), + 8 => pack_32_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u32::T]), + 9 => pack_32_9(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 9 / u32::T]), + + 10 => pack_32_10(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 10 / u32::T]), + 11 => pack_32_11(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 11 / u32::T]), + 12 => pack_32_12(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 12 / u32::T]), + 13 => pack_32_13(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 13 / u32::T]), + 14 => pack_32_14(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 14 / u32::T]), + 15 => pack_32_15(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 15 / u32::T]), + 16 => pack_32_16(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 16 / u32::T]), + 17 => pack_32_17(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 17 / u32::T]), + 18 => pack_32_18(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 18 / u32::T]), + 19 => pack_32_19(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 19 / u32::T]), + + 20 => pack_32_20(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 20 / u32::T]), + 21 => pack_32_21(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 21 / u32::T]), + 22 => pack_32_22(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 22 / u32::T]), + 23 => pack_32_23(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 23 / u32::T]), + 24 => pack_32_24(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 24 / u32::T]), + 25 => pack_32_25(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 25 / u32::T]), + 26 => pack_32_26(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 26 / u32::T]), + 27 => pack_32_27(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 27 / u32::T]), + 28 => pack_32_28(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 28 / u32::T]), + 29 => pack_32_29(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 29 / u32::T]), + + 30 => pack_32_30(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 30 / u32::T]), + 31 => pack_32_31(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 31 / u32::T]), + 32 => pack_32_32(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 32 / u32::T]), + + _ => unreachable!("Unsupported width: {}", width) + } + } + + unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); + debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => unpack_32_1(array_ref![input, 0, 1024 * 1 / u32::T], array_mut_ref![output, 0, 1024]), + 2 => unpack_32_2(array_ref![input, 0, 1024 * 2 / u32::T], array_mut_ref![output, 0, 1024]), + 3 => unpack_32_3(array_ref![input, 0, 1024 * 3 / u32::T], array_mut_ref![output, 0, 1024]), + 4 => unpack_32_4(array_ref![input, 0, 1024 * 4 / u32::T], array_mut_ref![output, 0, 1024]), + 5 => unpack_32_5(array_ref![input, 0, 1024 * 5 / u32::T], array_mut_ref![output, 0, 1024]), + 6 => unpack_32_6(array_ref![input, 0, 1024 * 6 / u32::T], array_mut_ref![output, 0, 1024]), + 7 => unpack_32_7(array_ref![input, 0, 1024 * 7 / u32::T], array_mut_ref![output, 0, 1024]), + 8 => unpack_32_8(array_ref![input, 0, 1024 * 8 / u32::T], array_mut_ref![output, 0, 1024]), + 9 => unpack_32_9(array_ref![input, 0, 1024 * 9 / u32::T], array_mut_ref![output, 0, 1024]), + + 10 => unpack_32_10(array_ref![input, 0, 1024 * 10 / u32::T], array_mut_ref![output, 0, 1024]), + 11 => unpack_32_11(array_ref![input, 0, 1024 * 11 / u32::T], array_mut_ref![output, 0, 1024]), + 12 => unpack_32_12(array_ref![input, 0, 1024 * 12 / u32::T], array_mut_ref![output, 0, 1024]), + 13 => unpack_32_13(array_ref![input, 0, 1024 * 13 / u32::T], array_mut_ref![output, 0, 1024]), + 14 => unpack_32_14(array_ref![input, 0, 1024 * 14 / u32::T], array_mut_ref![output, 0, 1024]), + 15 => unpack_32_15(array_ref![input, 0, 1024 * 15 / u32::T], array_mut_ref![output, 0, 1024]), + 16 => unpack_32_16(array_ref![input, 0, 1024 * 16 / u32::T], array_mut_ref![output, 0, 1024]), + 17 => unpack_32_17(array_ref![input, 0, 1024 * 17 / u32::T], array_mut_ref![output, 0, 1024]), + 18 => unpack_32_18(array_ref![input, 0, 1024 * 18 / u32::T], array_mut_ref![output, 0, 1024]), + 19 => unpack_32_19(array_ref![input, 0, 1024 * 19 / u32::T], array_mut_ref![output, 0, 1024]), + + 20 => unpack_32_20(array_ref![input, 0, 1024 * 20 / u32::T], array_mut_ref![output, 0, 1024]), + 21 => unpack_32_21(array_ref![input, 0, 1024 * 21 / u32::T], array_mut_ref![output, 0, 1024]), + 22 => unpack_32_22(array_ref![input, 0, 1024 * 22 / u32::T], array_mut_ref![output, 0, 1024]), + 23 => unpack_32_23(array_ref![input, 0, 1024 * 23 / u32::T], array_mut_ref![output, 0, 1024]), + 24 => unpack_32_24(array_ref![input, 0, 1024 * 24 / u32::T], array_mut_ref![output, 0, 1024]), + 25 => unpack_32_25(array_ref![input, 0, 1024 * 25 / u32::T], array_mut_ref![output, 0, 1024]), + 26 => unpack_32_26(array_ref![input, 0, 1024 * 26 / u32::T], array_mut_ref![output, 0, 1024]), + 27 => unpack_32_27(array_ref![input, 0, 1024 * 27 / u32::T], array_mut_ref![output, 0, 1024]), + 28 => unpack_32_28(array_ref![input, 0, 1024 * 28 / u32::T], array_mut_ref![output, 0, 1024]), + 29 => unpack_32_29(array_ref![input, 0, 1024 * 29 / u32::T], array_mut_ref![output, 0, 1024]), + + 30 => unpack_32_30(array_ref![input, 0, 1024 * 30 / u32::T], array_mut_ref![output, 0, 1024]), + 31 => unpack_32_31(array_ref![input, 0, 1024 * 31 / u32::T], array_mut_ref![output, 0, 1024]), + 32 => unpack_32_32(array_ref![input, 0, 1024 * 32 / u32::T], array_mut_ref![output, 0, 1024]), + + _ => unreachable!("Unsupported width: {}", width) + } + } +} + +impl BitPacking for u64 { + unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); + debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => pack_64_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u64::T]), + 2 => pack_64_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u64::T]), + 3 => pack_64_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u64::T]), + 4 => pack_64_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u64::T]), + 5 => pack_64_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u64::T]), + 6 => pack_64_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u64::T]), + 7 => pack_64_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u64::T]), + 8 => pack_64_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u64::T]), + 9 => pack_64_9(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 9 / u64::T]), + + 10 => pack_64_10(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 10 / u64::T]), + 11 => pack_64_11(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 11 / u64::T]), + 12 => pack_64_12(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 12 / u64::T]), + 13 => pack_64_13(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 13 / u64::T]), + 14 => pack_64_14(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 14 / u64::T]), + 15 => pack_64_15(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 15 / u64::T]), + 16 => pack_64_16(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 16 / u64::T]), + 17 => pack_64_17(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 17 / u64::T]), + 18 => pack_64_18(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 18 / u64::T]), + 19 => pack_64_19(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 19 / u64::T]), + + 20 => pack_64_20(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 20 / u64::T]), + 21 => pack_64_21(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 21 / u64::T]), + 22 => pack_64_22(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 22 / u64::T]), + 23 => pack_64_23(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 23 / u64::T]), + 24 => pack_64_24(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 24 / u64::T]), + 25 => pack_64_25(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 25 / u64::T]), + 26 => pack_64_26(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 26 / u64::T]), + 27 => pack_64_27(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 27 / u64::T]), + 28 => pack_64_28(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 28 / u64::T]), + 29 => pack_64_29(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 29 / u64::T]), + + 30 => pack_64_30(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 30 / u64::T]), + 31 => pack_64_31(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 31 / u64::T]), + 32 => pack_64_32(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 32 / u64::T]), + 33 => pack_64_33(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 33 / u64::T]), + 34 => pack_64_34(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 34 / u64::T]), + 35 => pack_64_35(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 35 / u64::T]), + 36 => pack_64_36(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 36 / u64::T]), + 37 => pack_64_37(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 37 / u64::T]), + 38 => pack_64_38(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 38 / u64::T]), + 39 => pack_64_39(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 39 / u64::T]), + + 40 => pack_64_40(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 40 / u64::T]), + 41 => pack_64_41(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 41 / u64::T]), + 42 => pack_64_42(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 42 / u64::T]), + 43 => pack_64_43(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 43 / u64::T]), + 44 => pack_64_44(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 44 / u64::T]), + 45 => pack_64_45(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 45 / u64::T]), + 46 => pack_64_46(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 46 / u64::T]), + 47 => pack_64_47(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 47 / u64::T]), + 48 => pack_64_48(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 48 / u64::T]), + 49 => pack_64_49(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 49 / u64::T]), + + 50 => pack_64_50(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 50 / u64::T]), + 51 => pack_64_51(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 51 / u64::T]), + 52 => pack_64_52(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 52 / u64::T]), + 53 => pack_64_53(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 53 / u64::T]), + 54 => pack_64_54(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 54 / u64::T]), + 55 => pack_64_55(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 55 / u64::T]), + 56 => pack_64_56(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 56 / u64::T]), + 57 => pack_64_57(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 57 / u64::T]), + 58 => pack_64_58(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 58 / u64::T]), + 59 => pack_64_59(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 59 / u64::T]), + + 60 => pack_64_60(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 60 / u64::T]), + 61 => pack_64_61(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 61 / u64::T]), + 62 => pack_64_62(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 62 / u64::T]), + 63 => pack_64_63(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 63 / u64::T]), + 64 => pack_64_64(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 64 / u64::T]), + + _ => unreachable!("Unsupported width: {}", width) + } + } + + unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); + debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => unpack_64_1(array_ref![input, 0, 1024 * 1 / u64::T], array_mut_ref![output, 0, 1024]), + 2 => unpack_64_2(array_ref![input, 0, 1024 * 2 / u64::T], array_mut_ref![output, 0, 1024]), + 3 => unpack_64_3(array_ref![input, 0, 1024 * 3 / u64::T], array_mut_ref![output, 0, 1024]), + 4 => unpack_64_4(array_ref![input, 0, 1024 * 4 / u64::T], array_mut_ref![output, 0, 1024]), + 5 => unpack_64_5(array_ref![input, 0, 1024 * 5 / u64::T], array_mut_ref![output, 0, 1024]), + 6 => unpack_64_6(array_ref![input, 0, 1024 * 6 / u64::T], array_mut_ref![output, 0, 1024]), + 7 => unpack_64_7(array_ref![input, 0, 1024 * 7 / u64::T], array_mut_ref![output, 0, 1024]), + 8 => unpack_64_8(array_ref![input, 0, 1024 * 8 / u64::T], array_mut_ref![output, 0, 1024]), + 9 => unpack_64_9(array_ref![input, 0, 1024 * 9 / u64::T], array_mut_ref![output, 0, 1024]), + + 10 => unpack_64_10(array_ref![input, 0, 1024 * 10 / u64::T], array_mut_ref![output, 0, 1024]), + 11 => unpack_64_11(array_ref![input, 0, 1024 * 11 / u64::T], array_mut_ref![output, 0, 1024]), + 12 => unpack_64_12(array_ref![input, 0, 1024 * 12 / u64::T], array_mut_ref![output, 0, 1024]), + 13 => unpack_64_13(array_ref![input, 0, 1024 * 13 / u64::T], array_mut_ref![output, 0, 1024]), + 14 => unpack_64_14(array_ref![input, 0, 1024 * 14 / u64::T], array_mut_ref![output, 0, 1024]), + 15 => unpack_64_15(array_ref![input, 0, 1024 * 15 / u64::T], array_mut_ref![output, 0, 1024]), + 16 => unpack_64_16(array_ref![input, 0, 1024 * 16 / u64::T], array_mut_ref![output, 0, 1024]), + 17 => unpack_64_17(array_ref![input, 0, 1024 * 17 / u64::T], array_mut_ref![output, 0, 1024]), + 18 => unpack_64_18(array_ref![input, 0, 1024 * 18 / u64::T], array_mut_ref![output, 0, 1024]), + 19 => unpack_64_19(array_ref![input, 0, 1024 * 19 / u64::T], array_mut_ref![output, 0, 1024]), + + 20 => unpack_64_20(array_ref![input, 0, 1024 * 20 / u64::T], array_mut_ref![output, 0, 1024]), + 21 => unpack_64_21(array_ref![input, 0, 1024 * 21 / u64::T], array_mut_ref![output, 0, 1024]), + 22 => unpack_64_22(array_ref![input, 0, 1024 * 22 / u64::T], array_mut_ref![output, 0, 1024]), + 23 => unpack_64_23(array_ref![input, 0, 1024 * 23 / u64::T], array_mut_ref![output, 0, 1024]), + 24 => unpack_64_24(array_ref![input, 0, 1024 * 24 / u64::T], array_mut_ref![output, 0, 1024]), + 25 => unpack_64_25(array_ref![input, 0, 1024 * 25 / u64::T], array_mut_ref![output, 0, 1024]), + 26 => unpack_64_26(array_ref![input, 0, 1024 * 26 / u64::T], array_mut_ref![output, 0, 1024]), + 27 => unpack_64_27(array_ref![input, 0, 1024 * 27 / u64::T], array_mut_ref![output, 0, 1024]), + 28 => unpack_64_28(array_ref![input, 0, 1024 * 28 / u64::T], array_mut_ref![output, 0, 1024]), + 29 => unpack_64_29(array_ref![input, 0, 1024 * 29 / u64::T], array_mut_ref![output, 0, 1024]), + + 30 => unpack_64_30(array_ref![input, 0, 1024 * 30 / u64::T], array_mut_ref![output, 0, 1024]), + 31 => unpack_64_31(array_ref![input, 0, 1024 * 31 / u64::T], array_mut_ref![output, 0, 1024]), + 32 => unpack_64_32(array_ref![input, 0, 1024 * 32 / u64::T], array_mut_ref![output, 0, 1024]), + 33 => unpack_64_33(array_ref![input, 0, 1024 * 33 / u64::T], array_mut_ref![output, 0, 1024]), + 34 => unpack_64_34(array_ref![input, 0, 1024 * 34 / u64::T], array_mut_ref![output, 0, 1024]), + 35 => unpack_64_35(array_ref![input, 0, 1024 * 35 / u64::T], array_mut_ref![output, 0, 1024]), + 36 => unpack_64_36(array_ref![input, 0, 1024 * 36 / u64::T], array_mut_ref![output, 0, 1024]), + 37 => unpack_64_37(array_ref![input, 0, 1024 * 37 / u64::T], array_mut_ref![output, 0, 1024]), + 38 => unpack_64_38(array_ref![input, 0, 1024 * 38 / u64::T], array_mut_ref![output, 0, 1024]), + 39 => unpack_64_39(array_ref![input, 0, 1024 * 39 / u64::T], array_mut_ref![output, 0, 1024]), + + 40 => unpack_64_40(array_ref![input, 0, 1024 * 40 / u64::T], array_mut_ref![output, 0, 1024]), + 41 => unpack_64_41(array_ref![input, 0, 1024 * 41 / u64::T], array_mut_ref![output, 0, 1024]), + 42 => unpack_64_42(array_ref![input, 0, 1024 * 42 / u64::T], array_mut_ref![output, 0, 1024]), + 43 => unpack_64_43(array_ref![input, 0, 1024 * 43 / u64::T], array_mut_ref![output, 0, 1024]), + 44 => unpack_64_44(array_ref![input, 0, 1024 * 44 / u64::T], array_mut_ref![output, 0, 1024]), + 45 => unpack_64_45(array_ref![input, 0, 1024 * 45 / u64::T], array_mut_ref![output, 0, 1024]), + 46 => unpack_64_46(array_ref![input, 0, 1024 * 46 / u64::T], array_mut_ref![output, 0, 1024]), + 47 => unpack_64_47(array_ref![input, 0, 1024 * 47 / u64::T], array_mut_ref![output, 0, 1024]), + 48 => unpack_64_48(array_ref![input, 0, 1024 * 48 / u64::T], array_mut_ref![output, 0, 1024]), + 49 => unpack_64_49(array_ref![input, 0, 1024 * 49 / u64::T], array_mut_ref![output, 0, 1024]), + + 50 => unpack_64_50(array_ref![input, 0, 1024 * 50 / u64::T], array_mut_ref![output, 0, 1024]), + 51 => unpack_64_51(array_ref![input, 0, 1024 * 51 / u64::T], array_mut_ref![output, 0, 1024]), + 52 => unpack_64_52(array_ref![input, 0, 1024 * 52 / u64::T], array_mut_ref![output, 0, 1024]), + 53 => unpack_64_53(array_ref![input, 0, 1024 * 53 / u64::T], array_mut_ref![output, 0, 1024]), + 54 => unpack_64_54(array_ref![input, 0, 1024 * 54 / u64::T], array_mut_ref![output, 0, 1024]), + 55 => unpack_64_55(array_ref![input, 0, 1024 * 55 / u64::T], array_mut_ref![output, 0, 1024]), + 56 => unpack_64_56(array_ref![input, 0, 1024 * 56 / u64::T], array_mut_ref![output, 0, 1024]), + 57 => unpack_64_57(array_ref![input, 0, 1024 * 57 / u64::T], array_mut_ref![output, 0, 1024]), + 58 => unpack_64_58(array_ref![input, 0, 1024 * 58 / u64::T], array_mut_ref![output, 0, 1024]), + 59 => unpack_64_59(array_ref![input, 0, 1024 * 59 / u64::T], array_mut_ref![output, 0, 1024]), + + 60 => unpack_64_60(array_ref![input, 0, 1024 * 60 / u64::T], array_mut_ref![output, 0, 1024]), + 61 => unpack_64_61(array_ref![input, 0, 1024 * 61 / u64::T], array_mut_ref![output, 0, 1024]), + 62 => unpack_64_62(array_ref![input, 0, 1024 * 62 / u64::T], array_mut_ref![output, 0, 1024]), + 63 => unpack_64_63(array_ref![input, 0, 1024 * 63 / u64::T], array_mut_ref![output, 0, 1024]), + 64 => unpack_64_64(array_ref![input, 0, 1024 * 64 / u64::T], array_mut_ref![output, 0, 1024]), + + _ => unreachable!("Unsupported width: {}", width) + } + } +} + +macro_rules! unpack_8 { + ($name:ident, $bits:expr) => { + fn $name(input: &[u8; 1024 * $bits / u8::T], output: &mut [u8; 1024]) { + for lane in 0..u8::LANES { + unpack!(u8, $bits, input, lane, |$idx, $elem| { + output[$idx] = $elem; + }); + } + } + }; +} + +unpack_8!(unpack_8_1, 1); +unpack_8!(unpack_8_2, 2); +unpack_8!(unpack_8_3, 3); +unpack_8!(unpack_8_4, 4); +unpack_8!(unpack_8_5, 5); +unpack_8!(unpack_8_6, 6); +unpack_8!(unpack_8_7, 7); +unpack_8!(unpack_8_8, 8); + +macro_rules! pack_8 { + ($name:ident, $bits:expr) => { + fn $name(input: &[u8; 1024], output: &mut [u8; 1024 * $bits / u8::T]) { + for lane in 0..u8::LANES { + pack!(u8, $bits, output, lane, |$idx| { + input[$idx] + }); + } + } + }; +} +pack_8!(pack_8_1, 1); +pack_8!(pack_8_2, 2); +pack_8!(pack_8_3, 3); +pack_8!(pack_8_4, 4); +pack_8!(pack_8_5, 5); +pack_8!(pack_8_6, 6); +pack_8!(pack_8_7, 7); +pack_8!(pack_8_8, 8); + +macro_rules! unpack_16 { + ($name:ident, $bits:expr) => { + fn $name(input: &[u16; 1024 * $bits / u16::T], output: &mut [u16; 1024]) { + for lane in 0..u16::LANES { + unpack!(u16, $bits, input, lane, |$idx, $elem| { + output[$idx] = $elem; + }); + } + } + }; +} + +unpack_16!(unpack_16_1, 1); +unpack_16!(unpack_16_2, 2); +unpack_16!(unpack_16_3, 3); +unpack_16!(unpack_16_4, 4); +unpack_16!(unpack_16_5, 5); +unpack_16!(unpack_16_6, 6); +unpack_16!(unpack_16_7, 7); +unpack_16!(unpack_16_8, 8); +unpack_16!(unpack_16_9, 9); +unpack_16!(unpack_16_10, 10); +unpack_16!(unpack_16_11, 11); +unpack_16!(unpack_16_12, 12); +unpack_16!(unpack_16_13, 13); +unpack_16!(unpack_16_14, 14); +unpack_16!(unpack_16_15, 15); +unpack_16!(unpack_16_16, 16); + +macro_rules! pack_16 { + ($name:ident, $bits:expr) => { + fn $name(input: &[u16; 1024], output: &mut [u16; 1024 * $bits / u16::T]) { + for lane in 0..u16::LANES { + pack!(u16, $bits, output, lane, |$idx| { + input[$idx] + }); + } + } + }; +} + +pack_16!(pack_16_1, 1); +pack_16!(pack_16_2, 2); +pack_16!(pack_16_3, 3); +pack_16!(pack_16_4, 4); +pack_16!(pack_16_5, 5); +pack_16!(pack_16_6, 6); +pack_16!(pack_16_7, 7); +pack_16!(pack_16_8, 8); +pack_16!(pack_16_9, 9); +pack_16!(pack_16_10, 10); +pack_16!(pack_16_11, 11); +pack_16!(pack_16_12, 12); +pack_16!(pack_16_13, 13); +pack_16!(pack_16_14, 14); +pack_16!(pack_16_15, 15); +pack_16!(pack_16_16, 16); + +macro_rules! unpack_32 { + ($name:ident, $bit_width:expr) => { + fn $name(input: &[u32; 1024 * $bit_width / u32::T], output: &mut [u32; 1024]) { + for lane in 0..u32::LANES { + unpack!(u32, $bit_width, input, lane, |$idx, $elem| { + output[$idx] = $elem + }); + } + } + }; +} + +unpack_32!(unpack_32_1, 1); +unpack_32!(unpack_32_2, 2); +unpack_32!(unpack_32_3, 3); +unpack_32!(unpack_32_4, 4); +unpack_32!(unpack_32_5, 5); +unpack_32!(unpack_32_6, 6); +unpack_32!(unpack_32_7, 7); +unpack_32!(unpack_32_8, 8); +unpack_32!(unpack_32_9, 9); +unpack_32!(unpack_32_10, 10); +unpack_32!(unpack_32_11, 11); +unpack_32!(unpack_32_12, 12); +unpack_32!(unpack_32_13, 13); +unpack_32!(unpack_32_14, 14); +unpack_32!(unpack_32_15, 15); +unpack_32!(unpack_32_16, 16); +unpack_32!(unpack_32_17, 17); +unpack_32!(unpack_32_18, 18); +unpack_32!(unpack_32_19, 19); +unpack_32!(unpack_32_20, 20); +unpack_32!(unpack_32_21, 21); +unpack_32!(unpack_32_22, 22); +unpack_32!(unpack_32_23, 23); +unpack_32!(unpack_32_24, 24); +unpack_32!(unpack_32_25, 25); +unpack_32!(unpack_32_26, 26); +unpack_32!(unpack_32_27, 27); +unpack_32!(unpack_32_28, 28); +unpack_32!(unpack_32_29, 29); +unpack_32!(unpack_32_30, 30); +unpack_32!(unpack_32_31, 31); +unpack_32!(unpack_32_32, 32); + +macro_rules! pack_32 { + ($name:ident, $bits:expr) => { + fn $name(input: &[u32; 1024], output: &mut [u32; 1024 * $bits / u32::BITS as usize]) { + for lane in 0..u32::LANES { + pack!(u32, $bits, output, lane, |$idx| { + input[$idx] + }); + } + } + }; +} + +pack_32!(pack_32_1, 1); +pack_32!(pack_32_2, 2); +pack_32!(pack_32_3, 3); +pack_32!(pack_32_4, 4); +pack_32!(pack_32_5, 5); +pack_32!(pack_32_6, 6); +pack_32!(pack_32_7, 7); +pack_32!(pack_32_8, 8); +pack_32!(pack_32_9, 9); +pack_32!(pack_32_10, 10); +pack_32!(pack_32_11, 11); +pack_32!(pack_32_12, 12); +pack_32!(pack_32_13, 13); +pack_32!(pack_32_14, 14); +pack_32!(pack_32_15, 15); +pack_32!(pack_32_16, 16); +pack_32!(pack_32_17, 17); +pack_32!(pack_32_18, 18); +pack_32!(pack_32_19, 19); +pack_32!(pack_32_20, 20); +pack_32!(pack_32_21, 21); +pack_32!(pack_32_22, 22); +pack_32!(pack_32_23, 23); +pack_32!(pack_32_24, 24); +pack_32!(pack_32_25, 25); +pack_32!(pack_32_26, 26); +pack_32!(pack_32_27, 27); +pack_32!(pack_32_28, 28); +pack_32!(pack_32_29, 29); +pack_32!(pack_32_30, 30); +pack_32!(pack_32_31, 31); +pack_32!(pack_32_32, 32); + +macro_rules! generate_unpack_64 { + ($($n:expr),*) => { + $( + paste::item! { + fn [](input: &[u64; 1024 * $n / u64::T], output: &mut [u64; 1024]) { + for lane in 0..u64::LANES { + unpack!(u64, $n, input, lane, |$idx, $elem| { + output[$idx] = $elem + }); + } + } + } + )* + }; +} + +generate_unpack_64!( + 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, + 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, + 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, + 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64 +); + +macro_rules! generate_pack_64 { + ($($n:expr),*) => { + $( + paste::item! { + fn [](input: &[u64; 1024], output: &mut [u64; 1024 * $n / u64::T]) { + for lane in 0..u64::LANES { + pack!(u64, $n, output, lane, |$idx| { + input[$idx] + }); + } + } + } + )* + }; +} + +generate_pack_64!( + 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, + 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, + 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, + 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64 +); + +#[cfg(test)] +mod test { + use core::array; + use super::*; + + #[test] + fn test_pack() { + let mut values: [u16; 1024] = [0; 1024]; + for i in 0..1024 { + values[i] = (i % (1 << 15)) as u16; + } + + let mut packed: [u16; 960] = [0; 960]; + for lane in 0..u16::LANES { + // Always loop over lanes first. This is what the compiler vectorizes. + pack!(u16, 15, packed, lane, |$pos| { + values[$pos] + }); + } + + let mut packed_orig: [u16; 960] = [0; 960]; + unsafe { + + BitPacking::unchecked_pack(15, &values, &mut packed_orig); + } + + let mut unpacked: [u16; 1024] = [0; 1024]; + for lane in 0..u16::LANES { + // Always loop over lanes first. This is what the compiler vectorizes. + unpack!(u16, 15, packed, lane, |$idx, $elem| { + unpacked[$idx] = $elem; + }); + } + + assert_eq!(values, unpacked); + } + + #[test] + fn test_unchecked_pack() { + let input = array::from_fn(|i| i as u32); + let mut packed = [0; 320]; + unsafe { BitPacking::unchecked_pack(10, &input, &mut packed) }; + let mut output = [0; 1024]; + unsafe { BitPacking::unchecked_unpack(10, &packed, &mut output) }; + assert_eq!(input, output); + } +} From dba9a48b764d9db9144e34e4c680fff19340bc8f Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Fri, 20 Sep 2024 22:39:11 +0000 Subject: [PATCH 14/31] fix lint --- .../encodings/physical/bitpack_fastlanes.rs | 81 +++++++++++++------ 1 file changed, 55 insertions(+), 26 deletions(-) diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index 39d865c1f1..da3c012f92 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -199,7 +199,7 @@ macro_rules! encode_fixed_width { let num_chunks = ($unpacked.num_values + ELEMS_PER_CHUNK - 1) / ELEMS_PER_CHUNK; let num_full_chunks = $unpacked.num_values / ELEMS_PER_CHUNK; let uncompressed_bit_width = std::mem::size_of::<$data_type>() as u64 * 8; - + // the output vector type is the same as the input type, for example, when input is u16, output is Vec let packed_chunk_size = 1024 * $self.compressed_bit_width as usize / uncompressed_bit_width as usize; @@ -290,7 +290,7 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { location: location!(), }); }; - + match data_type { DataType::UInt8 | DataType::Int8 => encode_fixed_width!(self, unpacked, u8, buffer_index), DataType::UInt16 | DataType::Int16 => encode_fixed_width!(self, unpacked, u16, buffer_index), @@ -444,7 +444,8 @@ fn bitpacked_for_non_neg_decode( match uncompressed_bits_per_value { 8 => { let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); - let packed_chunk_size: usize = ELEMS_PER_CHUNK as usize * compressed_bit_width as usize / 8; + let packed_chunk_size: usize = + ELEMS_PER_CHUNK as usize * compressed_bit_width as usize / 8; let mut decompress_chunk_buf = vec![0_u8; ELEMS_PER_CHUNK as usize]; for (i, bytes) in data.iter().enumerate() { let mut j = 0; @@ -460,10 +461,14 @@ fn bitpacked_for_non_neg_decode( ); } loop { - if curr_range_start + ELEMS_PER_CHUNK < bytes_idx_to_range_indices[i][ranges_idx].end { - let this_part_len = ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; + if curr_range_start + ELEMS_PER_CHUNK + < bytes_idx_to_range_indices[i][ranges_idx].end + { + let this_part_len = + ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; decompressed.extend_from_slice( - &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..], + &decompress_chunk_buf + [(curr_range_start % ELEMS_PER_CHUNK) as usize..], ); curr_range_start += this_part_len; break; @@ -471,7 +476,8 @@ fn bitpacked_for_non_neg_decode( let this_part_len = bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; decompressed.extend_from_slice( - &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..] + &decompress_chunk_buf + [(curr_range_start % ELEMS_PER_CHUNK) as usize..] [..this_part_len as usize], ); ranges_idx += 1; @@ -489,7 +495,8 @@ fn bitpacked_for_non_neg_decode( 16 => { let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); - let packed_chunk_size_in_byte: usize = (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; + let packed_chunk_size_in_byte: usize = + (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; let mut decompress_chunk_buf = vec![0_u16; ELEMS_PER_CHUNK as usize]; for (i, bytes) in data.iter().enumerate() { let mut j = 0; @@ -507,10 +514,14 @@ fn bitpacked_for_non_neg_decode( ); } loop { - if curr_range_start + ELEMS_PER_CHUNK < bytes_idx_to_range_indices[i][ranges_idx].end { - let this_part_len = ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; + if curr_range_start + ELEMS_PER_CHUNK + < bytes_idx_to_range_indices[i][ranges_idx].end + { + let this_part_len = + ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; decompressed.extend_from_slice( - &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..], + &decompress_chunk_buf + [(curr_range_start % ELEMS_PER_CHUNK) as usize..], ); curr_range_start += this_part_len; @@ -521,7 +532,8 @@ fn bitpacked_for_non_neg_decode( let this_part_len = bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; decompressed.extend_from_slice( - &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..] + &decompress_chunk_buf + [(curr_range_start % ELEMS_PER_CHUNK) as usize..] [..this_part_len as usize], ); ranges_idx += 1; @@ -539,14 +551,16 @@ fn bitpacked_for_non_neg_decode( 32 => { let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); - let packed_chunk_size_in_byte: usize = (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; + let packed_chunk_size_in_byte: usize = + (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; let mut decompress_chunk_buf = vec![0_u32; ELEMS_PER_CHUNK as usize]; for (i, bytes) in data.iter().enumerate() { let mut ranges_idx = 0; let mut curr_range_start = bytes_idx_to_range_indices[i][0].start; let mut chunk_num = 0; while chunk_num * packed_chunk_size_in_byte < bytes.len() { - let chunk_in_u8: &[u8] = &bytes[chunk_num * packed_chunk_size_in_byte..][..packed_chunk_size_in_byte]; + let chunk_in_u8: &[u8] = &bytes[chunk_num * packed_chunk_size_in_byte..] + [..packed_chunk_size_in_byte]; chunk_num += 1; let chunk = cast_slice(chunk_in_u8); unsafe { @@ -557,22 +571,31 @@ fn bitpacked_for_non_neg_decode( ); } loop { - // case 1: all the elements after (curr_range_start % ELEMS_PER_CHUNK) inside this chunk is needed. - let elems_after_curr_range_start_in_this_chunk = ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; - if curr_range_start + elems_after_curr_range_start_in_this_chunk <= bytes_idx_to_range_indices[i][ranges_idx].end { + // case 1: all the elements after (curr_range_start % ELEMS_PER_CHUNK) inside this chunk is needed. + let elems_after_curr_range_start_in_this_chunk = + ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; + if curr_range_start + elems_after_curr_range_start_in_this_chunk + <= bytes_idx_to_range_indices[i][ranges_idx].end + { decompressed.extend_from_slice( - &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..], + &decompress_chunk_buf + [(curr_range_start % ELEMS_PER_CHUNK) as usize..], ); curr_range_start += elems_after_curr_range_start_in_this_chunk; break; } else { // case 2: only part of the elements after (curr_range_start % ELEMS_PER_CHUNK) inside this chunk is needed. - let elems_this_range_needed_in_this_chunk = (bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start).min(ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK); + let elems_this_range_needed_in_this_chunk = + (bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start) + .min(ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK); decompressed.extend_from_slice( - &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..] + &decompress_chunk_buf + [(curr_range_start % ELEMS_PER_CHUNK) as usize..] [..elems_this_range_needed_in_this_chunk as usize], ); - if curr_range_start + elems_this_range_needed_in_this_chunk == bytes_idx_to_range_indices[i][ranges_idx].end { + if curr_range_start + elems_this_range_needed_in_this_chunk + == bytes_idx_to_range_indices[i][ranges_idx].end + { ranges_idx += 1; if ranges_idx == bytes_idx_to_range_indices[i].len() { break; @@ -590,7 +613,8 @@ fn bitpacked_for_non_neg_decode( 64 => { let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); - let packed_chunk_size_in_byte: usize = (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; + let packed_chunk_size_in_byte: usize = + (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; let mut decompress_chunk_buf = vec![0_u64; ELEMS_PER_CHUNK as usize]; for (i, bytes) in data.iter().enumerate() { let mut j = 0; @@ -608,10 +632,14 @@ fn bitpacked_for_non_neg_decode( ); } loop { - if curr_range_start + ELEMS_PER_CHUNK < bytes_idx_to_range_indices[i][ranges_idx].end { - let this_part_len = ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; + if curr_range_start + ELEMS_PER_CHUNK + < bytes_idx_to_range_indices[i][ranges_idx].end + { + let this_part_len = + ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; decompressed.extend_from_slice( - &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..], + &decompress_chunk_buf + [(curr_range_start % ELEMS_PER_CHUNK) as usize..], ); curr_range_start += this_part_len; break; @@ -619,7 +647,8 @@ fn bitpacked_for_non_neg_decode( let this_part_len = bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; decompressed.extend_from_slice( - &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..] + &decompress_chunk_buf + [(curr_range_start % ELEMS_PER_CHUNK) as usize..] [..this_part_len as usize], ); ranges_idx += 1; From 1eb75e245dc1eab7fd3059e45619f5f6eb0087bc Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Sat, 21 Sep 2024 02:03:47 +0000 Subject: [PATCH 15/31] remove external fastlanes crate --- rust/lance-encoding/Cargo.toml | 4 +- rust/lance-encoding/compression-algo/lib.rs | 3 + rust/lance-encoding/compression-algo/mod.rs | 0 .../src/compression_algo/fastlanes.rs | 862 ++++++++++++++++++ .../src/compression_algo/mod.rs | 1 + .../encodings/physical/bitpack_fastlanes.rs | 3 +- rust/lance-encoding/src/lib.rs | 1 + 7 files changed, 871 insertions(+), 3 deletions(-) create mode 100644 rust/lance-encoding/compression-algo/lib.rs create mode 100644 rust/lance-encoding/compression-algo/mod.rs create mode 100644 rust/lance-encoding/src/compression_algo/fastlanes.rs create mode 100644 rust/lance-encoding/src/compression_algo/mod.rs diff --git a/rust/lance-encoding/Cargo.toml b/rust/lance-encoding/Cargo.toml index 56641e0e35..c9272c41ad 100644 --- a/rust/lance-encoding/Cargo.toml +++ b/rust/lance-encoding/Cargo.toml @@ -37,8 +37,10 @@ snafu.workspace = true tokio.workspace = true tracing.workspace = true zstd.workspace = true -fastlanes = "0.1.5" bytemuck = "=1.18.0" +arrayref = "0.3.7" +paste = "1.0.15" +seq-macro = "0.3.5" [dev-dependencies] lance-testing.workspace = true diff --git a/rust/lance-encoding/compression-algo/lib.rs b/rust/lance-encoding/compression-algo/lib.rs new file mode 100644 index 0000000000..1c36cd0efb --- /dev/null +++ b/rust/lance-encoding/compression-algo/lib.rs @@ -0,0 +1,3 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright The Lance Authors +pub mod compression_algo; \ No newline at end of file diff --git a/rust/lance-encoding/compression-algo/mod.rs b/rust/lance-encoding/compression-algo/mod.rs new file mode 100644 index 0000000000..e69de29bb2 diff --git a/rust/lance-encoding/src/compression_algo/fastlanes.rs b/rust/lance-encoding/src/compression_algo/fastlanes.rs new file mode 100644 index 0000000000..cd10844bc3 --- /dev/null +++ b/rust/lance-encoding/src/compression_algo/fastlanes.rs @@ -0,0 +1,862 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright The Lance Authors + +// This file is a modification of the `fastlanes` crate: https://github.com/spiraldb/fastlanes +// It is modified to allow a rust stable build + +use arrayref::{array_mut_ref, array_ref}; +use core::mem::size_of; +use paste::paste; + +pub const FL_ORDER: [usize; 8] = [0, 4, 2, 6, 1, 5, 3, 7]; + +pub trait FastLanes: Sized + Copy { + const T: usize = size_of::() * 8; + const LANES: usize = 1024 / Self::T; +} + +// Implement the trait for basic unsigned integer types +impl FastLanes for u8 {} +impl FastLanes for u16 {} +impl FastLanes for u32 {} +impl FastLanes for u64 {} + +#[macro_export] +macro_rules! iterate { + ($T:ty, $lane: expr, | $_1:tt $idx:ident | $($body:tt)*) => { + macro_rules! __kernel__ {( $_1 $idx:ident ) => ( $($body)* )} + { + use $crate::{seq_t, FL_ORDER}; + use paste::paste; + + #[inline(always)] + fn index(row: usize, lane: usize) -> usize { + let o = row / 8; + let s = row % 8; + (FL_ORDER[o] * 16) + (s * 128) + lane + } + + paste!(seq_t!(row in $T { + let idx = index(row, $lane); + __kernel__!(idx); + })); + } + } +} + +#[macro_export] +macro_rules! pack { + ($T:ty, $W:expr, $packed:expr, $lane:expr, | $_1:tt $idx:ident | $($body:tt)*) => { + macro_rules! __kernel__ {( $_1 $idx:ident ) => ( $($body)* )} + { + use paste::paste; + + // The number of bits of T. + const T: usize = <$T>::T; + + #[inline(always)] + fn index(row: usize, lane: usize) -> usize { + let o = row / 8; + let s = row % 8; + (FL_ORDER[o] * 16) + (s * 128) + lane + } + + if $W == 0 { + // Nothing to do if W is 0, since the packed array is zero bytes. + } else if $W == T { + // Special case for W=T, we can just copy the input value directly to the packed value. + paste!(seq_t!(row in $T { + let idx = index(row, $lane); + $packed[<$T>::LANES * row + $lane] = __kernel__!(idx); + })); + } else { + // A mask of W bits. + let mask: $T = (1 << $W) - 1; + + // First we loop over each lane in the virtual 1024 bit word. + let mut tmp: $T = 0; + + // Loop over each of the rows of the lane. + // Inlining this loop means all branches are known at compile time and + // the code is auto-vectorized for SIMD execution. + paste!(seq_t!(row in $T { + let idx = index(row, $lane); + let src = __kernel__!(idx); + let src = src & mask; + + // Shift the src bits into their position in the tmp output variable. + if row == 0 { + tmp = src; + } else { + tmp |= src << (row * $W) % T; + } + + // If the next packed position is after our current one, then we have filled + // the current output and we can write the packed value. + let curr_word: usize = (row * $W) / T; + let next_word: usize = ((row + 1) * $W) / T; + + #[allow(unused_assignments)] + if next_word > curr_word { + $packed[<$T>::LANES * curr_word + $lane] = tmp; + let remaining_bits: usize = ((row + 1) * $W) % T; + // Keep the remaining bits for the next packed value. + tmp = src >> $W - remaining_bits; + } + })); + } + } + }; +} + +#[macro_export] +macro_rules! unpack { + ($T:ty, $W:expr, $packed:expr, $lane:expr, | $_1:tt $idx:ident, $_2:tt $elem:ident | $($body:tt)*) => { + macro_rules! __kernel__ {( $_1 $idx:ident, $_2 $elem:ident ) => ( $($body)* )} + { + use paste::paste; + + // The number of bits of T. + const T: usize = <$T>::T; + + #[inline(always)] + fn index(row: usize, lane: usize) -> usize { + let o = row / 8; + let s = row % 8; + (FL_ORDER[o] * 16) + (s * 128) + lane + } + + if $W == 0 { + // Special case for W=0, we just need to zero the output. + // We'll still respect the iteration order in case the kernel has side effects. + paste!(seq_t!(row in $T { + let idx = index(row, $lane); + let zero: $T = 0; + __kernel__!(idx, zero); + })); + } else if $W == T { + // Special case for W=T, we can just copy the packed value directly to the output. + paste!(seq_t!(row in $T { + let idx = index(row, $lane); + let src = $packed[<$T>::LANES * row + $lane]; + __kernel__!(idx, src); + })); + } else { + #[inline] + fn mask(width: usize) -> $T { + if width == T { <$T>::MAX } else { (1 << (width % T)) - 1 } + } + + let mut src: $T = $packed[$lane]; + let mut tmp: $T; + + paste!(seq_t!(row in $T { + // Figure out the packed positions + let curr_word: usize = (row * $W) / T; + let next_word = ((row + 1) * $W) / T; + + let shift = (row * $W) % T; + + if next_word > curr_word { + // Consume some bits from the curr packed input, the remainder are in the next + // packed input value + let remaining_bits = ((row + 1) * $W) % T; + let current_bits = $W - remaining_bits; + tmp = (src >> shift) & mask(current_bits); + + if next_word < $W { + // Load the next packed value + src = $packed[<$T>::LANES * next_word + $lane]; + // Consume the remaining bits from the next input value. + tmp |= (src & mask(remaining_bits)) << current_bits; + } + } else { + // Otherwise, just grab W bits from the src value + tmp = (src >> shift) & mask($W); + } + + // Write out the unpacked value + let idx = index(row, $lane); + __kernel__!(idx, tmp); + })); + } + } + }; +} + +// Macro for repeating a code block bit_size_of:: times. +#[macro_export] +macro_rules! seq_t { + ($ident:ident in u8 $body:tt) => {seq_macro::seq!($ident in 0..8 $body)}; + ($ident:ident in u16 $body:tt) => {seq_macro::seq!($ident in 0..16 $body)}; + ($ident:ident in u32 $body:tt) => {seq_macro::seq!($ident in 0..32 $body)}; + ($ident:ident in u64 $body:tt) => {seq_macro::seq!($ident in 0..64 $body)}; +} + +/// `BitPack` into a compile-time known bit-width. +pub trait BitPacking: FastLanes { + /// Packs 1024 elements into `W` bits each, where `W` is runtime-known instead of + /// compile-time known. + /// + /// # Safety + /// The input slice must be of exactly length 1024. The output slice must be of length + /// `1024 * W / T`, where `T` is the bit-width of Self and `W` is the packed width. + /// These lengths are checked only with `debug_assert` (i.e., not checked on release builds). + unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]); + + /// Unpacks 1024 elements from `W` bits each, where `W` is runtime-known instead of + /// compile-time known. + /// + /// # Safety + /// The input slice must be of length `1024 * W / T`, where `T` is the bit-width of Self and `W` + /// is the packed width. The output slice must be of exactly length 1024. + /// These lengths are checked only with `debug_assert` (i.e., not checked on release builds). + unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]); +} + +impl BitPacking for u8 { + unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); + debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => pack_8_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u8::T]), + 2 => pack_8_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u8::T]), + 3 => pack_8_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u8::T]), + 4 => pack_8_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u8::T]), + 5 => pack_8_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u8::T]), + 6 => pack_8_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u8::T]), + 7 => pack_8_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u8::T]), + 8 => pack_8_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u8::T]), + + _ => unreachable!("Unsupported width: {}", width) + } + } + + unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); + debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => unpack_8_1(array_ref![input, 0, 1024 * 1 / u8::T], array_mut_ref![output, 0, 1024]), + 2 => unpack_8_2(array_ref![input, 0, 1024 * 2 / u8::T], array_mut_ref![output, 0, 1024]), + 3 => unpack_8_3(array_ref![input, 0, 1024 * 3 / u8::T], array_mut_ref![output, 0, 1024]), + 4 => unpack_8_4(array_ref![input, 0, 1024 * 4 / u8::T], array_mut_ref![output, 0, 1024]), + 5 => unpack_8_5(array_ref![input, 0, 1024 * 5 / u8::T], array_mut_ref![output, 0, 1024]), + 6 => unpack_8_6(array_ref![input, 0, 1024 * 6 / u8::T], array_mut_ref![output, 0, 1024]), + 7 => unpack_8_7(array_ref![input, 0, 1024 * 7 / u8::T], array_mut_ref![output, 0, 1024]), + 8 => unpack_8_8(array_ref![input, 0, 1024 * 8 / u8::T], array_mut_ref![output, 0, 1024]), + + _ => unreachable!("Unsupported width: {}", width) + } + } +} + +impl BitPacking for u16 { + unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); + debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => pack_16_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u16::T]), + 2 => pack_16_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u16::T]), + 3 => pack_16_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u16::T]), + 4 => pack_16_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u16::T]), + 5 => pack_16_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u16::T]), + 6 => pack_16_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u16::T]), + 7 => pack_16_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u16::T]), + 8 => pack_16_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u16::T]), + 9 => pack_16_9(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 9 / u16::T]), + + 10 => pack_16_10(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 10 / u16::T]), + 11 => pack_16_11(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 11 / u16::T]), + 12 => pack_16_12(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 12 / u16::T]), + 13 => pack_16_13(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 13 / u16::T]), + 14 => pack_16_14(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 14 / u16::T]), + 15 => pack_16_15(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 15 / u16::T]), + 16 => pack_16_16(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 16 / u16::T]), + + _ => unreachable!("Unsupported width: {}", width) + } + } + + unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); + debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => unpack_16_1(array_ref![input, 0, 1024 * 1 / u16::T], array_mut_ref![output, 0, 1024]), + 2 => unpack_16_2(array_ref![input, 0, 1024 * 2 / u16::T], array_mut_ref![output, 0, 1024]), + 3 => unpack_16_3(array_ref![input, 0, 1024 * 3 / u16::T], array_mut_ref![output, 0, 1024]), + 4 => unpack_16_4(array_ref![input, 0, 1024 * 4 / u16::T], array_mut_ref![output, 0, 1024]), + 5 => unpack_16_5(array_ref![input, 0, 1024 * 5 / u16::T], array_mut_ref![output, 0, 1024]), + 6 => unpack_16_6(array_ref![input, 0, 1024 * 6 / u16::T], array_mut_ref![output, 0, 1024]), + 7 => unpack_16_7(array_ref![input, 0, 1024 * 7 / u16::T], array_mut_ref![output, 0, 1024]), + 8 => unpack_16_8(array_ref![input, 0, 1024 * 8 / u16::T], array_mut_ref![output, 0, 1024]), + 9 => unpack_16_9(array_ref![input, 0, 1024 * 9 / u16::T], array_mut_ref![output, 0, 1024]), + + 10 => unpack_16_10(array_ref![input, 0, 1024 * 10 / u16::T], array_mut_ref![output, 0, 1024]), + 11 => unpack_16_11(array_ref![input, 0, 1024 * 11 / u16::T], array_mut_ref![output, 0, 1024]), + 12 => unpack_16_12(array_ref![input, 0, 1024 * 12 / u16::T], array_mut_ref![output, 0, 1024]), + 13 => unpack_16_13(array_ref![input, 0, 1024 * 13 / u16::T], array_mut_ref![output, 0, 1024]), + 14 => unpack_16_14(array_ref![input, 0, 1024 * 14 / u16::T], array_mut_ref![output, 0, 1024]), + 15 => unpack_16_15(array_ref![input, 0, 1024 * 15 / u16::T], array_mut_ref![output, 0, 1024]), + 16 => unpack_16_16(array_ref![input, 0, 1024 * 16 / u16::T], array_mut_ref![output, 0, 1024]), + + _ => unreachable!("Unsupported width: {}", width) + } + } +} + +impl BitPacking for u32 { + unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); + debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => pack_32_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u32::T]), + 2 => pack_32_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u32::T]), + 3 => pack_32_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u32::T]), + 4 => pack_32_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u32::T]), + 5 => pack_32_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u32::T]), + 6 => pack_32_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u32::T]), + 7 => pack_32_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u32::T]), + 8 => pack_32_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u32::T]), + 9 => pack_32_9(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 9 / u32::T]), + + 10 => pack_32_10(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 10 / u32::T]), + 11 => pack_32_11(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 11 / u32::T]), + 12 => pack_32_12(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 12 / u32::T]), + 13 => pack_32_13(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 13 / u32::T]), + 14 => pack_32_14(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 14 / u32::T]), + 15 => pack_32_15(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 15 / u32::T]), + 16 => pack_32_16(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 16 / u32::T]), + 17 => pack_32_17(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 17 / u32::T]), + 18 => pack_32_18(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 18 / u32::T]), + 19 => pack_32_19(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 19 / u32::T]), + + 20 => pack_32_20(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 20 / u32::T]), + 21 => pack_32_21(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 21 / u32::T]), + 22 => pack_32_22(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 22 / u32::T]), + 23 => pack_32_23(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 23 / u32::T]), + 24 => pack_32_24(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 24 / u32::T]), + 25 => pack_32_25(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 25 / u32::T]), + 26 => pack_32_26(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 26 / u32::T]), + 27 => pack_32_27(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 27 / u32::T]), + 28 => pack_32_28(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 28 / u32::T]), + 29 => pack_32_29(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 29 / u32::T]), + + 30 => pack_32_30(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 30 / u32::T]), + 31 => pack_32_31(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 31 / u32::T]), + 32 => pack_32_32(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 32 / u32::T]), + + _ => unreachable!("Unsupported width: {}", width) + } + } + + unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); + debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => unpack_32_1(array_ref![input, 0, 1024 * 1 / u32::T], array_mut_ref![output, 0, 1024]), + 2 => unpack_32_2(array_ref![input, 0, 1024 * 2 / u32::T], array_mut_ref![output, 0, 1024]), + 3 => unpack_32_3(array_ref![input, 0, 1024 * 3 / u32::T], array_mut_ref![output, 0, 1024]), + 4 => unpack_32_4(array_ref![input, 0, 1024 * 4 / u32::T], array_mut_ref![output, 0, 1024]), + 5 => unpack_32_5(array_ref![input, 0, 1024 * 5 / u32::T], array_mut_ref![output, 0, 1024]), + 6 => unpack_32_6(array_ref![input, 0, 1024 * 6 / u32::T], array_mut_ref![output, 0, 1024]), + 7 => unpack_32_7(array_ref![input, 0, 1024 * 7 / u32::T], array_mut_ref![output, 0, 1024]), + 8 => unpack_32_8(array_ref![input, 0, 1024 * 8 / u32::T], array_mut_ref![output, 0, 1024]), + 9 => unpack_32_9(array_ref![input, 0, 1024 * 9 / u32::T], array_mut_ref![output, 0, 1024]), + + 10 => unpack_32_10(array_ref![input, 0, 1024 * 10 / u32::T], array_mut_ref![output, 0, 1024]), + 11 => unpack_32_11(array_ref![input, 0, 1024 * 11 / u32::T], array_mut_ref![output, 0, 1024]), + 12 => unpack_32_12(array_ref![input, 0, 1024 * 12 / u32::T], array_mut_ref![output, 0, 1024]), + 13 => unpack_32_13(array_ref![input, 0, 1024 * 13 / u32::T], array_mut_ref![output, 0, 1024]), + 14 => unpack_32_14(array_ref![input, 0, 1024 * 14 / u32::T], array_mut_ref![output, 0, 1024]), + 15 => unpack_32_15(array_ref![input, 0, 1024 * 15 / u32::T], array_mut_ref![output, 0, 1024]), + 16 => unpack_32_16(array_ref![input, 0, 1024 * 16 / u32::T], array_mut_ref![output, 0, 1024]), + 17 => unpack_32_17(array_ref![input, 0, 1024 * 17 / u32::T], array_mut_ref![output, 0, 1024]), + 18 => unpack_32_18(array_ref![input, 0, 1024 * 18 / u32::T], array_mut_ref![output, 0, 1024]), + 19 => unpack_32_19(array_ref![input, 0, 1024 * 19 / u32::T], array_mut_ref![output, 0, 1024]), + + 20 => unpack_32_20(array_ref![input, 0, 1024 * 20 / u32::T], array_mut_ref![output, 0, 1024]), + 21 => unpack_32_21(array_ref![input, 0, 1024 * 21 / u32::T], array_mut_ref![output, 0, 1024]), + 22 => unpack_32_22(array_ref![input, 0, 1024 * 22 / u32::T], array_mut_ref![output, 0, 1024]), + 23 => unpack_32_23(array_ref![input, 0, 1024 * 23 / u32::T], array_mut_ref![output, 0, 1024]), + 24 => unpack_32_24(array_ref![input, 0, 1024 * 24 / u32::T], array_mut_ref![output, 0, 1024]), + 25 => unpack_32_25(array_ref![input, 0, 1024 * 25 / u32::T], array_mut_ref![output, 0, 1024]), + 26 => unpack_32_26(array_ref![input, 0, 1024 * 26 / u32::T], array_mut_ref![output, 0, 1024]), + 27 => unpack_32_27(array_ref![input, 0, 1024 * 27 / u32::T], array_mut_ref![output, 0, 1024]), + 28 => unpack_32_28(array_ref![input, 0, 1024 * 28 / u32::T], array_mut_ref![output, 0, 1024]), + 29 => unpack_32_29(array_ref![input, 0, 1024 * 29 / u32::T], array_mut_ref![output, 0, 1024]), + + 30 => unpack_32_30(array_ref![input, 0, 1024 * 30 / u32::T], array_mut_ref![output, 0, 1024]), + 31 => unpack_32_31(array_ref![input, 0, 1024 * 31 / u32::T], array_mut_ref![output, 0, 1024]), + 32 => unpack_32_32(array_ref![input, 0, 1024 * 32 / u32::T], array_mut_ref![output, 0, 1024]), + + _ => unreachable!("Unsupported width: {}", width) + } + } +} + +impl BitPacking for u64 { + unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); + debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => pack_64_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u64::T]), + 2 => pack_64_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u64::T]), + 3 => pack_64_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u64::T]), + 4 => pack_64_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u64::T]), + 5 => pack_64_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u64::T]), + 6 => pack_64_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u64::T]), + 7 => pack_64_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u64::T]), + 8 => pack_64_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u64::T]), + 9 => pack_64_9(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 9 / u64::T]), + + 10 => pack_64_10(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 10 / u64::T]), + 11 => pack_64_11(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 11 / u64::T]), + 12 => pack_64_12(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 12 / u64::T]), + 13 => pack_64_13(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 13 / u64::T]), + 14 => pack_64_14(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 14 / u64::T]), + 15 => pack_64_15(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 15 / u64::T]), + 16 => pack_64_16(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 16 / u64::T]), + 17 => pack_64_17(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 17 / u64::T]), + 18 => pack_64_18(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 18 / u64::T]), + 19 => pack_64_19(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 19 / u64::T]), + + 20 => pack_64_20(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 20 / u64::T]), + 21 => pack_64_21(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 21 / u64::T]), + 22 => pack_64_22(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 22 / u64::T]), + 23 => pack_64_23(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 23 / u64::T]), + 24 => pack_64_24(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 24 / u64::T]), + 25 => pack_64_25(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 25 / u64::T]), + 26 => pack_64_26(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 26 / u64::T]), + 27 => pack_64_27(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 27 / u64::T]), + 28 => pack_64_28(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 28 / u64::T]), + 29 => pack_64_29(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 29 / u64::T]), + + 30 => pack_64_30(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 30 / u64::T]), + 31 => pack_64_31(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 31 / u64::T]), + 32 => pack_64_32(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 32 / u64::T]), + 33 => pack_64_33(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 33 / u64::T]), + 34 => pack_64_34(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 34 / u64::T]), + 35 => pack_64_35(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 35 / u64::T]), + 36 => pack_64_36(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 36 / u64::T]), + 37 => pack_64_37(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 37 / u64::T]), + 38 => pack_64_38(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 38 / u64::T]), + 39 => pack_64_39(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 39 / u64::T]), + + 40 => pack_64_40(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 40 / u64::T]), + 41 => pack_64_41(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 41 / u64::T]), + 42 => pack_64_42(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 42 / u64::T]), + 43 => pack_64_43(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 43 / u64::T]), + 44 => pack_64_44(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 44 / u64::T]), + 45 => pack_64_45(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 45 / u64::T]), + 46 => pack_64_46(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 46 / u64::T]), + 47 => pack_64_47(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 47 / u64::T]), + 48 => pack_64_48(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 48 / u64::T]), + 49 => pack_64_49(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 49 / u64::T]), + + 50 => pack_64_50(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 50 / u64::T]), + 51 => pack_64_51(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 51 / u64::T]), + 52 => pack_64_52(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 52 / u64::T]), + 53 => pack_64_53(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 53 / u64::T]), + 54 => pack_64_54(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 54 / u64::T]), + 55 => pack_64_55(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 55 / u64::T]), + 56 => pack_64_56(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 56 / u64::T]), + 57 => pack_64_57(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 57 / u64::T]), + 58 => pack_64_58(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 58 / u64::T]), + 59 => pack_64_59(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 59 / u64::T]), + + 60 => pack_64_60(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 60 / u64::T]), + 61 => pack_64_61(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 61 / u64::T]), + 62 => pack_64_62(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 62 / u64::T]), + 63 => pack_64_63(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 63 / u64::T]), + 64 => pack_64_64(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 64 / u64::T]), + + _ => unreachable!("Unsupported width: {}", width) + } + } + + unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { + let packed_len = 128 * width / size_of::(); + debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); + debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); + debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + + match width { + 1 => unpack_64_1(array_ref![input, 0, 1024 * 1 / u64::T], array_mut_ref![output, 0, 1024]), + 2 => unpack_64_2(array_ref![input, 0, 1024 * 2 / u64::T], array_mut_ref![output, 0, 1024]), + 3 => unpack_64_3(array_ref![input, 0, 1024 * 3 / u64::T], array_mut_ref![output, 0, 1024]), + 4 => unpack_64_4(array_ref![input, 0, 1024 * 4 / u64::T], array_mut_ref![output, 0, 1024]), + 5 => unpack_64_5(array_ref![input, 0, 1024 * 5 / u64::T], array_mut_ref![output, 0, 1024]), + 6 => unpack_64_6(array_ref![input, 0, 1024 * 6 / u64::T], array_mut_ref![output, 0, 1024]), + 7 => unpack_64_7(array_ref![input, 0, 1024 * 7 / u64::T], array_mut_ref![output, 0, 1024]), + 8 => unpack_64_8(array_ref![input, 0, 1024 * 8 / u64::T], array_mut_ref![output, 0, 1024]), + 9 => unpack_64_9(array_ref![input, 0, 1024 * 9 / u64::T], array_mut_ref![output, 0, 1024]), + + 10 => unpack_64_10(array_ref![input, 0, 1024 * 10 / u64::T], array_mut_ref![output, 0, 1024]), + 11 => unpack_64_11(array_ref![input, 0, 1024 * 11 / u64::T], array_mut_ref![output, 0, 1024]), + 12 => unpack_64_12(array_ref![input, 0, 1024 * 12 / u64::T], array_mut_ref![output, 0, 1024]), + 13 => unpack_64_13(array_ref![input, 0, 1024 * 13 / u64::T], array_mut_ref![output, 0, 1024]), + 14 => unpack_64_14(array_ref![input, 0, 1024 * 14 / u64::T], array_mut_ref![output, 0, 1024]), + 15 => unpack_64_15(array_ref![input, 0, 1024 * 15 / u64::T], array_mut_ref![output, 0, 1024]), + 16 => unpack_64_16(array_ref![input, 0, 1024 * 16 / u64::T], array_mut_ref![output, 0, 1024]), + 17 => unpack_64_17(array_ref![input, 0, 1024 * 17 / u64::T], array_mut_ref![output, 0, 1024]), + 18 => unpack_64_18(array_ref![input, 0, 1024 * 18 / u64::T], array_mut_ref![output, 0, 1024]), + 19 => unpack_64_19(array_ref![input, 0, 1024 * 19 / u64::T], array_mut_ref![output, 0, 1024]), + + 20 => unpack_64_20(array_ref![input, 0, 1024 * 20 / u64::T], array_mut_ref![output, 0, 1024]), + 21 => unpack_64_21(array_ref![input, 0, 1024 * 21 / u64::T], array_mut_ref![output, 0, 1024]), + 22 => unpack_64_22(array_ref![input, 0, 1024 * 22 / u64::T], array_mut_ref![output, 0, 1024]), + 23 => unpack_64_23(array_ref![input, 0, 1024 * 23 / u64::T], array_mut_ref![output, 0, 1024]), + 24 => unpack_64_24(array_ref![input, 0, 1024 * 24 / u64::T], array_mut_ref![output, 0, 1024]), + 25 => unpack_64_25(array_ref![input, 0, 1024 * 25 / u64::T], array_mut_ref![output, 0, 1024]), + 26 => unpack_64_26(array_ref![input, 0, 1024 * 26 / u64::T], array_mut_ref![output, 0, 1024]), + 27 => unpack_64_27(array_ref![input, 0, 1024 * 27 / u64::T], array_mut_ref![output, 0, 1024]), + 28 => unpack_64_28(array_ref![input, 0, 1024 * 28 / u64::T], array_mut_ref![output, 0, 1024]), + 29 => unpack_64_29(array_ref![input, 0, 1024 * 29 / u64::T], array_mut_ref![output, 0, 1024]), + + 30 => unpack_64_30(array_ref![input, 0, 1024 * 30 / u64::T], array_mut_ref![output, 0, 1024]), + 31 => unpack_64_31(array_ref![input, 0, 1024 * 31 / u64::T], array_mut_ref![output, 0, 1024]), + 32 => unpack_64_32(array_ref![input, 0, 1024 * 32 / u64::T], array_mut_ref![output, 0, 1024]), + 33 => unpack_64_33(array_ref![input, 0, 1024 * 33 / u64::T], array_mut_ref![output, 0, 1024]), + 34 => unpack_64_34(array_ref![input, 0, 1024 * 34 / u64::T], array_mut_ref![output, 0, 1024]), + 35 => unpack_64_35(array_ref![input, 0, 1024 * 35 / u64::T], array_mut_ref![output, 0, 1024]), + 36 => unpack_64_36(array_ref![input, 0, 1024 * 36 / u64::T], array_mut_ref![output, 0, 1024]), + 37 => unpack_64_37(array_ref![input, 0, 1024 * 37 / u64::T], array_mut_ref![output, 0, 1024]), + 38 => unpack_64_38(array_ref![input, 0, 1024 * 38 / u64::T], array_mut_ref![output, 0, 1024]), + 39 => unpack_64_39(array_ref![input, 0, 1024 * 39 / u64::T], array_mut_ref![output, 0, 1024]), + + 40 => unpack_64_40(array_ref![input, 0, 1024 * 40 / u64::T], array_mut_ref![output, 0, 1024]), + 41 => unpack_64_41(array_ref![input, 0, 1024 * 41 / u64::T], array_mut_ref![output, 0, 1024]), + 42 => unpack_64_42(array_ref![input, 0, 1024 * 42 / u64::T], array_mut_ref![output, 0, 1024]), + 43 => unpack_64_43(array_ref![input, 0, 1024 * 43 / u64::T], array_mut_ref![output, 0, 1024]), + 44 => unpack_64_44(array_ref![input, 0, 1024 * 44 / u64::T], array_mut_ref![output, 0, 1024]), + 45 => unpack_64_45(array_ref![input, 0, 1024 * 45 / u64::T], array_mut_ref![output, 0, 1024]), + 46 => unpack_64_46(array_ref![input, 0, 1024 * 46 / u64::T], array_mut_ref![output, 0, 1024]), + 47 => unpack_64_47(array_ref![input, 0, 1024 * 47 / u64::T], array_mut_ref![output, 0, 1024]), + 48 => unpack_64_48(array_ref![input, 0, 1024 * 48 / u64::T], array_mut_ref![output, 0, 1024]), + 49 => unpack_64_49(array_ref![input, 0, 1024 * 49 / u64::T], array_mut_ref![output, 0, 1024]), + + 50 => unpack_64_50(array_ref![input, 0, 1024 * 50 / u64::T], array_mut_ref![output, 0, 1024]), + 51 => unpack_64_51(array_ref![input, 0, 1024 * 51 / u64::T], array_mut_ref![output, 0, 1024]), + 52 => unpack_64_52(array_ref![input, 0, 1024 * 52 / u64::T], array_mut_ref![output, 0, 1024]), + 53 => unpack_64_53(array_ref![input, 0, 1024 * 53 / u64::T], array_mut_ref![output, 0, 1024]), + 54 => unpack_64_54(array_ref![input, 0, 1024 * 54 / u64::T], array_mut_ref![output, 0, 1024]), + 55 => unpack_64_55(array_ref![input, 0, 1024 * 55 / u64::T], array_mut_ref![output, 0, 1024]), + 56 => unpack_64_56(array_ref![input, 0, 1024 * 56 / u64::T], array_mut_ref![output, 0, 1024]), + 57 => unpack_64_57(array_ref![input, 0, 1024 * 57 / u64::T], array_mut_ref![output, 0, 1024]), + 58 => unpack_64_58(array_ref![input, 0, 1024 * 58 / u64::T], array_mut_ref![output, 0, 1024]), + 59 => unpack_64_59(array_ref![input, 0, 1024 * 59 / u64::T], array_mut_ref![output, 0, 1024]), + + 60 => unpack_64_60(array_ref![input, 0, 1024 * 60 / u64::T], array_mut_ref![output, 0, 1024]), + 61 => unpack_64_61(array_ref![input, 0, 1024 * 61 / u64::T], array_mut_ref![output, 0, 1024]), + 62 => unpack_64_62(array_ref![input, 0, 1024 * 62 / u64::T], array_mut_ref![output, 0, 1024]), + 63 => unpack_64_63(array_ref![input, 0, 1024 * 63 / u64::T], array_mut_ref![output, 0, 1024]), + 64 => unpack_64_64(array_ref![input, 0, 1024 * 64 / u64::T], array_mut_ref![output, 0, 1024]), + + _ => unreachable!("Unsupported width: {}", width) + } + } +} + +macro_rules! unpack_8 { + ($name:ident, $bits:expr) => { + fn $name(input: &[u8; 1024 * $bits / u8::T], output: &mut [u8; 1024]) { + for lane in 0..u8::LANES { + unpack!(u8, $bits, input, lane, |$idx, $elem| { + output[$idx] = $elem; + }); + } + } + }; +} + +unpack_8!(unpack_8_1, 1); +unpack_8!(unpack_8_2, 2); +unpack_8!(unpack_8_3, 3); +unpack_8!(unpack_8_4, 4); +unpack_8!(unpack_8_5, 5); +unpack_8!(unpack_8_6, 6); +unpack_8!(unpack_8_7, 7); +unpack_8!(unpack_8_8, 8); + +macro_rules! pack_8 { + ($name:ident, $bits:expr) => { + fn $name(input: &[u8; 1024], output: &mut [u8; 1024 * $bits / u8::T]) { + for lane in 0..u8::LANES { + pack!(u8, $bits, output, lane, |$idx| { + input[$idx] + }); + } + } + }; +} +pack_8!(pack_8_1, 1); +pack_8!(pack_8_2, 2); +pack_8!(pack_8_3, 3); +pack_8!(pack_8_4, 4); +pack_8!(pack_8_5, 5); +pack_8!(pack_8_6, 6); +pack_8!(pack_8_7, 7); +pack_8!(pack_8_8, 8); + +macro_rules! unpack_16 { + ($name:ident, $bits:expr) => { + fn $name(input: &[u16; 1024 * $bits / u16::T], output: &mut [u16; 1024]) { + for lane in 0..u16::LANES { + unpack!(u16, $bits, input, lane, |$idx, $elem| { + output[$idx] = $elem; + }); + } + } + }; +} + +unpack_16!(unpack_16_1, 1); +unpack_16!(unpack_16_2, 2); +unpack_16!(unpack_16_3, 3); +unpack_16!(unpack_16_4, 4); +unpack_16!(unpack_16_5, 5); +unpack_16!(unpack_16_6, 6); +unpack_16!(unpack_16_7, 7); +unpack_16!(unpack_16_8, 8); +unpack_16!(unpack_16_9, 9); +unpack_16!(unpack_16_10, 10); +unpack_16!(unpack_16_11, 11); +unpack_16!(unpack_16_12, 12); +unpack_16!(unpack_16_13, 13); +unpack_16!(unpack_16_14, 14); +unpack_16!(unpack_16_15, 15); +unpack_16!(unpack_16_16, 16); + +macro_rules! pack_16 { + ($name:ident, $bits:expr) => { + fn $name(input: &[u16; 1024], output: &mut [u16; 1024 * $bits / u16::T]) { + for lane in 0..u16::LANES { + pack!(u16, $bits, output, lane, |$idx| { + input[$idx] + }); + } + } + }; +} + +pack_16!(pack_16_1, 1); +pack_16!(pack_16_2, 2); +pack_16!(pack_16_3, 3); +pack_16!(pack_16_4, 4); +pack_16!(pack_16_5, 5); +pack_16!(pack_16_6, 6); +pack_16!(pack_16_7, 7); +pack_16!(pack_16_8, 8); +pack_16!(pack_16_9, 9); +pack_16!(pack_16_10, 10); +pack_16!(pack_16_11, 11); +pack_16!(pack_16_12, 12); +pack_16!(pack_16_13, 13); +pack_16!(pack_16_14, 14); +pack_16!(pack_16_15, 15); +pack_16!(pack_16_16, 16); + +macro_rules! unpack_32 { + ($name:ident, $bit_width:expr) => { + fn $name(input: &[u32; 1024 * $bit_width / u32::T], output: &mut [u32; 1024]) { + for lane in 0..u32::LANES { + unpack!(u32, $bit_width, input, lane, |$idx, $elem| { + output[$idx] = $elem + }); + } + } + }; +} + +unpack_32!(unpack_32_1, 1); +unpack_32!(unpack_32_2, 2); +unpack_32!(unpack_32_3, 3); +unpack_32!(unpack_32_4, 4); +unpack_32!(unpack_32_5, 5); +unpack_32!(unpack_32_6, 6); +unpack_32!(unpack_32_7, 7); +unpack_32!(unpack_32_8, 8); +unpack_32!(unpack_32_9, 9); +unpack_32!(unpack_32_10, 10); +unpack_32!(unpack_32_11, 11); +unpack_32!(unpack_32_12, 12); +unpack_32!(unpack_32_13, 13); +unpack_32!(unpack_32_14, 14); +unpack_32!(unpack_32_15, 15); +unpack_32!(unpack_32_16, 16); +unpack_32!(unpack_32_17, 17); +unpack_32!(unpack_32_18, 18); +unpack_32!(unpack_32_19, 19); +unpack_32!(unpack_32_20, 20); +unpack_32!(unpack_32_21, 21); +unpack_32!(unpack_32_22, 22); +unpack_32!(unpack_32_23, 23); +unpack_32!(unpack_32_24, 24); +unpack_32!(unpack_32_25, 25); +unpack_32!(unpack_32_26, 26); +unpack_32!(unpack_32_27, 27); +unpack_32!(unpack_32_28, 28); +unpack_32!(unpack_32_29, 29); +unpack_32!(unpack_32_30, 30); +unpack_32!(unpack_32_31, 31); +unpack_32!(unpack_32_32, 32); + +macro_rules! pack_32 { + ($name:ident, $bits:expr) => { + fn $name(input: &[u32; 1024], output: &mut [u32; 1024 * $bits / u32::BITS as usize]) { + for lane in 0..u32::LANES { + pack!(u32, $bits, output, lane, |$idx| { + input[$idx] + }); + } + } + }; +} + +pack_32!(pack_32_1, 1); +pack_32!(pack_32_2, 2); +pack_32!(pack_32_3, 3); +pack_32!(pack_32_4, 4); +pack_32!(pack_32_5, 5); +pack_32!(pack_32_6, 6); +pack_32!(pack_32_7, 7); +pack_32!(pack_32_8, 8); +pack_32!(pack_32_9, 9); +pack_32!(pack_32_10, 10); +pack_32!(pack_32_11, 11); +pack_32!(pack_32_12, 12); +pack_32!(pack_32_13, 13); +pack_32!(pack_32_14, 14); +pack_32!(pack_32_15, 15); +pack_32!(pack_32_16, 16); +pack_32!(pack_32_17, 17); +pack_32!(pack_32_18, 18); +pack_32!(pack_32_19, 19); +pack_32!(pack_32_20, 20); +pack_32!(pack_32_21, 21); +pack_32!(pack_32_22, 22); +pack_32!(pack_32_23, 23); +pack_32!(pack_32_24, 24); +pack_32!(pack_32_25, 25); +pack_32!(pack_32_26, 26); +pack_32!(pack_32_27, 27); +pack_32!(pack_32_28, 28); +pack_32!(pack_32_29, 29); +pack_32!(pack_32_30, 30); +pack_32!(pack_32_31, 31); +pack_32!(pack_32_32, 32); + +macro_rules! generate_unpack_64 { + ($($n:expr),*) => { + $( + paste::item! { + fn [](input: &[u64; 1024 * $n / u64::T], output: &mut [u64; 1024]) { + for lane in 0..u64::LANES { + unpack!(u64, $n, input, lane, |$idx, $elem| { + output[$idx] = $elem + }); + } + } + } + )* + }; +} + +generate_unpack_64!( + 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, + 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, + 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, + 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64 +); + +macro_rules! generate_pack_64 { + ($($n:expr),*) => { + $( + paste::item! { + fn [](input: &[u64; 1024], output: &mut [u64; 1024 * $n / u64::T]) { + for lane in 0..u64::LANES { + pack!(u64, $n, output, lane, |$idx| { + input[$idx] + }); + } + } + } + )* + }; +} + +generate_pack_64!( + 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, + 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, + 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, + 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64 +); + +#[cfg(test)] +mod test { + use core::array; + use super::*; + + #[test] + fn test_pack() { + let mut values: [u16; 1024] = [0; 1024]; + for i in 0..1024 { + values[i] = (i % (1 << 15)) as u16; + } + + let mut packed: [u16; 960] = [0; 960]; + for lane in 0..u16::LANES { + // Always loop over lanes first. This is what the compiler vectorizes. + pack!(u16, 15, packed, lane, |$pos| { + values[$pos] + }); + } + + let mut packed_orig: [u16; 960] = [0; 960]; + unsafe { + + BitPacking::unchecked_pack(15, &values, &mut packed_orig); + } + + let mut unpacked: [u16; 1024] = [0; 1024]; + for lane in 0..u16::LANES { + // Always loop over lanes first. This is what the compiler vectorizes. + unpack!(u16, 15, packed, lane, |$idx, $elem| { + unpacked[$idx] = $elem; + }); + } + + assert_eq!(values, unpacked); + } + + #[test] + fn test_unchecked_pack() { + let input = array::from_fn(|i| i as u32); + let mut packed = [0; 320]; + unsafe { BitPacking::unchecked_pack(10, &input, &mut packed) }; + let mut output = [0; 1024]; + unsafe { BitPacking::unchecked_unpack(10, &packed, &mut output) }; + assert_eq!(input, output); + } +} diff --git a/rust/lance-encoding/src/compression_algo/mod.rs b/rust/lance-encoding/src/compression_algo/mod.rs new file mode 100644 index 0000000000..c823ac9cd0 --- /dev/null +++ b/rust/lance-encoding/src/compression_algo/mod.rs @@ -0,0 +1 @@ +pub mod fastlanes; \ No newline at end of file diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index da3c012f92..11438b9f64 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -23,8 +23,7 @@ use crate::encoder::{ArrayEncoder, EncodedArray}; use crate::format::ProtobufUtils; use arrow::array::ArrayRef; use bytemuck::cast_slice; -use fastlanes::BitPacking; - +use crate::compression_algo::fastlanes::BitPacking; const ELEMS_PER_CHUNK: u64 = 1024; // Compute the compressed_bit_width for a given array of integers diff --git a/rust/lance-encoding/src/lib.rs b/rust/lance-encoding/src/lib.rs index 17a897df9f..4bfa9193ae 100644 --- a/rust/lance-encoding/src/lib.rs +++ b/rust/lance-encoding/src/lib.rs @@ -14,6 +14,7 @@ pub mod decoder; pub mod encoder; pub mod encodings; pub mod format; +pub mod compression_algo; #[cfg(test)] pub mod testing; pub mod version; From 148575995de0e2c52bd51966defe996b7ec20c5c Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Sat, 21 Sep 2024 02:06:12 +0000 Subject: [PATCH 16/31] license header --- rust/lance-encoding/src/compression_algo/mod.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/rust/lance-encoding/src/compression_algo/mod.rs b/rust/lance-encoding/src/compression_algo/mod.rs index c823ac9cd0..2cddf46de5 100644 --- a/rust/lance-encoding/src/compression_algo/mod.rs +++ b/rust/lance-encoding/src/compression_algo/mod.rs @@ -1 +1,4 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright The Lance Authors + pub mod fastlanes; \ No newline at end of file From 8543f542a0d5178c237218c83c141bae21f5ac27 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Sat, 21 Sep 2024 02:43:58 +0000 Subject: [PATCH 17/31] fix lint --- .../src/compression_algo/fastlanes.rs | 1572 +++++++++++++---- .../src/compression_algo/mod.rs | 2 +- .../encodings/physical/bitpack_fastlanes.rs | 2 +- rust/lance-encoding/src/lib.rs | 2 +- 4 files changed, 1231 insertions(+), 347 deletions(-) diff --git a/rust/lance-encoding/src/compression_algo/fastlanes.rs b/rust/lance-encoding/src/compression_algo/fastlanes.rs index cd10844bc3..3ef964dcf3 100644 --- a/rust/lance-encoding/src/compression_algo/fastlanes.rs +++ b/rust/lance-encoding/src/compression_algo/fastlanes.rs @@ -2,7 +2,7 @@ // SPDX-FileCopyrightText: Copyright The Lance Authors // This file is a modification of the `fastlanes` crate: https://github.com/spiraldb/fastlanes -// It is modified to allow a rust stable build +// It is modified to allow a rust stable build use arrayref::{array_mut_ref, array_ref}; use core::mem::size_of; @@ -217,41 +217,105 @@ pub trait BitPacking: FastLanes { impl BitPacking for u8 { unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { let packed_len = 128 * width / size_of::(); - debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); + debug_assert_eq!( + output.len(), + packed_len, + "Output buffer must be of size 1024 * W / T" + ); debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + debug_assert!( + width <= Self::T, + "Width must be less than or equal to {}", + Self::T + ); match width { - 1 => pack_8_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u8::T]), - 2 => pack_8_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u8::T]), - 3 => pack_8_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u8::T]), - 4 => pack_8_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u8::T]), - 5 => pack_8_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u8::T]), - 6 => pack_8_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u8::T]), - 7 => pack_8_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u8::T]), - 8 => pack_8_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u8::T]), - - _ => unreachable!("Unsupported width: {}", width) + 1 => pack_8_1( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 / 8], + ), + 2 => pack_8_2( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 2 / 8], + ), + 3 => pack_8_3( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 3 / 8], + ), + 4 => pack_8_4( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 4 / 8], + ), + 5 => pack_8_5( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 5 / 8], + ), + 6 => pack_8_6( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 6 / 8], + ), + 7 => pack_8_7( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 7 / 8], + ), + 8 => pack_8_8( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 8 / 8], + ), + + _ => unreachable!("Unsupported width: {}", width), } } unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { let packed_len = 128 * width / size_of::(); - debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); + debug_assert_eq!( + input.len(), + packed_len, + "Input buffer must be of size 1024 * W / T" + ); debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + debug_assert!( + width <= Self::T, + "Width must be less than or equal to {}", + Self::T + ); match width { - 1 => unpack_8_1(array_ref![input, 0, 1024 * 1 / u8::T], array_mut_ref![output, 0, 1024]), - 2 => unpack_8_2(array_ref![input, 0, 1024 * 2 / u8::T], array_mut_ref![output, 0, 1024]), - 3 => unpack_8_3(array_ref![input, 0, 1024 * 3 / u8::T], array_mut_ref![output, 0, 1024]), - 4 => unpack_8_4(array_ref![input, 0, 1024 * 4 / u8::T], array_mut_ref![output, 0, 1024]), - 5 => unpack_8_5(array_ref![input, 0, 1024 * 5 / u8::T], array_mut_ref![output, 0, 1024]), - 6 => unpack_8_6(array_ref![input, 0, 1024 * 6 / u8::T], array_mut_ref![output, 0, 1024]), - 7 => unpack_8_7(array_ref![input, 0, 1024 * 7 / u8::T], array_mut_ref![output, 0, 1024]), - 8 => unpack_8_8(array_ref![input, 0, 1024 * 8 / u8::T], array_mut_ref![output, 0, 1024]), - - _ => unreachable!("Unsupported width: {}", width) + 1 => unpack_8_1( + array_ref![input, 0, 1024 / 8], + array_mut_ref![output, 0, 1024], + ), + 2 => unpack_8_2( + array_ref![input, 0, 1024 * 2 / 8], + array_mut_ref![output, 0, 1024], + ), + 3 => unpack_8_3( + array_ref![input, 0, 1024 * 3 / 8], + array_mut_ref![output, 0, 1024], + ), + 4 => unpack_8_4( + array_ref![input, 0, 1024 * 4 / 8], + array_mut_ref![output, 0, 1024], + ), + 5 => unpack_8_5( + array_ref![input, 0, 1024 * 5 / 8], + array_mut_ref![output, 0, 1024], + ), + 6 => unpack_8_6( + array_ref![input, 0, 1024 * 6 / 8], + array_mut_ref![output, 0, 1024], + ), + 7 => unpack_8_7( + array_ref![input, 0, 1024 * 7 / 8], + array_mut_ref![output, 0, 1024], + ), + 8 => unpack_8_8( + array_ref![input, 0, 1024 * 8 / 8], + array_mut_ref![output, 0, 1024], + ), + + _ => unreachable!("Unsupported width: {}", width), } } } @@ -259,59 +323,171 @@ impl BitPacking for u8 { impl BitPacking for u16 { unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { let packed_len = 128 * width / size_of::(); - debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); + debug_assert_eq!( + output.len(), + packed_len, + "Output buffer must be of size 1024 * W / T" + ); debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + debug_assert!( + width <= Self::T, + "Width must be less than or equal to {}", + Self::T + ); match width { - 1 => pack_16_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u16::T]), - 2 => pack_16_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u16::T]), - 3 => pack_16_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u16::T]), - 4 => pack_16_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u16::T]), - 5 => pack_16_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u16::T]), - 6 => pack_16_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u16::T]), - 7 => pack_16_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u16::T]), - 8 => pack_16_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u16::T]), - 9 => pack_16_9(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 9 / u16::T]), - - 10 => pack_16_10(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 10 / u16::T]), - 11 => pack_16_11(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 11 / u16::T]), - 12 => pack_16_12(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 12 / u16::T]), - 13 => pack_16_13(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 13 / u16::T]), - 14 => pack_16_14(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 14 / u16::T]), - 15 => pack_16_15(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 15 / u16::T]), - 16 => pack_16_16(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 16 / u16::T]), - - _ => unreachable!("Unsupported width: {}", width) + 1 => pack_16_1( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 / 16], + ), + 2 => pack_16_2( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 2 / 16], + ), + 3 => pack_16_3( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 3 / 16], + ), + 4 => pack_16_4( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 4 / 16], + ), + 5 => pack_16_5( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 5 / 16], + ), + 6 => pack_16_6( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 6 / 16], + ), + 7 => pack_16_7( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 7 / 16], + ), + 8 => pack_16_8( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 8 / 16], + ), + 9 => pack_16_9( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 9 / 16], + ), + + 10 => pack_16_10( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 10 / 16], + ), + 11 => pack_16_11( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 11 / 16], + ), + 12 => pack_16_12( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 12 / 16], + ), + 13 => pack_16_13( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 13 / 16], + ), + 14 => pack_16_14( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 14 / 16], + ), + 15 => pack_16_15( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 15 / 16], + ), + 16 => pack_16_16( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 16 / 16], + ), + + _ => unreachable!("Unsupported width: {}", width), } } unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { let packed_len = 128 * width / size_of::(); - debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); + debug_assert_eq!( + input.len(), + packed_len, + "Input buffer must be of size 1024 * W / T" + ); debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + debug_assert!( + width <= Self::T, + "Width must be less than or equal to {}", + Self::T + ); match width { - 1 => unpack_16_1(array_ref![input, 0, 1024 * 1 / u16::T], array_mut_ref![output, 0, 1024]), - 2 => unpack_16_2(array_ref![input, 0, 1024 * 2 / u16::T], array_mut_ref![output, 0, 1024]), - 3 => unpack_16_3(array_ref![input, 0, 1024 * 3 / u16::T], array_mut_ref![output, 0, 1024]), - 4 => unpack_16_4(array_ref![input, 0, 1024 * 4 / u16::T], array_mut_ref![output, 0, 1024]), - 5 => unpack_16_5(array_ref![input, 0, 1024 * 5 / u16::T], array_mut_ref![output, 0, 1024]), - 6 => unpack_16_6(array_ref![input, 0, 1024 * 6 / u16::T], array_mut_ref![output, 0, 1024]), - 7 => unpack_16_7(array_ref![input, 0, 1024 * 7 / u16::T], array_mut_ref![output, 0, 1024]), - 8 => unpack_16_8(array_ref![input, 0, 1024 * 8 / u16::T], array_mut_ref![output, 0, 1024]), - 9 => unpack_16_9(array_ref![input, 0, 1024 * 9 / u16::T], array_mut_ref![output, 0, 1024]), - - 10 => unpack_16_10(array_ref![input, 0, 1024 * 10 / u16::T], array_mut_ref![output, 0, 1024]), - 11 => unpack_16_11(array_ref![input, 0, 1024 * 11 / u16::T], array_mut_ref![output, 0, 1024]), - 12 => unpack_16_12(array_ref![input, 0, 1024 * 12 / u16::T], array_mut_ref![output, 0, 1024]), - 13 => unpack_16_13(array_ref![input, 0, 1024 * 13 / u16::T], array_mut_ref![output, 0, 1024]), - 14 => unpack_16_14(array_ref![input, 0, 1024 * 14 / u16::T], array_mut_ref![output, 0, 1024]), - 15 => unpack_16_15(array_ref![input, 0, 1024 * 15 / u16::T], array_mut_ref![output, 0, 1024]), - 16 => unpack_16_16(array_ref![input, 0, 1024 * 16 / u16::T], array_mut_ref![output, 0, 1024]), - - _ => unreachable!("Unsupported width: {}", width) + 1 => unpack_16_1( + array_ref![input, 0, 1024 / 16], + array_mut_ref![output, 0, 1024], + ), + 2 => unpack_16_2( + array_ref![input, 0, 1024 * 2 / 16], + array_mut_ref![output, 0, 1024], + ), + 3 => unpack_16_3( + array_ref![input, 0, 1024 * 3 / 16], + array_mut_ref![output, 0, 1024], + ), + 4 => unpack_16_4( + array_ref![input, 0, 1024 * 4 / 16], + array_mut_ref![output, 0, 1024], + ), + 5 => unpack_16_5( + array_ref![input, 0, 1024 * 5 / 16], + array_mut_ref![output, 0, 1024], + ), + 6 => unpack_16_6( + array_ref![input, 0, 1024 * 6 / 16], + array_mut_ref![output, 0, 1024], + ), + 7 => unpack_16_7( + array_ref![input, 0, 1024 * 7 / 16], + array_mut_ref![output, 0, 1024], + ), + 8 => unpack_16_8( + array_ref![input, 0, 1024 * 8 / 16], + array_mut_ref![output, 0, 1024], + ), + 9 => unpack_16_9( + array_ref![input, 0, 1024 * 9 / 16], + array_mut_ref![output, 0, 1024], + ), + + 10 => unpack_16_10( + array_ref![input, 0, 1024 * 10 / 16], + array_mut_ref![output, 0, 1024], + ), + 11 => unpack_16_11( + array_ref![input, 0, 1024 * 11 / 16], + array_mut_ref![output, 0, 1024], + ), + 12 => unpack_16_12( + array_ref![input, 0, 1024 * 12 / 16], + array_mut_ref![output, 0, 1024], + ), + 13 => unpack_16_13( + array_ref![input, 0, 1024 * 13 / 16], + array_mut_ref![output, 0, 1024], + ), + 14 => unpack_16_14( + array_ref![input, 0, 1024 * 14 / 16], + array_mut_ref![output, 0, 1024], + ), + 15 => unpack_16_15( + array_ref![input, 0, 1024 * 15 / 16], + array_mut_ref![output, 0, 1024], + ), + 16 => unpack_16_16( + array_ref![input, 0, 1024 * 16 / 16], + array_mut_ref![output, 0, 1024], + ), + + _ => unreachable!("Unsupported width: {}", width), } } } @@ -319,95 +495,303 @@ impl BitPacking for u16 { impl BitPacking for u32 { unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { let packed_len = 128 * width / size_of::(); - debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); + debug_assert_eq!( + output.len(), + packed_len, + "Output buffer must be of size 1024 * W / T" + ); debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + debug_assert!( + width <= Self::T, + "Width must be less than or equal to {}", + Self::T + ); match width { - 1 => pack_32_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u32::T]), - 2 => pack_32_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u32::T]), - 3 => pack_32_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u32::T]), - 4 => pack_32_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u32::T]), - 5 => pack_32_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u32::T]), - 6 => pack_32_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u32::T]), - 7 => pack_32_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u32::T]), - 8 => pack_32_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u32::T]), - 9 => pack_32_9(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 9 / u32::T]), - - 10 => pack_32_10(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 10 / u32::T]), - 11 => pack_32_11(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 11 / u32::T]), - 12 => pack_32_12(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 12 / u32::T]), - 13 => pack_32_13(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 13 / u32::T]), - 14 => pack_32_14(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 14 / u32::T]), - 15 => pack_32_15(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 15 / u32::T]), - 16 => pack_32_16(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 16 / u32::T]), - 17 => pack_32_17(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 17 / u32::T]), - 18 => pack_32_18(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 18 / u32::T]), - 19 => pack_32_19(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 19 / u32::T]), - - 20 => pack_32_20(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 20 / u32::T]), - 21 => pack_32_21(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 21 / u32::T]), - 22 => pack_32_22(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 22 / u32::T]), - 23 => pack_32_23(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 23 / u32::T]), - 24 => pack_32_24(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 24 / u32::T]), - 25 => pack_32_25(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 25 / u32::T]), - 26 => pack_32_26(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 26 / u32::T]), - 27 => pack_32_27(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 27 / u32::T]), - 28 => pack_32_28(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 28 / u32::T]), - 29 => pack_32_29(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 29 / u32::T]), - - 30 => pack_32_30(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 30 / u32::T]), - 31 => pack_32_31(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 31 / u32::T]), - 32 => pack_32_32(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 32 / u32::T]), - - _ => unreachable!("Unsupported width: {}", width) + 1 => pack_32_1( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 / 32], + ), + 2 => pack_32_2( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 2 / 32], + ), + 3 => pack_32_3( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 3 / 32], + ), + 4 => pack_32_4( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 4 / 32], + ), + 5 => pack_32_5( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 5 / 32], + ), + 6 => pack_32_6( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 6 / 32], + ), + 7 => pack_32_7( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 7 / 32], + ), + 8 => pack_32_8( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 8 / 32], + ), + 9 => pack_32_9( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 9 / 32], + ), + + 10 => pack_32_10( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 10 / 32], + ), + 11 => pack_32_11( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 11 / 32], + ), + 12 => pack_32_12( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 12 / 32], + ), + 13 => pack_32_13( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 13 / 32], + ), + 14 => pack_32_14( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 14 / 32], + ), + 15 => pack_32_15( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 15 / 32], + ), + 16 => pack_32_16( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 16 / 32], + ), + 17 => pack_32_17( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 17 / 32], + ), + 18 => pack_32_18( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 18 / 32], + ), + 19 => pack_32_19( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 19 / 32], + ), + + 20 => pack_32_20( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 20 / 32], + ), + 21 => pack_32_21( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 21 / 32], + ), + 22 => pack_32_22( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 22 / 32], + ), + 23 => pack_32_23( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 23 / 32], + ), + 24 => pack_32_24( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 24 / 32], + ), + 25 => pack_32_25( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 25 / 32], + ), + 26 => pack_32_26( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 26 / 32], + ), + 27 => pack_32_27( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 27 / 32], + ), + 28 => pack_32_28( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 28 / 32], + ), + 29 => pack_32_29( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 29 / 32], + ), + + 30 => pack_32_30( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 30 / 32], + ), + 31 => pack_32_31( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 31 / 32], + ), + 32 => pack_32_32( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 32 / 32], + ), + + _ => unreachable!("Unsupported width: {}", width), } } unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { let packed_len = 128 * width / size_of::(); - debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); + debug_assert_eq!( + input.len(), + packed_len, + "Input buffer must be of size 1024 * W / T" + ); debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + debug_assert!( + width <= Self::T, + "Width must be less than or equal to {}", + Self::T + ); match width { - 1 => unpack_32_1(array_ref![input, 0, 1024 * 1 / u32::T], array_mut_ref![output, 0, 1024]), - 2 => unpack_32_2(array_ref![input, 0, 1024 * 2 / u32::T], array_mut_ref![output, 0, 1024]), - 3 => unpack_32_3(array_ref![input, 0, 1024 * 3 / u32::T], array_mut_ref![output, 0, 1024]), - 4 => unpack_32_4(array_ref![input, 0, 1024 * 4 / u32::T], array_mut_ref![output, 0, 1024]), - 5 => unpack_32_5(array_ref![input, 0, 1024 * 5 / u32::T], array_mut_ref![output, 0, 1024]), - 6 => unpack_32_6(array_ref![input, 0, 1024 * 6 / u32::T], array_mut_ref![output, 0, 1024]), - 7 => unpack_32_7(array_ref![input, 0, 1024 * 7 / u32::T], array_mut_ref![output, 0, 1024]), - 8 => unpack_32_8(array_ref![input, 0, 1024 * 8 / u32::T], array_mut_ref![output, 0, 1024]), - 9 => unpack_32_9(array_ref![input, 0, 1024 * 9 / u32::T], array_mut_ref![output, 0, 1024]), - - 10 => unpack_32_10(array_ref![input, 0, 1024 * 10 / u32::T], array_mut_ref![output, 0, 1024]), - 11 => unpack_32_11(array_ref![input, 0, 1024 * 11 / u32::T], array_mut_ref![output, 0, 1024]), - 12 => unpack_32_12(array_ref![input, 0, 1024 * 12 / u32::T], array_mut_ref![output, 0, 1024]), - 13 => unpack_32_13(array_ref![input, 0, 1024 * 13 / u32::T], array_mut_ref![output, 0, 1024]), - 14 => unpack_32_14(array_ref![input, 0, 1024 * 14 / u32::T], array_mut_ref![output, 0, 1024]), - 15 => unpack_32_15(array_ref![input, 0, 1024 * 15 / u32::T], array_mut_ref![output, 0, 1024]), - 16 => unpack_32_16(array_ref![input, 0, 1024 * 16 / u32::T], array_mut_ref![output, 0, 1024]), - 17 => unpack_32_17(array_ref![input, 0, 1024 * 17 / u32::T], array_mut_ref![output, 0, 1024]), - 18 => unpack_32_18(array_ref![input, 0, 1024 * 18 / u32::T], array_mut_ref![output, 0, 1024]), - 19 => unpack_32_19(array_ref![input, 0, 1024 * 19 / u32::T], array_mut_ref![output, 0, 1024]), - - 20 => unpack_32_20(array_ref![input, 0, 1024 * 20 / u32::T], array_mut_ref![output, 0, 1024]), - 21 => unpack_32_21(array_ref![input, 0, 1024 * 21 / u32::T], array_mut_ref![output, 0, 1024]), - 22 => unpack_32_22(array_ref![input, 0, 1024 * 22 / u32::T], array_mut_ref![output, 0, 1024]), - 23 => unpack_32_23(array_ref![input, 0, 1024 * 23 / u32::T], array_mut_ref![output, 0, 1024]), - 24 => unpack_32_24(array_ref![input, 0, 1024 * 24 / u32::T], array_mut_ref![output, 0, 1024]), - 25 => unpack_32_25(array_ref![input, 0, 1024 * 25 / u32::T], array_mut_ref![output, 0, 1024]), - 26 => unpack_32_26(array_ref![input, 0, 1024 * 26 / u32::T], array_mut_ref![output, 0, 1024]), - 27 => unpack_32_27(array_ref![input, 0, 1024 * 27 / u32::T], array_mut_ref![output, 0, 1024]), - 28 => unpack_32_28(array_ref![input, 0, 1024 * 28 / u32::T], array_mut_ref![output, 0, 1024]), - 29 => unpack_32_29(array_ref![input, 0, 1024 * 29 / u32::T], array_mut_ref![output, 0, 1024]), - - 30 => unpack_32_30(array_ref![input, 0, 1024 * 30 / u32::T], array_mut_ref![output, 0, 1024]), - 31 => unpack_32_31(array_ref![input, 0, 1024 * 31 / u32::T], array_mut_ref![output, 0, 1024]), - 32 => unpack_32_32(array_ref![input, 0, 1024 * 32 / u32::T], array_mut_ref![output, 0, 1024]), - - _ => unreachable!("Unsupported width: {}", width) + 1 => unpack_32_1( + array_ref![input, 0, 1024 / 32], + array_mut_ref![output, 0, 1024], + ), + 2 => unpack_32_2( + array_ref![input, 0, 1024 * 2 / 32], + array_mut_ref![output, 0, 1024], + ), + 3 => unpack_32_3( + array_ref![input, 0, 1024 * 3 / 32], + array_mut_ref![output, 0, 1024], + ), + 4 => unpack_32_4( + array_ref![input, 0, 1024 * 4 / 32], + array_mut_ref![output, 0, 1024], + ), + 5 => unpack_32_5( + array_ref![input, 0, 1024 * 5 / 32], + array_mut_ref![output, 0, 1024], + ), + 6 => unpack_32_6( + array_ref![input, 0, 1024 * 6 / 32], + array_mut_ref![output, 0, 1024], + ), + 7 => unpack_32_7( + array_ref![input, 0, 1024 * 7 / 32], + array_mut_ref![output, 0, 1024], + ), + 8 => unpack_32_8( + array_ref![input, 0, 1024 * 8 / 32], + array_mut_ref![output, 0, 1024], + ), + 9 => unpack_32_9( + array_ref![input, 0, 1024 * 9 / 32], + array_mut_ref![output, 0, 1024], + ), + + 10 => unpack_32_10( + array_ref![input, 0, 1024 * 10 / 32], + array_mut_ref![output, 0, 1024], + ), + 11 => unpack_32_11( + array_ref![input, 0, 1024 * 11 / 32], + array_mut_ref![output, 0, 1024], + ), + 12 => unpack_32_12( + array_ref![input, 0, 1024 * 12 / 32], + array_mut_ref![output, 0, 1024], + ), + 13 => unpack_32_13( + array_ref![input, 0, 1024 * 13 / 32], + array_mut_ref![output, 0, 1024], + ), + 14 => unpack_32_14( + array_ref![input, 0, 1024 * 14 / 32], + array_mut_ref![output, 0, 1024], + ), + 15 => unpack_32_15( + array_ref![input, 0, 1024 * 15 / 32], + array_mut_ref![output, 0, 1024], + ), + 16 => unpack_32_16( + array_ref![input, 0, 1024 * 16 / 32], + array_mut_ref![output, 0, 1024], + ), + 17 => unpack_32_17( + array_ref![input, 0, 1024 * 17 / 32], + array_mut_ref![output, 0, 1024], + ), + 18 => unpack_32_18( + array_ref![input, 0, 1024 * 18 / 32], + array_mut_ref![output, 0, 1024], + ), + 19 => unpack_32_19( + array_ref![input, 0, 1024 * 19 / 32], + array_mut_ref![output, 0, 1024], + ), + + 20 => unpack_32_20( + array_ref![input, 0, 1024 * 20 / 32], + array_mut_ref![output, 0, 1024], + ), + 21 => unpack_32_21( + array_ref![input, 0, 1024 * 21 / 32], + array_mut_ref![output, 0, 1024], + ), + 22 => unpack_32_22( + array_ref![input, 0, 1024 * 22 / 32], + array_mut_ref![output, 0, 1024], + ), + 23 => unpack_32_23( + array_ref![input, 0, 1024 * 23 / 32], + array_mut_ref![output, 0, 1024], + ), + 24 => unpack_32_24( + array_ref![input, 0, 1024 * 24 / 32], + array_mut_ref![output, 0, 1024], + ), + 25 => unpack_32_25( + array_ref![input, 0, 1024 * 25 / 32], + array_mut_ref![output, 0, 1024], + ), + 26 => unpack_32_26( + array_ref![input, 0, 1024 * 26 / 32], + array_mut_ref![output, 0, 1024], + ), + 27 => unpack_32_27( + array_ref![input, 0, 1024 * 27 / 32], + array_mut_ref![output, 0, 1024], + ), + 28 => unpack_32_28( + array_ref![input, 0, 1024 * 28 / 32], + array_mut_ref![output, 0, 1024], + ), + 29 => unpack_32_29( + array_ref![input, 0, 1024 * 29 / 32], + array_mut_ref![output, 0, 1024], + ), + + 30 => unpack_32_30( + array_ref![input, 0, 1024 * 30 / 32], + array_mut_ref![output, 0, 1024], + ), + 31 => unpack_32_31( + array_ref![input, 0, 1024 * 31 / 32], + array_mut_ref![output, 0, 1024], + ), + 32 => unpack_32_32( + array_ref![input, 0, 1024 * 32 / 32], + array_mut_ref![output, 0, 1024], + ), + + _ => unreachable!("Unsupported width: {}", width), } } } @@ -415,165 +799,565 @@ impl BitPacking for u32 { impl BitPacking for u64 { unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { let packed_len = 128 * width / size_of::(); - debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); + debug_assert_eq!( + output.len(), + packed_len, + "Output buffer must be of size 1024 * W / T" + ); debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + debug_assert!( + width <= Self::T, + "Width must be less than or equal to {}", + Self::T + ); match width { - 1 => pack_64_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u64::T]), - 2 => pack_64_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u64::T]), - 3 => pack_64_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u64::T]), - 4 => pack_64_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u64::T]), - 5 => pack_64_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u64::T]), - 6 => pack_64_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u64::T]), - 7 => pack_64_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u64::T]), - 8 => pack_64_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u64::T]), - 9 => pack_64_9(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 9 / u64::T]), - - 10 => pack_64_10(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 10 / u64::T]), - 11 => pack_64_11(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 11 / u64::T]), - 12 => pack_64_12(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 12 / u64::T]), - 13 => pack_64_13(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 13 / u64::T]), - 14 => pack_64_14(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 14 / u64::T]), - 15 => pack_64_15(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 15 / u64::T]), - 16 => pack_64_16(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 16 / u64::T]), - 17 => pack_64_17(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 17 / u64::T]), - 18 => pack_64_18(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 18 / u64::T]), - 19 => pack_64_19(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 19 / u64::T]), - - 20 => pack_64_20(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 20 / u64::T]), - 21 => pack_64_21(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 21 / u64::T]), - 22 => pack_64_22(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 22 / u64::T]), - 23 => pack_64_23(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 23 / u64::T]), - 24 => pack_64_24(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 24 / u64::T]), - 25 => pack_64_25(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 25 / u64::T]), - 26 => pack_64_26(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 26 / u64::T]), - 27 => pack_64_27(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 27 / u64::T]), - 28 => pack_64_28(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 28 / u64::T]), - 29 => pack_64_29(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 29 / u64::T]), - - 30 => pack_64_30(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 30 / u64::T]), - 31 => pack_64_31(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 31 / u64::T]), - 32 => pack_64_32(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 32 / u64::T]), - 33 => pack_64_33(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 33 / u64::T]), - 34 => pack_64_34(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 34 / u64::T]), - 35 => pack_64_35(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 35 / u64::T]), - 36 => pack_64_36(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 36 / u64::T]), - 37 => pack_64_37(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 37 / u64::T]), - 38 => pack_64_38(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 38 / u64::T]), - 39 => pack_64_39(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 39 / u64::T]), - - 40 => pack_64_40(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 40 / u64::T]), - 41 => pack_64_41(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 41 / u64::T]), - 42 => pack_64_42(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 42 / u64::T]), - 43 => pack_64_43(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 43 / u64::T]), - 44 => pack_64_44(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 44 / u64::T]), - 45 => pack_64_45(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 45 / u64::T]), - 46 => pack_64_46(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 46 / u64::T]), - 47 => pack_64_47(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 47 / u64::T]), - 48 => pack_64_48(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 48 / u64::T]), - 49 => pack_64_49(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 49 / u64::T]), - - 50 => pack_64_50(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 50 / u64::T]), - 51 => pack_64_51(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 51 / u64::T]), - 52 => pack_64_52(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 52 / u64::T]), - 53 => pack_64_53(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 53 / u64::T]), - 54 => pack_64_54(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 54 / u64::T]), - 55 => pack_64_55(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 55 / u64::T]), - 56 => pack_64_56(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 56 / u64::T]), - 57 => pack_64_57(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 57 / u64::T]), - 58 => pack_64_58(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 58 / u64::T]), - 59 => pack_64_59(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 59 / u64::T]), - - 60 => pack_64_60(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 60 / u64::T]), - 61 => pack_64_61(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 61 / u64::T]), - 62 => pack_64_62(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 62 / u64::T]), - 63 => pack_64_63(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 63 / u64::T]), - 64 => pack_64_64(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 64 / u64::T]), - - _ => unreachable!("Unsupported width: {}", width) + 1 => pack_64_1( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 / 64], + ), + 2 => pack_64_2( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 2 / 64], + ), + 3 => pack_64_3( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 3 / 64], + ), + 4 => pack_64_4( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 4 / 64], + ), + 5 => pack_64_5( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 5 / 64], + ), + 6 => pack_64_6( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 6 / 64], + ), + 7 => pack_64_7( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 7 / 64], + ), + 8 => pack_64_8( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 8 / 64], + ), + 9 => pack_64_9( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 9 / 64], + ), + + 10 => pack_64_10( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 10 / 64], + ), + 11 => pack_64_11( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 11 / 64], + ), + 12 => pack_64_12( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 12 / 64], + ), + 13 => pack_64_13( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 13 / 64], + ), + 14 => pack_64_14( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 14 / 64], + ), + 15 => pack_64_15( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 15 / 64], + ), + 16 => pack_64_16( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 16 / 64], + ), + 17 => pack_64_17( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 17 / 64], + ), + 18 => pack_64_18( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 18 / 64], + ), + 19 => pack_64_19( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 19 / 64], + ), + + 20 => pack_64_20( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 20 / 64], + ), + 21 => pack_64_21( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 21 / 64], + ), + 22 => pack_64_22( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 22 / 64], + ), + 23 => pack_64_23( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 23 / 64], + ), + 24 => pack_64_24( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 24 / 64], + ), + 25 => pack_64_25( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 25 / 64], + ), + 26 => pack_64_26( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 26 / 64], + ), + 27 => pack_64_27( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 27 / 64], + ), + 28 => pack_64_28( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 28 / 64], + ), + 29 => pack_64_29( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 29 / 64], + ), + + 30 => pack_64_30( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 30 / 64], + ), + 31 => pack_64_31( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 31 / 64], + ), + 32 => pack_64_32( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 32 / 64], + ), + 33 => pack_64_33( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 33 / 64], + ), + 34 => pack_64_34( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 34 / 64], + ), + 35 => pack_64_35( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 35 / 64], + ), + 36 => pack_64_36( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 36 / 64], + ), + 37 => pack_64_37( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 37 / 64], + ), + 38 => pack_64_38( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 38 / 64], + ), + 39 => pack_64_39( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 39 / 64], + ), + + 40 => pack_64_40( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 40 / 64], + ), + 41 => pack_64_41( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 41 / 64], + ), + 42 => pack_64_42( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 42 / 64], + ), + 43 => pack_64_43( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 43 / 64], + ), + 44 => pack_64_44( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 44 / 64], + ), + 45 => pack_64_45( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 45 / 64], + ), + 46 => pack_64_46( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 46 / 64], + ), + 47 => pack_64_47( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 47 / 64], + ), + 48 => pack_64_48( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 48 / 64], + ), + 49 => pack_64_49( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 49 / 64], + ), + + 50 => pack_64_50( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 50 / 64], + ), + 51 => pack_64_51( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 51 / 64], + ), + 52 => pack_64_52( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 52 / 64], + ), + 53 => pack_64_53( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 53 / 64], + ), + 54 => pack_64_54( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 54 / 64], + ), + 55 => pack_64_55( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 55 / 64], + ), + 56 => pack_64_56( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 56 / 64], + ), + 57 => pack_64_57( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 57 / 64], + ), + 58 => pack_64_58( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 58 / 64], + ), + 59 => pack_64_59( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 59 / 64], + ), + + 60 => pack_64_60( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 60 / 64], + ), + 61 => pack_64_61( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 61 / 64], + ), + 62 => pack_64_62( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 62 / 64], + ), + 63 => pack_64_63( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 63 / 64], + ), + 64 => pack_64_64( + array_ref![input, 0, 1024], + array_mut_ref![output, 0, 1024 * 64 / 64], + ), + + _ => unreachable!("Unsupported width: {}", width), } } unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { let packed_len = 128 * width / size_of::(); - debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); + debug_assert_eq!( + input.len(), + packed_len, + "Input buffer must be of size 1024 * W / T" + ); debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); + debug_assert!( + width <= Self::T, + "Width must be less than or equal to {}", + Self::T + ); match width { - 1 => unpack_64_1(array_ref![input, 0, 1024 * 1 / u64::T], array_mut_ref![output, 0, 1024]), - 2 => unpack_64_2(array_ref![input, 0, 1024 * 2 / u64::T], array_mut_ref![output, 0, 1024]), - 3 => unpack_64_3(array_ref![input, 0, 1024 * 3 / u64::T], array_mut_ref![output, 0, 1024]), - 4 => unpack_64_4(array_ref![input, 0, 1024 * 4 / u64::T], array_mut_ref![output, 0, 1024]), - 5 => unpack_64_5(array_ref![input, 0, 1024 * 5 / u64::T], array_mut_ref![output, 0, 1024]), - 6 => unpack_64_6(array_ref![input, 0, 1024 * 6 / u64::T], array_mut_ref![output, 0, 1024]), - 7 => unpack_64_7(array_ref![input, 0, 1024 * 7 / u64::T], array_mut_ref![output, 0, 1024]), - 8 => unpack_64_8(array_ref![input, 0, 1024 * 8 / u64::T], array_mut_ref![output, 0, 1024]), - 9 => unpack_64_9(array_ref![input, 0, 1024 * 9 / u64::T], array_mut_ref![output, 0, 1024]), - - 10 => unpack_64_10(array_ref![input, 0, 1024 * 10 / u64::T], array_mut_ref![output, 0, 1024]), - 11 => unpack_64_11(array_ref![input, 0, 1024 * 11 / u64::T], array_mut_ref![output, 0, 1024]), - 12 => unpack_64_12(array_ref![input, 0, 1024 * 12 / u64::T], array_mut_ref![output, 0, 1024]), - 13 => unpack_64_13(array_ref![input, 0, 1024 * 13 / u64::T], array_mut_ref![output, 0, 1024]), - 14 => unpack_64_14(array_ref![input, 0, 1024 * 14 / u64::T], array_mut_ref![output, 0, 1024]), - 15 => unpack_64_15(array_ref![input, 0, 1024 * 15 / u64::T], array_mut_ref![output, 0, 1024]), - 16 => unpack_64_16(array_ref![input, 0, 1024 * 16 / u64::T], array_mut_ref![output, 0, 1024]), - 17 => unpack_64_17(array_ref![input, 0, 1024 * 17 / u64::T], array_mut_ref![output, 0, 1024]), - 18 => unpack_64_18(array_ref![input, 0, 1024 * 18 / u64::T], array_mut_ref![output, 0, 1024]), - 19 => unpack_64_19(array_ref![input, 0, 1024 * 19 / u64::T], array_mut_ref![output, 0, 1024]), - - 20 => unpack_64_20(array_ref![input, 0, 1024 * 20 / u64::T], array_mut_ref![output, 0, 1024]), - 21 => unpack_64_21(array_ref![input, 0, 1024 * 21 / u64::T], array_mut_ref![output, 0, 1024]), - 22 => unpack_64_22(array_ref![input, 0, 1024 * 22 / u64::T], array_mut_ref![output, 0, 1024]), - 23 => unpack_64_23(array_ref![input, 0, 1024 * 23 / u64::T], array_mut_ref![output, 0, 1024]), - 24 => unpack_64_24(array_ref![input, 0, 1024 * 24 / u64::T], array_mut_ref![output, 0, 1024]), - 25 => unpack_64_25(array_ref![input, 0, 1024 * 25 / u64::T], array_mut_ref![output, 0, 1024]), - 26 => unpack_64_26(array_ref![input, 0, 1024 * 26 / u64::T], array_mut_ref![output, 0, 1024]), - 27 => unpack_64_27(array_ref![input, 0, 1024 * 27 / u64::T], array_mut_ref![output, 0, 1024]), - 28 => unpack_64_28(array_ref![input, 0, 1024 * 28 / u64::T], array_mut_ref![output, 0, 1024]), - 29 => unpack_64_29(array_ref![input, 0, 1024 * 29 / u64::T], array_mut_ref![output, 0, 1024]), - - 30 => unpack_64_30(array_ref![input, 0, 1024 * 30 / u64::T], array_mut_ref![output, 0, 1024]), - 31 => unpack_64_31(array_ref![input, 0, 1024 * 31 / u64::T], array_mut_ref![output, 0, 1024]), - 32 => unpack_64_32(array_ref![input, 0, 1024 * 32 / u64::T], array_mut_ref![output, 0, 1024]), - 33 => unpack_64_33(array_ref![input, 0, 1024 * 33 / u64::T], array_mut_ref![output, 0, 1024]), - 34 => unpack_64_34(array_ref![input, 0, 1024 * 34 / u64::T], array_mut_ref![output, 0, 1024]), - 35 => unpack_64_35(array_ref![input, 0, 1024 * 35 / u64::T], array_mut_ref![output, 0, 1024]), - 36 => unpack_64_36(array_ref![input, 0, 1024 * 36 / u64::T], array_mut_ref![output, 0, 1024]), - 37 => unpack_64_37(array_ref![input, 0, 1024 * 37 / u64::T], array_mut_ref![output, 0, 1024]), - 38 => unpack_64_38(array_ref![input, 0, 1024 * 38 / u64::T], array_mut_ref![output, 0, 1024]), - 39 => unpack_64_39(array_ref![input, 0, 1024 * 39 / u64::T], array_mut_ref![output, 0, 1024]), - - 40 => unpack_64_40(array_ref![input, 0, 1024 * 40 / u64::T], array_mut_ref![output, 0, 1024]), - 41 => unpack_64_41(array_ref![input, 0, 1024 * 41 / u64::T], array_mut_ref![output, 0, 1024]), - 42 => unpack_64_42(array_ref![input, 0, 1024 * 42 / u64::T], array_mut_ref![output, 0, 1024]), - 43 => unpack_64_43(array_ref![input, 0, 1024 * 43 / u64::T], array_mut_ref![output, 0, 1024]), - 44 => unpack_64_44(array_ref![input, 0, 1024 * 44 / u64::T], array_mut_ref![output, 0, 1024]), - 45 => unpack_64_45(array_ref![input, 0, 1024 * 45 / u64::T], array_mut_ref![output, 0, 1024]), - 46 => unpack_64_46(array_ref![input, 0, 1024 * 46 / u64::T], array_mut_ref![output, 0, 1024]), - 47 => unpack_64_47(array_ref![input, 0, 1024 * 47 / u64::T], array_mut_ref![output, 0, 1024]), - 48 => unpack_64_48(array_ref![input, 0, 1024 * 48 / u64::T], array_mut_ref![output, 0, 1024]), - 49 => unpack_64_49(array_ref![input, 0, 1024 * 49 / u64::T], array_mut_ref![output, 0, 1024]), - - 50 => unpack_64_50(array_ref![input, 0, 1024 * 50 / u64::T], array_mut_ref![output, 0, 1024]), - 51 => unpack_64_51(array_ref![input, 0, 1024 * 51 / u64::T], array_mut_ref![output, 0, 1024]), - 52 => unpack_64_52(array_ref![input, 0, 1024 * 52 / u64::T], array_mut_ref![output, 0, 1024]), - 53 => unpack_64_53(array_ref![input, 0, 1024 * 53 / u64::T], array_mut_ref![output, 0, 1024]), - 54 => unpack_64_54(array_ref![input, 0, 1024 * 54 / u64::T], array_mut_ref![output, 0, 1024]), - 55 => unpack_64_55(array_ref![input, 0, 1024 * 55 / u64::T], array_mut_ref![output, 0, 1024]), - 56 => unpack_64_56(array_ref![input, 0, 1024 * 56 / u64::T], array_mut_ref![output, 0, 1024]), - 57 => unpack_64_57(array_ref![input, 0, 1024 * 57 / u64::T], array_mut_ref![output, 0, 1024]), - 58 => unpack_64_58(array_ref![input, 0, 1024 * 58 / u64::T], array_mut_ref![output, 0, 1024]), - 59 => unpack_64_59(array_ref![input, 0, 1024 * 59 / u64::T], array_mut_ref![output, 0, 1024]), - - 60 => unpack_64_60(array_ref![input, 0, 1024 * 60 / u64::T], array_mut_ref![output, 0, 1024]), - 61 => unpack_64_61(array_ref![input, 0, 1024 * 61 / u64::T], array_mut_ref![output, 0, 1024]), - 62 => unpack_64_62(array_ref![input, 0, 1024 * 62 / u64::T], array_mut_ref![output, 0, 1024]), - 63 => unpack_64_63(array_ref![input, 0, 1024 * 63 / u64::T], array_mut_ref![output, 0, 1024]), - 64 => unpack_64_64(array_ref![input, 0, 1024 * 64 / u64::T], array_mut_ref![output, 0, 1024]), - - _ => unreachable!("Unsupported width: {}", width) + 1 => unpack_64_1( + array_ref![input, 0, 1024 / 64], + array_mut_ref![output, 0, 1024], + ), + 2 => unpack_64_2( + array_ref![input, 0, 1024 * 2 / 64], + array_mut_ref![output, 0, 1024], + ), + 3 => unpack_64_3( + array_ref![input, 0, 1024 * 3 / 64], + array_mut_ref![output, 0, 1024], + ), + 4 => unpack_64_4( + array_ref![input, 0, 1024 * 4 / 64], + array_mut_ref![output, 0, 1024], + ), + 5 => unpack_64_5( + array_ref![input, 0, 1024 * 5 / 64], + array_mut_ref![output, 0, 1024], + ), + 6 => unpack_64_6( + array_ref![input, 0, 1024 * 6 / 64], + array_mut_ref![output, 0, 1024], + ), + 7 => unpack_64_7( + array_ref![input, 0, 1024 * 7 / 64], + array_mut_ref![output, 0, 1024], + ), + 8 => unpack_64_8( + array_ref![input, 0, 1024 * 8 / 64], + array_mut_ref![output, 0, 1024], + ), + 9 => unpack_64_9( + array_ref![input, 0, 1024 * 9 / 64], + array_mut_ref![output, 0, 1024], + ), + + 10 => unpack_64_10( + array_ref![input, 0, 1024 * 10 / 64], + array_mut_ref![output, 0, 1024], + ), + 11 => unpack_64_11( + array_ref![input, 0, 1024 * 11 / 64], + array_mut_ref![output, 0, 1024], + ), + 12 => unpack_64_12( + array_ref![input, 0, 1024 * 12 / 64], + array_mut_ref![output, 0, 1024], + ), + 13 => unpack_64_13( + array_ref![input, 0, 1024 * 13 / 64], + array_mut_ref![output, 0, 1024], + ), + 14 => unpack_64_14( + array_ref![input, 0, 1024 * 14 / 64], + array_mut_ref![output, 0, 1024], + ), + 15 => unpack_64_15( + array_ref![input, 0, 1024 * 15 / 64], + array_mut_ref![output, 0, 1024], + ), + 16 => unpack_64_16( + array_ref![input, 0, 1024 * 16 / 64], + array_mut_ref![output, 0, 1024], + ), + 17 => unpack_64_17( + array_ref![input, 0, 1024 * 17 / 64], + array_mut_ref![output, 0, 1024], + ), + 18 => unpack_64_18( + array_ref![input, 0, 1024 * 18 / 64], + array_mut_ref![output, 0, 1024], + ), + 19 => unpack_64_19( + array_ref![input, 0, 1024 * 19 / 64], + array_mut_ref![output, 0, 1024], + ), + + 20 => unpack_64_20( + array_ref![input, 0, 1024 * 20 / 64], + array_mut_ref![output, 0, 1024], + ), + 21 => unpack_64_21( + array_ref![input, 0, 1024 * 21 / 64], + array_mut_ref![output, 0, 1024], + ), + 22 => unpack_64_22( + array_ref![input, 0, 1024 * 22 / 64], + array_mut_ref![output, 0, 1024], + ), + 23 => unpack_64_23( + array_ref![input, 0, 1024 * 23 / 64], + array_mut_ref![output, 0, 1024], + ), + 24 => unpack_64_24( + array_ref![input, 0, 1024 * 24 / 64], + array_mut_ref![output, 0, 1024], + ), + 25 => unpack_64_25( + array_ref![input, 0, 1024 * 25 / 64], + array_mut_ref![output, 0, 1024], + ), + 26 => unpack_64_26( + array_ref![input, 0, 1024 * 26 / 64], + array_mut_ref![output, 0, 1024], + ), + 27 => unpack_64_27( + array_ref![input, 0, 1024 * 27 / 64], + array_mut_ref![output, 0, 1024], + ), + 28 => unpack_64_28( + array_ref![input, 0, 1024 * 28 / 64], + array_mut_ref![output, 0, 1024], + ), + 29 => unpack_64_29( + array_ref![input, 0, 1024 * 29 / 64], + array_mut_ref![output, 0, 1024], + ), + + 30 => unpack_64_30( + array_ref![input, 0, 1024 * 30 / 64], + array_mut_ref![output, 0, 1024], + ), + 31 => unpack_64_31( + array_ref![input, 0, 1024 * 31 / 64], + array_mut_ref![output, 0, 1024], + ), + 32 => unpack_64_32( + array_ref![input, 0, 1024 * 32 / 64], + array_mut_ref![output, 0, 1024], + ), + 33 => unpack_64_33( + array_ref![input, 0, 1024 * 33 / 64], + array_mut_ref![output, 0, 1024], + ), + 34 => unpack_64_34( + array_ref![input, 0, 1024 * 34 / 64], + array_mut_ref![output, 0, 1024], + ), + 35 => unpack_64_35( + array_ref![input, 0, 1024 * 35 / 64], + array_mut_ref![output, 0, 1024], + ), + 36 => unpack_64_36( + array_ref![input, 0, 1024 * 36 / 64], + array_mut_ref![output, 0, 1024], + ), + 37 => unpack_64_37( + array_ref![input, 0, 1024 * 37 / 64], + array_mut_ref![output, 0, 1024], + ), + 38 => unpack_64_38( + array_ref![input, 0, 1024 * 38 / 64], + array_mut_ref![output, 0, 1024], + ), + 39 => unpack_64_39( + array_ref![input, 0, 1024 * 39 / 64], + array_mut_ref![output, 0, 1024], + ), + + 40 => unpack_64_40( + array_ref![input, 0, 1024 * 40 / 64], + array_mut_ref![output, 0, 1024], + ), + 41 => unpack_64_41( + array_ref![input, 0, 1024 * 41 / 64], + array_mut_ref![output, 0, 1024], + ), + 42 => unpack_64_42( + array_ref![input, 0, 1024 * 42 / 64], + array_mut_ref![output, 0, 1024], + ), + 43 => unpack_64_43( + array_ref![input, 0, 1024 * 43 / 64], + array_mut_ref![output, 0, 1024], + ), + 44 => unpack_64_44( + array_ref![input, 0, 1024 * 44 / 64], + array_mut_ref![output, 0, 1024], + ), + 45 => unpack_64_45( + array_ref![input, 0, 1024 * 45 / 64], + array_mut_ref![output, 0, 1024], + ), + 46 => unpack_64_46( + array_ref![input, 0, 1024 * 46 / 64], + array_mut_ref![output, 0, 1024], + ), + 47 => unpack_64_47( + array_ref![input, 0, 1024 * 47 / 64], + array_mut_ref![output, 0, 1024], + ), + 48 => unpack_64_48( + array_ref![input, 0, 1024 * 48 / 64], + array_mut_ref![output, 0, 1024], + ), + 49 => unpack_64_49( + array_ref![input, 0, 1024 * 49 / 64], + array_mut_ref![output, 0, 1024], + ), + + 50 => unpack_64_50( + array_ref![input, 0, 1024 * 50 / 64], + array_mut_ref![output, 0, 1024], + ), + 51 => unpack_64_51( + array_ref![input, 0, 1024 * 51 / 64], + array_mut_ref![output, 0, 1024], + ), + 52 => unpack_64_52( + array_ref![input, 0, 1024 * 52 / 64], + array_mut_ref![output, 0, 1024], + ), + 53 => unpack_64_53( + array_ref![input, 0, 1024 * 53 / 64], + array_mut_ref![output, 0, 1024], + ), + 54 => unpack_64_54( + array_ref![input, 0, 1024 * 54 / 64], + array_mut_ref![output, 0, 1024], + ), + 55 => unpack_64_55( + array_ref![input, 0, 1024 * 55 / 64], + array_mut_ref![output, 0, 1024], + ), + 56 => unpack_64_56( + array_ref![input, 0, 1024 * 56 / 64], + array_mut_ref![output, 0, 1024], + ), + 57 => unpack_64_57( + array_ref![input, 0, 1024 * 57 / 64], + array_mut_ref![output, 0, 1024], + ), + 58 => unpack_64_58( + array_ref![input, 0, 1024 * 58 / 64], + array_mut_ref![output, 0, 1024], + ), + 59 => unpack_64_59( + array_ref![input, 0, 1024 * 59 / 64], + array_mut_ref![output, 0, 1024], + ), + + 60 => unpack_64_60( + array_ref![input, 0, 1024 * 60 / 64], + array_mut_ref![output, 0, 1024], + ), + 61 => unpack_64_61( + array_ref![input, 0, 1024 * 61 / 64], + array_mut_ref![output, 0, 1024], + ), + 62 => unpack_64_62( + array_ref![input, 0, 1024 * 62 / 64], + array_mut_ref![output, 0, 1024], + ), + 63 => unpack_64_63( + array_ref![input, 0, 1024 * 63 / 64], + array_mut_ref![output, 0, 1024], + ), + 64 => unpack_64_64( + array_ref![input, 0, 1024 * 64 / 64], + array_mut_ref![output, 0, 1024], + ), + + _ => unreachable!("Unsupported width: {}", width), } } } @@ -603,9 +1387,7 @@ macro_rules! pack_8 { ($name:ident, $bits:expr) => { fn $name(input: &[u8; 1024], output: &mut [u8; 1024 * $bits / u8::T]) { for lane in 0..u8::LANES { - pack!(u8, $bits, output, lane, |$idx| { - input[$idx] - }); + pack!(u8, $bits, output, lane, |$idx| { input[$idx] }); } } }; @@ -652,9 +1434,7 @@ macro_rules! pack_16 { ($name:ident, $bits:expr) => { fn $name(input: &[u16; 1024], output: &mut [u16; 1024 * $bits / u16::T]) { for lane in 0..u16::LANES { - pack!(u16, $bits, output, lane, |$idx| { - input[$idx] - }); + pack!(u16, $bits, output, lane, |$idx| { input[$idx] }); } } }; @@ -726,9 +1506,7 @@ macro_rules! pack_32 { ($name:ident, $bits:expr) => { fn $name(input: &[u32; 1024], output: &mut [u32; 1024 * $bits / u32::BITS as usize]) { for lane in 0..u32::LANES { - pack!(u32, $bits, output, lane, |$idx| { - input[$idx] - }); + pack!(u32, $bits, output, lane, |$idx| { input[$idx] }); } } }; @@ -767,62 +1545,169 @@ pack_32!(pack_32_30, 30); pack_32!(pack_32_31, 31); pack_32!(pack_32_32, 32); -macro_rules! generate_unpack_64 { - ($($n:expr),*) => { - $( - paste::item! { - fn [](input: &[u64; 1024 * $n / u64::T], output: &mut [u64; 1024]) { - for lane in 0..u64::LANES { - unpack!(u64, $n, input, lane, |$idx, $elem| { - output[$idx] = $elem - }); - } - } +macro_rules! unpack_64 { + ($name:ident, $bit_width:expr) => { + fn $name(input: &[u64; 1024 * $bit_width / u64::T], output: &mut [u64; 1024]) { + for lane in 0..u64::LANES { + unpack!(u64, $bit_width, input, lane, |$idx, $elem| { + output[$idx] = $elem + }); } - )* + } }; } -generate_unpack_64!( - 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, - 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, - 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, - 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64 -); - -macro_rules! generate_pack_64 { - ($($n:expr),*) => { - $( - paste::item! { - fn [](input: &[u64; 1024], output: &mut [u64; 1024 * $n / u64::T]) { - for lane in 0..u64::LANES { - pack!(u64, $n, output, lane, |$idx| { - input[$idx] - }); - } - } +unpack_64!(unpack_64_1, 1); +unpack_64!(unpack_64_2, 2); +unpack_64!(unpack_64_3, 3); +unpack_64!(unpack_64_4, 4); +unpack_64!(unpack_64_5, 5); +unpack_64!(unpack_64_6, 6); +unpack_64!(unpack_64_7, 7); +unpack_64!(unpack_64_8, 8); +unpack_64!(unpack_64_9, 9); +unpack_64!(unpack_64_10, 10); +unpack_64!(unpack_64_11, 11); +unpack_64!(unpack_64_12, 12); +unpack_64!(unpack_64_13, 13); +unpack_64!(unpack_64_14, 14); +unpack_64!(unpack_64_15, 15); +unpack_64!(unpack_64_16, 16); +unpack_64!(unpack_64_17, 17); +unpack_64!(unpack_64_18, 18); +unpack_64!(unpack_64_19, 19); +unpack_64!(unpack_64_20, 20); +unpack_64!(unpack_64_21, 21); +unpack_64!(unpack_64_22, 22); +unpack_64!(unpack_64_23, 23); +unpack_64!(unpack_64_24, 24); +unpack_64!(unpack_64_25, 25); +unpack_64!(unpack_64_26, 26); +unpack_64!(unpack_64_27, 27); +unpack_64!(unpack_64_28, 28); +unpack_64!(unpack_64_29, 29); +unpack_64!(unpack_64_30, 30); +unpack_64!(unpack_64_31, 31); +unpack_64!(unpack_64_32, 32); + +unpack_64!(unpack_64_33, 33); +unpack_64!(unpack_64_34, 34); +unpack_64!(unpack_64_35, 35); +unpack_64!(unpack_64_36, 36); +unpack_64!(unpack_64_37, 37); +unpack_64!(unpack_64_38, 38); +unpack_64!(unpack_64_39, 39); +unpack_64!(unpack_64_40, 40); +unpack_64!(unpack_64_41, 41); +unpack_64!(unpack_64_42, 42); +unpack_64!(unpack_64_43, 43); +unpack_64!(unpack_64_44, 44); +unpack_64!(unpack_64_45, 45); +unpack_64!(unpack_64_46, 46); +unpack_64!(unpack_64_47, 47); +unpack_64!(unpack_64_48, 48); +unpack_64!(unpack_64_49, 49); +unpack_64!(unpack_64_50, 50); +unpack_64!(unpack_64_51, 51); +unpack_64!(unpack_64_52, 52); +unpack_64!(unpack_64_53, 53); +unpack_64!(unpack_64_54, 54); +unpack_64!(unpack_64_55, 55); +unpack_64!(unpack_64_56, 56); +unpack_64!(unpack_64_57, 57); +unpack_64!(unpack_64_58, 58); +unpack_64!(unpack_64_59, 59); +unpack_64!(unpack_64_60, 60); +unpack_64!(unpack_64_61, 61); +unpack_64!(unpack_64_62, 62); +unpack_64!(unpack_64_63, 63); +unpack_64!(unpack_64_64, 64); + +macro_rules! pack_64 { + ($name:ident, $bits:expr) => { + fn $name(input: &[u64; 1024], output: &mut [u64; 1024 * $bits / u64::BITS as usize]) { + for lane in 0..u64::LANES { + pack!(u64, $bits, output, lane, |$idx| { input[$idx] }); } - )* + } }; } -generate_pack_64!( - 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, - 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, - 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, - 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64 -); - +pack_64!(pack_64_1, 1); +pack_64!(pack_64_2, 2); +pack_64!(pack_64_3, 3); +pack_64!(pack_64_4, 4); +pack_64!(pack_64_5, 5); +pack_64!(pack_64_6, 6); +pack_64!(pack_64_7, 7); +pack_64!(pack_64_8, 8); +pack_64!(pack_64_9, 9); +pack_64!(pack_64_10, 10); +pack_64!(pack_64_11, 11); +pack_64!(pack_64_12, 12); +pack_64!(pack_64_13, 13); +pack_64!(pack_64_14, 14); +pack_64!(pack_64_15, 15); +pack_64!(pack_64_16, 16); +pack_64!(pack_64_17, 17); +pack_64!(pack_64_18, 18); +pack_64!(pack_64_19, 19); +pack_64!(pack_64_20, 20); +pack_64!(pack_64_21, 21); +pack_64!(pack_64_22, 22); +pack_64!(pack_64_23, 23); +pack_64!(pack_64_24, 24); +pack_64!(pack_64_25, 25); +pack_64!(pack_64_26, 26); +pack_64!(pack_64_27, 27); +pack_64!(pack_64_28, 28); +pack_64!(pack_64_29, 29); +pack_64!(pack_64_30, 30); +pack_64!(pack_64_31, 31); +pack_64!(pack_64_32, 32); + +pack_64!(pack_64_33, 33); +pack_64!(pack_64_34, 34); +pack_64!(pack_64_35, 35); +pack_64!(pack_64_36, 36); +pack_64!(pack_64_37, 37); +pack_64!(pack_64_38, 38); +pack_64!(pack_64_39, 39); +pack_64!(pack_64_40, 40); +pack_64!(pack_64_41, 41); +pack_64!(pack_64_42, 42); +pack_64!(pack_64_43, 43); +pack_64!(pack_64_44, 44); +pack_64!(pack_64_45, 45); +pack_64!(pack_64_46, 46); +pack_64!(pack_64_47, 47); +pack_64!(pack_64_48, 48); +pack_64!(pack_64_49, 49); +pack_64!(pack_64_50, 50); +pack_64!(pack_64_51, 51); +pack_64!(pack_64_52, 52); +pack_64!(pack_64_53, 53); +pack_64!(pack_64_54, 54); +pack_64!(pack_64_55, 55); +pack_64!(pack_64_56, 56); +pack_64!(pack_64_57, 57); +pack_64!(pack_64_58, 58); +pack_64!(pack_64_59, 59); +pack_64!(pack_64_60, 60); +pack_64!(pack_64_61, 61); +pack_64!(pack_64_62, 62); +pack_64!(pack_64_63, 63); +pack_64!(pack_64_64, 64); #[cfg(test)] mod test { - use core::array; use super::*; + use core::array; #[test] fn test_pack() { let mut values: [u16; 1024] = [0; 1024]; - for i in 0..1024 { - values[i] = (i % (1 << 15)) as u16; + for (i, value) in values.iter_mut().enumerate() { + *value = (i % (1 << 15)) as u16; } let mut packed: [u16; 960] = [0; 960]; @@ -835,7 +1720,6 @@ mod test { let mut packed_orig: [u16; 960] = [0; 960]; unsafe { - BitPacking::unchecked_pack(15, &values, &mut packed_orig); } diff --git a/rust/lance-encoding/src/compression_algo/mod.rs b/rust/lance-encoding/src/compression_algo/mod.rs index 2cddf46de5..4b133f003e 100644 --- a/rust/lance-encoding/src/compression_algo/mod.rs +++ b/rust/lance-encoding/src/compression_algo/mod.rs @@ -1,4 +1,4 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright The Lance Authors -pub mod fastlanes; \ No newline at end of file +pub mod fastlanes; diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index 11438b9f64..08d4fac3dc 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -17,13 +17,13 @@ use lance_arrow::DataTypeExt; use lance_core::{Error, Result}; use crate::buffer::LanceBuffer; +use crate::compression_algo::fastlanes::BitPacking; use crate::data::{DataBlock, FixedWidthDataBlock}; use crate::decoder::{PageScheduler, PrimitivePageDecoder}; use crate::encoder::{ArrayEncoder, EncodedArray}; use crate::format::ProtobufUtils; use arrow::array::ArrayRef; use bytemuck::cast_slice; -use crate::compression_algo::fastlanes::BitPacking; const ELEMS_PER_CHUNK: u64 = 1024; // Compute the compressed_bit_width for a given array of integers diff --git a/rust/lance-encoding/src/lib.rs b/rust/lance-encoding/src/lib.rs index 4bfa9193ae..7d0d45bbc8 100644 --- a/rust/lance-encoding/src/lib.rs +++ b/rust/lance-encoding/src/lib.rs @@ -9,12 +9,12 @@ use futures::{future::BoxFuture, FutureExt, TryFutureExt}; use lance_core::Result; pub mod buffer; +pub mod compression_algo; pub mod data; pub mod decoder; pub mod encoder; pub mod encodings; pub mod format; -pub mod compression_algo; #[cfg(test)] pub mod testing; pub mod version; From ee78fc62451b19315e0cb43f4ebfaa85ba959557 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Sat, 21 Sep 2024 02:48:34 +0000 Subject: [PATCH 18/31] delete a unnecessary file rust/lance-encoding/compression-algo/mod.rs --- rust/lance-encoding/compression-algo/mod.rs | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 rust/lance-encoding/compression-algo/mod.rs diff --git a/rust/lance-encoding/compression-algo/mod.rs b/rust/lance-encoding/compression-algo/mod.rs deleted file mode 100644 index e69de29bb2..0000000000 From fe3fda84fcbabead789813dd898b990389cf73d4 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Sat, 21 Sep 2024 16:05:06 +0000 Subject: [PATCH 19/31] delete two redundant file --- .../compression-algo/fastlanes.rs | 865 ------------------ rust/lance-encoding/compression-algo/lib.rs | 3 - 2 files changed, 868 deletions(-) delete mode 100644 rust/lance-encoding/compression-algo/fastlanes.rs delete mode 100644 rust/lance-encoding/compression-algo/lib.rs diff --git a/rust/lance-encoding/compression-algo/fastlanes.rs b/rust/lance-encoding/compression-algo/fastlanes.rs deleted file mode 100644 index a514fadbe6..0000000000 --- a/rust/lance-encoding/compression-algo/fastlanes.rs +++ /dev/null @@ -1,865 +0,0 @@ -// SPDX-License-Identifier: Apache-2.0 -// SPDX-FileCopyrightText: Copyright The Lance Authors - -// This file is a modification of the `fastlanes` crate: https://github.com/spiraldb/fastlanes -// It is modified to allow a rust stable build - -use arrayref::{array_mut_ref, array_ref}; -use core::mem::size_of; -use paste::paste; - -pub const FL_ORDER: [usize; 8] = [0, 4, 2, 6, 1, 5, 3, 7]; - - -pub trait FastLanes: Sized + Copy { - const T: usize = size_of::() * 8; - const LANES: usize = 1024 / Self::T; -} - -// Implement the trait for basic unsigned integer types -impl FastLanes for u8 {} -impl FastLanes for u16 {} -impl FastLanes for u32 {} -impl FastLanes for u64 {} - -#[macro_export] -macro_rules! iterate { - ($T:ty, $lane: expr, | $_1:tt $idx:ident | $($body:tt)*) => { - macro_rules! __kernel__ {( $_1 $idx:ident ) => ( $($body)* )} - { - use $crate::{seq_t, FL_ORDER}; - use paste::paste; - - #[inline(always)] - fn index(row: usize, lane: usize) -> usize { - let o = row / 8; - let s = row % 8; - (FL_ORDER[o] * 16) + (s * 128) + lane - } - - paste!(seq_t!(row in $T { - let idx = index(row, $lane); - __kernel__!(idx); - })); - } - } -} - -#[macro_export] -macro_rules! pack { - ($T:ty, $W:expr, $packed:expr, $lane:expr, | $_1:tt $idx:ident | $($body:tt)*) => { - macro_rules! __kernel__ {( $_1 $idx:ident ) => ( $($body)* )} - { - use $crate::{seq_t, FL_ORDER}; - use paste::paste; - - // The number of bits of T. - const T: usize = <$T>::T; - - #[inline(always)] - fn index(row: usize, lane: usize) -> usize { - let o = row / 8; - let s = row % 8; - (FL_ORDER[o] * 16) + (s * 128) + lane - } - - if $W == 0 { - // Nothing to do if W is 0, since the packed array is zero bytes. - } else if $W == T { - // Special case for W=T, we can just copy the input value directly to the packed value. - paste!(seq_t!(row in $T { - let idx = index(row, $lane); - $packed[<$T>::LANES * row + $lane] = __kernel__!(idx); - })); - } else { - // A mask of W bits. - let mask: $T = (1 << $W) - 1; - - // First we loop over each lane in the virtual 1024 bit word. - let mut tmp: $T = 0; - - // Loop over each of the rows of the lane. - // Inlining this loop means all branches are known at compile time and - // the code is auto-vectorized for SIMD execution. - paste!(seq_t!(row in $T { - let idx = index(row, $lane); - let src = __kernel__!(idx); - let src = src & mask; - - // Shift the src bits into their position in the tmp output variable. - if row == 0 { - tmp = src; - } else { - tmp |= src << (row * $W) % T; - } - - // If the next packed position is after our current one, then we have filled - // the current output and we can write the packed value. - let curr_word: usize = (row * $W) / T; - let next_word: usize = ((row + 1) * $W) / T; - - #[allow(unused_assignments)] - if next_word > curr_word { - $packed[<$T>::LANES * curr_word + $lane] = tmp; - let remaining_bits: usize = ((row + 1) * $W) % T; - // Keep the remaining bits for the next packed value. - tmp = src >> $W - remaining_bits; - } - })); - } - } - }; -} - -#[macro_export] -macro_rules! unpack { - ($T:ty, $W:expr, $packed:expr, $lane:expr, | $_1:tt $idx:ident, $_2:tt $elem:ident | $($body:tt)*) => { - macro_rules! __kernel__ {( $_1 $idx:ident, $_2 $elem:ident ) => ( $($body)* )} - { - use $crate::{seq_t, FL_ORDER}; - use paste::paste; - - // The number of bits of T. - const T: usize = <$T>::T; - - #[inline(always)] - fn index(row: usize, lane: usize) -> usize { - let o = row / 8; - let s = row % 8; - (FL_ORDER[o] * 16) + (s * 128) + lane - } - - if $W == 0 { - // Special case for W=0, we just need to zero the output. - // We'll still respect the iteration order in case the kernel has side effects. - paste!(seq_t!(row in $T { - let idx = index(row, $lane); - let zero: $T = 0; - __kernel__!(idx, zero); - })); - } else if $W == T { - // Special case for W=T, we can just copy the packed value directly to the output. - paste!(seq_t!(row in $T { - let idx = index(row, $lane); - let src = $packed[<$T>::LANES * row + $lane]; - __kernel__!(idx, src); - })); - } else { - #[inline] - fn mask(width: usize) -> $T { - if width == T { <$T>::MAX } else { (1 << (width % T)) - 1 } - } - - let mut src: $T = $packed[$lane]; - let mut tmp: $T; - - paste!(seq_t!(row in $T { - // Figure out the packed positions - let curr_word: usize = (row * $W) / T; - let next_word = ((row + 1) * $W) / T; - - let shift = (row * $W) % T; - - if next_word > curr_word { - // Consume some bits from the curr packed input, the remainder are in the next - // packed input value - let remaining_bits = ((row + 1) * $W) % T; - let current_bits = $W - remaining_bits; - tmp = (src >> shift) & mask(current_bits); - - if next_word < $W { - // Load the next packed value - src = $packed[<$T>::LANES * next_word + $lane]; - // Consume the remaining bits from the next input value. - tmp |= (src & mask(remaining_bits)) << current_bits; - } - } else { - // Otherwise, just grab W bits from the src value - tmp = (src >> shift) & mask($W); - } - - // Write out the unpacked value - let idx = index(row, $lane); - __kernel__!(idx, tmp); - })); - } - } - }; -} - -// Macro for repeating a code block bit_size_of:: times. -#[macro_export] -macro_rules! seq_t { - ($ident:ident in u8 $body:tt) => {seq_macro::seq!($ident in 0..8 $body)}; - ($ident:ident in u16 $body:tt) => {seq_macro::seq!($ident in 0..16 $body)}; - ($ident:ident in u32 $body:tt) => {seq_macro::seq!($ident in 0..32 $body)}; - ($ident:ident in u64 $body:tt) => {seq_macro::seq!($ident in 0..64 $body)}; -} - -/// `BitPack` into a compile-time known bit-width. -pub trait BitPacking: FastLanes { - /// Packs 1024 elements into `W` bits each, where `W` is runtime-known instead of - /// compile-time known. - /// - /// # Safety - /// The input slice must be of exactly length 1024. The output slice must be of length - /// `1024 * W / T`, where `T` is the bit-width of Self and `W` is the packed width. - /// These lengths are checked only with `debug_assert` (i.e., not checked on release builds). - pub unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]); - - /// Unpacks 1024 elements from `W` bits each, where `W` is runtime-known instead of - /// compile-time known. - /// - /// # Safety - /// The input slice must be of length `1024 * W / T`, where `T` is the bit-width of Self and `W` - /// is the packed width. The output slice must be of exactly length 1024. - /// These lengths are checked only with `debug_assert` (i.e., not checked on release builds). - pub unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]); -} - -impl BitPacking for u8 { - unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { - let packed_len = 128 * width / size_of::(); - debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); - debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); - - match width { - 1 => pack_8_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u8::T]), - 2 => pack_8_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u8::T]), - 3 => pack_8_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u8::T]), - 4 => pack_8_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u8::T]), - 5 => pack_8_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u8::T]), - 6 => pack_8_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u8::T]), - 7 => pack_8_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u8::T]), - 8 => pack_8_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u8::T]), - - _ => unreachable!("Unsupported width: {}", width) - } - } - - unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { - let packed_len = 128 * width / size_of::(); - debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); - debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); - - match width { - 1 => unpack_8_1(array_ref![input, 0, 1024 * 1 / u8::T], array_mut_ref![output, 0, 1024]), - 2 => unpack_8_2(array_ref![input, 0, 1024 * 2 / u8::T], array_mut_ref![output, 0, 1024]), - 3 => unpack_8_3(array_ref![input, 0, 1024 * 3 / u8::T], array_mut_ref![output, 0, 1024]), - 4 => unpack_8_4(array_ref![input, 0, 1024 * 4 / u8::T], array_mut_ref![output, 0, 1024]), - 5 => unpack_8_5(array_ref![input, 0, 1024 * 5 / u8::T], array_mut_ref![output, 0, 1024]), - 6 => unpack_8_6(array_ref![input, 0, 1024 * 6 / u8::T], array_mut_ref![output, 0, 1024]), - 7 => unpack_8_7(array_ref![input, 0, 1024 * 7 / u8::T], array_mut_ref![output, 0, 1024]), - 8 => unpack_8_8(array_ref![input, 0, 1024 * 8 / u8::T], array_mut_ref![output, 0, 1024]), - - _ => unreachable!("Unsupported width: {}", width) - } - } -} - -impl BitPacking for u16 { - unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { - let packed_len = 128 * width / size_of::(); - debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); - debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); - - match width { - 1 => pack_16_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u16::T]), - 2 => pack_16_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u16::T]), - 3 => pack_16_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u16::T]), - 4 => pack_16_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u16::T]), - 5 => pack_16_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u16::T]), - 6 => pack_16_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u16::T]), - 7 => pack_16_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u16::T]), - 8 => pack_16_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u16::T]), - 9 => pack_16_9(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 9 / u16::T]), - - 10 => pack_16_10(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 10 / u16::T]), - 11 => pack_16_11(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 11 / u16::T]), - 12 => pack_16_12(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 12 / u16::T]), - 13 => pack_16_13(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 13 / u16::T]), - 14 => pack_16_14(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 14 / u16::T]), - 15 => pack_16_15(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 15 / u16::T]), - 16 => pack_16_16(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 16 / u16::T]), - - _ => unreachable!("Unsupported width: {}", width) - } - } - - unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { - let packed_len = 128 * width / size_of::(); - debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); - debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); - - match width { - 1 => unpack_16_1(array_ref![input, 0, 1024 * 1 / u16::T], array_mut_ref![output, 0, 1024]), - 2 => unpack_16_2(array_ref![input, 0, 1024 * 2 / u16::T], array_mut_ref![output, 0, 1024]), - 3 => unpack_16_3(array_ref![input, 0, 1024 * 3 / u16::T], array_mut_ref![output, 0, 1024]), - 4 => unpack_16_4(array_ref![input, 0, 1024 * 4 / u16::T], array_mut_ref![output, 0, 1024]), - 5 => unpack_16_5(array_ref![input, 0, 1024 * 5 / u16::T], array_mut_ref![output, 0, 1024]), - 6 => unpack_16_6(array_ref![input, 0, 1024 * 6 / u16::T], array_mut_ref![output, 0, 1024]), - 7 => unpack_16_7(array_ref![input, 0, 1024 * 7 / u16::T], array_mut_ref![output, 0, 1024]), - 8 => unpack_16_8(array_ref![input, 0, 1024 * 8 / u16::T], array_mut_ref![output, 0, 1024]), - 9 => unpack_16_9(array_ref![input, 0, 1024 * 9 / u16::T], array_mut_ref![output, 0, 1024]), - - 10 => unpack_16_10(array_ref![input, 0, 1024 * 10 / u16::T], array_mut_ref![output, 0, 1024]), - 11 => unpack_16_11(array_ref![input, 0, 1024 * 11 / u16::T], array_mut_ref![output, 0, 1024]), - 12 => unpack_16_12(array_ref![input, 0, 1024 * 12 / u16::T], array_mut_ref![output, 0, 1024]), - 13 => unpack_16_13(array_ref![input, 0, 1024 * 13 / u16::T], array_mut_ref![output, 0, 1024]), - 14 => unpack_16_14(array_ref![input, 0, 1024 * 14 / u16::T], array_mut_ref![output, 0, 1024]), - 15 => unpack_16_15(array_ref![input, 0, 1024 * 15 / u16::T], array_mut_ref![output, 0, 1024]), - 16 => unpack_16_16(array_ref![input, 0, 1024 * 16 / u16::T], array_mut_ref![output, 0, 1024]), - - _ => unreachable!("Unsupported width: {}", width) - } - } -} - -impl BitPacking for u32 { - unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { - let packed_len = 128 * width / size_of::(); - debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); - debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); - - match width { - 1 => pack_32_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u32::T]), - 2 => pack_32_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u32::T]), - 3 => pack_32_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u32::T]), - 4 => pack_32_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u32::T]), - 5 => pack_32_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u32::T]), - 6 => pack_32_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u32::T]), - 7 => pack_32_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u32::T]), - 8 => pack_32_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u32::T]), - 9 => pack_32_9(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 9 / u32::T]), - - 10 => pack_32_10(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 10 / u32::T]), - 11 => pack_32_11(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 11 / u32::T]), - 12 => pack_32_12(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 12 / u32::T]), - 13 => pack_32_13(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 13 / u32::T]), - 14 => pack_32_14(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 14 / u32::T]), - 15 => pack_32_15(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 15 / u32::T]), - 16 => pack_32_16(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 16 / u32::T]), - 17 => pack_32_17(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 17 / u32::T]), - 18 => pack_32_18(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 18 / u32::T]), - 19 => pack_32_19(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 19 / u32::T]), - - 20 => pack_32_20(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 20 / u32::T]), - 21 => pack_32_21(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 21 / u32::T]), - 22 => pack_32_22(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 22 / u32::T]), - 23 => pack_32_23(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 23 / u32::T]), - 24 => pack_32_24(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 24 / u32::T]), - 25 => pack_32_25(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 25 / u32::T]), - 26 => pack_32_26(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 26 / u32::T]), - 27 => pack_32_27(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 27 / u32::T]), - 28 => pack_32_28(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 28 / u32::T]), - 29 => pack_32_29(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 29 / u32::T]), - - 30 => pack_32_30(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 30 / u32::T]), - 31 => pack_32_31(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 31 / u32::T]), - 32 => pack_32_32(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 32 / u32::T]), - - _ => unreachable!("Unsupported width: {}", width) - } - } - - unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { - let packed_len = 128 * width / size_of::(); - debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); - debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); - - match width { - 1 => unpack_32_1(array_ref![input, 0, 1024 * 1 / u32::T], array_mut_ref![output, 0, 1024]), - 2 => unpack_32_2(array_ref![input, 0, 1024 * 2 / u32::T], array_mut_ref![output, 0, 1024]), - 3 => unpack_32_3(array_ref![input, 0, 1024 * 3 / u32::T], array_mut_ref![output, 0, 1024]), - 4 => unpack_32_4(array_ref![input, 0, 1024 * 4 / u32::T], array_mut_ref![output, 0, 1024]), - 5 => unpack_32_5(array_ref![input, 0, 1024 * 5 / u32::T], array_mut_ref![output, 0, 1024]), - 6 => unpack_32_6(array_ref![input, 0, 1024 * 6 / u32::T], array_mut_ref![output, 0, 1024]), - 7 => unpack_32_7(array_ref![input, 0, 1024 * 7 / u32::T], array_mut_ref![output, 0, 1024]), - 8 => unpack_32_8(array_ref![input, 0, 1024 * 8 / u32::T], array_mut_ref![output, 0, 1024]), - 9 => unpack_32_9(array_ref![input, 0, 1024 * 9 / u32::T], array_mut_ref![output, 0, 1024]), - - 10 => unpack_32_10(array_ref![input, 0, 1024 * 10 / u32::T], array_mut_ref![output, 0, 1024]), - 11 => unpack_32_11(array_ref![input, 0, 1024 * 11 / u32::T], array_mut_ref![output, 0, 1024]), - 12 => unpack_32_12(array_ref![input, 0, 1024 * 12 / u32::T], array_mut_ref![output, 0, 1024]), - 13 => unpack_32_13(array_ref![input, 0, 1024 * 13 / u32::T], array_mut_ref![output, 0, 1024]), - 14 => unpack_32_14(array_ref![input, 0, 1024 * 14 / u32::T], array_mut_ref![output, 0, 1024]), - 15 => unpack_32_15(array_ref![input, 0, 1024 * 15 / u32::T], array_mut_ref![output, 0, 1024]), - 16 => unpack_32_16(array_ref![input, 0, 1024 * 16 / u32::T], array_mut_ref![output, 0, 1024]), - 17 => unpack_32_17(array_ref![input, 0, 1024 * 17 / u32::T], array_mut_ref![output, 0, 1024]), - 18 => unpack_32_18(array_ref![input, 0, 1024 * 18 / u32::T], array_mut_ref![output, 0, 1024]), - 19 => unpack_32_19(array_ref![input, 0, 1024 * 19 / u32::T], array_mut_ref![output, 0, 1024]), - - 20 => unpack_32_20(array_ref![input, 0, 1024 * 20 / u32::T], array_mut_ref![output, 0, 1024]), - 21 => unpack_32_21(array_ref![input, 0, 1024 * 21 / u32::T], array_mut_ref![output, 0, 1024]), - 22 => unpack_32_22(array_ref![input, 0, 1024 * 22 / u32::T], array_mut_ref![output, 0, 1024]), - 23 => unpack_32_23(array_ref![input, 0, 1024 * 23 / u32::T], array_mut_ref![output, 0, 1024]), - 24 => unpack_32_24(array_ref![input, 0, 1024 * 24 / u32::T], array_mut_ref![output, 0, 1024]), - 25 => unpack_32_25(array_ref![input, 0, 1024 * 25 / u32::T], array_mut_ref![output, 0, 1024]), - 26 => unpack_32_26(array_ref![input, 0, 1024 * 26 / u32::T], array_mut_ref![output, 0, 1024]), - 27 => unpack_32_27(array_ref![input, 0, 1024 * 27 / u32::T], array_mut_ref![output, 0, 1024]), - 28 => unpack_32_28(array_ref![input, 0, 1024 * 28 / u32::T], array_mut_ref![output, 0, 1024]), - 29 => unpack_32_29(array_ref![input, 0, 1024 * 29 / u32::T], array_mut_ref![output, 0, 1024]), - - 30 => unpack_32_30(array_ref![input, 0, 1024 * 30 / u32::T], array_mut_ref![output, 0, 1024]), - 31 => unpack_32_31(array_ref![input, 0, 1024 * 31 / u32::T], array_mut_ref![output, 0, 1024]), - 32 => unpack_32_32(array_ref![input, 0, 1024 * 32 / u32::T], array_mut_ref![output, 0, 1024]), - - _ => unreachable!("Unsupported width: {}", width) - } - } -} - -impl BitPacking for u64 { - unsafe fn unchecked_pack(width: usize, input: &[Self], output: &mut [Self]) { - let packed_len = 128 * width / size_of::(); - debug_assert_eq!(output.len(), packed_len, "Output buffer must be of size 1024 * W / T"); - debug_assert_eq!(input.len(), 1024, "Input buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); - - match width { - 1 => pack_64_1(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 1 / u64::T]), - 2 => pack_64_2(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 2 / u64::T]), - 3 => pack_64_3(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 3 / u64::T]), - 4 => pack_64_4(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 4 / u64::T]), - 5 => pack_64_5(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 5 / u64::T]), - 6 => pack_64_6(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 6 / u64::T]), - 7 => pack_64_7(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 7 / u64::T]), - 8 => pack_64_8(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 8 / u64::T]), - 9 => pack_64_9(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 9 / u64::T]), - - 10 => pack_64_10(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 10 / u64::T]), - 11 => pack_64_11(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 11 / u64::T]), - 12 => pack_64_12(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 12 / u64::T]), - 13 => pack_64_13(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 13 / u64::T]), - 14 => pack_64_14(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 14 / u64::T]), - 15 => pack_64_15(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 15 / u64::T]), - 16 => pack_64_16(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 16 / u64::T]), - 17 => pack_64_17(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 17 / u64::T]), - 18 => pack_64_18(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 18 / u64::T]), - 19 => pack_64_19(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 19 / u64::T]), - - 20 => pack_64_20(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 20 / u64::T]), - 21 => pack_64_21(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 21 / u64::T]), - 22 => pack_64_22(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 22 / u64::T]), - 23 => pack_64_23(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 23 / u64::T]), - 24 => pack_64_24(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 24 / u64::T]), - 25 => pack_64_25(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 25 / u64::T]), - 26 => pack_64_26(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 26 / u64::T]), - 27 => pack_64_27(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 27 / u64::T]), - 28 => pack_64_28(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 28 / u64::T]), - 29 => pack_64_29(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 29 / u64::T]), - - 30 => pack_64_30(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 30 / u64::T]), - 31 => pack_64_31(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 31 / u64::T]), - 32 => pack_64_32(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 32 / u64::T]), - 33 => pack_64_33(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 33 / u64::T]), - 34 => pack_64_34(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 34 / u64::T]), - 35 => pack_64_35(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 35 / u64::T]), - 36 => pack_64_36(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 36 / u64::T]), - 37 => pack_64_37(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 37 / u64::T]), - 38 => pack_64_38(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 38 / u64::T]), - 39 => pack_64_39(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 39 / u64::T]), - - 40 => pack_64_40(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 40 / u64::T]), - 41 => pack_64_41(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 41 / u64::T]), - 42 => pack_64_42(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 42 / u64::T]), - 43 => pack_64_43(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 43 / u64::T]), - 44 => pack_64_44(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 44 / u64::T]), - 45 => pack_64_45(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 45 / u64::T]), - 46 => pack_64_46(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 46 / u64::T]), - 47 => pack_64_47(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 47 / u64::T]), - 48 => pack_64_48(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 48 / u64::T]), - 49 => pack_64_49(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 49 / u64::T]), - - 50 => pack_64_50(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 50 / u64::T]), - 51 => pack_64_51(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 51 / u64::T]), - 52 => pack_64_52(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 52 / u64::T]), - 53 => pack_64_53(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 53 / u64::T]), - 54 => pack_64_54(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 54 / u64::T]), - 55 => pack_64_55(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 55 / u64::T]), - 56 => pack_64_56(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 56 / u64::T]), - 57 => pack_64_57(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 57 / u64::T]), - 58 => pack_64_58(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 58 / u64::T]), - 59 => pack_64_59(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 59 / u64::T]), - - 60 => pack_64_60(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 60 / u64::T]), - 61 => pack_64_61(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 61 / u64::T]), - 62 => pack_64_62(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 62 / u64::T]), - 63 => pack_64_63(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 63 / u64::T]), - 64 => pack_64_64(array_ref![input, 0, 1024], array_mut_ref![output, 0, 1024 * 64 / u64::T]), - - _ => unreachable!("Unsupported width: {}", width) - } - } - - unsafe fn unchecked_unpack(width: usize, input: &[Self], output: &mut [Self]) { - let packed_len = 128 * width / size_of::(); - debug_assert_eq!(input.len(), packed_len, "Input buffer must be of size 1024 * W / T"); - debug_assert_eq!(output.len(), 1024, "Output buffer must be of size 1024"); - debug_assert!(width <= Self::T, "Width must be less than or equal to {}", Self::T); - - match width { - 1 => unpack_64_1(array_ref![input, 0, 1024 * 1 / u64::T], array_mut_ref![output, 0, 1024]), - 2 => unpack_64_2(array_ref![input, 0, 1024 * 2 / u64::T], array_mut_ref![output, 0, 1024]), - 3 => unpack_64_3(array_ref![input, 0, 1024 * 3 / u64::T], array_mut_ref![output, 0, 1024]), - 4 => unpack_64_4(array_ref![input, 0, 1024 * 4 / u64::T], array_mut_ref![output, 0, 1024]), - 5 => unpack_64_5(array_ref![input, 0, 1024 * 5 / u64::T], array_mut_ref![output, 0, 1024]), - 6 => unpack_64_6(array_ref![input, 0, 1024 * 6 / u64::T], array_mut_ref![output, 0, 1024]), - 7 => unpack_64_7(array_ref![input, 0, 1024 * 7 / u64::T], array_mut_ref![output, 0, 1024]), - 8 => unpack_64_8(array_ref![input, 0, 1024 * 8 / u64::T], array_mut_ref![output, 0, 1024]), - 9 => unpack_64_9(array_ref![input, 0, 1024 * 9 / u64::T], array_mut_ref![output, 0, 1024]), - - 10 => unpack_64_10(array_ref![input, 0, 1024 * 10 / u64::T], array_mut_ref![output, 0, 1024]), - 11 => unpack_64_11(array_ref![input, 0, 1024 * 11 / u64::T], array_mut_ref![output, 0, 1024]), - 12 => unpack_64_12(array_ref![input, 0, 1024 * 12 / u64::T], array_mut_ref![output, 0, 1024]), - 13 => unpack_64_13(array_ref![input, 0, 1024 * 13 / u64::T], array_mut_ref![output, 0, 1024]), - 14 => unpack_64_14(array_ref![input, 0, 1024 * 14 / u64::T], array_mut_ref![output, 0, 1024]), - 15 => unpack_64_15(array_ref![input, 0, 1024 * 15 / u64::T], array_mut_ref![output, 0, 1024]), - 16 => unpack_64_16(array_ref![input, 0, 1024 * 16 / u64::T], array_mut_ref![output, 0, 1024]), - 17 => unpack_64_17(array_ref![input, 0, 1024 * 17 / u64::T], array_mut_ref![output, 0, 1024]), - 18 => unpack_64_18(array_ref![input, 0, 1024 * 18 / u64::T], array_mut_ref![output, 0, 1024]), - 19 => unpack_64_19(array_ref![input, 0, 1024 * 19 / u64::T], array_mut_ref![output, 0, 1024]), - - 20 => unpack_64_20(array_ref![input, 0, 1024 * 20 / u64::T], array_mut_ref![output, 0, 1024]), - 21 => unpack_64_21(array_ref![input, 0, 1024 * 21 / u64::T], array_mut_ref![output, 0, 1024]), - 22 => unpack_64_22(array_ref![input, 0, 1024 * 22 / u64::T], array_mut_ref![output, 0, 1024]), - 23 => unpack_64_23(array_ref![input, 0, 1024 * 23 / u64::T], array_mut_ref![output, 0, 1024]), - 24 => unpack_64_24(array_ref![input, 0, 1024 * 24 / u64::T], array_mut_ref![output, 0, 1024]), - 25 => unpack_64_25(array_ref![input, 0, 1024 * 25 / u64::T], array_mut_ref![output, 0, 1024]), - 26 => unpack_64_26(array_ref![input, 0, 1024 * 26 / u64::T], array_mut_ref![output, 0, 1024]), - 27 => unpack_64_27(array_ref![input, 0, 1024 * 27 / u64::T], array_mut_ref![output, 0, 1024]), - 28 => unpack_64_28(array_ref![input, 0, 1024 * 28 / u64::T], array_mut_ref![output, 0, 1024]), - 29 => unpack_64_29(array_ref![input, 0, 1024 * 29 / u64::T], array_mut_ref![output, 0, 1024]), - - 30 => unpack_64_30(array_ref![input, 0, 1024 * 30 / u64::T], array_mut_ref![output, 0, 1024]), - 31 => unpack_64_31(array_ref![input, 0, 1024 * 31 / u64::T], array_mut_ref![output, 0, 1024]), - 32 => unpack_64_32(array_ref![input, 0, 1024 * 32 / u64::T], array_mut_ref![output, 0, 1024]), - 33 => unpack_64_33(array_ref![input, 0, 1024 * 33 / u64::T], array_mut_ref![output, 0, 1024]), - 34 => unpack_64_34(array_ref![input, 0, 1024 * 34 / u64::T], array_mut_ref![output, 0, 1024]), - 35 => unpack_64_35(array_ref![input, 0, 1024 * 35 / u64::T], array_mut_ref![output, 0, 1024]), - 36 => unpack_64_36(array_ref![input, 0, 1024 * 36 / u64::T], array_mut_ref![output, 0, 1024]), - 37 => unpack_64_37(array_ref![input, 0, 1024 * 37 / u64::T], array_mut_ref![output, 0, 1024]), - 38 => unpack_64_38(array_ref![input, 0, 1024 * 38 / u64::T], array_mut_ref![output, 0, 1024]), - 39 => unpack_64_39(array_ref![input, 0, 1024 * 39 / u64::T], array_mut_ref![output, 0, 1024]), - - 40 => unpack_64_40(array_ref![input, 0, 1024 * 40 / u64::T], array_mut_ref![output, 0, 1024]), - 41 => unpack_64_41(array_ref![input, 0, 1024 * 41 / u64::T], array_mut_ref![output, 0, 1024]), - 42 => unpack_64_42(array_ref![input, 0, 1024 * 42 / u64::T], array_mut_ref![output, 0, 1024]), - 43 => unpack_64_43(array_ref![input, 0, 1024 * 43 / u64::T], array_mut_ref![output, 0, 1024]), - 44 => unpack_64_44(array_ref![input, 0, 1024 * 44 / u64::T], array_mut_ref![output, 0, 1024]), - 45 => unpack_64_45(array_ref![input, 0, 1024 * 45 / u64::T], array_mut_ref![output, 0, 1024]), - 46 => unpack_64_46(array_ref![input, 0, 1024 * 46 / u64::T], array_mut_ref![output, 0, 1024]), - 47 => unpack_64_47(array_ref![input, 0, 1024 * 47 / u64::T], array_mut_ref![output, 0, 1024]), - 48 => unpack_64_48(array_ref![input, 0, 1024 * 48 / u64::T], array_mut_ref![output, 0, 1024]), - 49 => unpack_64_49(array_ref![input, 0, 1024 * 49 / u64::T], array_mut_ref![output, 0, 1024]), - - 50 => unpack_64_50(array_ref![input, 0, 1024 * 50 / u64::T], array_mut_ref![output, 0, 1024]), - 51 => unpack_64_51(array_ref![input, 0, 1024 * 51 / u64::T], array_mut_ref![output, 0, 1024]), - 52 => unpack_64_52(array_ref![input, 0, 1024 * 52 / u64::T], array_mut_ref![output, 0, 1024]), - 53 => unpack_64_53(array_ref![input, 0, 1024 * 53 / u64::T], array_mut_ref![output, 0, 1024]), - 54 => unpack_64_54(array_ref![input, 0, 1024 * 54 / u64::T], array_mut_ref![output, 0, 1024]), - 55 => unpack_64_55(array_ref![input, 0, 1024 * 55 / u64::T], array_mut_ref![output, 0, 1024]), - 56 => unpack_64_56(array_ref![input, 0, 1024 * 56 / u64::T], array_mut_ref![output, 0, 1024]), - 57 => unpack_64_57(array_ref![input, 0, 1024 * 57 / u64::T], array_mut_ref![output, 0, 1024]), - 58 => unpack_64_58(array_ref![input, 0, 1024 * 58 / u64::T], array_mut_ref![output, 0, 1024]), - 59 => unpack_64_59(array_ref![input, 0, 1024 * 59 / u64::T], array_mut_ref![output, 0, 1024]), - - 60 => unpack_64_60(array_ref![input, 0, 1024 * 60 / u64::T], array_mut_ref![output, 0, 1024]), - 61 => unpack_64_61(array_ref![input, 0, 1024 * 61 / u64::T], array_mut_ref![output, 0, 1024]), - 62 => unpack_64_62(array_ref![input, 0, 1024 * 62 / u64::T], array_mut_ref![output, 0, 1024]), - 63 => unpack_64_63(array_ref![input, 0, 1024 * 63 / u64::T], array_mut_ref![output, 0, 1024]), - 64 => unpack_64_64(array_ref![input, 0, 1024 * 64 / u64::T], array_mut_ref![output, 0, 1024]), - - _ => unreachable!("Unsupported width: {}", width) - } - } -} - -macro_rules! unpack_8 { - ($name:ident, $bits:expr) => { - fn $name(input: &[u8; 1024 * $bits / u8::T], output: &mut [u8; 1024]) { - for lane in 0..u8::LANES { - unpack!(u8, $bits, input, lane, |$idx, $elem| { - output[$idx] = $elem; - }); - } - } - }; -} - -unpack_8!(unpack_8_1, 1); -unpack_8!(unpack_8_2, 2); -unpack_8!(unpack_8_3, 3); -unpack_8!(unpack_8_4, 4); -unpack_8!(unpack_8_5, 5); -unpack_8!(unpack_8_6, 6); -unpack_8!(unpack_8_7, 7); -unpack_8!(unpack_8_8, 8); - -macro_rules! pack_8 { - ($name:ident, $bits:expr) => { - fn $name(input: &[u8; 1024], output: &mut [u8; 1024 * $bits / u8::T]) { - for lane in 0..u8::LANES { - pack!(u8, $bits, output, lane, |$idx| { - input[$idx] - }); - } - } - }; -} -pack_8!(pack_8_1, 1); -pack_8!(pack_8_2, 2); -pack_8!(pack_8_3, 3); -pack_8!(pack_8_4, 4); -pack_8!(pack_8_5, 5); -pack_8!(pack_8_6, 6); -pack_8!(pack_8_7, 7); -pack_8!(pack_8_8, 8); - -macro_rules! unpack_16 { - ($name:ident, $bits:expr) => { - fn $name(input: &[u16; 1024 * $bits / u16::T], output: &mut [u16; 1024]) { - for lane in 0..u16::LANES { - unpack!(u16, $bits, input, lane, |$idx, $elem| { - output[$idx] = $elem; - }); - } - } - }; -} - -unpack_16!(unpack_16_1, 1); -unpack_16!(unpack_16_2, 2); -unpack_16!(unpack_16_3, 3); -unpack_16!(unpack_16_4, 4); -unpack_16!(unpack_16_5, 5); -unpack_16!(unpack_16_6, 6); -unpack_16!(unpack_16_7, 7); -unpack_16!(unpack_16_8, 8); -unpack_16!(unpack_16_9, 9); -unpack_16!(unpack_16_10, 10); -unpack_16!(unpack_16_11, 11); -unpack_16!(unpack_16_12, 12); -unpack_16!(unpack_16_13, 13); -unpack_16!(unpack_16_14, 14); -unpack_16!(unpack_16_15, 15); -unpack_16!(unpack_16_16, 16); - -macro_rules! pack_16 { - ($name:ident, $bits:expr) => { - fn $name(input: &[u16; 1024], output: &mut [u16; 1024 * $bits / u16::T]) { - for lane in 0..u16::LANES { - pack!(u16, $bits, output, lane, |$idx| { - input[$idx] - }); - } - } - }; -} - -pack_16!(pack_16_1, 1); -pack_16!(pack_16_2, 2); -pack_16!(pack_16_3, 3); -pack_16!(pack_16_4, 4); -pack_16!(pack_16_5, 5); -pack_16!(pack_16_6, 6); -pack_16!(pack_16_7, 7); -pack_16!(pack_16_8, 8); -pack_16!(pack_16_9, 9); -pack_16!(pack_16_10, 10); -pack_16!(pack_16_11, 11); -pack_16!(pack_16_12, 12); -pack_16!(pack_16_13, 13); -pack_16!(pack_16_14, 14); -pack_16!(pack_16_15, 15); -pack_16!(pack_16_16, 16); - -macro_rules! unpack_32 { - ($name:ident, $bit_width:expr) => { - fn $name(input: &[u32; 1024 * $bit_width / u32::T], output: &mut [u32; 1024]) { - for lane in 0..u32::LANES { - unpack!(u32, $bit_width, input, lane, |$idx, $elem| { - output[$idx] = $elem - }); - } - } - }; -} - -unpack_32!(unpack_32_1, 1); -unpack_32!(unpack_32_2, 2); -unpack_32!(unpack_32_3, 3); -unpack_32!(unpack_32_4, 4); -unpack_32!(unpack_32_5, 5); -unpack_32!(unpack_32_6, 6); -unpack_32!(unpack_32_7, 7); -unpack_32!(unpack_32_8, 8); -unpack_32!(unpack_32_9, 9); -unpack_32!(unpack_32_10, 10); -unpack_32!(unpack_32_11, 11); -unpack_32!(unpack_32_12, 12); -unpack_32!(unpack_32_13, 13); -unpack_32!(unpack_32_14, 14); -unpack_32!(unpack_32_15, 15); -unpack_32!(unpack_32_16, 16); -unpack_32!(unpack_32_17, 17); -unpack_32!(unpack_32_18, 18); -unpack_32!(unpack_32_19, 19); -unpack_32!(unpack_32_20, 20); -unpack_32!(unpack_32_21, 21); -unpack_32!(unpack_32_22, 22); -unpack_32!(unpack_32_23, 23); -unpack_32!(unpack_32_24, 24); -unpack_32!(unpack_32_25, 25); -unpack_32!(unpack_32_26, 26); -unpack_32!(unpack_32_27, 27); -unpack_32!(unpack_32_28, 28); -unpack_32!(unpack_32_29, 29); -unpack_32!(unpack_32_30, 30); -unpack_32!(unpack_32_31, 31); -unpack_32!(unpack_32_32, 32); - -macro_rules! pack_32 { - ($name:ident, $bits:expr) => { - fn $name(input: &[u32; 1024], output: &mut [u32; 1024 * $bits / u32::BITS as usize]) { - for lane in 0..u32::LANES { - pack!(u32, $bits, output, lane, |$idx| { - input[$idx] - }); - } - } - }; -} - -pack_32!(pack_32_1, 1); -pack_32!(pack_32_2, 2); -pack_32!(pack_32_3, 3); -pack_32!(pack_32_4, 4); -pack_32!(pack_32_5, 5); -pack_32!(pack_32_6, 6); -pack_32!(pack_32_7, 7); -pack_32!(pack_32_8, 8); -pack_32!(pack_32_9, 9); -pack_32!(pack_32_10, 10); -pack_32!(pack_32_11, 11); -pack_32!(pack_32_12, 12); -pack_32!(pack_32_13, 13); -pack_32!(pack_32_14, 14); -pack_32!(pack_32_15, 15); -pack_32!(pack_32_16, 16); -pack_32!(pack_32_17, 17); -pack_32!(pack_32_18, 18); -pack_32!(pack_32_19, 19); -pack_32!(pack_32_20, 20); -pack_32!(pack_32_21, 21); -pack_32!(pack_32_22, 22); -pack_32!(pack_32_23, 23); -pack_32!(pack_32_24, 24); -pack_32!(pack_32_25, 25); -pack_32!(pack_32_26, 26); -pack_32!(pack_32_27, 27); -pack_32!(pack_32_28, 28); -pack_32!(pack_32_29, 29); -pack_32!(pack_32_30, 30); -pack_32!(pack_32_31, 31); -pack_32!(pack_32_32, 32); - -macro_rules! generate_unpack_64 { - ($($n:expr),*) => { - $( - paste::item! { - fn [](input: &[u64; 1024 * $n / u64::T], output: &mut [u64; 1024]) { - for lane in 0..u64::LANES { - unpack!(u64, $n, input, lane, |$idx, $elem| { - output[$idx] = $elem - }); - } - } - } - )* - }; -} - -generate_unpack_64!( - 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, - 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, - 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, - 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64 -); - -macro_rules! generate_pack_64 { - ($($n:expr),*) => { - $( - paste::item! { - fn [](input: &[u64; 1024], output: &mut [u64; 1024 * $n / u64::T]) { - for lane in 0..u64::LANES { - pack!(u64, $n, output, lane, |$idx| { - input[$idx] - }); - } - } - } - )* - }; -} - -generate_pack_64!( - 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, - 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, - 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, - 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64 -); - -#[cfg(test)] -mod test { - use core::array; - use super::*; - - #[test] - fn test_pack() { - let mut values: [u16; 1024] = [0; 1024]; - for i in 0..1024 { - values[i] = (i % (1 << 15)) as u16; - } - - let mut packed: [u16; 960] = [0; 960]; - for lane in 0..u16::LANES { - // Always loop over lanes first. This is what the compiler vectorizes. - pack!(u16, 15, packed, lane, |$pos| { - values[$pos] - }); - } - - let mut packed_orig: [u16; 960] = [0; 960]; - unsafe { - - BitPacking::unchecked_pack(15, &values, &mut packed_orig); - } - - let mut unpacked: [u16; 1024] = [0; 1024]; - for lane in 0..u16::LANES { - // Always loop over lanes first. This is what the compiler vectorizes. - unpack!(u16, 15, packed, lane, |$idx, $elem| { - unpacked[$idx] = $elem; - }); - } - - assert_eq!(values, unpacked); - } - - #[test] - fn test_unchecked_pack() { - let input = array::from_fn(|i| i as u32); - let mut packed = [0; 320]; - unsafe { BitPacking::unchecked_pack(10, &input, &mut packed) }; - let mut output = [0; 1024]; - unsafe { BitPacking::unchecked_unpack(10, &packed, &mut output) }; - assert_eq!(input, output); - } -} diff --git a/rust/lance-encoding/compression-algo/lib.rs b/rust/lance-encoding/compression-algo/lib.rs deleted file mode 100644 index 1c36cd0efb..0000000000 --- a/rust/lance-encoding/compression-algo/lib.rs +++ /dev/null @@ -1,3 +0,0 @@ -// SPDX-License-Identifier: Apache-2.0 -// SPDX-FileCopyrightText: Copyright The Lance Authors -pub mod compression_algo; \ No newline at end of file From 697af4a88308df456a42a382edd688bf0ea9a3bd Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Mon, 23 Sep 2024 19:36:18 +0000 Subject: [PATCH 20/31] hangle nullable and all null data block in `encode`. --- .../encodings/physical/bitpack_fastlanes.rs | 82 +++++++++++++++---- 1 file changed, 64 insertions(+), 18 deletions(-) diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index 08d4fac3dc..d43a0b396d 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -18,7 +18,7 @@ use lance_core::{Error, Result}; use crate::buffer::LanceBuffer; use crate::compression_algo::fastlanes::BitPacking; -use crate::data::{DataBlock, FixedWidthDataBlock}; +use crate::data::{DataBlock, FixedWidthDataBlock, NullableDataBlock}; use crate::decoder::{PageScheduler, PrimitivePageDecoder}; use crate::encoder::{ArrayEncoder, EncodedArray}; use crate::format::ProtobufUtils; @@ -254,7 +254,7 @@ macro_rules! encode_fixed_width { num_values: $unpacked.num_values, }); - Ok(EncodedArray { + Result::Ok(EncodedArray { data: packed, encoding, }) @@ -283,19 +283,64 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { data_type: &DataType, buffer_index: &mut u32, ) -> Result { - let DataBlock::FixedWidth(mut unpacked) = data else { - return Err(Error::InvalidInput { - source: "Bitpacking only supports fixed width data blocks".into(), - location: location!(), - }); - }; - - match data_type { - DataType::UInt8 | DataType::Int8 => encode_fixed_width!(self, unpacked, u8, buffer_index), - DataType::UInt16 | DataType::Int16 => encode_fixed_width!(self, unpacked, u16, buffer_index), - DataType::UInt32 | DataType::Int32 => encode_fixed_width!(self, unpacked, u32, buffer_index), - DataType::UInt64 | DataType::Int64 => encode_fixed_width!(self, unpacked, u64, buffer_index), - _ => unreachable!("BitpackedForNonNegArrayEncoder only supports data types of UInt8, Int8, UInt16, Int16, UInt32, Int32, UInt64, Int64"), + match data { + DataBlock::AllNull(_) => { + let encoding = ProtobufUtils::basic_all_null_encoding(); + Ok(EncodedArray { data, encoding }) + } + DataBlock::FixedWidth(mut unpacked) => { + match data_type { + DataType::UInt8 | DataType::Int8 => encode_fixed_width!(self, unpacked, u8, buffer_index), + DataType::UInt16 | DataType::Int16 => encode_fixed_width!(self, unpacked, u16, buffer_index), + DataType::UInt32 | DataType::Int32 => encode_fixed_width!(self, unpacked, u32, buffer_index), + DataType::UInt64 | DataType::Int64 => encode_fixed_width!(self, unpacked, u64, buffer_index), + _ => unreachable!("BitpackedForNonNegArrayEncoder only supports data types of UInt8, Int8, UInt16, Int16, UInt32, Int32, UInt64, Int64"), + } + } + DataBlock::Nullable(nullable) => { + let validity_buffer_index = *buffer_index; + *buffer_index += 1; + + let validity_desc = ProtobufUtils::flat_encoding( + 1, + validity_buffer_index, + /*compression=*/ None, + ); + let encoded_values: EncodedArray; + match *nullable.data { + DataBlock::FixedWidth(mut unpacked) => { + match data_type { + DataType::UInt8 | DataType::Int8 => encoded_values = encode_fixed_width!(self, unpacked, u8, buffer_index)?, + DataType::UInt16 | DataType::Int16 => encoded_values = encode_fixed_width!(self, unpacked, u16, buffer_index)?, + DataType::UInt32 | DataType::Int32 => encoded_values = encode_fixed_width!(self, unpacked, u32, buffer_index)?, + DataType::UInt64 | DataType::Int64 => encoded_values = encode_fixed_width!(self, unpacked, u64, buffer_index)?, + _ => unreachable!("BitpackedForNonNegArrayEncoder only supports data types of UInt8, Int8, UInt16, Int16, UInt32, Int32, UInt64, Int64"), + } + } + _ => { + return Err(Error::InvalidInput { + source: "Bitpacking only supports fixed width data blocks or a nullable data block with fixed width data block inside or a all null data block".into(), + location: location!(), + }); + } + } + let encoding = + ProtobufUtils::basic_some_null_encoding(validity_desc, encoded_values.encoding); + let encoded = DataBlock::Nullable(NullableDataBlock { + data: Box::new(encoded_values.data), + nulls: nullable.nulls, + }); + Ok(EncodedArray { + data: encoded, + encoding, + }) + } + _ => { + return Err(Error::InvalidInput { + source: "Bitpacking only supports fixed width data blocks or a nullable data block with fixed width data block inside or a all null data block".into(), + location: location!(), + }); + } } } } @@ -558,10 +603,11 @@ fn bitpacked_for_non_neg_decode( let mut curr_range_start = bytes_idx_to_range_indices[i][0].start; let mut chunk_num = 0; while chunk_num * packed_chunk_size_in_byte < bytes.len() { - let chunk_in_u8: &[u8] = &bytes[chunk_num * packed_chunk_size_in_byte..] - [..packed_chunk_size_in_byte]; + // I have to do a copy here for memory alignment + let chunk_in_u8: Vec = bytes[chunk_num * packed_chunk_size_in_byte..] + [..packed_chunk_size_in_byte].to_vec(); chunk_num += 1; - let chunk = cast_slice(chunk_in_u8); + let chunk = cast_slice(&chunk_in_u8); unsafe { BitPacking::unchecked_unpack( compressed_bit_width as usize, From 922c2fe1d1bd4ed4b864ea77de5df495e66bb36e Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Tue, 24 Sep 2024 15:52:56 +0000 Subject: [PATCH 21/31] fix `choose_array_encoder` issue when enable V2.1 --- rust/lance-encoding/src/encoder.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rust/lance-encoding/src/encoder.rs b/rust/lance-encoding/src/encoder.rs index 12c1503b11..0cc9a18cc3 100644 --- a/rust/lance-encoding/src/encoder.rs +++ b/rust/lance-encoding/src/encoder.rs @@ -338,7 +338,7 @@ impl CoreArrayEncodingStrategy { Ok(Box::new(PackedStructEncoder::new(inner_encoders))) } DataType::UInt8 | DataType::UInt16 | DataType::UInt32 | DataType::UInt64 => { - if version >= LanceFileVersion::V2_1 { + if version >= LanceFileVersion::V2_1 && arrays[0].data_type() == data_type { let compressed_bit_width = compute_compressed_bit_width_for_non_neg(arrays); Ok(Box::new(BitpackedForNonNegArrayEncoder::new( compressed_bit_width as usize, @@ -354,7 +354,7 @@ impl CoreArrayEncodingStrategy { // for signed integers, I intend to make it a cascaded encoding, a sparse array for the negative values and very wide(bit-width) values, // then a bitpacked array for the narrow(bit-width) values, I need `BitpackedForNeg` to be merged first DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64 => { - if version >= LanceFileVersion::V2_1 { + if version >= LanceFileVersion::V2_1 && arrays[0].data_type() == data_type { let compressed_bit_width = compute_compressed_bit_width_for_non_neg(arrays); Ok(Box::new(BitpackedForNonNegArrayEncoder::new( compressed_bit_width as usize, From f09cad72ccfb0fe63bc8cf6c8e2c844d44859fcc Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Tue, 24 Sep 2024 15:56:42 +0000 Subject: [PATCH 22/31] fix lint --- .../src/encodings/physical/bitpack_fastlanes.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index d43a0b396d..35842166a4 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -336,10 +336,10 @@ impl ArrayEncoder for BitpackedForNonNegArrayEncoder { }) } _ => { - return Err(Error::InvalidInput { + Err(Error::InvalidInput { source: "Bitpacking only supports fixed width data blocks or a nullable data block with fixed width data block inside or a all null data block".into(), location: location!(), - }); + }) } } } @@ -605,7 +605,8 @@ fn bitpacked_for_non_neg_decode( while chunk_num * packed_chunk_size_in_byte < bytes.len() { // I have to do a copy here for memory alignment let chunk_in_u8: Vec = bytes[chunk_num * packed_chunk_size_in_byte..] - [..packed_chunk_size_in_byte].to_vec(); + [..packed_chunk_size_in_byte] + .to_vec(); chunk_num += 1; let chunk = cast_slice(&chunk_in_u8); unsafe { From fc89bf4da57e45776daedfb2fc390d893a5f43b4 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Tue, 24 Sep 2024 18:21:32 +0000 Subject: [PATCH 23/31] fix a bug scheduling ranges for data types other than 32-bit width --- rust/lance-encoding/benches/decoder.rs | 2 +- .../encodings/physical/bitpack_fastlanes.rs | 309 +++++------------- 2 files changed, 91 insertions(+), 220 deletions(-) diff --git a/rust/lance-encoding/benches/decoder.rs b/rust/lance-encoding/benches/decoder.rs index cdc13479e4..f8bca14585 100644 --- a/rust/lance-encoding/benches/decoder.rs +++ b/rust/lance-encoding/benches/decoder.rs @@ -66,7 +66,7 @@ fn bench_decode(c: &mut Criterion) { for data_type in PRIMITIVE_TYPES { let data = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(data_type)) - .into_batch_rows(lance_datagen::RowCount::from(1024 * 1024)) + .into_batch_rows(lance_datagen::RowCount::from(1024 * 1024 * 1024)) .unwrap(); let lance_schema = Arc::new(lance_core::datatypes::Schema::try_from(data.schema().as_ref()).unwrap()); diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index 35842166a4..6c3d56d211 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -186,8 +186,7 @@ pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { } } _ => { - // in dictionary encoding, they route it to here when array is utf8, don't know what we should do yet. - res = 8; + panic!("BitpackedForNonNegArrayEncoder only supports data types of UInt8, Int8, UInt16, Int16, UInt32, Int32, UInt64, Int64"); } }; res @@ -478,237 +477,109 @@ impl PrimitivePageDecoder for BitpackedForNonNegPageDecoder { } } -fn bitpacked_for_non_neg_decode( - compressed_bit_width: u64, - uncompressed_bits_per_value: u64, - data: &[Bytes], - bytes_idx_to_range_indices: &[Vec>], - num_rows: u64, -) -> LanceBuffer { - match uncompressed_bits_per_value { - 8 => { - let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); - let packed_chunk_size: usize = - ELEMS_PER_CHUNK as usize * compressed_bit_width as usize / 8; - let mut decompress_chunk_buf = vec![0_u8; ELEMS_PER_CHUNK as usize]; - for (i, bytes) in data.iter().enumerate() { - let mut j = 0; - let mut ranges_idx = 0; - let mut curr_range_start = bytes_idx_to_range_indices[i][0].start; - while j * packed_chunk_size < bytes.len() { - let chunk: &[u8] = &bytes[j * packed_chunk_size..][..packed_chunk_size]; - unsafe { - BitPacking::unchecked_unpack( - compressed_bit_width as usize, - chunk, - &mut decompress_chunk_buf[..ELEMS_PER_CHUNK as usize], - ); - } - loop { - if curr_range_start + ELEMS_PER_CHUNK - < bytes_idx_to_range_indices[i][ranges_idx].end - { - let this_part_len = - ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; - decompressed.extend_from_slice( - &decompress_chunk_buf - [(curr_range_start % ELEMS_PER_CHUNK) as usize..], - ); - curr_range_start += this_part_len; - break; - } else { - let this_part_len = - bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; - decompressed.extend_from_slice( - &decompress_chunk_buf - [(curr_range_start % ELEMS_PER_CHUNK) as usize..] - [..this_part_len as usize], - ); - ranges_idx += 1; - if ranges_idx == bytes_idx_to_range_indices[i].len() { - break; - } - curr_range_start = bytes_idx_to_range_indices[i][ranges_idx].start; - } - } - j += 1; +macro_rules! bitpacked_decode { + ($uncompressed_type:ty, $compressed_bit_width:expr, $data:expr, $bytes_idx_to_range_indices:expr, $num_rows:expr) => {{ + let mut decompressed: Vec<$uncompressed_type> = Vec::with_capacity($num_rows as usize); + let packed_chunk_size_in_byte: usize = (ELEMS_PER_CHUNK * $compressed_bit_width) as usize / 8; + let mut decompress_chunk_buf = vec![0 as $uncompressed_type; ELEMS_PER_CHUNK as usize]; + + for (i, bytes) in $data.iter().enumerate() { + let mut ranges_idx = 0; + let mut curr_range_start = $bytes_idx_to_range_indices[i][0].start; + let mut chunk_num = 0; + + while chunk_num * packed_chunk_size_in_byte < bytes.len() { + // Copy for memory alignment + let chunk_in_u8: Vec = bytes[chunk_num * packed_chunk_size_in_byte..] + [..packed_chunk_size_in_byte] + .to_vec(); + chunk_num += 1; + let chunk = cast_slice(&chunk_in_u8); + unsafe { + BitPacking::unchecked_unpack( + $compressed_bit_width as usize, + chunk, + &mut decompress_chunk_buf, + ); } - } - LanceBuffer::Owned(decompressed) - } - 16 => { - let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); - let packed_chunk_size_in_byte: usize = - (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; - let mut decompress_chunk_buf = vec![0_u16; ELEMS_PER_CHUNK as usize]; - for (i, bytes) in data.iter().enumerate() { - let mut j = 0; - let mut ranges_idx = 0; - let mut curr_range_start = bytes_idx_to_range_indices[i][0].start; - while j * packed_chunk_size_in_byte < bytes.len() { - let chunk_in_u8: &[u8] = - &bytes[j * packed_chunk_size_in_byte..][..packed_chunk_size_in_byte]; - let chunk = cast_slice(chunk_in_u8); - unsafe { - BitPacking::unchecked_unpack( - compressed_bit_width as usize, - chunk, - &mut decompress_chunk_buf, + loop { + // Case 1: All the elements after (curr_range_start % ELEMS_PER_CHUNK) inside this chunk are needed. + let elems_after_curr_range_start_in_this_chunk = + ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; + if curr_range_start + elems_after_curr_range_start_in_this_chunk + <= $bytes_idx_to_range_indices[i][ranges_idx].end + { + decompressed.extend_from_slice( + &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..], ); - } - loop { - if curr_range_start + ELEMS_PER_CHUNK - < bytes_idx_to_range_indices[i][ranges_idx].end + curr_range_start += elems_after_curr_range_start_in_this_chunk; + break; + } else { + // Case 2: Only part of the elements after (curr_range_start % ELEMS_PER_CHUNK) inside this chunk are needed. + let elems_this_range_needed_in_this_chunk = + ($bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start) + .min(ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK); + decompressed.extend_from_slice( + &decompress_chunk_buf[(curr_range_start % ELEMS_PER_CHUNK) as usize..] + [..elems_this_range_needed_in_this_chunk as usize], + ); + if curr_range_start + elems_this_range_needed_in_this_chunk + == $bytes_idx_to_range_indices[i][ranges_idx].end { - let this_part_len = - ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; - decompressed.extend_from_slice( - &decompress_chunk_buf - [(curr_range_start % ELEMS_PER_CHUNK) as usize..], - ); - curr_range_start += this_part_len; - - // when `curr_range_start + 1024 < bytes_idx_to_range_indices[i][ranges_idx].end`, - // we know this chunk has only data of this range - break; - } else { - let this_part_len = - bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; - decompressed.extend_from_slice( - &decompress_chunk_buf - [(curr_range_start % ELEMS_PER_CHUNK) as usize..] - [..this_part_len as usize], - ); ranges_idx += 1; - if ranges_idx == bytes_idx_to_range_indices[i].len() { + if ranges_idx == $bytes_idx_to_range_indices[i].len() { break; } - curr_range_start = bytes_idx_to_range_indices[i][ranges_idx].start; - } - } - j += 1; - } - } - LanceBuffer::reinterpret_vec(decompressed).to_owned() - } - - 32 => { - let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); - let packed_chunk_size_in_byte: usize = - (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; - let mut decompress_chunk_buf = vec![0_u32; ELEMS_PER_CHUNK as usize]; - for (i, bytes) in data.iter().enumerate() { - let mut ranges_idx = 0; - let mut curr_range_start = bytes_idx_to_range_indices[i][0].start; - let mut chunk_num = 0; - while chunk_num * packed_chunk_size_in_byte < bytes.len() { - // I have to do a copy here for memory alignment - let chunk_in_u8: Vec = bytes[chunk_num * packed_chunk_size_in_byte..] - [..packed_chunk_size_in_byte] - .to_vec(); - chunk_num += 1; - let chunk = cast_slice(&chunk_in_u8); - unsafe { - BitPacking::unchecked_unpack( - compressed_bit_width as usize, - chunk, - &mut decompress_chunk_buf, - ); - } - loop { - // case 1: all the elements after (curr_range_start % ELEMS_PER_CHUNK) inside this chunk is needed. - let elems_after_curr_range_start_in_this_chunk = - ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; - if curr_range_start + elems_after_curr_range_start_in_this_chunk - <= bytes_idx_to_range_indices[i][ranges_idx].end - { - decompressed.extend_from_slice( - &decompress_chunk_buf - [(curr_range_start % ELEMS_PER_CHUNK) as usize..], - ); - curr_range_start += elems_after_curr_range_start_in_this_chunk; - break; + curr_range_start = $bytes_idx_to_range_indices[i][ranges_idx].start; } else { - // case 2: only part of the elements after (curr_range_start % ELEMS_PER_CHUNK) inside this chunk is needed. - let elems_this_range_needed_in_this_chunk = - (bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start) - .min(ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK); - decompressed.extend_from_slice( - &decompress_chunk_buf - [(curr_range_start % ELEMS_PER_CHUNK) as usize..] - [..elems_this_range_needed_in_this_chunk as usize], - ); - if curr_range_start + elems_this_range_needed_in_this_chunk - == bytes_idx_to_range_indices[i][ranges_idx].end - { - ranges_idx += 1; - if ranges_idx == bytes_idx_to_range_indices[i].len() { - break; - } - curr_range_start = bytes_idx_to_range_indices[i][ranges_idx].start; - } else { - curr_range_start += elems_this_range_needed_in_this_chunk; - } + curr_range_start += elems_this_range_needed_in_this_chunk; } } } } - LanceBuffer::reinterpret_vec(decompressed).to_owned() } - 64 => { - let mut decompressed: Vec = Vec::with_capacity(num_rows as usize); - let packed_chunk_size_in_byte: usize = - (ELEMS_PER_CHUNK * compressed_bit_width) as usize / 8; - let mut decompress_chunk_buf = vec![0_u64; ELEMS_PER_CHUNK as usize]; - for (i, bytes) in data.iter().enumerate() { - let mut j = 0; - let mut ranges_idx = 0; - let mut curr_range_start = bytes_idx_to_range_indices[i][0].start; - while j * packed_chunk_size_in_byte < bytes.len() { - let chunk_in_u8: &[u8] = - &bytes[j * packed_chunk_size_in_byte..][..packed_chunk_size_in_byte]; - let chunk = cast_slice(chunk_in_u8); - unsafe { - BitPacking::unchecked_unpack( - compressed_bit_width as usize, - chunk, - &mut decompress_chunk_buf, - ); - } - loop { - if curr_range_start + ELEMS_PER_CHUNK - < bytes_idx_to_range_indices[i][ranges_idx].end - { - let this_part_len = - ELEMS_PER_CHUNK - curr_range_start % ELEMS_PER_CHUNK; - decompressed.extend_from_slice( - &decompress_chunk_buf - [(curr_range_start % ELEMS_PER_CHUNK) as usize..], - ); - curr_range_start += this_part_len; - break; - } else { - let this_part_len = - bytes_idx_to_range_indices[i][ranges_idx].end - curr_range_start; - decompressed.extend_from_slice( - &decompress_chunk_buf - [(curr_range_start % ELEMS_PER_CHUNK) as usize..] - [..this_part_len as usize], - ); - ranges_idx += 1; - if ranges_idx == bytes_idx_to_range_indices[i].len() { - break; - } - curr_range_start = bytes_idx_to_range_indices[i][ranges_idx].start; - } - } - j += 1; - } - } - LanceBuffer::reinterpret_vec(decompressed).to_owned() - } + LanceBuffer::reinterpret_vec(decompressed) + }}; +} + +fn bitpacked_for_non_neg_decode( + compressed_bit_width: u64, + uncompressed_bits_per_value: u64, + data: &[Bytes], + bytes_idx_to_range_indices: &[Vec>], + num_rows: u64, +) -> LanceBuffer { + match uncompressed_bits_per_value { + 8 => bitpacked_decode!( + u8, + compressed_bit_width, + data, + bytes_idx_to_range_indices, + num_rows + ), + 16 => bitpacked_decode!( + u16, + compressed_bit_width, + data, + bytes_idx_to_range_indices, + num_rows + ), + 32 => bitpacked_decode!( + u32, + compressed_bit_width, + data, + bytes_idx_to_range_indices, + num_rows + ), + 64 => bitpacked_decode!( + u64, + compressed_bit_width, + data, + bytes_idx_to_range_indices, + num_rows + ), _ => unreachable!( "bitpacked_for_non_neg_decode only supports 8, 16, 32, 64 uncompressed_bits_per_value" ), From d5b9201ae335e7ced05d62127e358c6f655c167a Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 24 Sep 2024 14:44:40 -0700 Subject: [PATCH 24/31] Make sure to use version 2.1 in tests for bitpacking --- .../encodings/physical/bitpack_fastlanes.rs | 294 +++++++++--------- rust/lance-encoding/src/testing.rs | 14 +- 2 files changed, 163 insertions(+), 145 deletions(-) diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index 6c3d56d211..4bde319f53 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -603,35 +603,41 @@ mod tests { use lance_datagen::RowCount; use crate::testing::{check_round_trip_encoding_of_data, TestCases}; + use crate::version::LanceFileVersion; + + async fn check_round_trip_bitpacked(array: Arc) { + let test_cases = TestCases::default().with_file_version(LanceFileVersion::V2_1); + check_round_trip_encoding_of_data(vec![array], &test_cases, HashMap::new()).await; + } #[test_log::test(tokio::test)] async fn test_bitpack_fastlanes_u8() { let values: Vec = vec![5; 1024]; let array = UInt8Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![66; 1000]; let array = UInt8Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![77; 2000]; let array = UInt8Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![0; 10000]; let array = UInt8Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![88; 10000]; let array = UInt8Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) @@ -639,7 +645,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) @@ -647,7 +653,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) @@ -655,7 +661,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) @@ -663,7 +669,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) @@ -671,7 +677,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) @@ -679,7 +685,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) @@ -687,7 +693,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt8)) @@ -695,7 +701,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; } #[test_log::test(tokio::test)] @@ -703,39 +709,39 @@ mod tests { let values: Vec = vec![5; 1024]; let array = UInt16Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![66; 1000]; let array = UInt16Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![77; 2000]; let array = UInt16Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![0; 10000]; let array = UInt16Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![88; 10000]; let array = UInt16Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![300; 100]; let array = UInt16Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![800; 100]; let array = UInt16Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) @@ -743,7 +749,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) @@ -751,7 +757,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) @@ -759,7 +765,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) @@ -767,7 +773,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) @@ -775,7 +781,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) @@ -783,7 +789,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt16)) @@ -791,7 +797,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; } #[test_log::test(tokio::test)] @@ -799,72 +805,72 @@ mod tests { let values: Vec = vec![5; 1024]; let array = UInt32Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![7; 2000]; let array = UInt32Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![66; 1000]; let array = UInt32Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![666; 1000]; let array = UInt32Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![77; 2000]; let array = UInt32Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![0; 10000]; let array = UInt32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![1; 10000]; let array = UInt32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![88; 10000]; let array = UInt32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![300; 100]; let array = UInt32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![3000; 100]; let array = UInt32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![800; 100]; let array = UInt32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![8000; 100]; let array = UInt32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![65536; 100]; let array = UInt32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![655360; 100]; let array = UInt32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) @@ -872,7 +878,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) @@ -880,7 +886,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) @@ -888,7 +894,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) @@ -896,7 +902,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) @@ -904,7 +910,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) @@ -912,7 +918,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) @@ -920,7 +926,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt32)) @@ -928,7 +934,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; } #[test_log::test(tokio::test)] @@ -936,72 +942,72 @@ mod tests { let values: Vec = vec![5; 1024]; let array = UInt64Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![7; 2000]; let array = UInt64Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![66; 1000]; let array = UInt64Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![666; 1000]; let array = UInt64Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![77; 2000]; let array = UInt64Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![0; 10000]; let array = UInt64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![1; 10000]; let array = UInt64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![88; 10000]; let array = UInt64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![300; 100]; let array = UInt64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![3000; 100]; let array = UInt64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![800; 100]; let array = UInt64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![8000; 100]; let array = UInt64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![65536; 100]; let array = UInt64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![655360; 100]; let array = UInt64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) @@ -1009,7 +1015,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) @@ -1017,7 +1023,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) @@ -1025,7 +1031,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) @@ -1033,7 +1039,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) @@ -1041,7 +1047,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) @@ -1049,7 +1055,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) @@ -1057,7 +1063,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::UInt64)) @@ -1065,7 +1071,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; } #[test_log::test(tokio::test)] @@ -1073,34 +1079,34 @@ mod tests { let values: Vec = vec![-5; 1024]; let array = Int8Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![66; 1000]; let array = Int8Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![77; 2000]; let array = Int8Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![0; 10000]; let array = Int8Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![88; 10000]; let array = Int8Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![-88; 10000]; let array = Int8Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) @@ -1108,7 +1114,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) @@ -1116,7 +1122,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) @@ -1124,7 +1130,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) @@ -1132,7 +1138,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) @@ -1140,7 +1146,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) @@ -1148,7 +1154,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) @@ -1156,7 +1162,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int8)) @@ -1164,7 +1170,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; } #[test_log::test(tokio::test)] @@ -1172,39 +1178,39 @@ mod tests { let values: Vec = vec![-5; 1024]; let array = Int16Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![66; 1000]; let array = Int16Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![77; 2000]; let array = Int16Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![0; 10000]; let array = Int16Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![88; 10000]; let array = Int16Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![300; 100]; let array = Int16Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![800; 100]; let array = Int16Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) @@ -1212,7 +1218,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) @@ -1220,7 +1226,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) @@ -1228,7 +1234,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) @@ -1236,7 +1242,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) @@ -1244,7 +1250,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) @@ -1252,7 +1258,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) @@ -1260,7 +1266,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int16)) @@ -1268,7 +1274,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; } #[test_log::test(tokio::test)] @@ -1276,72 +1282,72 @@ mod tests { let values: Vec = vec![-5; 1024]; let array = Int32Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![66; 1000]; let array = Int32Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![-66; 1000]; let array = Int32Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![77; 2000]; let array = Int32Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![-77; 2000]; let array = Int32Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![0; 10000]; let array = Int32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![88; 10000]; let array = Int32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![-88; 10000]; let array = Int32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![300; 100]; let array = Int32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![-300; 100]; let array = Int32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![800; 100]; let array = Int32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![-800; 100]; let array = Int32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![65536; 100]; let array = Int32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![-65536; 100]; let array = Int32Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) @@ -1349,7 +1355,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) @@ -1357,7 +1363,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) @@ -1365,7 +1371,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) @@ -1373,7 +1379,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) @@ -1381,7 +1387,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) @@ -1389,7 +1395,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) @@ -1397,7 +1403,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int32)) @@ -1405,7 +1411,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; } #[test_log::test(tokio::test)] @@ -1413,72 +1419,72 @@ mod tests { let values: Vec = vec![-5; 1024]; let array = Int64Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![66; 1000]; let array = Int64Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![-66; 1000]; let array = Int64Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![77; 2000]; let array = Int64Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![-77; 2000]; let array = Int64Array::from(values); let array: Arc = Arc::new(array); - check_round_trip_encoding_of_data(vec![array], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(array).await; let values: Vec = vec![0; 10000]; let array = Int64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![88; 10000]; let array = Int64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![-88; 10000]; let array = Int64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![300; 100]; let array = Int64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![-300; 100]; let array = Int64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![800; 100]; let array = Int64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![-800; 100]; let array = Int64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![65536; 100]; let array = Int64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let values: Vec = vec![-65536; 100]; let array = Int64Array::from(values); let arr = Arc::new(array) as ArrayRef; - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) @@ -1486,7 +1492,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) @@ -1494,7 +1500,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) @@ -1502,7 +1508,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) @@ -1510,7 +1516,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) @@ -1518,7 +1524,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) @@ -1526,7 +1532,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) @@ -1534,7 +1540,7 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; let arr = lance_datagen::gen() .anon_col(lance_datagen::array::rand_type(&DataType::Int64)) @@ -1542,6 +1548,6 @@ mod tests { .unwrap() .column(0) .clone(); - check_round_trip_encoding_of_data(vec![arr], &TestCases::default(), HashMap::new()).await; + check_round_trip_bitpacked(arr).await; } } diff --git a/rust/lance-encoding/src/testing.rs b/rust/lance-encoding/src/testing.rs index 4089291f2b..2d27087414 100644 --- a/rust/lance-encoding/src/testing.rs +++ b/rust/lance-encoding/src/testing.rs @@ -269,6 +269,7 @@ pub struct TestCases { batch_size: u32, skip_validation: bool, max_page_size: Option, + file_version: LanceFileVersion, } impl Default for TestCases { @@ -279,6 +280,7 @@ impl Default for TestCases { indices: Vec::new(), skip_validation: false, max_page_size: None, + file_version: LanceFileVersion::default(), } } } @@ -312,6 +314,11 @@ impl TestCases { fn get_max_page_size(&self) -> u64 { self.max_page_size.unwrap_or(MAX_PAGE_BYTES) } + + pub fn with_file_version(mut self, version: LanceFileVersion) -> Self { + self.file_version = version; + self + } } /// Given specific data and test cases we check round trip encoding and decoding @@ -330,7 +337,12 @@ pub async fn check_round_trip_encoding_of_data( field = field.with_metadata(metadata); let lance_field = lance_core::datatypes::Field::try_from(&field).unwrap(); for page_size in [4096, 1024 * 1024] { - let encoding_strategy = CoreFieldEncodingStrategy::default(); + let encoding_strategy = CoreFieldEncodingStrategy { + array_encoding_strategy: Arc::new(CoreArrayEncodingStrategy { + version: test_cases.file_version, + }), + version: test_cases.file_version, + }; let mut column_index_seq = ColumnIndexSequence::default(); let encoding_options = EncodingOptions { cache_bytes_per_column: page_size, From 13b757a431a85b91151373d9a5c1f8d5716a03f7 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Tue, 24 Sep 2024 22:11:26 +0000 Subject: [PATCH 25/31] make `locate_chunk_start` and `locate_chunk_end` a method --- .../src/compression_algo/fastlanes.rs | 2 +- .../encodings/physical/bitpack_fastlanes.rs | 34 +++++++++---------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/rust/lance-encoding/src/compression_algo/fastlanes.rs b/rust/lance-encoding/src/compression_algo/fastlanes.rs index 3ef964dcf3..e8a2d2d1ba 100644 --- a/rust/lance-encoding/src/compression_algo/fastlanes.rs +++ b/rust/lance-encoding/src/compression_algo/fastlanes.rs @@ -1,7 +1,7 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright The Lance Authors -// This file is a modification of the `fastlanes` crate: https://github.com/spiraldb/fastlanes +// NOTICE: This file is a modification of the `fastlanes` crate: https://github.com/spiraldb/fastlanes // It is modified to allow a rust stable build use arrayref::{array_mut_ref, array_ref}; diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index 6c3d56d211..1ced7f61f5 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -351,16 +351,6 @@ pub struct BitpackedForNonNegScheduler { buffer_offset: u64, } -fn locate_chunk_start(scheduler: &BitpackedForNonNegScheduler, relative_row_num: u64) -> u64 { - let chunk_size = ELEMS_PER_CHUNK * scheduler.compressed_bit_width / 8; - relative_row_num / ELEMS_PER_CHUNK * chunk_size -} - -fn locate_chunk_end(scheduler: &BitpackedForNonNegScheduler, relative_row_num: u64) -> u64 { - let chunk_size = ELEMS_PER_CHUNK * scheduler.compressed_bit_width / 8; - relative_row_num / ELEMS_PER_CHUNK * chunk_size + chunk_size -} - impl BitpackedForNonNegScheduler { pub fn new( compressed_bit_width: u64, @@ -373,6 +363,16 @@ impl BitpackedForNonNegScheduler { buffer_offset, } } + + fn locate_chunk_start(&self, relative_row_num: u64) -> u64 { + let chunk_size = ELEMS_PER_CHUNK * self.compressed_bit_width / 8; + self.buffer_offset + (relative_row_num / ELEMS_PER_CHUNK * chunk_size) + } + + fn locate_chunk_end(&self, relative_row_num: u64) -> u64 { + let chunk_size = ELEMS_PER_CHUNK * self.compressed_bit_width / 8; + self.buffer_offset + (relative_row_num / ELEMS_PER_CHUNK * chunk_size) + chunk_size + } } impl PageScheduler for BitpackedForNonNegScheduler { @@ -391,26 +391,26 @@ impl PageScheduler for BitpackedForNonNegScheduler { // map one bytes to multiple ranges, one bytes has at least one range corresponding to it let mut bytes_idx_to_range_indices = vec![]; let first_byte_range = std::ops::Range { - start: self.buffer_offset + locate_chunk_start(self, ranges[0].start), - end: self.buffer_offset + locate_chunk_end(self, ranges[0].end - 1), + start: self.locate_chunk_start(ranges[0].start), + end: self.locate_chunk_end(ranges[0].end - 1), }; // the ranges are half-open byte_ranges.push(first_byte_range); bytes_idx_to_range_indices.push(vec![ranges[0].clone()]); for (i, range) in ranges.iter().enumerate().skip(1) { - let this_start = locate_chunk_start(self, range.start); - let this_end = locate_chunk_end(self, range.end - 1); + let this_start = self.locate_chunk_start(range.start); + let this_end = self.locate_chunk_end(range.end - 1); // when the current range start is in the same chunk as the previous range's end, we colaesce this two bytes ranges // when the current range start is not in the same chunk as the previous range's end, we create a new bytes range - if this_start == locate_chunk_start(self, ranges[i - 1].end - 1) { - byte_ranges.last_mut().unwrap().end = self.buffer_offset + this_end; + if this_start == self.locate_chunk_start(ranges[i - 1].end - 1) { + byte_ranges.last_mut().unwrap().end = this_end; bytes_idx_to_range_indices .last_mut() .unwrap() .push(range.clone()); } else { - byte_ranges.push(self.buffer_offset + this_start..self.buffer_offset + this_end); + byte_ranges.push(this_start..this_end); bytes_idx_to_range_indices.push(vec![range.clone()]); } } From f42af4c4464728558fecf64f33cb2dfa785bb7dd Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Wed, 25 Sep 2024 16:38:53 +0000 Subject: [PATCH 26/31] add test_pack --- .../src/compression_algo/fastlanes.rs | 256 ++++++++++++++++-- 1 file changed, 239 insertions(+), 17 deletions(-) diff --git a/rust/lance-encoding/src/compression_algo/fastlanes.rs b/rust/lance-encoding/src/compression_algo/fastlanes.rs index e8a2d2d1ba..0df159d2b7 100644 --- a/rust/lance-encoding/src/compression_algo/fastlanes.rs +++ b/rust/lance-encoding/src/compression_algo/fastlanes.rs @@ -21,7 +21,6 @@ impl FastLanes for u16 {} impl FastLanes for u32 {} impl FastLanes for u64 {} -#[macro_export] macro_rules! iterate { ($T:ty, $lane: expr, | $_1:tt $idx:ident | $($body:tt)*) => { macro_rules! __kernel__ {( $_1 $idx:ident ) => ( $($body)* )} @@ -44,7 +43,6 @@ macro_rules! iterate { } } -#[macro_export] macro_rules! pack { ($T:ty, $W:expr, $packed:expr, $lane:expr, | $_1:tt $idx:ident | $($body:tt)*) => { macro_rules! __kernel__ {( $_1 $idx:ident ) => ( $($body)* )} @@ -109,7 +107,6 @@ macro_rules! pack { }; } -#[macro_export] macro_rules! unpack { ($T:ty, $W:expr, $packed:expr, $lane:expr, | $_1:tt $idx:ident, $_2:tt $elem:ident | $($body:tt)*) => { macro_rules! __kernel__ {( $_1 $idx:ident, $_2 $elem:ident ) => ( $($body)* )} @@ -185,7 +182,6 @@ macro_rules! unpack { } // Macro for repeating a code block bit_size_of:: times. -#[macro_export] macro_rules! seq_t { ($ident:ident in u8 $body:tt) => {seq_macro::seq!($ident in 0..8 $body)}; ($ident:ident in u16 $body:tt) => {seq_macro::seq!($ident in 0..16 $body)}; @@ -1698,35 +1694,130 @@ pack_64!(pack_64_61, 61); pack_64!(pack_64_62, 62); pack_64!(pack_64_63, 63); pack_64!(pack_64_64, 64); + #[cfg(test)] mod test { use super::*; use core::array; + // a fast random number generator + pub struct XorShift { + state: u64, + } + + impl XorShift { + pub fn new(seed: u64) -> Self { + XorShift { state: seed } + } + + pub fn next(&mut self) -> u64 { + let mut x = self.state; + x ^= x << 13; + x ^= x >> 7; + x ^= x << 17; + self.state = x; + x + } + } - #[test] - fn test_pack() { - let mut values: [u16; 1024] = [0; 1024]; - for (i, value) in values.iter_mut().enumerate() { - *value = (i % (1 << 15)) as u16; + fn pack_unpack_u8(bit_width: usize) { + let mut values: [u8; 1024] = [0; 1024]; + let mut rng = XorShift::new(123456789); + for i in 0..1024 { + values[i] = (rng.next() % (1 << bit_width)) as u8; } - let mut packed: [u16; 960] = [0; 960]; - for lane in 0..u16::LANES { + let mut packed = vec![0; 1024 * bit_width / 8]; + for lane in 0..u8::LANES { // Always loop over lanes first. This is what the compiler vectorizes. - pack!(u16, 15, packed, lane, |$pos| { + pack!(u8, bit_width, packed, lane, |$pos| { values[$pos] }); } - let mut packed_orig: [u16; 960] = [0; 960]; - unsafe { - BitPacking::unchecked_pack(15, &values, &mut packed_orig); + let mut unpacked: [u8; 1024] = [0; 1024]; + for lane in 0..u8::LANES { + // Always loop over lanes first. This is what the compiler vectorizes. + unpack!(u8, bit_width, packed, lane, |$idx, $elem| { + unpacked[$idx] = $elem; + }); + } + + assert_eq!(values, unpacked); + } + + fn pack_unpack_u16(bit_width: usize) { + let mut values: [u16; 1024] = [0; 1024]; + let mut rng = XorShift::new(123456789); + for i in 0..1024 { + values[i] = (rng.next() % (1 << bit_width)) as u16; + } + + let mut packed = vec![0; 1024 * bit_width / 16]; + for lane in 0..u16::LANES { + // Always loop over lanes first. This is what the compiler vectorizes. + pack!(u16, bit_width, packed, lane, |$pos| { + values[$pos] + }); } let mut unpacked: [u16; 1024] = [0; 1024]; for lane in 0..u16::LANES { // Always loop over lanes first. This is what the compiler vectorizes. - unpack!(u16, 15, packed, lane, |$idx, $elem| { + unpack!(u16, bit_width, packed, lane, |$idx, $elem| { + unpacked[$idx] = $elem; + }); + } + + assert_eq!(values, unpacked); + } + + fn pack_unpack_u32(bit_width: usize) { + let mut values: [u32; 1024] = [0; 1024]; + let mut rng = XorShift::new(123456789); + for i in 0..1024 { + values[i] = (rng.next() % (1 << bit_width)) as u32; + } + + let mut packed = vec![0; 1024 * bit_width / 32]; + for lane in 0..u32::LANES { + // Always loop over lanes first. This is what the compiler vectorizes. + pack!(u32, bit_width, packed, lane, |$pos| { + values[$pos] + }); + } + + let mut unpacked: [u32; 1024] = [0; 1024]; + for lane in 0..u32::LANES { + // Always loop over lanes first. This is what the compiler vectorizes. + unpack!(u32, bit_width, packed, lane, |$idx, $elem| { + unpacked[$idx] = $elem; + }); + } + + assert_eq!(values, unpacked); + } + + fn pack_unpack_u64(bit_width: usize) { + let mut values: [u64; 1024] = [0; 1024]; + let mut rng = XorShift::new(123456789); + if bit_width > 0 { + for i in 0..1024 { + values[i] = (rng.next() % (1 << (bit_width - 1))) as u64; + } + } + + let mut packed = vec![0; 1024 * bit_width / 64]; + for lane in 0..u64::LANES { + // Always loop over lanes first. This is what the compiler vectorizes. + pack!(u64, bit_width, packed, lane, |$pos| { + values[$pos] + }); + } + + let mut unpacked: [u64; 1024] = [0; 1024]; + for lane in 0..u64::LANES { + // Always loop over lanes first. This is what the compiler vectorizes. + unpack!(u64, bit_width, packed, lane, |$idx, $elem| { unpacked[$idx] = $elem; }); } @@ -1734,6 +1825,137 @@ mod test { assert_eq!(values, unpacked); } + #[test] + fn test_pack() { + pack_unpack_u8(0); + pack_unpack_u8(1); + pack_unpack_u8(2); + pack_unpack_u8(3); + pack_unpack_u8(4); + pack_unpack_u8(5); + pack_unpack_u8(6); + pack_unpack_u8(7); + pack_unpack_u8(8); + + pack_unpack_u16(0); + pack_unpack_u16(1); + pack_unpack_u16(2); + pack_unpack_u16(3); + pack_unpack_u16(4); + pack_unpack_u16(5); + pack_unpack_u16(6); + pack_unpack_u16(7); + pack_unpack_u16(8); + pack_unpack_u16(9); + pack_unpack_u16(10); + pack_unpack_u16(11); + pack_unpack_u16(12); + pack_unpack_u16(13); + pack_unpack_u16(14); + pack_unpack_u16(15); + pack_unpack_u16(16); + + pack_unpack_u32(0); + pack_unpack_u32(1); + pack_unpack_u32(2); + pack_unpack_u32(3); + pack_unpack_u32(4); + pack_unpack_u32(5); + pack_unpack_u32(6); + pack_unpack_u32(7); + pack_unpack_u32(8); + pack_unpack_u32(9); + pack_unpack_u32(10); + pack_unpack_u32(11); + pack_unpack_u32(12); + pack_unpack_u32(13); + pack_unpack_u32(14); + pack_unpack_u32(15); + pack_unpack_u32(16); + pack_unpack_u32(17); + pack_unpack_u32(18); + pack_unpack_u32(19); + pack_unpack_u32(20); + pack_unpack_u32(21); + pack_unpack_u32(22); + pack_unpack_u32(23); + pack_unpack_u32(24); + pack_unpack_u32(25); + pack_unpack_u32(26); + pack_unpack_u32(27); + pack_unpack_u32(28); + pack_unpack_u32(29); + pack_unpack_u32(30); + pack_unpack_u32(31); + pack_unpack_u32(32); + + pack_unpack_u64(0); + pack_unpack_u64(1); + pack_unpack_u64(2); + pack_unpack_u64(3); + pack_unpack_u64(4); + pack_unpack_u64(5); + pack_unpack_u64(6); + pack_unpack_u64(7); + pack_unpack_u64(8); + pack_unpack_u64(9); + pack_unpack_u64(10); + pack_unpack_u64(11); + pack_unpack_u64(12); + pack_unpack_u64(13); + pack_unpack_u64(14); + pack_unpack_u64(15); + pack_unpack_u64(16); + pack_unpack_u64(17); + pack_unpack_u64(18); + pack_unpack_u64(19); + pack_unpack_u64(20); + pack_unpack_u64(21); + pack_unpack_u64(22); + pack_unpack_u64(23); + pack_unpack_u64(24); + pack_unpack_u64(25); + pack_unpack_u64(26); + pack_unpack_u64(27); + pack_unpack_u64(28); + pack_unpack_u64(29); + pack_unpack_u64(30); + pack_unpack_u64(31); + pack_unpack_u64(32); + pack_unpack_u64(33); + pack_unpack_u64(34); + pack_unpack_u64(35); + pack_unpack_u64(36); + pack_unpack_u64(37); + pack_unpack_u64(38); + pack_unpack_u64(39); + pack_unpack_u64(40); + pack_unpack_u64(41); + pack_unpack_u64(42); + pack_unpack_u64(43); + pack_unpack_u64(44); + pack_unpack_u64(45); + pack_unpack_u64(46); + pack_unpack_u64(47); + pack_unpack_u64(48); + pack_unpack_u64(49); + pack_unpack_u64(50); + pack_unpack_u64(51); + pack_unpack_u64(52); + pack_unpack_u64(53); + pack_unpack_u64(54); + pack_unpack_u64(55); + pack_unpack_u64(56); + pack_unpack_u64(57); + pack_unpack_u64(58); + pack_unpack_u64(59); + pack_unpack_u64(60); + pack_unpack_u64(61); + pack_unpack_u64(62); + pack_unpack_u64(63); + pack_unpack_u64(64); + } + #[test] fn test_unchecked_pack() { let input = array::from_fn(|i| i as u32); @@ -1743,4 +1965,4 @@ mod test { unsafe { BitPacking::unchecked_unpack(10, &packed, &mut output) }; assert_eq!(input, output); } -} +} \ No newline at end of file From 1c2878b8d0f201cb9e8a0e81d221326c421fa9a5 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Wed, 25 Sep 2024 17:54:05 +0000 Subject: [PATCH 27/31] add test_unchecked_pack --- .../src/compression_algo/fastlanes.rs | 240 +++++++++++++++--- 1 file changed, 205 insertions(+), 35 deletions(-) diff --git a/rust/lance-encoding/src/compression_algo/fastlanes.rs b/rust/lance-encoding/src/compression_algo/fastlanes.rs index 0df159d2b7..9f9d1dd8ab 100644 --- a/rust/lance-encoding/src/compression_algo/fastlanes.rs +++ b/rust/lance-encoding/src/compression_algo/fastlanes.rs @@ -21,28 +21,6 @@ impl FastLanes for u16 {} impl FastLanes for u32 {} impl FastLanes for u64 {} -macro_rules! iterate { - ($T:ty, $lane: expr, | $_1:tt $idx:ident | $($body:tt)*) => { - macro_rules! __kernel__ {( $_1 $idx:ident ) => ( $($body)* )} - { - use $crate::{seq_t, FL_ORDER}; - use paste::paste; - - #[inline(always)] - fn index(row: usize, lane: usize) -> usize { - let o = row / 8; - let s = row % 8; - (FL_ORDER[o] * 16) + (s * 128) + lane - } - - paste!(seq_t!(row in $T { - let idx = index(row, $lane); - __kernel__!(idx); - })); - } - } -} - macro_rules! pack { ($T:ty, $W:expr, $packed:expr, $lane:expr, | $_1:tt $idx:ident | $($body:tt)*) => { macro_rules! __kernel__ {( $_1 $idx:ident ) => ( $($body)* )} @@ -1703,12 +1681,12 @@ mod test { pub struct XorShift { state: u64, } - + impl XorShift { pub fn new(seed: u64) -> Self { - XorShift { state: seed } + Self { state: seed } } - + pub fn next(&mut self) -> u64 { let mut x = self.state; x ^= x << 13; @@ -1719,11 +1697,13 @@ mod test { } } + // a macro version of this function generalize u8, u16, u32, u64 takes very long time for a test build, so I + // write it for each type separately fn pack_unpack_u8(bit_width: usize) { let mut values: [u8; 1024] = [0; 1024]; let mut rng = XorShift::new(123456789); - for i in 0..1024 { - values[i] = (rng.next() % (1 << bit_width)) as u8; + for value in &mut values { + *value = (rng.next() % (1 << bit_width)) as u8; } let mut packed = vec![0; 1024 * bit_width / 8]; @@ -1748,8 +1728,8 @@ mod test { fn pack_unpack_u16(bit_width: usize) { let mut values: [u16; 1024] = [0; 1024]; let mut rng = XorShift::new(123456789); - for i in 0..1024 { - values[i] = (rng.next() % (1 << bit_width)) as u16; + for value in &mut values { + *value = (rng.next() % (1 << bit_width)) as u16; } let mut packed = vec![0; 1024 * bit_width / 16]; @@ -1774,8 +1754,8 @@ mod test { fn pack_unpack_u32(bit_width: usize) { let mut values: [u32; 1024] = [0; 1024]; let mut rng = XorShift::new(123456789); - for i in 0..1024 { - values[i] = (rng.next() % (1 << bit_width)) as u32; + for value in &mut values { + *value = (rng.next() % (1 << bit_width)) as u32; } let mut packed = vec![0; 1024 * bit_width / 32]; @@ -1800,9 +1780,13 @@ mod test { fn pack_unpack_u64(bit_width: usize) { let mut values: [u64; 1024] = [0; 1024]; let mut rng = XorShift::new(123456789); - if bit_width > 0 { - for i in 0..1024 { - values[i] = (rng.next() % (1 << (bit_width - 1))) as u64; + if bit_width == 64 { + for value in &mut values { + *value = rng.next(); + } + } else { + for value in &mut values { + *value = rng.next() % (1 << bit_width); } } @@ -1956,6 +1940,68 @@ mod test { pack_unpack_u64(64); } + fn unchecked_pack_unpack_u8(bit_width: usize) { + let mut values = [0u8; 1024]; + let mut rng = XorShift::new(123456789); + for value in &mut values { + *value = (rng.next() % (1 << bit_width)) as u8; + } + let mut packed = vec![0; 1024 * bit_width / 8]; + unsafe { + BitPacking::unchecked_pack(bit_width, &values, &mut packed); + } + let mut output = [0; 1024]; + unsafe { BitPacking::unchecked_unpack(bit_width, &packed, &mut output) }; + assert_eq!(values, output); + } + + fn unchecked_pack_unpack_u16(bit_width: usize) { + let mut values = [0u16; 1024]; + let mut rng = XorShift::new(123456789); + for value in &mut values { + *value = (rng.next() % (1 << bit_width)) as u16; + } + let mut packed = vec![0; 1024 * bit_width / u16::T]; + unsafe { + BitPacking::unchecked_pack(bit_width, &values, &mut packed); + } + let mut output = [0; 1024]; + unsafe { BitPacking::unchecked_unpack(bit_width, &packed, &mut output) }; + assert_eq!(values, output); + } + + fn unchecked_pack_unpack_u32(bit_width: usize) { + let mut values = [0u32; 1024]; + let mut rng = XorShift::new(123456789); + for value in &mut values { + *value = (rng.next() % (1 << bit_width)) as u32; + } + let mut packed = vec![0; 1024 * bit_width / u32::T]; + unsafe { + BitPacking::unchecked_pack(bit_width, &values, &mut packed); + } + let mut output = [0; 1024]; + unsafe { BitPacking::unchecked_unpack(bit_width, &packed, &mut output) }; + assert_eq!(values, output); + } + + fn unchecked_pack_unpack_u64(bit_width: usize) { + let mut values = [0u64; 1024]; + let mut rng = XorShift::new(123456789); + if bit_width == 64 { + for value in &mut values { + *value = rng.next(); + } + } + let mut packed = vec![0; 1024 * bit_width / u64::T]; + unsafe { + BitPacking::unchecked_pack(bit_width, &values, &mut packed); + } + let mut output = [0; 1024]; + unsafe { BitPacking::unchecked_unpack(bit_width, &packed, &mut output) }; + assert_eq!(values, output); + } + #[test] fn test_unchecked_pack() { let input = array::from_fn(|i| i as u32); @@ -1964,5 +2010,129 @@ mod test { let mut output = [0; 1024]; unsafe { BitPacking::unchecked_unpack(10, &packed, &mut output) }; assert_eq!(input, output); + + unchecked_pack_unpack_u8(1); + unchecked_pack_unpack_u8(2); + unchecked_pack_unpack_u8(3); + unchecked_pack_unpack_u8(4); + unchecked_pack_unpack_u8(5); + unchecked_pack_unpack_u8(6); + unchecked_pack_unpack_u8(7); + unchecked_pack_unpack_u8(8); + + unchecked_pack_unpack_u16(1); + unchecked_pack_unpack_u16(2); + unchecked_pack_unpack_u16(3); + unchecked_pack_unpack_u16(4); + unchecked_pack_unpack_u16(5); + unchecked_pack_unpack_u16(6); + unchecked_pack_unpack_u16(7); + unchecked_pack_unpack_u16(8); + unchecked_pack_unpack_u16(9); + unchecked_pack_unpack_u16(10); + unchecked_pack_unpack_u16(11); + unchecked_pack_unpack_u16(12); + unchecked_pack_unpack_u16(13); + unchecked_pack_unpack_u16(14); + unchecked_pack_unpack_u16(15); + unchecked_pack_unpack_u16(16); + + unchecked_pack_unpack_u32(1); + unchecked_pack_unpack_u32(2); + unchecked_pack_unpack_u32(3); + unchecked_pack_unpack_u32(4); + unchecked_pack_unpack_u32(5); + unchecked_pack_unpack_u32(6); + unchecked_pack_unpack_u32(7); + unchecked_pack_unpack_u32(8); + unchecked_pack_unpack_u32(9); + unchecked_pack_unpack_u32(10); + unchecked_pack_unpack_u32(11); + unchecked_pack_unpack_u32(12); + unchecked_pack_unpack_u32(13); + unchecked_pack_unpack_u32(14); + unchecked_pack_unpack_u32(15); + unchecked_pack_unpack_u32(16); + unchecked_pack_unpack_u32(17); + unchecked_pack_unpack_u32(18); + unchecked_pack_unpack_u32(19); + unchecked_pack_unpack_u32(20); + unchecked_pack_unpack_u32(21); + unchecked_pack_unpack_u32(22); + unchecked_pack_unpack_u32(23); + unchecked_pack_unpack_u32(24); + unchecked_pack_unpack_u32(25); + unchecked_pack_unpack_u32(26); + unchecked_pack_unpack_u32(27); + unchecked_pack_unpack_u32(28); + unchecked_pack_unpack_u32(29); + unchecked_pack_unpack_u32(30); + unchecked_pack_unpack_u32(31); + unchecked_pack_unpack_u32(32); + + unchecked_pack_unpack_u64(1); + unchecked_pack_unpack_u64(2); + unchecked_pack_unpack_u64(3); + unchecked_pack_unpack_u64(4); + unchecked_pack_unpack_u64(5); + unchecked_pack_unpack_u64(6); + unchecked_pack_unpack_u64(7); + unchecked_pack_unpack_u64(8); + unchecked_pack_unpack_u64(9); + unchecked_pack_unpack_u64(10); + unchecked_pack_unpack_u64(11); + unchecked_pack_unpack_u64(12); + unchecked_pack_unpack_u64(13); + unchecked_pack_unpack_u64(14); + unchecked_pack_unpack_u64(15); + unchecked_pack_unpack_u64(16); + unchecked_pack_unpack_u64(17); + unchecked_pack_unpack_u64(18); + unchecked_pack_unpack_u64(19); + unchecked_pack_unpack_u64(20); + unchecked_pack_unpack_u64(21); + unchecked_pack_unpack_u64(22); + unchecked_pack_unpack_u64(23); + unchecked_pack_unpack_u64(24); + unchecked_pack_unpack_u64(25); + unchecked_pack_unpack_u64(26); + unchecked_pack_unpack_u64(27); + unchecked_pack_unpack_u64(28); + unchecked_pack_unpack_u64(29); + unchecked_pack_unpack_u64(30); + unchecked_pack_unpack_u64(31); + unchecked_pack_unpack_u64(32); + unchecked_pack_unpack_u64(33); + unchecked_pack_unpack_u64(34); + unchecked_pack_unpack_u64(35); + unchecked_pack_unpack_u64(36); + unchecked_pack_unpack_u64(37); + unchecked_pack_unpack_u64(38); + unchecked_pack_unpack_u64(39); + unchecked_pack_unpack_u64(40); + unchecked_pack_unpack_u64(41); + unchecked_pack_unpack_u64(42); + unchecked_pack_unpack_u64(43); + unchecked_pack_unpack_u64(44); + unchecked_pack_unpack_u64(45); + unchecked_pack_unpack_u64(46); + unchecked_pack_unpack_u64(47); + unchecked_pack_unpack_u64(48); + unchecked_pack_unpack_u64(49); + unchecked_pack_unpack_u64(50); + unchecked_pack_unpack_u64(51); + unchecked_pack_unpack_u64(52); + unchecked_pack_unpack_u64(53); + unchecked_pack_unpack_u64(54); + unchecked_pack_unpack_u64(55); + unchecked_pack_unpack_u64(56); + unchecked_pack_unpack_u64(57); + unchecked_pack_unpack_u64(58); + unchecked_pack_unpack_u64(59); + unchecked_pack_unpack_u64(60); + unchecked_pack_unpack_u64(61); + unchecked_pack_unpack_u64(62); + unchecked_pack_unpack_u64(63); + unchecked_pack_unpack_u64(64); } -} \ No newline at end of file +} From 688bb1fd03751aae6b165e92c17ca92297fb14e7 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Wed, 25 Sep 2024 20:23:07 +0000 Subject: [PATCH 28/31] address PR comments --- rust/lance-encoding/src/buffer.rs | 19 +++++++++++-------- .../src/compression_algo/fastlanes.rs | 6 +++++- rust/lance-encoding/src/encoder.rs | 5 +++-- .../encodings/physical/bitpack_fastlanes.rs | 18 ++++++++++-------- 4 files changed, 29 insertions(+), 19 deletions(-) diff --git a/rust/lance-encoding/src/buffer.rs b/rust/lance-encoding/src/buffer.rs index d7b15bdc71..542338d9a8 100644 --- a/rust/lance-encoding/src/buffer.rs +++ b/rust/lance-encoding/src/buffer.rs @@ -284,26 +284,29 @@ impl LanceBuffer { Self::Owned(Vec::from(array)) } + #[allow(clippy::len_without_is_empty)] + pub fn len(&self) -> usize { + match self { + Self::Borrowed(buffer) => buffer.len(), + Self::Owned(buffer) => buffer.len(), + } + } + /// Returns a new [LanceBuffer] that is a slice of this buffer starting at `offset`, /// with `length` bytes. /// Doing so allows the same memory region to be shared between lance buffers. /// # Panics /// Panics if `(offset + length)` is larger than the existing length. + /// If the buffer is owned this method will require a copy. pub fn slice_with_length(&self, offset: usize, length: usize) -> Self { - let original_buffer_len = match self { - Self::Borrowed(buffer) => buffer.len(), - Self::Owned(buffer) => buffer.len(), - }; + let original_buffer_len = self.len(); assert!( offset.saturating_add(length) <= original_buffer_len, "the offset of the new Buffer cannot exceed the existing length" ); match self { Self::Borrowed(buffer) => Self::Borrowed(buffer.slice_with_length(offset, length)), - // A copy happened during `Buffer::from_slice_ref` - Self::Owned(buffer) => { - Self::Borrowed(Buffer::from_slice_ref(&buffer[offset..offset + length])) - } + Self::Owned(buffer) => Self::Owned(buffer[offset..offset + length].to_vec()), } } } diff --git a/rust/lance-encoding/src/compression_algo/fastlanes.rs b/rust/lance-encoding/src/compression_algo/fastlanes.rs index 9f9d1dd8ab..daeb2600c4 100644 --- a/rust/lance-encoding/src/compression_algo/fastlanes.rs +++ b/rust/lance-encoding/src/compression_algo/fastlanes.rs @@ -1,8 +1,12 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright The Lance Authors -// NOTICE: This file is a modification of the `fastlanes` crate: https://github.com/spiraldb/fastlanes +// NOTICE: +// This file is a modification of the `fastlanes` crate: https://github.com/spiraldb/fastlanes // It is modified to allow a rust stable build +// The original code can be accessed at https://github.com/spiraldb/fastlanes/blob/8e0ff374f815... +// The original code is licensed under the Apache Software License: +// https://github.com/spiraldb/fastlanes/blob/8e0ff374f815d919d0c0ebdccf5ffd9e6dc7d663/LICENSE use arrayref::{array_mut_ref, array_ref}; use core::mem::size_of; diff --git a/rust/lance-encoding/src/encoder.rs b/rust/lance-encoding/src/encoder.rs index 3731cc2902..eb264c8fac 100644 --- a/rust/lance-encoding/src/encoder.rs +++ b/rust/lance-encoding/src/encoder.rs @@ -363,8 +363,9 @@ impl CoreArrayEncodingStrategy { } } - // for signed integers, I intend to make it a cascaded encoding, a sparse array for the negative values and very wide(bit-width) values, - // then a bitpacked array for the narrow(bit-width) values, I need `BitpackedForNeg` to be merged first + // TODO: for signed integers, I intend to make it a cascaded encoding, a sparse array for the negative values and very wide(bit-width) values, + // then a bitpacked array for the narrow(bit-width) values, I need `BitpackedForNeg` to be merged first, I am + // thinking about putting this sparse array in the metadata so bitpacking remain using one page buffer only. DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64 => { if version >= LanceFileVersion::V2_1 && arrays[0].data_type() == data_type { let compressed_bit_width = compute_compressed_bit_width_for_non_neg(arrays); diff --git a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs index 1ced7f61f5..cbd3499acc 100644 --- a/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs +++ b/rust/lance-encoding/src/encodings/physical/bitpack_fastlanes.rs @@ -30,10 +30,7 @@ const ELEMS_PER_CHUNK: u64 = 1024; // todo: compute all statistics before encoding // todo: see how to use rust macro to rewrite this function pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { - // is it possible to get here? - if arrays.is_empty() { - return 0; - } + debug_assert!(!arrays.is_empty()); let res; @@ -192,6 +189,13 @@ pub fn compute_compressed_bit_width_for_non_neg(arrays: &[ArrayRef]) -> u64 { res } +// Bitpack integers using fastlanes algorithm, the input is sliced into chunks of 1024 integers, and bitpacked +// chunk by chunk. when the input is not a multiple of 1024, the last chunk is padded with zeros, this is fine because +// we also know the number of rows we have. +// Here self is a borrow of BitpackedForNonNegArrayEncoder, unpacked is a mutable borrow of FixedWidthDataBlock, +// data_type can be one of u8, u16, u32, or u64. +// buffer_index is a mutable borrow of u32, indicating the buffer index of the output EncodedArray. +// It outputs an fastlanes bitpacked EncodedArray macro_rules! encode_fixed_width { ($self:expr, $unpacked:expr, $data_type:ty, $buffer_index:expr) => {{ let num_chunks = ($unpacked.num_values + ELEMS_PER_CHUNK - 1) / ELEMS_PER_CHUNK; @@ -382,10 +386,8 @@ impl PageScheduler for BitpackedForNonNegScheduler { scheduler: &Arc, top_level_row: u64, ) -> BoxFuture<'static, Result>> { - // can we get here? - if ranges.is_empty() { - panic!("cannot schedule empty ranges"); - } + assert!(!ranges.is_empty()); + let mut byte_ranges = vec![]; // map one bytes to multiple ranges, one bytes has at least one range corresponding to it From 4d7557fc7792f373175a21da6250c7342d505b09 Mon Sep 17 00:00:00 2001 From: broccoliSpicy <93440049+broccoliSpicy@users.noreply.github.com> Date: Wed, 25 Sep 2024 16:34:42 -0400 Subject: [PATCH 29/31] Update rust/lance-encoding/src/buffer.rs Co-authored-by: Weston Pace --- rust/lance-encoding/src/buffer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/lance-encoding/src/buffer.rs b/rust/lance-encoding/src/buffer.rs index 542338d9a8..57255565bc 100644 --- a/rust/lance-encoding/src/buffer.rs +++ b/rust/lance-encoding/src/buffer.rs @@ -302,7 +302,7 @@ impl LanceBuffer { let original_buffer_len = self.len(); assert!( offset.saturating_add(length) <= original_buffer_len, - "the offset of the new Buffer cannot exceed the existing length" + "the offset + length of the sliced Buffer cannot exceed the existing length" ); match self { Self::Borrowed(buffer) => Self::Borrowed(buffer.slice_with_length(offset, length)), From dedb3065cf6343e75748c1620b88a8e21078cd72 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Fri, 27 Sep 2024 19:32:12 +0000 Subject: [PATCH 30/31] fix fastlanes original code link --- rust/lance-encoding/src/compression_algo/fastlanes.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/rust/lance-encoding/src/compression_algo/fastlanes.rs b/rust/lance-encoding/src/compression_algo/fastlanes.rs index daeb2600c4..dd5f4f26d7 100644 --- a/rust/lance-encoding/src/compression_algo/fastlanes.rs +++ b/rust/lance-encoding/src/compression_algo/fastlanes.rs @@ -4,7 +4,12 @@ // NOTICE: // This file is a modification of the `fastlanes` crate: https://github.com/spiraldb/fastlanes // It is modified to allow a rust stable build -// The original code can be accessed at https://github.com/spiraldb/fastlanes/blob/8e0ff374f815... +// +// The original code can be accessed at +// https://github.com/spiraldb/fastlanes/blob/8e0ff374f815d919d0c0ebdccf5ffd9e6dc7d663/src/bitpacking.rs +// https://github.com/spiraldb/fastlanes/blob/8e0ff374f815d919d0c0ebdccf5ffd9e6dc7d663/src/lib.rs +// https://github.com/spiraldb/fastlanes/blob/8e0ff374f815d919d0c0ebdccf5ffd9e6dc7d663/src/macros.rs +// // The original code is licensed under the Apache Software License: // https://github.com/spiraldb/fastlanes/blob/8e0ff374f815d919d0c0ebdccf5ffd9e6dc7d663/LICENSE From 655a06316e8fe6a35414bf5f6397fb465d90b518 Mon Sep 17 00:00:00 2001 From: broccoliSpicy Date: Fri, 27 Sep 2024 19:34:43 +0000 Subject: [PATCH 31/31] lint --- rust/lance-encoding/src/compression_algo/fastlanes.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rust/lance-encoding/src/compression_algo/fastlanes.rs b/rust/lance-encoding/src/compression_algo/fastlanes.rs index dd5f4f26d7..346c7ab219 100644 --- a/rust/lance-encoding/src/compression_algo/fastlanes.rs +++ b/rust/lance-encoding/src/compression_algo/fastlanes.rs @@ -4,8 +4,8 @@ // NOTICE: // This file is a modification of the `fastlanes` crate: https://github.com/spiraldb/fastlanes // It is modified to allow a rust stable build -// -// The original code can be accessed at +// +// The original code can be accessed at // https://github.com/spiraldb/fastlanes/blob/8e0ff374f815d919d0c0ebdccf5ffd9e6dc7d663/src/bitpacking.rs // https://github.com/spiraldb/fastlanes/blob/8e0ff374f815d919d0c0ebdccf5ffd9e6dc7d663/src/lib.rs // https://github.com/spiraldb/fastlanes/blob/8e0ff374f815d919d0c0ebdccf5ffd9e6dc7d663/src/macros.rs