diff --git a/columnar/Cargo.toml b/columnar/Cargo.toml index b67c4cbfd7..fedb252f68 100644 --- a/columnar/Cargo.toml +++ b/columnar/Cargo.toml @@ -11,8 +11,22 @@ thiserror = "1" fnv = "1" sstable = { path = "../sstable", package = "tantivy-sstable" } common = { path = "../common", package = "tantivy-common" } -fastfield_codecs = { path = "../fastfield_codecs"} itertools = "0.10" +log = "0.4" +tantivy-bitpacker = { version= "0.3", path = "../bitpacker/" } +prettytable-rs = {version="0.10.0", optional= true} +rand = {version="0.8.3", optional= true} +fastdivide = "0.4" +measure_time = { version="0.8.2", optional=true} [dev-dependencies] proptest = "1" +more-asserts = "0.3.0" +rand = "0.8.3" + +# temporary +[workspace] +members = [] + +[features] +unstable = [] diff --git a/columnar/README.md b/columnar/README.md index 2c875415ec..8b42df0867 100644 --- a/columnar/README.md +++ b/columnar/README.md @@ -6,13 +6,12 @@ This crate describes columnar format used in tantivy. This format is special in the following way. - it needs to be compact -- it does not required to be loaded in memory. -- it is designed to fit well with quickwit's strange constraint: -we need to be able to load columns rapidly. +- accessing a specific column does not require to load the entire columnar. It can be done in 2 to 3 random access. - columns of several types can be associated with the same column name. - it needs to support columns with different types `(str, u64, i64, f64)` and different cardinality `(required, optional, multivalued)`. - columns, once loaded, offer cheap random access. +- it is designed to allow range queries. # Coercion rules @@ -65,3 +64,46 @@ be done by listing all keys prefixed by The associated range of bytes refer to a range of bytes +This crate exposes a columnar format for tantivy. +This format is described in README.md + + +The crate introduces the following concepts. + +`Columnar` is an equivalent of a dataframe. +It maps `column_key` to `Column`. + +A `Column` asssociates a `RowId` (u32) to any +number of values. + +This is made possible by wrapping a `ColumnIndex` and a `ColumnValue` object. +The `ColumnValue` represents a mapping that associates each `RowId` to +exactly one single value. + +The `ColumnIndex` then maps each RowId to a set of `RowId` in the +`ColumnValue`. + +For optimization, and compression purposes, the `ColumnIndex` has three +possible representation, each for different cardinalities. + +- Full + +All RowId have exactly one value. The ColumnIndex is the trivial mapping. + +- Optional + +All RowIds can have at most one value. The ColumnIndex is the trivial mapping `ColumnRowId -> Option`. + +- Multivalued + +All RowIds can have any number of values. +The column index is mapping values to a range. + + +All these objects are implemented an unit tested independently +in their own module: + +- columnar +- column_index +- column_values +- column diff --git a/columnar/src/TODO.md b/columnar/src/TODO.md new file mode 100644 index 0000000000..ec1aca931e --- /dev/null +++ b/columnar/src/TODO.md @@ -0,0 +1,45 @@ +# zero to one +* merges +* full still needs a num_values +* replug u128 +* add dictionary encoded stuff +* fix multivalued +* find a way to make columnar work with strict types +* plug to tantivy + - indexing + - aggregations + - merge + +# Perf and Size +* re-add ZSTD compression for dictionaries +no systematic monotonic mapping +consider removing multilinear +f32? +adhoc solution for bool? + +add metrics helper for aggregate. sum(row_id) +review inline absence/presence +improv perf of select using PDEP +compare with roaring bitmap/elias fano etc etc. +SIMD range? (see blog post) +Add alignment? +Consider another codec to bridge the gap between few and 5k elements + +# Cleanup and rationalization +in benchmark, unify percent vs ratio, f32 vs f64. +investigate if should have better errors? io::Error is overused at the moment. +rename rank/select in unit tests +Review the public API via cargo doc +go through TODOs +remove all doc_id occurences -> row_id +use the rank & select naming in unit tests branch. +multi-linear -> blockwise +linear codec -> simply a multiplication for the index column + +# Other +fix enhance column-cli + +# Santa claus + +autodetect datetime ipaddr, plug customizable tokenizer. + diff --git a/columnar/src/column/dictionary_encoded.rs b/columnar/src/column/dictionary_encoded.rs new file mode 100644 index 0000000000..586ddc0431 --- /dev/null +++ b/columnar/src/column/dictionary_encoded.rs @@ -0,0 +1,40 @@ +use std::io; +use std::ops::Deref; +use std::sync::Arc; + +use sstable::{Dictionary, VoidSSTable}; + +use crate::column::Column; +use crate::column_index::ColumnIndex; + +/// Dictionary encoded column. +#[derive(Clone)] +pub struct BytesColumn { + pub(crate) dictionary: Arc>, + pub(crate) term_ord_column: Column, +} + +impl BytesColumn { + /// Returns `false` if the term does not exist (e.g. `term_ord` is greater or equal to the + /// overll number of terms). + pub fn term_ord_to_str(&self, term_ord: u64, output: &mut Vec) -> io::Result { + self.dictionary.ord_to_term(term_ord, output) + } + + pub fn term_ords(&self) -> &Column { + &self.term_ord_column + } +} + +impl Deref for BytesColumn { + type Target = ColumnIndex<'static>; + + fn deref(&self) -> &Self::Target { + &**self.term_ords() + } +} + +#[cfg(test)] +mod tests { + use crate::{ColumnarReader, ColumnarWriter}; +} diff --git a/columnar/src/column/mod.rs b/columnar/src/column/mod.rs new file mode 100644 index 0000000000..1c8830c08d --- /dev/null +++ b/columnar/src/column/mod.rs @@ -0,0 +1,56 @@ +mod dictionary_encoded; +mod serialize; + +use std::ops::Deref; +use std::sync::Arc; + +use common::BinarySerializable; +pub use dictionary_encoded::BytesColumn; +pub use serialize::{open_column_bytes, open_column_u64, serialize_column_u64}; + +use crate::column_index::ColumnIndex; +use crate::column_values::ColumnValues; +use crate::{Cardinality, RowId}; + +#[derive(Clone)] +pub struct Column { + pub idx: ColumnIndex<'static>, + pub values: Arc>, +} + +use crate::column_index::Set; + +impl Column { + pub fn first(&self, row_id: RowId) -> Option { + match &self.idx { + ColumnIndex::Full => Some(self.values.get_val(row_id)), + ColumnIndex::Optional(opt_idx) => { + let value_row_idx = opt_idx.rank_if_exists(row_id)?; + Some(self.values.get_val(value_row_idx)) + } + ColumnIndex::Multivalued(_multivalued_index) => { + todo!(); + } + } + } +} + +impl Deref for Column { + type Target = ColumnIndex<'static>; + + fn deref(&self) -> &Self::Target { + &self.idx + } +} + +impl BinarySerializable for Cardinality { + fn serialize(&self, writer: &mut W) -> std::io::Result<()> { + self.to_code().serialize(writer) + } + + fn deserialize(reader: &mut R) -> std::io::Result { + let cardinality_code = u8::deserialize(reader)?; + let cardinality = Cardinality::try_from_code(cardinality_code)?; + Ok(cardinality) + } +} diff --git a/columnar/src/column/serialize.rs b/columnar/src/column/serialize.rs new file mode 100644 index 0000000000..79adce6d62 --- /dev/null +++ b/columnar/src/column/serialize.rs @@ -0,0 +1,54 @@ +use std::io; +use std::io::Write; +use std::sync::Arc; + +use common::{CountingWriter, OwnedBytes}; +use sstable::Dictionary; + +use crate::column::{BytesColumn, Column}; +use crate::column_index::{serialize_column_index, SerializableColumnIndex}; +use crate::column_values::{ + serialize_column_values, ColumnValues, MonotonicallyMappableToU64, ALL_CODEC_TYPES, +}; +pub fn serialize_column_u64( + column_index: SerializableColumnIndex<'_>, + column_values: &impl ColumnValues, + output: &mut impl Write, +) -> io::Result<()> { + let mut counting_writer = CountingWriter::wrap(output); + serialize_column_index(column_index, &mut counting_writer)?; + let column_index_num_bytes = counting_writer.written_bytes() as u32; + let output = counting_writer.finish(); + serialize_column_values(column_values, &ALL_CODEC_TYPES[..], output)?; + output.write_all(&column_index_num_bytes.to_le_bytes())?; + Ok(()) +} + +pub fn open_column_u64(bytes: OwnedBytes) -> io::Result> { + let (body, column_index_num_bytes_payload) = bytes.rsplit(4); + let column_index_num_bytes = u32::from_le_bytes( + column_index_num_bytes_payload + .as_slice() + .try_into() + .unwrap(), + ); + let (column_index_data, column_values_data) = body.split(column_index_num_bytes as usize); + let column_index = crate::column_index::open_column_index(column_index_data)?; + let column_values = crate::column_values::open_u64_mapped(column_values_data)?; + Ok(Column { + idx: column_index, + values: column_values, + }) +} + +pub fn open_column_bytes(data: OwnedBytes) -> io::Result { + let (body, dictionary_len_bytes) = data.rsplit(4); + let dictionary_len = u32::from_le_bytes(dictionary_len_bytes.as_slice().try_into().unwrap()); + let (dictionary_bytes, column_bytes) = body.split(dictionary_len as usize); + let dictionary = Arc::new(Dictionary::from_bytes(dictionary_bytes)?); + let term_ord_column = crate::column::open_column_u64::(column_bytes)?; + Ok(BytesColumn { + dictionary, + term_ord_column, + }) +} diff --git a/columnar/src/column_index/mod.rs b/columnar/src/column_index/mod.rs new file mode 100644 index 0000000000..64a7092e7a --- /dev/null +++ b/columnar/src/column_index/mod.rs @@ -0,0 +1,40 @@ +mod multivalued_index; +mod optional_index; +mod serialize; + +use std::sync::Arc; + +pub use optional_index::{OptionalIndex, SerializableOptionalIndex, Set}; +pub use serialize::{open_column_index, serialize_column_index, SerializableColumnIndex}; + +use crate::column_values::ColumnValues; +use crate::{Cardinality, RowId}; + +#[derive(Clone)] +pub enum ColumnIndex<'a> { + Full, + Optional(OptionalIndex), + // TODO remove the Arc apart from serialization this is not + // dynamic at all. + Multivalued(Arc + 'a>), +} + +impl<'a> ColumnIndex<'a> { + pub fn get_cardinality(&self) -> Cardinality { + match self { + ColumnIndex::Full => Cardinality::Full, + ColumnIndex::Optional(_) => Cardinality::Optional, + ColumnIndex::Multivalued(_) => Cardinality::Multivalued, + } + } + + pub fn num_rows(&self) -> RowId { + match self { + ColumnIndex::Full => { + todo!() + } + ColumnIndex::Optional(optional_index) => optional_index.num_rows(), + ColumnIndex::Multivalued(multivalued_index) => multivalued_index.num_vals() - 1, + } + } +} diff --git a/columnar/src/column_index/multivalued_index.rs b/columnar/src/column_index/multivalued_index.rs new file mode 100644 index 0000000000..de5c5bb589 --- /dev/null +++ b/columnar/src/column_index/multivalued_index.rs @@ -0,0 +1,27 @@ +use std::io; +use std::io::Write; +use std::sync::Arc; + +use common::OwnedBytes; + +use crate::column_values::{ColumnValues, FastFieldCodecType}; +use crate::RowId; + +#[derive(Clone)] +pub struct MultivaluedIndex(Arc>); + +pub fn serialize_multivalued_index( + multivalued_index: MultivaluedIndex, + output: &mut impl Write, +) -> io::Result<()> { + crate::column_values::serialize_column_values( + &*multivalued_index.0, + &[FastFieldCodecType::Bitpacked, FastFieldCodecType::Linear], + output, + )?; + Ok(()) +} + +pub fn open_multivalued_index(bytes: OwnedBytes) -> io::Result>> { + todo!(); +} diff --git a/columnar/src/column_index/optional_index/mod.rs b/columnar/src/column_index/optional_index/mod.rs new file mode 100644 index 0000000000..615e2a8373 --- /dev/null +++ b/columnar/src/column_index/optional_index/mod.rs @@ -0,0 +1,453 @@ +use std::io::{self, Write}; +use std::ops::Range; +use std::sync::Arc; + +mod set; +mod set_block; + +use common::{BinarySerializable, GroupByIteratorExtended, OwnedBytes, VInt}; +pub use set::{Set, SetCodec}; +use set_block::{ + DenseBlock, DenseBlockCodec, SparseBlock, SparseBlockCodec, DENSE_BLOCK_NUM_BYTES, +}; + +use crate::{InvalidData, RowId}; + +/// The threshold for for number of elements after which we switch to dense block encoding. +/// +/// We simply pick the value that minimize the size of the blocks. +const DENSE_BLOCK_THRESHOLD: u32 = + set_block::DENSE_BLOCK_NUM_BYTES / std::mem::size_of::() as u32; //< 5_120 + +const ELEMENTS_PER_BLOCK: u32 = u16::MAX as u32 + 1; + +const BLOCK_SIZE: RowId = 1 << 16; + +#[derive(Copy, Clone, Debug)] +struct BlockMeta { + non_null_rows_before_block: u32, + start_byte_offset: u32, + block_variant: BlockVariant, +} + +#[derive(Clone, Copy, Debug)] +enum BlockVariant { + Dense, + Sparse { num_vals: u16 }, +} + +impl BlockVariant { + pub fn empty() -> Self { + Self::Sparse { num_vals: 0 } + } + pub fn num_bytes_in_block(&self) -> u32 { + match *self { + BlockVariant::Dense => set_block::DENSE_BLOCK_NUM_BYTES, + BlockVariant::Sparse { num_vals } => num_vals as u32 * 2, + } + } +} + +/// This codec is inspired by roaring bitmaps. +/// In the dense blocks, however, in order to accelerate `select` +/// we interleave an offset over two bytes. (more on this lower) +/// +/// The lower 16 bits of doc ids are stored as u16 while the upper 16 bits are given by the block +/// id. Each block contains 1<<16 docids. +/// +/// # Serialized Data Layout +/// The data starts with the block data. Each block is either dense or sparse encoded, depending on +/// the number of values in the block. A block is sparse when it contains less than +/// DENSE_BLOCK_THRESHOLD (6144) values. +/// [Sparse data block | dense data block, .. #repeat*; Desc: Either a sparse or dense encoded +/// block] +/// ### Sparse block data +/// [u16 LE, .. #repeat*; Desc: Positions with values in a block] +/// ### Dense block data +/// [Dense codec for the whole block; Desc: Similar to a bitvec(0..ELEMENTS_PER_BLOCK) + Metadata +/// for faster lookups. See dense.rs] +/// +/// The data is followed by block metadata, to know which area of the raw block data belongs to +/// which block. Only metadata for blocks with elements is recorded to +/// keep the overhead low for scenarios with many very sparse columns. The block metadata consists +/// of the block index and the number of values in the block. Since we don't store empty blocks +/// num_vals is incremented by 1, e.g. 0 means 1 value. +/// +/// The last u16 is storing the number of metadata blocks. +/// [u16 LE, .. #repeat*; Desc: Positions with values in a block][(u16 LE, u16 LE), .. #repeat*; +/// Desc: (Block Id u16, Num Elements u16)][u16 LE; Desc: num blocks with values u16] +/// +/// # Opening +/// When opening the data layout, the data is expanded to `Vec`, where the +/// index is the block index. For each block `byte_start` and `offset` is computed. +#[derive(Clone)] +pub struct OptionalIndex { + num_rows: RowId, + num_non_null_rows: RowId, + block_data: OwnedBytes, + block_metas: Arc<[BlockMeta]>, +} + +impl OptionalIndex { + pub fn num_rows(&self) -> RowId { + self.num_rows + } + + pub fn num_non_nulls(&self) -> RowId { + self.num_non_null_rows + } +} + +/// Splits a value address into lower and upper 16bits. +/// The lower 16 bits are the value in the block +/// The upper 16 bits are the block index +#[derive(Copy, Debug, Clone)] +struct RowAddr { + block_id: u16, + in_block_row_id: u16, +} + +#[inline(always)] +fn row_addr_from_row_id(row_id: RowId) -> RowAddr { + RowAddr { + block_id: (row_id / BLOCK_SIZE) as u16, + in_block_row_id: (row_id % BLOCK_SIZE) as u16, + } +} + +impl Set for OptionalIndex { + // Check if value at position is not null. + #[inline] + fn contains(&self, row_id: RowId) -> bool { + let RowAddr { + block_id, + in_block_row_id, + } = row_addr_from_row_id(row_id); + let block_meta = self.block_metas[block_id as usize]; + match self.block(block_meta) { + Block::Dense(dense_block) => dense_block.contains(in_block_row_id), + Block::Sparse(sparse_block) => sparse_block.contains(in_block_row_id), + } + } + + #[inline] + fn rank_if_exists(&self, row_id: RowId) -> Option { + let RowAddr { + block_id, + in_block_row_id, + } = row_addr_from_row_id(row_id); + let block_meta = self.block_metas[block_id as usize]; + let block = self.block(block_meta); + let block_offset_row_id = match block { + Block::Dense(dense_block) => dense_block.rank_if_exists(in_block_row_id), + Block::Sparse(sparse_block) => sparse_block.rank_if_exists(in_block_row_id), + }? as u32; + Some(block_meta.non_null_rows_before_block + block_offset_row_id) + } + + #[inline] + fn select(&self, rank: RowId) -> RowId { + let block_pos = self.find_block(rank, 0); + let block_doc_idx_start = block_pos * ELEMENTS_PER_BLOCK; + let block_meta = self.block_metas[block_pos as usize]; + let block: Block<'_> = self.block(block_meta); + let index_in_block = (rank - block_meta.non_null_rows_before_block) as u16; + let in_block_rank = match block { + Block::Dense(dense_block) => dense_block.select(index_in_block), + Block::Sparse(sparse_block) => sparse_block.select(index_in_block), + }; + block_doc_idx_start + in_block_rank as u32 + } + + fn select_batch(&self, ranks: &[u32], output_idxs: &mut [u32]) { + let mut block_pos = 0u32; + let mut start = 0; + let group_by_it = ranks.iter().copied().group_by(move |codec_idx| { + block_pos = self.find_block(*codec_idx, block_pos); + block_pos + }); + for (block_pos, block_iter) in group_by_it { + let block_doc_idx_start = block_pos * ELEMENTS_PER_BLOCK; + let block_meta = self.block_metas[block_pos as usize]; + let block: Block<'_> = self.block(block_meta); + let offset = block_meta.non_null_rows_before_block; + let indexes_in_block_iter = + block_iter.map(move |codec_idx| (codec_idx - offset) as u16); + match block { + Block::Dense(dense_block) => { + for in_offset in dense_block.select_iter(indexes_in_block_iter) { + output_idxs[start] = in_offset as u32 + block_doc_idx_start; + start += 1; + } + } + Block::Sparse(sparse_block) => { + for in_offset in sparse_block.select_iter(indexes_in_block_iter) { + output_idxs[start] = in_offset as u32 + block_doc_idx_start; + start += 1; + } + } + }; + } + } +} + +impl OptionalIndex { + #[inline] + fn block<'a>(&'a self, block_meta: BlockMeta) -> Block<'a> { + let BlockMeta { + start_byte_offset, + block_variant, + .. + } = block_meta; + let start_byte_offset = start_byte_offset as usize; + let bytes = self.block_data.as_slice(); + match block_variant { + BlockVariant::Dense => Block::Dense(DenseBlockCodec::open( + &bytes[start_byte_offset..start_byte_offset + DENSE_BLOCK_NUM_BYTES as usize], + )), + BlockVariant::Sparse { num_vals } => { + let end_byte_offset = start_byte_offset + num_vals as usize * 2; + let sparse_bytes = &bytes[start_byte_offset..end_byte_offset]; + Block::Sparse(SparseBlockCodec::open(sparse_bytes)) + } + } + } + + #[inline] + fn find_block(&self, dense_idx: u32, start_block_pos: u32) -> u32 { + for block_pos in start_block_pos..self.block_metas.len() as u32 { + let offset = self.block_metas[block_pos as usize].non_null_rows_before_block; + if offset > dense_idx { + return block_pos - 1; + } + } + self.block_metas.len() as u32 - 1u32 + } + + // TODO Add a good API for the codec_idx to original_idx translation. + // The Iterator API is a probably a bad idea +} + +#[derive(Copy, Clone)] +enum Block<'a> { + Dense(DenseBlock<'a>), + Sparse(SparseBlock<'a>), +} + +#[derive(Debug, Copy, Clone)] +enum OptionalIndexCodec { + Dense = 0, + Sparse = 1, +} + +impl OptionalIndexCodec { + fn to_code(self) -> u8 { + self as u8 + } + + fn try_from_code(code: u8) -> Result { + match code { + 0 => Ok(Self::Dense), + 1 => Ok(Self::Sparse), + _ => Err(InvalidData), + } + } +} + +impl BinarySerializable for OptionalIndexCodec { + fn serialize(&self, writer: &mut W) -> io::Result<()> { + writer.write_all(&[self.to_code()]) + } + + fn deserialize(reader: &mut R) -> io::Result { + let optional_codec_code = u8::deserialize(reader)?; + let optional_codec = Self::try_from_code(optional_codec_code)?; + Ok(optional_codec) + } +} + +fn serialize_optional_index_block(block_els: &[u16], out: &mut impl io::Write) -> io::Result<()> { + let is_sparse = is_sparse(block_els.len() as u32); + if is_sparse { + SparseBlockCodec::serialize(block_els.iter().copied(), out)?; + } else { + DenseBlockCodec::serialize(block_els.iter().copied(), out)?; + } + Ok(()) +} + +pub fn serialize_optional_index<'a, W: io::Write>( + serializable_optional_index: &dyn SerializableOptionalIndex<'a>, + output: &mut W, +) -> io::Result<()> { + VInt(serializable_optional_index.num_rows() as u64).serialize(output)?; + + let mut rows_it = serializable_optional_index.non_null_rows(); + let mut block_metadata: Vec = Vec::new(); + let mut current_block = Vec::new(); + + // This if-statement for the first element ensures that + // `block_metadata` is not empty in the loop below. + let Some(idx) = rows_it.next() else { + output.write_all(&0u16.to_le_bytes())?; + return Ok(()); + }; + + let row_addr = row_addr_from_row_id(idx); + + let mut current_block_id = row_addr.block_id; + current_block.push(row_addr.in_block_row_id); + + for idx in rows_it { + let value_addr = row_addr_from_row_id(idx); + if current_block_id != value_addr.block_id { + serialize_optional_index_block(¤t_block[..], output)?; + block_metadata.push(SerializedBlockMeta { + block_id: current_block_id, + num_non_null_rows: current_block.len() as u32, + }); + current_block.clear(); + current_block_id = value_addr.block_id; + } + current_block.push(value_addr.in_block_row_id); + } + + // handle last block + serialize_optional_index_block(¤t_block[..], output)?; + + block_metadata.push(SerializedBlockMeta { + block_id: current_block_id, + num_non_null_rows: current_block.len() as u32, + }); + + for block in &block_metadata { + output.write_all(&block.to_bytes())?; + } + + output.write_all((block_metadata.len() as u16).to_le_bytes().as_ref())?; + + Ok(()) +} + +const SERIALIZED_BLOCK_META_NUM_BYTES: usize = 4; + +#[derive(Clone, Copy, Debug)] +struct SerializedBlockMeta { + block_id: u16, + num_non_null_rows: u32, //< takes values in 1..=u16::MAX +} + +// TODO unit tests +impl SerializedBlockMeta { + #[inline] + fn from_bytes(bytes: [u8; SERIALIZED_BLOCK_META_NUM_BYTES]) -> SerializedBlockMeta { + let block_id = u16::from_le_bytes(bytes[0..2].try_into().unwrap()); + let num_non_null_rows: u32 = + u16::from_le_bytes(bytes[2..4].try_into().unwrap()) as u32 + 1u32; + SerializedBlockMeta { + block_id, + num_non_null_rows, + } + } + + #[inline] + fn to_bytes(&self) -> [u8; SERIALIZED_BLOCK_META_NUM_BYTES] { + assert!(self.num_non_null_rows > 0); + let mut bytes = [0u8; SERIALIZED_BLOCK_META_NUM_BYTES]; + bytes[0..2].copy_from_slice(&self.block_id.to_le_bytes()); + // We don't store empty blocks, therefore we can subtract 1. + // This way we will be able to use u16 when the number of elements is 1 << 16 or u16::MAX+1 + bytes[2..4].copy_from_slice(&((self.num_non_null_rows - 1u32) as u16).to_le_bytes()); + bytes + } +} + +#[inline] +fn is_sparse(num_rows_in_block: u32) -> bool { + num_rows_in_block < DENSE_BLOCK_THRESHOLD as u32 +} + +fn deserialize_optional_index_block_metadatas( + data: &[u8], + num_rows: u32, +) -> (Box<[BlockMeta]>, u32) { + let num_blocks = data.len() / SERIALIZED_BLOCK_META_NUM_BYTES; + let mut block_metas = Vec::with_capacity(num_blocks as usize + 1); + let mut start_byte_offset = 0; + let mut non_null_rows_before_block = 0; + for block_meta_bytes in data.chunks_exact(SERIALIZED_BLOCK_META_NUM_BYTES) { + let block_meta_bytes: [u8; SERIALIZED_BLOCK_META_NUM_BYTES] = + block_meta_bytes.try_into().unwrap(); + let SerializedBlockMeta { + block_id, + num_non_null_rows, + } = SerializedBlockMeta::from_bytes(block_meta_bytes); + block_metas.resize( + block_id as usize, + BlockMeta { + non_null_rows_before_block, + start_byte_offset, + block_variant: BlockVariant::empty(), + }, + ); + let block_variant = if is_sparse(num_non_null_rows) { + BlockVariant::Sparse { + num_vals: num_non_null_rows as u16, + } + } else { + BlockVariant::Dense + }; + block_metas.push(BlockMeta { + non_null_rows_before_block, + start_byte_offset, + block_variant, + }); + start_byte_offset += block_variant.num_bytes_in_block(); + non_null_rows_before_block += num_non_null_rows as u32; + } + block_metas.resize( + ((num_rows + BLOCK_SIZE - 1) / BLOCK_SIZE) as usize, + BlockMeta { + non_null_rows_before_block, + start_byte_offset, + block_variant: BlockVariant::empty(), + }, + ); + (block_metas.into_boxed_slice(), non_null_rows_before_block) +} + +pub fn open_optional_index(bytes: OwnedBytes) -> io::Result { + let (mut bytes, num_non_empty_blocks_bytes) = bytes.rsplit(2); + let num_non_empty_block_bytes = + u16::from_le_bytes(num_non_empty_blocks_bytes.as_slice().try_into().unwrap()); + let num_rows = VInt::deserialize_u64(&mut bytes)? as u32; + let block_metas_num_bytes = + num_non_empty_block_bytes as usize * SERIALIZED_BLOCK_META_NUM_BYTES; + let (block_data, block_metas) = bytes.rsplit(block_metas_num_bytes); + let (block_metas, num_non_null_rows) = + deserialize_optional_index_block_metadatas(block_metas.as_slice(), num_rows).into(); + let optional_index = OptionalIndex { + num_rows, + num_non_null_rows, + block_data, + block_metas: block_metas.into(), + }; + Ok(optional_index) +} + +pub trait SerializableOptionalIndex<'a> { + fn num_rows(&self) -> RowId; + fn non_null_rows(&self) -> Box + 'a>; +} + +impl SerializableOptionalIndex<'static> for Range { + fn num_rows(&self) -> RowId { + self.end + } + fn non_null_rows(&self) -> Box + 'static> { + Box::new(self.clone()) + } +} + +#[cfg(test)] +mod tests; diff --git a/columnar/src/column_index/optional_index/set.rs b/columnar/src/column_index/optional_index/set.rs new file mode 100644 index 0000000000..f447b60d9a --- /dev/null +++ b/columnar/src/column_index/optional_index/set.rs @@ -0,0 +1,38 @@ +use std::io; + +/// A codec makes it possible to serialize a set of +/// elements, and open the resulting Set representation. +pub trait SetCodec { + type Item: Copy + TryFrom + Eq + std::hash::Hash + std::fmt::Debug; + type Reader<'a>: Set; + + /// Serializes a set of unique sorted u16 elements. + /// + /// May panic if the elements are not sorted. + fn serialize(els: impl Iterator, wrt: impl io::Write) -> io::Result<()>; + fn open<'a>(data: &'a [u8]) -> Self::Reader<'a>; +} + +pub trait Set { + /// Returns true if the elements is contained in the Set + fn contains(&self, el: T) -> bool; + + /// If the set contains `el` returns its position in the sortd set of elements. + /// If the set does not contain the element, it returns `None`. + fn rank_if_exists(&self, el: T) -> Option; + + /// Return the rank-th value stored in this bitmap. + /// + /// # Panics + /// + /// May panic if rank is greater than the number of elements in the Set. + fn select(&self, rank: T) -> T; + + /// Batch version of select. + /// `ranks` is assumed to be sorted. + /// + /// # Panics + /// + /// May panic if rank is greater than the number of elements in the Set. + fn select_batch(&self, ranks: &[T], outputs: &mut [T]); +} diff --git a/columnar/src/column_index/optional_index/set_block/mod.rs b/columnar/src/column_index/optional_index/set_block/mod.rs new file mode 100644 index 0000000000..7eb26f12aa --- /dev/null +++ b/columnar/src/column_index/optional_index/set_block/mod.rs @@ -0,0 +1,8 @@ +mod set_block; +mod sparse; + +pub use set_block::{DenseBlock, DenseBlockCodec, DENSE_BLOCK_NUM_BYTES}; +pub use sparse::{SparseBlock, SparseBlockCodec}; + +#[cfg(test)] +mod tests; diff --git a/columnar/src/column_index/optional_index/set_block/set_block.rs b/columnar/src/column_index/optional_index/set_block/set_block.rs new file mode 100644 index 0000000000..b01ea1f07f --- /dev/null +++ b/columnar/src/column_index/optional_index/set_block/set_block.rs @@ -0,0 +1,271 @@ +use std::convert::TryInto; +use std::io::{self, Write}; + +use common::BinarySerializable; + +use crate::column_index::optional_index::{Set, SetCodec, ELEMENTS_PER_BLOCK}; + +#[inline(always)] +fn get_bit_at(input: u64, n: u16) -> bool { + input & (1 << n) != 0 +} + +#[inline] +fn set_bit_at(input: &mut u64, n: u16) { + *input |= 1 << n; +} + +/// For the `DenseCodec`, `data` which contains the encoded blocks. +/// Each block consists of [u8; 12]. The first 8 bytes is a bitvec for 64 elements. +/// The last 4 bytes are the offset, the number of set bits so far. +/// +/// When translating the original index to a dense index, the correct block can be computed +/// directly `orig_idx/64`. Inside the block the position is `orig_idx%64`. +/// +/// When translating a dense index to the original index, we can use the offset to find the correct +/// block. Direct computation is not possible, but we can employ a linear or binary search. + +const ELEMENTS_PER_MINI_BLOCK: u16 = 64; +const MINI_BLOCK_BITVEC_NUM_BYTES: usize = 8; +const MINI_BLOCK_OFFSET_NUM_BYTES: usize = 2; +pub const MINI_BLOCK_NUM_BYTES: usize = MINI_BLOCK_BITVEC_NUM_BYTES + MINI_BLOCK_OFFSET_NUM_BYTES; + +/// Number of bytes in a dense block. +pub const DENSE_BLOCK_NUM_BYTES: u32 = + (ELEMENTS_PER_BLOCK as u32 / ELEMENTS_PER_MINI_BLOCK as u32) * MINI_BLOCK_NUM_BYTES as u32; + +pub struct DenseBlockCodec; + +impl SetCodec for DenseBlockCodec { + type Item = u16; + type Reader<'a> = DenseBlock<'a>; + + fn serialize(els: impl Iterator, wrt: impl io::Write) -> io::Result<()> { + serialize_dense_codec(els, wrt) + } + + #[inline] + fn open<'a>(data: &'a [u8]) -> Self::Reader<'a> { + assert_eq!(data.len(), DENSE_BLOCK_NUM_BYTES as usize); + DenseBlock(data) + } +} + +/// Interpreting the bitvec as a set of integer within 0..=63 +/// and given an element, returns the number of elements in the +/// set lesser than the element. +/// +/// # Panics +/// +/// May panic or return a wrong result if el <= 64. +#[inline(always)] +fn rank_u64(bitvec: u64, el: u16) -> u16 { + debug_assert!(el < 64); + let mask = (1u64 << el) - 1; + let masked_bitvec = bitvec & mask; + masked_bitvec.count_ones() as u16 +} + +#[inline(always)] +fn select_u64(mut bitvec: u64, rank: u16) -> u16 { + for _ in 0..rank { + bitvec &= bitvec - 1; + } + bitvec.trailing_zeros() as u16 +} + +// TODO test the following solution on Intel... on Ryzen Zen <3 it is a catastrophy. +// #[target_feature(enable = "bmi2")] +// unsafe fn select_bitvec_unsafe(bitvec: u64, rank: u16) -> u16 { +// let pdep = _pdep_u64(1u64 << rank, bitvec); +// pdep.trailing_zeros() as u16 +// } + +#[derive(Clone, Copy, Debug)] +struct DenseMiniBlock { + bitvec: u64, + rank: u16, +} + +impl DenseMiniBlock { + fn from_bytes(data: [u8; MINI_BLOCK_NUM_BYTES]) -> Self { + let bitvec = u64::from_le_bytes(data[..MINI_BLOCK_BITVEC_NUM_BYTES].try_into().unwrap()); + let rank = u16::from_le_bytes(data[MINI_BLOCK_BITVEC_NUM_BYTES..].try_into().unwrap()); + Self { bitvec, rank } + } + + fn to_bytes(&self) -> [u8; MINI_BLOCK_NUM_BYTES] { + let mut bytes = [0u8; MINI_BLOCK_NUM_BYTES]; + bytes[..MINI_BLOCK_BITVEC_NUM_BYTES].copy_from_slice(&self.bitvec.to_le_bytes()); + bytes[MINI_BLOCK_BITVEC_NUM_BYTES..].copy_from_slice(&self.rank.to_le_bytes()); + bytes + } +} + +#[derive(Copy, Clone)] +pub struct DenseBlock<'a>(&'a [u8]); + +impl<'a> Set for DenseBlock<'a> { + #[inline(always)] + fn contains(&self, el: u16) -> bool { + let mini_block_id = el / ELEMENTS_PER_MINI_BLOCK; + let bitvec = self.mini_block(mini_block_id).bitvec; + let pos_in_bitvec = el % ELEMENTS_PER_MINI_BLOCK; + get_bit_at(bitvec, pos_in_bitvec) + } + + #[inline(always)] + fn rank_if_exists(&self, el: u16) -> Option { + let block_pos = el / ELEMENTS_PER_MINI_BLOCK; + let index_block = self.mini_block(block_pos); + let pos_in_block_bit_vec = el % ELEMENTS_PER_MINI_BLOCK; + let ones_in_block = rank_u64(index_block.bitvec, pos_in_block_bit_vec); + let rank = index_block.rank + ones_in_block; + if get_bit_at(index_block.bitvec, pos_in_block_bit_vec) { + Some(rank) + } else { + None + } + } + + #[inline(always)] + fn select(&self, rank: u16) -> u16 { + let block_id = self.find_miniblock_containing_rank(rank, 0).unwrap(); + let index_block = self.mini_block(block_id); + let in_block_rank = rank - index_block.rank; + block_id * ELEMENTS_PER_MINI_BLOCK + select_u64(index_block.bitvec, in_block_rank) + } + + fn select_batch(&self, ranks: &[u16], outputs: &mut [u16]) { + let orig_ids = self.select_iter(ranks.iter().copied()); + for (output, original_id) in outputs.iter_mut().zip(orig_ids) { + *output = original_id; + } + } +} + +impl<'a> DenseBlock<'a> { + /// Iterator verison of select. + /// + /// # Panics + /// Panics if one of the rank is higher than the number of elements in the set. + pub fn select_iter<'b>( + &self, + rank_it: impl Iterator + 'b, + ) -> impl Iterator + 'b + where + Self: 'b, + { + let mut block_id = 0u16; + let me = *self; + rank_it.map(move |rank| { + block_id = me.find_miniblock_containing_rank(rank, block_id).unwrap(); + let index_block = me.mini_block(block_id); + let in_block_rank = rank - index_block.rank; + block_id * ELEMENTS_PER_MINI_BLOCK + select_u64(index_block.bitvec, in_block_rank) + }) + } +} + +impl<'a> DenseBlock<'a> { + #[inline] + fn mini_block(&self, mini_block_id: u16) -> DenseMiniBlock { + let data_start_pos = mini_block_id as usize * MINI_BLOCK_NUM_BYTES; + DenseMiniBlock::from_bytes( + self.0[data_start_pos..data_start_pos + MINI_BLOCK_NUM_BYTES] + .try_into() + .unwrap(), + ) + } + + #[inline] + fn iter_miniblocks( + &self, + from_block_id: u16, + ) -> impl Iterator + '_ { + self.0 + .chunks_exact(MINI_BLOCK_NUM_BYTES) + .enumerate() + .skip(from_block_id as usize) + .map(|(block_id, bytes)| { + let mini_block = DenseMiniBlock::from_bytes(bytes.try_into().unwrap()); + (block_id as u16, mini_block) + }) + } + + /// Finds the block position containing the dense_idx. + /// + /// # Correctness + /// dense_idx needs to be smaller than the number of values in the index + /// + /// The last offset number is equal to the number of values in the index. + #[inline] + fn find_miniblock_containing_rank(&self, rank: u16, from_block_id: u16) -> Option { + self.iter_miniblocks(from_block_id) + .take_while(|(_, block)| block.rank <= rank) + .map(|(block_id, _)| block_id) + .last() + } +} + +/// Iterator over all values, true if set, otherwise false +pub fn serialize_dense_codec( + els: impl Iterator, + mut output: impl Write, +) -> io::Result<()> { + let mut non_null_rows_before: u16 = 0u16; + let mut block = 0u64; + let mut current_block_id = 0u16; + for el in els { + let block_id = el / ELEMENTS_PER_MINI_BLOCK; + let in_offset = el % ELEMENTS_PER_MINI_BLOCK; + while block_id > current_block_id { + let dense_mini_block = DenseMiniBlock { + bitvec: block, + rank: non_null_rows_before as u16, + }; + output.write_all(&dense_mini_block.to_bytes())?; + non_null_rows_before += block.count_ones() as u16; + block = 0u64; + current_block_id += 1u16; + } + set_bit_at(&mut block, in_offset); + } + while current_block_id <= u16::MAX / ELEMENTS_PER_MINI_BLOCK { + block.serialize(&mut output)?; + non_null_rows_before.serialize(&mut output)?; + // This will overflow to 0 exactly if all bits are set. + // This is however not problem as we won't use this last value. + non_null_rows_before = non_null_rows_before.wrapping_add(block.count_ones() as u16); + block = 0u64; + current_block_id += 1u16; + } + Ok(()) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_select_bitvec() { + assert_eq!(select_u64(1u64, 0), 0); + assert_eq!(select_u64(2u64, 0), 1); + assert_eq!(select_u64(4u64, 0), 2); + assert_eq!(select_u64(8u64, 0), 3); + assert_eq!(select_u64(1 | 8u64, 0), 0); + assert_eq!(select_u64(1 | 8u64, 1), 3); + } + + #[test] + fn test_count_ones() { + for i in 0..=63 { + assert_eq!(rank_u64(u64::MAX, i), i); + } + } + + #[test] + fn test_dense() { + assert_eq!(DENSE_BLOCK_NUM_BYTES, 10_240); + } +} diff --git a/columnar/src/column_index/optional_index/set_block/sparse.rs b/columnar/src/column_index/optional_index/set_block/sparse.rs new file mode 100644 index 0000000000..486dc70a50 --- /dev/null +++ b/columnar/src/column_index/optional_index/set_block/sparse.rs @@ -0,0 +1,112 @@ +use crate::column_index::optional_index::{Set, SetCodec}; + +pub struct SparseBlockCodec; + +impl SetCodec for SparseBlockCodec { + type Item = u16; + type Reader<'a> = SparseBlock<'a>; + + fn serialize( + els: impl Iterator, + mut wrt: impl std::io::Write, + ) -> std::io::Result<()> { + for el in els { + wrt.write_all(&el.to_le_bytes())?; + } + Ok(()) + } + + fn open<'a>(data: &'a [u8]) -> Self::Reader<'a> { + SparseBlock(data) + } +} + +#[derive(Copy, Clone)] +pub struct SparseBlock<'a>(&'a [u8]); + +impl<'a> Set for SparseBlock<'a> { + #[inline(always)] + fn contains(&self, el: u16) -> bool { + self.binary_search(el).is_ok() + } + + #[inline(always)] + fn rank_if_exists(&self, el: u16) -> Option { + self.binary_search(el).ok() + } + + #[inline(always)] + fn select(&self, rank: u16) -> u16 { + let offset = rank as usize * 2; + u16::from_le_bytes(self.0[offset..offset + 2].try_into().unwrap()) + } + + fn select_batch(&self, ranks: &[u16], outputs: &mut [u16]) { + let orig_ids = self.select_iter(ranks.iter().copied()); + for (output, original_id) in outputs.iter_mut().zip(orig_ids) { + *output = original_id; + } + } +} + +#[inline(always)] +fn get_u16(data: &[u8], byte_position: usize) -> u16 { + let bytes: [u8; 2] = data[byte_position..byte_position + 2].try_into().unwrap(); + u16::from_le_bytes(bytes) +} + +impl<'a> SparseBlock<'a> { + #[inline(always)] + fn value_at_idx(&self, data: &[u8], idx: u16) -> u16 { + let start_offset: usize = idx as usize * 2; + get_u16(data, start_offset) + } + + #[inline] + fn num_vals(&self) -> u16 { + (self.0.len() / 2) as u16 + } + + #[inline] + #[allow(clippy::comparison_chain)] + // Looks for the element in the block. Returns the positions if found. + fn binary_search(&self, target: u16) -> Result { + let data = &self.0; + let mut size = self.num_vals(); + let mut left = 0; + let mut right = size; + // TODO try different implem. + // e.g. exponential search into binary search + while left < right { + let mid = left + size / 2; + + // TODO do boundary check only once, and then use an + // unsafe `value_at_idx` + let mid_val = self.value_at_idx(data, mid); + + if target > mid_val { + left = mid + 1; + } else if target < mid_val { + right = mid; + } else { + return Ok(mid); + } + + size = right - left; + } + Err(left) + } + + pub fn select_iter<'b>( + &self, + iter: impl Iterator + 'b, + ) -> impl Iterator + 'b + where + Self: 'b, + { + iter.map(|codec_id| { + let offset = codec_id as usize * 2; + u16::from_le_bytes(self.0[offset..offset + 2].try_into().unwrap()) + }) + } +} diff --git a/columnar/src/column_index/optional_index/set_block/tests.rs b/columnar/src/column_index/optional_index/set_block/tests.rs new file mode 100644 index 0000000000..6e21b1b5ba --- /dev/null +++ b/columnar/src/column_index/optional_index/set_block/tests.rs @@ -0,0 +1,110 @@ +use std::collections::HashMap; + +use crate::column_index::optional_index::set_block::set_block::DENSE_BLOCK_NUM_BYTES; +use crate::column_index::optional_index::set_block::{DenseBlockCodec, SparseBlockCodec}; +use crate::column_index::optional_index::{Set, SetCodec}; + +fn test_set_helper>(vals: &[u16]) -> usize { + let mut buffer = Vec::new(); + C::serialize(vals.iter().copied(), &mut buffer).unwrap(); + let tested_set = C::open(buffer.as_slice()); + let hash_set: HashMap = vals + .iter() + .copied() + .enumerate() + .map(|(ord, val)| (val, C::Item::try_from(ord).ok().unwrap())) + .collect(); + for val in 0u16..=u16::MAX { + assert_eq!(tested_set.contains(val), hash_set.contains_key(&val)); + assert_eq!(tested_set.rank_if_exists(val), hash_set.get(&val).copied()); + } + for rank in 0..vals.len() { + assert_eq!(tested_set.select(rank as u16), vals[rank]); + } + buffer.len() +} + +#[test] +fn test_dense_block_set_u16_empty() { + let buffer_len = test_set_helper::(&[]); + assert_eq!(buffer_len, DENSE_BLOCK_NUM_BYTES as usize); +} + +#[test] +fn test_dense_block_set_u16_max() { + let buffer_len = test_set_helper::(&[u16::MAX]); + assert_eq!(buffer_len, DENSE_BLOCK_NUM_BYTES as usize); +} + +#[test] +fn test_sparse_block_set_u16_empty() { + let buffer_len = test_set_helper::(&[]); + assert_eq!(buffer_len, 0); +} + +#[test] +fn test_sparse_block_set_u16_max() { + let buffer_len = test_set_helper::(&[u16::MAX]); + assert_eq!(buffer_len, 2); +} + +use proptest::prelude::*; + +proptest! { + #[test] + fn test_prop_test_dense(els in proptest::collection::btree_set(0..=u16::MAX, 0..=u16::MAX as usize)) { + let vals: Vec = els.into_iter().collect(); + let buffer_len = test_set_helper::(&vals); + assert_eq!(buffer_len, DENSE_BLOCK_NUM_BYTES as usize); + } + + #[test] + fn test_prop_test_sparse(els in proptest::collection::btree_set(0..=u16::MAX, 0..=u16::MAX as usize)) { + let vals: Vec = els.into_iter().collect(); + let buffer_len = test_set_helper::(&vals); + assert_eq!(buffer_len, vals.len() * 2); + } +} + +#[test] +fn test_simple_translate_codec_codec_idx_to_original_idx_dense() { + let mut buffer = Vec::new(); + DenseBlockCodec::serialize([1, 3, 17, 32, 30_000, 30_001].iter().copied(), &mut buffer) + .unwrap(); + let tested_set = DenseBlockCodec::open(buffer.as_slice()); + assert!(tested_set.contains(1)); + assert_eq!( + &tested_set + .select_iter([0, 1, 2, 5].iter().copied()) + .collect::>(), + &[1, 3, 17, 30_001] + ); +} + +#[test] +fn test_simple_translate_codec_idx_to_original_idx_sparse() { + let mut buffer = Vec::new(); + SparseBlockCodec::serialize([1, 3, 17].iter().copied(), &mut buffer).unwrap(); + let tested_set = SparseBlockCodec::open(buffer.as_slice()); + assert!(tested_set.contains(1)); + assert_eq!( + &tested_set + .select_iter([0, 1, 2].iter().copied()) + .collect::>(), + &[1, 3, 17] + ); +} + +#[test] +fn test_simple_translate_codec_idx_to_original_idx_dense() { + let mut buffer = Vec::new(); + DenseBlockCodec::serialize(0u16..150u16, &mut buffer).unwrap(); + let tested_set = DenseBlockCodec::open(buffer.as_slice()); + assert!(tested_set.contains(1)); + let rg = 0u16..150u16; + let els: Vec = rg.clone().collect(); + assert_eq!( + &tested_set.select_iter(rg.clone()).collect::>(), + &els + ); +} diff --git a/columnar/src/column_index/optional_index/tests.rs b/columnar/src/column_index/optional_index/tests.rs new file mode 100644 index 0000000000..e34700b4d0 --- /dev/null +++ b/columnar/src/column_index/optional_index/tests.rs @@ -0,0 +1,327 @@ +use proptest::prelude::{any, prop, *}; +use proptest::strategy::Strategy; +use proptest::{prop_oneof, proptest}; + +use super::*; + +#[test] +fn test_dense_block_threshold() { + assert_eq!(super::DENSE_BLOCK_THRESHOLD, 5_120); +} + +fn random_bitvec() -> BoxedStrategy> { + prop_oneof![ + 1 => prop::collection::vec(proptest::bool::weighted(1.0), 0..100), + 1 => prop::collection::vec(proptest::bool::weighted(0.00), 0..(ELEMENTS_PER_BLOCK as usize * 3)), // empty blocks + 1 => prop::collection::vec(proptest::bool::weighted(1.00), 0..(ELEMENTS_PER_BLOCK as usize + 10)), // full block + 1 => prop::collection::vec(proptest::bool::weighted(0.01), 0..100), + 1 => prop::collection::vec(proptest::bool::weighted(0.01), 0..u16::MAX as usize), + 8 => vec![any::()], + ] + .boxed() +} + +proptest! { + #![proptest_config(ProptestConfig::with_cases(50))] + #[test] + fn test_with_random_bitvecs(bitvec1 in random_bitvec(), bitvec2 in random_bitvec(), bitvec3 in random_bitvec()) { + let mut bitvec = Vec::new(); + bitvec.extend_from_slice(&bitvec1); + bitvec.extend_from_slice(&bitvec2); + bitvec.extend_from_slice(&bitvec3); + test_null_index(&bitvec[..]); + } +} + +#[test] +fn test_with_random_sets_simple() { + let vals = 10..BLOCK_SIZE * 2; + let mut out: Vec = Vec::new(); + serialize_optional_index(&vals.clone(), &mut out).unwrap(); + let null_index = open_optional_index(OwnedBytes::new(out)).unwrap(); + let ranks: Vec = (65_472u32..65_473u32).collect(); + let els: Vec = ranks.iter().copied().map(|rank| rank + 10).collect(); + let mut output = vec![0u32; ranks.len()]; + null_index.select_batch(&ranks[..], &mut output[..]); + assert_eq!(&output, &els); +} + +#[test] +fn test_optional_index_trailing_empty_blocks() { + test_null_index(&[false]); +} + +#[test] +fn test_optional_index_one_block_false() { + let mut iter = vec![false; ELEMENTS_PER_BLOCK as usize]; + iter.push(true); + test_null_index(&iter[..]); +} + +#[test] +fn test_optional_index_one_block_true() { + let mut iter = vec![true; ELEMENTS_PER_BLOCK as usize]; + iter.push(true); + test_null_index(&iter[..]); +} + +impl<'a> SerializableOptionalIndex<'a> for &'a [bool] { + fn num_rows(&self) -> RowId { + self.len() as u32 + } + + fn non_null_rows(&self) -> Box + 'a> { + Box::new( + self.iter() + .cloned() + .enumerate() + .filter(|(_pos, val)| *val) + .map(|(pos, _val)| pos as u32), + ) + } +} + +fn test_null_index(data: &[bool]) { + let mut out: Vec = Vec::new(); + serialize_optional_index(&data, &mut out).unwrap(); + let null_index = open_optional_index(OwnedBytes::new(out)).unwrap(); + let orig_idx_with_value: Vec = data + .iter() + .enumerate() + .filter(|(_pos, val)| **val) + .map(|(pos, _val)| pos as u32) + .collect(); + let ids: Vec = (0..orig_idx_with_value.len() as u32).collect(); + let mut output = vec![0u32; ids.len()]; + null_index.select_batch(&ids[..], &mut output); + // assert_eq!(&output[0..100], &orig_idx_with_value[0..100]); + assert_eq!(output, orig_idx_with_value); + + let step_size = (orig_idx_with_value.len() / 100).max(1); + for (dense_idx, orig_idx) in orig_idx_with_value.iter().enumerate().step_by(step_size) { + assert_eq!(null_index.rank_if_exists(*orig_idx), Some(dense_idx as u32)); + } + + // 100 samples + let step_size = (data.len() / 100).max(1); + for (pos, value) in data.iter().enumerate().step_by(step_size) { + assert_eq!(null_index.contains(pos as u32), *value); + } +} + +#[test] +fn test_optional_index_test_translation() { + let mut out = vec![]; + let iter = &[true, false, true, false]; + serialize_optional_index(&&iter[..], &mut out).unwrap(); + let null_index = open_optional_index(OwnedBytes::new(out)).unwrap(); + let mut output = vec![0u32; 2]; + null_index.select_batch(&[0, 1], &mut output); + assert_eq!(output, &[0, 2]); +} + +#[test] +fn test_optional_index_translate() { + let mut out = vec![]; + let iter = &[true, false, true, false]; + serialize_optional_index(&&iter[..], &mut out).unwrap(); + let null_index = open_optional_index(OwnedBytes::new(out)).unwrap(); + assert_eq!(null_index.rank_if_exists(0), Some(0)); + assert_eq!(null_index.rank_if_exists(2), Some(1)); +} + +#[test] +fn test_optional_index_small() { + let mut out = vec![]; + let iter = &[true, false, true, false]; + serialize_optional_index(&&iter[..], &mut out).unwrap(); + let null_index = open_optional_index(OwnedBytes::new(out)).unwrap(); + assert!(null_index.contains(0)); + assert!(!null_index.contains(1)); + assert!(null_index.contains(2)); + assert!(!null_index.contains(3)); +} + +#[test] +fn test_optional_index_large() { + let mut docs = vec![]; + docs.extend((0..ELEMENTS_PER_BLOCK).map(|_idx| false)); + docs.extend((0..=1).map(|_idx| true)); + + let mut out = vec![]; + serialize_optional_index(&&docs[..], &mut out).unwrap(); + let null_index = open_optional_index(OwnedBytes::new(out)).unwrap(); + assert!(!null_index.contains(0)); + assert!(!null_index.contains(100)); + assert!(!null_index.contains(ELEMENTS_PER_BLOCK - 1)); + assert!(null_index.contains(ELEMENTS_PER_BLOCK)); + assert!(null_index.contains(ELEMENTS_PER_BLOCK + 1)); +} + +#[cfg(all(test, feature = "unstable"))] +mod bench { + + use rand::rngs::StdRng; + use rand::{Rng, SeedableRng}; + use test::Bencher; + + use super::*; + + const TOTAL_NUM_VALUES: u32 = 1_000_000; + fn gen_bools(fill_ratio: f64) -> OptionalIndex { + let mut out = Vec::new(); + let mut rng: StdRng = StdRng::from_seed([1u8; 32]); + let vals: Vec = (0..TOTAL_NUM_VALUES) + .map(|_| rng.gen_bool(fill_ratio)) + .collect(); + serialize_optional_index(&&vals[..], &mut out).unwrap(); + + let codec = open_optional_index(OwnedBytes::new(out)).unwrap(); + codec + } + + fn random_range_iterator( + start: u32, + end: u32, + avg_step_size: u32, + avg_deviation: u32, + ) -> impl Iterator { + let mut rng: StdRng = StdRng::from_seed([1u8; 32]); + let mut current = start; + std::iter::from_fn(move || { + current += rng.gen_range(avg_step_size - avg_deviation..=avg_step_size + avg_deviation); + if current >= end { + None + } else { + Some(current) + } + }) + } + + fn n_percent_step_iterator(percent: f32, num_values: u32) -> impl Iterator { + let ratio = percent as f32 / 100.0; + let step_size = (1f32 / ratio) as u32; + let deviation = step_size - 1; + random_range_iterator(0, num_values, step_size, deviation) + } + + fn walk_over_data(codec: &OptionalIndex, avg_step_size: u32) -> Option { + walk_over_data_from_positions( + codec, + random_range_iterator(0, TOTAL_NUM_VALUES, avg_step_size, 0), + ) + } + + fn walk_over_data_from_positions( + codec: &OptionalIndex, + positions: impl Iterator, + ) -> Option { + let mut dense_idx: Option = None; + for idx in positions { + dense_idx = dense_idx.or(codec.rank_if_exists(idx)); + } + dense_idx + } + + #[bench] + fn bench_translate_orig_to_codec_1percent_filled_10percent_hit(bench: &mut Bencher) { + let codec = gen_bools(0.01f64); + bench.iter(|| walk_over_data(&codec, 100)); + } + + #[bench] + fn bench_translate_orig_to_codec_5percent_filled_10percent_hit(bench: &mut Bencher) { + let codec = gen_bools(0.05f64); + bench.iter(|| walk_over_data(&codec, 100)); + } + + #[bench] + fn bench_translate_orig_to_codec_5percent_filled_1percent_hit(bench: &mut Bencher) { + let codec = gen_bools(0.05f64); + bench.iter(|| walk_over_data(&codec, 1000)); + } + + #[bench] + fn bench_translate_orig_to_codec_full_scan_1percent_filled(bench: &mut Bencher) { + let codec = gen_bools(0.01f64); + bench.iter(|| walk_over_data_from_positions(&codec, 0..TOTAL_NUM_VALUES)); + } + + #[bench] + fn bench_translate_orig_to_codec_full_scan_10percent_filled(bench: &mut Bencher) { + let codec = gen_bools(0.1f64); + bench.iter(|| walk_over_data_from_positions(&codec, 0..TOTAL_NUM_VALUES)); + } + + #[bench] + fn bench_translate_orig_to_codec_full_scan_90percent_filled(bench: &mut Bencher) { + let codec = gen_bools(0.9f64); + bench.iter(|| walk_over_data_from_positions(&codec, 0..TOTAL_NUM_VALUES)); + } + + #[bench] + fn bench_translate_orig_to_codec_10percent_filled_1percent_hit(bench: &mut Bencher) { + let codec = gen_bools(0.1f64); + bench.iter(|| walk_over_data(&codec, 100)); + } + + #[bench] + fn bench_translate_orig_to_codec_50percent_filled_1percent_hit(bench: &mut Bencher) { + let codec = gen_bools(0.5f64); + bench.iter(|| walk_over_data(&codec, 100)); + } + + #[bench] + fn bench_translate_orig_to_codec_90percent_filled_1percent_hit(bench: &mut Bencher) { + let codec = gen_bools(0.9f64); + bench.iter(|| walk_over_data(&codec, 100)); + } + + #[bench] + fn bench_translate_codec_to_orig_1percent_filled_0comma005percent_hit(bench: &mut Bencher) { + bench_translate_codec_to_orig_util(0.01f64, 0.005f32, bench); + } + + #[bench] + fn bench_translate_codec_to_orig_10percent_filled_0comma005percent_hit(bench: &mut Bencher) { + bench_translate_codec_to_orig_util(0.1f64, 0.005f32, bench); + } + + #[bench] + fn bench_translate_codec_to_orig_1percent_filled_10percent_hit(bench: &mut Bencher) { + bench_translate_codec_to_orig_util(0.01f64, 10f32, bench); + } + + #[bench] + fn bench_translate_codec_to_orig_1percent_filled_full_scan(bench: &mut Bencher) { + bench_translate_codec_to_orig_util(0.01f64, 100f32, bench); + } + + fn bench_translate_codec_to_orig_util( + percent_filled: f64, + percent_hit: f32, + bench: &mut Bencher, + ) { + let codec = gen_bools(percent_filled); + let num_non_nulls = codec.num_non_nulls(); + let idxs: Vec = if percent_hit == 100.0f32 { + (0..num_non_nulls).collect() + } else { + n_percent_step_iterator(percent_hit, num_non_nulls).collect() + }; + let mut output = vec![0u32; idxs.len()]; + bench.iter(|| { + codec.select_batch(&idxs[..], &mut output); + }); + } + + #[bench] + fn bench_translate_codec_to_orig_90percent_filled_0comma005percent_hit(bench: &mut Bencher) { + bench_translate_codec_to_orig_util(0.9f64, 0.005, bench); + } + + #[bench] + fn bench_translate_codec_to_orig_90percent_filled_full_scan(bench: &mut Bencher) { + bench_translate_codec_to_orig_util(0.9f64, 100.0f32, bench); + } +} diff --git a/columnar/src/column_index/serialize.rs b/columnar/src/column_index/serialize.rs new file mode 100644 index 0000000000..bc9168bf32 --- /dev/null +++ b/columnar/src/column_index/serialize.rs @@ -0,0 +1,70 @@ +use std::io; +use std::io::Write; + +use common::OwnedBytes; + +use crate::column_index::multivalued_index::{serialize_multivalued_index, MultivaluedIndex}; +use crate::column_index::optional_index::serialize_optional_index; +use crate::column_index::{ColumnIndex, SerializableOptionalIndex}; +use crate::Cardinality; + +pub enum SerializableColumnIndex<'a> { + Full, + Optional(Box + 'a>), + // TODO remove the Arc apart from serialization this is not + // dynamic at all. + Multivalued(MultivaluedIndex), +} + +impl<'a> SerializableColumnIndex<'a> { + pub fn get_cardinality(&self) -> Cardinality { + match self { + SerializableColumnIndex::Full => Cardinality::Full, + SerializableColumnIndex::Optional(_) => Cardinality::Optional, + SerializableColumnIndex::Multivalued(_) => Cardinality::Multivalued, + } + } +} + +pub fn serialize_column_index( + column_index: SerializableColumnIndex, + output: &mut impl Write, +) -> io::Result<()> { + let cardinality = column_index.get_cardinality().to_code(); + output.write_all(&[cardinality])?; + match column_index { + SerializableColumnIndex::Full => {} + SerializableColumnIndex::Optional(optional_index) => { + serialize_optional_index(&*optional_index, output)? + } + SerializableColumnIndex::Multivalued(multivalued_index) => { + serialize_multivalued_index(multivalued_index, output)? + } + } + Ok(()) +} + +pub fn open_column_index(mut bytes: OwnedBytes) -> io::Result> { + if bytes.is_empty() { + return Err(io::Error::new( + io::ErrorKind::UnexpectedEof, + "Failed to deserialize column index. Empty buffer.", + )); + } + let cardinality_code = bytes[0]; + let cardinality = Cardinality::try_from_code(cardinality_code)?; + bytes.advance(1); + match cardinality { + Cardinality::Full => Ok(ColumnIndex::Full), + Cardinality::Optional => { + let optional_index = super::optional_index::open_optional_index(bytes)?; + Ok(ColumnIndex::Optional(optional_index)) + } + Cardinality::Multivalued => { + let multivalued_index = super::multivalued_index::open_multivalued_index(bytes)?; + Ok(ColumnIndex::Multivalued(multivalued_index)) + } + } +} + +// TODO unit tests diff --git a/columnar/src/column_values/bitpacked.rs b/columnar/src/column_values/bitpacked.rs new file mode 100644 index 0000000000..4d799890ba --- /dev/null +++ b/columnar/src/column_values/bitpacked.rs @@ -0,0 +1,115 @@ +use std::io::{self, Write}; + +use common::OwnedBytes; +use tantivy_bitpacker::{compute_num_bits, BitPacker, BitUnpacker}; + +use super::serialize::NormalizedHeader; +use super::{ColumnValues, FastFieldCodec, FastFieldCodecType}; + +/// Depending on the field type, a different +/// fast field is required. +#[derive(Clone)] +pub struct BitpackedReader { + data: OwnedBytes, + bit_unpacker: BitUnpacker, + normalized_header: NormalizedHeader, +} + +impl ColumnValues for BitpackedReader { + #[inline] + fn get_val(&self, doc: u32) -> u64 { + self.bit_unpacker.get(doc, &self.data) + } + #[inline] + fn min_value(&self) -> u64 { + // The BitpackedReader assumes a normalized vector. + 0 + } + #[inline] + fn max_value(&self) -> u64 { + self.normalized_header.max_value + } + #[inline] + fn num_vals(&self) -> u32 { + self.normalized_header.num_vals + } +} + +pub struct BitpackedCodec; + +impl FastFieldCodec for BitpackedCodec { + /// The CODEC_TYPE is an enum value used for serialization. + const CODEC_TYPE: FastFieldCodecType = FastFieldCodecType::Bitpacked; + + type Reader = BitpackedReader; + + /// Opens a fast field given a file. + fn open_from_bytes( + data: OwnedBytes, + normalized_header: NormalizedHeader, + ) -> io::Result { + let num_bits = compute_num_bits(normalized_header.max_value); + let bit_unpacker = BitUnpacker::new(num_bits); + Ok(BitpackedReader { + data, + bit_unpacker, + normalized_header, + }) + } + + /// Serializes data with the BitpackedFastFieldSerializer. + /// + /// The bitpacker assumes that the column has been normalized. + /// i.e. It has already been shifted by its minimum value, so that its + /// current minimum value is 0. + /// + /// Ideally, we made a shift upstream on the column so that `col.min_value() == 0`. + fn serialize(column: &dyn ColumnValues, write: &mut impl Write) -> io::Result<()> { + assert_eq!(column.min_value(), 0u64); + let num_bits = compute_num_bits(column.max_value()); + let mut bit_packer = BitPacker::new(); + for val in column.iter() { + bit_packer.write(val, num_bits, write)?; + } + bit_packer.close(write)?; + Ok(()) + } + + fn estimate(column: &dyn ColumnValues) -> Option { + let num_bits = compute_num_bits(column.max_value()); + let num_bits_uncompressed = 64; + Some(num_bits as f32 / num_bits_uncompressed as f32) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::column_values::tests::create_and_validate; + + fn create_and_validate_bitpacked_codec(data: &[u64], name: &str) { + create_and_validate::(data, name); + } + + #[test] + fn test_with_codec_data_sets() { + let data_sets = crate::column_values::tests::get_codec_test_datasets(); + for (mut data, name) in data_sets { + create_and_validate_bitpacked_codec(&data, name); + data.reverse(); + create_and_validate::(&data, name); + } + } + + #[test] + fn bitpacked_fast_field_rand() { + for _ in 0..500 { + let mut data = (0..1 + rand::random::() as usize) + .map(|_| rand::random::() as u64 / 2) + .collect::>(); + create_and_validate_bitpacked_codec(&data, "rand"); + data.reverse(); + create_and_validate::(&data, "rand"); + } + } +} diff --git a/columnar/src/column_values/blockwise_linear.rs b/columnar/src/column_values/blockwise_linear.rs new file mode 100644 index 0000000000..69fa21975c --- /dev/null +++ b/columnar/src/column_values/blockwise_linear.rs @@ -0,0 +1,188 @@ +use std::sync::Arc; +use std::{io, iter}; + +use common::{BinarySerializable, CountingWriter, DeserializeFrom, OwnedBytes}; +use tantivy_bitpacker::{compute_num_bits, BitPacker, BitUnpacker}; + +use crate::column_values::line::Line; +use crate::column_values::serialize::NormalizedHeader; +use crate::column_values::{ColumnValues, FastFieldCodec, FastFieldCodecType, VecColumn}; + +const CHUNK_SIZE: usize = 512; + +#[derive(Debug, Default)] +struct Block { + line: Line, + bit_unpacker: BitUnpacker, + data_start_offset: usize, +} + +impl BinarySerializable for Block { + fn serialize(&self, writer: &mut W) -> io::Result<()> { + self.line.serialize(writer)?; + self.bit_unpacker.bit_width().serialize(writer)?; + Ok(()) + } + + fn deserialize(reader: &mut R) -> io::Result { + let line = Line::deserialize(reader)?; + let bit_width = u8::deserialize(reader)?; + Ok(Block { + line, + bit_unpacker: BitUnpacker::new(bit_width), + data_start_offset: 0, + }) + } +} + +fn compute_num_blocks(num_vals: u32) -> usize { + (num_vals as usize + CHUNK_SIZE - 1) / CHUNK_SIZE +} + +pub struct BlockwiseLinearCodec; + +impl FastFieldCodec for BlockwiseLinearCodec { + const CODEC_TYPE: FastFieldCodecType = FastFieldCodecType::BlockwiseLinear; + type Reader = BlockwiseLinearReader; + + fn open_from_bytes( + bytes: common::OwnedBytes, + normalized_header: NormalizedHeader, + ) -> io::Result { + let footer_len: u32 = (&bytes[bytes.len() - 4..]).deserialize()?; + let footer_offset = bytes.len() - 4 - footer_len as usize; + let (data, mut footer) = bytes.split(footer_offset); + let num_blocks = compute_num_blocks(normalized_header.num_vals); + let mut blocks: Vec = iter::repeat_with(|| Block::deserialize(&mut footer)) + .take(num_blocks) + .collect::>()?; + + let mut start_offset = 0; + for block in &mut blocks { + block.data_start_offset = start_offset; + start_offset += (block.bit_unpacker.bit_width() as usize) * CHUNK_SIZE / 8; + } + Ok(BlockwiseLinearReader { + blocks: Arc::new(blocks), + data, + normalized_header, + }) + } + + // Estimate first_chunk and extrapolate + fn estimate(column: &dyn ColumnValues) -> Option { + if column.num_vals() < 10 * CHUNK_SIZE as u32 { + return None; + } + let mut first_chunk: Vec = column.iter().take(CHUNK_SIZE).collect(); + let line = Line::train(&VecColumn::from(&first_chunk)); + for (i, buffer_val) in first_chunk.iter_mut().enumerate() { + let interpolated_val = line.eval(i as u32); + *buffer_val = buffer_val.wrapping_sub(interpolated_val); + } + let estimated_bit_width = first_chunk + .iter() + .map(|el| ((el + 1) as f32 * 3.0) as u64) + .map(compute_num_bits) + .max() + .unwrap(); + + let metadata_per_block = { + let mut out = vec![]; + Block::default().serialize(&mut out).unwrap(); + out.len() + }; + let num_bits = estimated_bit_width as u64 * column.num_vals() as u64 + // function metadata per block + + metadata_per_block as u64 * (column.num_vals() as u64 / CHUNK_SIZE as u64); + let num_bits_uncompressed = 64 * column.num_vals(); + Some(num_bits as f32 / num_bits_uncompressed as f32) + } + + fn serialize(column: &dyn ColumnValues, wrt: &mut impl io::Write) -> io::Result<()> { + // The BitpackedReader assumes a normalized vector. + assert_eq!(column.min_value(), 0); + let mut buffer = Vec::with_capacity(CHUNK_SIZE); + let num_vals = column.num_vals(); + + let num_blocks = compute_num_blocks(num_vals); + let mut blocks = Vec::with_capacity(num_blocks); + + let mut vals = column.iter(); + + let mut bit_packer = BitPacker::new(); + + for _ in 0..num_blocks { + buffer.clear(); + buffer.extend((&mut vals).take(CHUNK_SIZE)); + let line = Line::train(&VecColumn::from(&buffer)); + + assert!(!buffer.is_empty()); + + for (i, buffer_val) in buffer.iter_mut().enumerate() { + let interpolated_val = line.eval(i as u32); + *buffer_val = buffer_val.wrapping_sub(interpolated_val); + } + let bit_width = buffer.iter().copied().map(compute_num_bits).max().unwrap(); + + for &buffer_val in &buffer { + bit_packer.write(buffer_val, bit_width, wrt)?; + } + + blocks.push(Block { + line, + bit_unpacker: BitUnpacker::new(bit_width), + data_start_offset: 0, + }); + } + + bit_packer.close(wrt)?; + + assert_eq!(blocks.len(), compute_num_blocks(num_vals)); + + let mut counting_wrt = CountingWriter::wrap(wrt); + for block in &blocks { + block.serialize(&mut counting_wrt)?; + } + let footer_len = counting_wrt.written_bytes(); + (footer_len as u32).serialize(&mut counting_wrt)?; + + Ok(()) + } +} + +#[derive(Clone)] +pub struct BlockwiseLinearReader { + blocks: Arc>, + normalized_header: NormalizedHeader, + data: OwnedBytes, +} + +impl ColumnValues for BlockwiseLinearReader { + #[inline(always)] + fn get_val(&self, idx: u32) -> u64 { + let block_id = (idx / CHUNK_SIZE as u32) as usize; + let idx_within_block = idx % (CHUNK_SIZE as u32); + let block = &self.blocks[block_id]; + let interpoled_val: u64 = block.line.eval(idx_within_block); + let block_bytes = &self.data[block.data_start_offset..]; + let bitpacked_diff = block.bit_unpacker.get(idx_within_block, block_bytes); + interpoled_val.wrapping_add(bitpacked_diff) + } + + #[inline(always)] + fn min_value(&self) -> u64 { + // The BlockwiseLinearReader assumes a normalized vector. + 0u64 + } + + #[inline(always)] + fn max_value(&self) -> u64 { + self.normalized_header.max_value + } + + #[inline(always)] + fn num_vals(&self) -> u32 { + self.normalized_header.num_vals + } +} diff --git a/columnar/src/column_values/column.rs b/columnar/src/column_values/column.rs new file mode 100644 index 0000000000..e53c5b265e --- /dev/null +++ b/columnar/src/column_values/column.rs @@ -0,0 +1,350 @@ +use std::marker::PhantomData; +use std::ops::{Range, RangeInclusive}; + +use tantivy_bitpacker::minmax; + +use crate::column_values::monotonic_mapping::StrictlyMonotonicFn; + +/// `ColumnValues` provides access to a dense field column. +/// +/// `Column` are just a wrapper over `ColumnValues` and a `ColumnIndex`. +pub trait ColumnValues: Send + Sync { + /// Return the value associated with the given idx. + /// + /// This accessor should return as fast as possible. + /// + /// # Panics + /// + /// May panic if `idx` is greater than the column length. + fn get_val(&self, idx: u32) -> T; + + /// Fills an output buffer with the fast field values + /// associated with the `DocId` going from + /// `start` to `start + output.len()`. + /// + /// # Panics + /// + /// Must panic if `start + output.len()` is greater than + /// the segment's `maxdoc`. + #[inline] + fn get_range(&self, start: u64, output: &mut [T]) { + for (out, idx) in output.iter_mut().zip(start..) { + *out = self.get_val(idx as u32); + } + } + + /// Get the positions of values which are in the provided value range. + /// + /// Note that position == docid for single value fast fields + #[inline] + fn get_docids_for_value_range( + &self, + value_range: RangeInclusive, + doc_id_range: Range, + positions: &mut Vec, + ) { + let doc_id_range = doc_id_range.start..doc_id_range.end.min(self.num_vals()); + + for idx in doc_id_range.start..doc_id_range.end { + let val = self.get_val(idx); + if value_range.contains(&val) { + positions.push(idx); + } + } + } + + /// Returns the minimum value for this fast field. + /// + /// This min_value may not be exact. + /// For instance, the min value does not take in account of possible + /// deleted document. All values are however guaranteed to be higher than + /// `.min_value()`. + fn min_value(&self) -> T; + + /// Returns the maximum value for this fast field. + /// + /// This max_value may not be exact. + /// For instance, the max value does not take in account of possible + /// deleted document. All values are however guaranteed to be higher than + /// `.max_value()`. + fn max_value(&self) -> T; + + /// The number of values in the column. + fn num_vals(&self) -> u32; + + /// Returns a iterator over the data + fn iter<'a>(&'a self) -> Box + 'a> { + Box::new((0..self.num_vals()).map(|idx| self.get_val(idx))) + } +} + +impl<'a, C: ColumnValues + ?Sized, T: Copy + PartialOrd> ColumnValues for &'a C { + fn get_val(&self, idx: u32) -> T { + (*self).get_val(idx) + } + + fn min_value(&self) -> T { + (*self).min_value() + } + + fn max_value(&self) -> T { + (*self).max_value() + } + + fn num_vals(&self) -> u32 { + (*self).num_vals() + } + + fn iter<'b>(&'b self) -> Box + 'b> { + (*self).iter() + } + + fn get_range(&self, start: u64, output: &mut [T]) { + (*self).get_range(start, output) + } +} + +/// VecColumn provides `Column` over a slice. +pub struct VecColumn<'a, T = u64> { + pub(crate) values: &'a [T], + pub(crate) min_value: T, + pub(crate) max_value: T, +} + +impl<'a, T: Copy + PartialOrd + Send + Sync> ColumnValues for VecColumn<'a, T> { + fn get_val(&self, position: u32) -> T { + self.values[position as usize] + } + + fn iter(&self) -> Box + '_> { + Box::new(self.values.iter().copied()) + } + + fn min_value(&self) -> T { + self.min_value + } + + fn max_value(&self) -> T { + self.max_value + } + + fn num_vals(&self) -> u32 { + self.values.len() as u32 + } + + fn get_range(&self, start: u64, output: &mut [T]) { + output.copy_from_slice(&self.values[start as usize..][..output.len()]) + } +} + +impl<'a, T: Copy + PartialOrd + Default, V> From<&'a V> for VecColumn<'a, T> +where V: AsRef<[T]> + ?Sized +{ + fn from(values: &'a V) -> Self { + let values = values.as_ref(); + let (min_value, max_value) = minmax(values.iter().copied()).unwrap_or_default(); + Self { + values, + min_value, + max_value, + } + } +} + +struct MonotonicMappingColumn { + from_column: C, + monotonic_mapping: T, + _phantom: PhantomData, +} + +/// Creates a view of a column transformed by a strictly monotonic mapping. See +/// [`StrictlyMonotonicFn`]. +/// +/// E.g. apply a gcd monotonic_mapping([100, 200, 300]) == [1, 2, 3] +/// monotonic_mapping.mapping() is expected to be injective, and we should always have +/// monotonic_mapping.inverse(monotonic_mapping.mapping(el)) == el +/// +/// The inverse of the mapping is required for: +/// `fn get_positions_for_value_range(&self, range: RangeInclusive) -> Vec ` +/// The user provides the original value range and we need to monotonic map them in the same way the +/// serialization does before calling the underlying column. +/// +/// Note that when opening a codec, the monotonic_mapping should be the inverse of the mapping +/// during serialization. And therefore the monotonic_mapping_inv when opening is the same as +/// monotonic_mapping during serialization. +pub fn monotonic_map_column( + from_column: C, + monotonic_mapping: T, +) -> impl ColumnValues +where + C: ColumnValues, + T: StrictlyMonotonicFn + Send + Sync, + Input: PartialOrd + Send + Sync + Clone, + Output: PartialOrd + Send + Sync + Clone, +{ + MonotonicMappingColumn { + from_column, + monotonic_mapping, + _phantom: PhantomData, + } +} + +impl ColumnValues for MonotonicMappingColumn +where + C: ColumnValues, + T: StrictlyMonotonicFn + Send + Sync, + Input: PartialOrd + Send + Sync + Clone, + Output: PartialOrd + Send + Sync + Clone, +{ + #[inline] + fn get_val(&self, idx: u32) -> Output { + let from_val = self.from_column.get_val(idx); + self.monotonic_mapping.mapping(from_val) + } + + fn min_value(&self) -> Output { + let from_min_value = self.from_column.min_value(); + self.monotonic_mapping.mapping(from_min_value) + } + + fn max_value(&self) -> Output { + let from_max_value = self.from_column.max_value(); + self.monotonic_mapping.mapping(from_max_value) + } + + fn num_vals(&self) -> u32 { + self.from_column.num_vals() + } + + fn iter(&self) -> Box + '_> { + Box::new( + self.from_column + .iter() + .map(|el| self.monotonic_mapping.mapping(el)), + ) + } + + fn get_docids_for_value_range( + &self, + range: RangeInclusive, + doc_id_range: Range, + positions: &mut Vec, + ) { + self.from_column.get_docids_for_value_range( + self.monotonic_mapping.inverse(range.start().clone()) + ..=self.monotonic_mapping.inverse(range.end().clone()), + doc_id_range, + positions, + ) + } + + // We voluntarily do not implement get_range as it yields a regression, + // and we do not have any specialized implementation anyway. +} + +/// Wraps an iterator into a `Column`. +pub struct IterColumn(T); + +impl From for IterColumn +where T: Iterator + Clone + ExactSizeIterator +{ + fn from(iter: T) -> Self { + IterColumn(iter) + } +} + +impl ColumnValues for IterColumn +where + T: Iterator + Clone + ExactSizeIterator + Send + Sync, + T::Item: PartialOrd, +{ + fn get_val(&self, idx: u32) -> T::Item { + self.0.clone().nth(idx as usize).unwrap() + } + + fn min_value(&self) -> T::Item { + self.0.clone().next().unwrap() + } + + fn max_value(&self) -> T::Item { + self.0.clone().last().unwrap() + } + + fn num_vals(&self) -> u32 { + self.0.len() as u32 + } + + fn iter(&self) -> Box + '_> { + Box::new(self.0.clone()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::column_values::monotonic_mapping::{ + StrictlyMonotonicMappingInverter, StrictlyMonotonicMappingToInternalBaseval, + StrictlyMonotonicMappingToInternalGCDBaseval, + }; + + #[test] + fn test_monotonic_mapping() { + let vals = &[3u64, 5u64][..]; + let col = VecColumn::from(vals); + let mapped = monotonic_map_column(col, StrictlyMonotonicMappingToInternalBaseval::new(2)); + assert_eq!(mapped.min_value(), 1u64); + assert_eq!(mapped.max_value(), 3u64); + assert_eq!(mapped.num_vals(), 2); + assert_eq!(mapped.num_vals(), 2); + assert_eq!(mapped.get_val(0), 1); + assert_eq!(mapped.get_val(1), 3); + } + + #[test] + fn test_range_as_col() { + let col = IterColumn::from(10..100); + assert_eq!(col.num_vals(), 90); + assert_eq!(col.max_value(), 99); + } + + #[test] + fn test_monotonic_mapping_iter() { + let vals: Vec = (10..110u64).map(|el| el * 10).collect(); + let col = VecColumn::from(&vals); + let mapped = monotonic_map_column( + col, + StrictlyMonotonicMappingInverter::from( + StrictlyMonotonicMappingToInternalGCDBaseval::new(10, 100), + ), + ); + let val_i64s: Vec = mapped.iter().collect(); + for i in 0..100 { + assert_eq!(val_i64s[i as usize], mapped.get_val(i)); + } + } + + #[test] + fn test_monotonic_mapping_get_range() { + let vals: Vec = (0..100u64).map(|el| el * 10).collect(); + let col = VecColumn::from(&vals); + let mapped = monotonic_map_column( + col, + StrictlyMonotonicMappingInverter::from( + StrictlyMonotonicMappingToInternalGCDBaseval::new(10, 0), + ), + ); + + assert_eq!(mapped.min_value(), 0u64); + assert_eq!(mapped.max_value(), 9900u64); + assert_eq!(mapped.num_vals(), 100); + let val_u64s: Vec = mapped.iter().collect(); + assert_eq!(val_u64s.len(), 100); + for i in 0..100 { + assert_eq!(val_u64s[i as usize], mapped.get_val(i)); + assert_eq!(val_u64s[i as usize], vals[i as usize] * 10); + } + let mut buf = [0u64; 20]; + mapped.get_range(7, &mut buf[..]); + assert_eq!(&val_u64s[7..][..20], &buf); + } +} diff --git a/columnar/src/column_values/column_with_cardinality.rs b/columnar/src/column_values/column_with_cardinality.rs new file mode 100644 index 0000000000..65c332bf7c --- /dev/null +++ b/columnar/src/column_values/column_with_cardinality.rs @@ -0,0 +1,19 @@ +// Copyright (C) 2022 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . +// diff --git a/columnar/src/column_values/compact_space/blank_range.rs b/columnar/src/column_values/compact_space/blank_range.rs new file mode 100644 index 0000000000..a1f265f00d --- /dev/null +++ b/columnar/src/column_values/compact_space/blank_range.rs @@ -0,0 +1,43 @@ +use std::ops::RangeInclusive; + +/// The range of a blank in value space. +/// +/// A blank is an unoccupied space in the data. +/// Use try_into() to construct. +/// A range has to have at least length of 3. Invalid ranges will be rejected. +/// +/// Ordered by range length. +#[derive(Debug, Eq, PartialEq, Clone)] +pub(crate) struct BlankRange { + blank_range: RangeInclusive, +} +impl TryFrom> for BlankRange { + type Error = &'static str; + fn try_from(range: RangeInclusive) -> Result { + let blank_size = range.end().saturating_sub(*range.start()); + if blank_size < 2 { + Err("invalid range") + } else { + Ok(BlankRange { blank_range: range }) + } + } +} +impl BlankRange { + pub(crate) fn blank_size(&self) -> u128 { + self.blank_range.end() - self.blank_range.start() + 1 + } + pub(crate) fn blank_range(&self) -> RangeInclusive { + self.blank_range.clone() + } +} + +impl Ord for BlankRange { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + self.blank_size().cmp(&other.blank_size()) + } +} +impl PartialOrd for BlankRange { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.blank_size().cmp(&other.blank_size())) + } +} diff --git a/columnar/src/column_values/compact_space/build_compact_space.rs b/columnar/src/column_values/compact_space/build_compact_space.rs new file mode 100644 index 0000000000..51d1912ca9 --- /dev/null +++ b/columnar/src/column_values/compact_space/build_compact_space.rs @@ -0,0 +1,231 @@ +use std::collections::{BTreeSet, BinaryHeap}; +use std::iter; +use std::ops::RangeInclusive; + +use itertools::Itertools; + +use super::blank_range::BlankRange; +use super::{CompactSpace, RangeMapping}; + +/// Put the blanks for the sorted values into a binary heap +fn get_blanks(values_sorted: &BTreeSet) -> BinaryHeap { + let mut blanks: BinaryHeap = BinaryHeap::new(); + for (first, second) in values_sorted.iter().tuple_windows() { + // Correctness Overflow: the values are deduped and sorted (BTreeSet property), that means + // there's always space between two values. + let blank_range = first + 1..=second - 1; + let blank_range: Result = blank_range.try_into(); + if let Ok(blank_range) = blank_range { + blanks.push(blank_range); + } + } + + blanks +} + +struct BlankCollector { + blanks: Vec, + staged_blanks_sum: u128, +} +impl BlankCollector { + fn new() -> Self { + Self { + blanks: vec![], + staged_blanks_sum: 0, + } + } + fn stage_blank(&mut self, blank: BlankRange) { + self.staged_blanks_sum += blank.blank_size(); + self.blanks.push(blank); + } + fn drain(&mut self) -> impl Iterator + '_ { + self.staged_blanks_sum = 0; + self.blanks.drain(..) + } + fn staged_blanks_sum(&self) -> u128 { + self.staged_blanks_sum + } + fn num_staged_blanks(&self) -> usize { + self.blanks.len() + } +} +fn num_bits(val: u128) -> u8 { + (128u32 - val.leading_zeros()) as u8 +} + +/// Will collect blanks and add them to compact space if more bits are saved than cost from +/// metadata. +pub fn get_compact_space( + values_deduped_sorted: &BTreeSet, + total_num_values: u32, + cost_per_blank: usize, +) -> CompactSpace { + let mut compact_space_builder = CompactSpaceBuilder::new(); + if values_deduped_sorted.is_empty() { + return compact_space_builder.finish(); + } + + let mut blanks: BinaryHeap = get_blanks(values_deduped_sorted); + // Replace after stabilization of https://github.com/rust-lang/rust/issues/62924 + + // We start by space that's limited to min_value..=max_value + let min_value = *values_deduped_sorted.iter().next().unwrap_or(&0); + let max_value = *values_deduped_sorted.iter().last().unwrap_or(&0); + + // +1 for null, in case min and max covers the whole space, we are off by one. + let mut amplitude_compact_space = (max_value - min_value).saturating_add(1); + if min_value != 0 { + compact_space_builder.add_blanks(iter::once(0..=min_value - 1)); + } + if max_value != u128::MAX { + compact_space_builder.add_blanks(iter::once(max_value + 1..=u128::MAX)); + } + + let mut amplitude_bits: u8 = num_bits(amplitude_compact_space); + + let mut blank_collector = BlankCollector::new(); + // We will stage blanks until they reduce the compact space by at least 1 bit and then flush + // them if the metadata cost is lower than the total number of saved bits. + // Binary heap to process the gaps by their size + while let Some(blank_range) = blanks.pop() { + blank_collector.stage_blank(blank_range); + + let staged_spaces_sum: u128 = blank_collector.staged_blanks_sum(); + let amplitude_new_compact_space = amplitude_compact_space - staged_spaces_sum; + let amplitude_new_bits = num_bits(amplitude_new_compact_space); + if amplitude_bits == amplitude_new_bits { + continue; + } + let saved_bits = (amplitude_bits - amplitude_new_bits) as usize * total_num_values as usize; + // TODO: Maybe calculate exact cost of blanks and run this more expensive computation only, + // when amplitude_new_bits changes + let cost = blank_collector.num_staged_blanks() * cost_per_blank; + if cost >= saved_bits { + // Continue here, since although we walk over the blanks by size, + // we can potentially save a lot at the last bits, which are smaller blanks + // + // E.g. if the first range reduces the compact space by 1000 from 2000 to 1000, which + // saves 11-10=1 bit and the next range reduces the compact space by 950 to + // 50, which saves 10-6=4 bit + continue; + } + + amplitude_compact_space = amplitude_new_compact_space; + amplitude_bits = amplitude_new_bits; + compact_space_builder.add_blanks(blank_collector.drain().map(|blank| blank.blank_range())); + } + + // special case, when we don't collected any blanks because: + // * the data is empty (early exit) + // * the algorithm did decide it's not worth the cost, which can be the case for single values + // + // We drain one collected blank unconditionally, so the empty case is reserved for empty + // data, and therefore empty compact_space means the data is empty and no data is covered + // (conversely to all data) and we can assign null to it. + if compact_space_builder.is_empty() { + compact_space_builder.add_blanks( + blank_collector + .drain() + .map(|blank| blank.blank_range()) + .take(1), + ); + } + + let compact_space = compact_space_builder.finish(); + if max_value - min_value != u128::MAX { + debug_assert_eq!( + compact_space.amplitude_compact_space(), + amplitude_compact_space + ); + } + compact_space +} + +#[derive(Debug, Clone, Eq, PartialEq)] +struct CompactSpaceBuilder { + blanks: Vec>, +} + +impl CompactSpaceBuilder { + /// Creates a new compact space builder which will initially cover the whole space. + fn new() -> Self { + Self { blanks: Vec::new() } + } + + /// Assumes that repeated add_blank calls don't overlap and are not adjacent, + /// e.g. [3..=5, 5..=10] is not allowed + /// + /// Both of those assumptions are true when blanks are produced from sorted values. + fn add_blanks(&mut self, blank: impl Iterator>) { + self.blanks.extend(blank); + } + + fn is_empty(&self) -> bool { + self.blanks.is_empty() + } + + /// Convert blanks to covered space and assign null value + fn finish(mut self) -> CompactSpace { + // sort by start. ranges are not allowed to overlap + self.blanks.sort_unstable_by_key(|blank| *blank.start()); + + let mut covered_space = Vec::with_capacity(self.blanks.len()); + + // begining of the blanks + if let Some(first_blank_start) = self.blanks.first().map(RangeInclusive::start) { + if *first_blank_start != 0 { + covered_space.push(0..=first_blank_start - 1); + } + } + + // Between the blanks + let between_blanks = self.blanks.iter().tuple_windows().map(|(left, right)| { + assert!( + left.end() < right.start(), + "overlapping or adjacent ranges detected" + ); + *left.end() + 1..=*right.start() - 1 + }); + covered_space.extend(between_blanks); + + // end of the blanks + if let Some(last_blank_end) = self.blanks.last().map(RangeInclusive::end) { + if *last_blank_end != u128::MAX { + covered_space.push(last_blank_end + 1..=u128::MAX); + } + } + + if covered_space.is_empty() { + covered_space.push(0..=0); // empty data case + }; + + let mut compact_start: u64 = 1; // 0 is reserved for `null` + let mut ranges_mapping: Vec = Vec::with_capacity(covered_space.len()); + for cov in covered_space { + let range_mapping = super::RangeMapping { + value_range: cov, + compact_start, + }; + let covered_range_len = range_mapping.range_length(); + ranges_mapping.push(range_mapping); + compact_start += covered_range_len; + } + // println!("num ranges {}", ranges_mapping.len()); + CompactSpace { ranges_mapping } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_binary_heap_pop_order() { + let mut blanks: BinaryHeap = BinaryHeap::new(); + blanks.push((0..=10).try_into().unwrap()); + blanks.push((100..=200).try_into().unwrap()); + blanks.push((100..=110).try_into().unwrap()); + assert_eq!(blanks.pop().unwrap().blank_size(), 101); + assert_eq!(blanks.pop().unwrap().blank_size(), 11); + } +} diff --git a/columnar/src/column_values/compact_space/mod.rs b/columnar/src/column_values/compact_space/mod.rs new file mode 100644 index 0000000000..86f58c67d0 --- /dev/null +++ b/columnar/src/column_values/compact_space/mod.rs @@ -0,0 +1,813 @@ +/// This codec takes a large number space (u128) and reduces it to a compact number space. +/// +/// It will find spaces in the number range. For example: +/// +/// 100, 101, 102, 103, 104, 50000, 50001 +/// could be mapped to +/// 100..104 -> 0..4 +/// 50000..50001 -> 5..6 +/// +/// Compact space 0..=6 requires much less bits than 100..=50001 +/// +/// The codec is created to compress ip addresses, but may be employed in other use cases. +use std::{ + cmp::Ordering, + collections::BTreeSet, + io::{self, Write}, + ops::{Range, RangeInclusive}, +}; + +use common::{BinarySerializable, CountingWriter, OwnedBytes, VInt, VIntU128}; +use tantivy_bitpacker::{self, BitPacker, BitUnpacker}; + +use crate::column_values::compact_space::build_compact_space::get_compact_space; +use crate::column_values::ColumnValues; + +mod blank_range; +mod build_compact_space; + +/// The cost per blank is quite hard actually, since blanks are delta encoded, the actual cost of +/// blanks depends on the number of blanks. +/// +/// The number is taken by looking at a real dataset. It is optimized for larger datasets. +const COST_PER_BLANK_IN_BITS: usize = 36; + +#[derive(Debug, Clone, Eq, PartialEq)] +pub struct CompactSpace { + ranges_mapping: Vec, +} + +/// Maps the range from the original space to compact_start + range.len() +#[derive(Debug, Clone, Eq, PartialEq)] +struct RangeMapping { + value_range: RangeInclusive, + compact_start: u64, +} +impl RangeMapping { + fn range_length(&self) -> u64 { + (self.value_range.end() - self.value_range.start()) as u64 + 1 + } + + // The last value of the compact space in this range + fn compact_end(&self) -> u64 { + self.compact_start + self.range_length() - 1 + } +} + +impl BinarySerializable for CompactSpace { + fn serialize(&self, writer: &mut W) -> io::Result<()> { + VInt(self.ranges_mapping.len() as u64).serialize(writer)?; + + let mut prev_value = 0; + for value_range in self + .ranges_mapping + .iter() + .map(|range_mapping| &range_mapping.value_range) + { + let blank_delta_start = value_range.start() - prev_value; + VIntU128(blank_delta_start).serialize(writer)?; + prev_value = *value_range.start(); + + let blank_delta_end = value_range.end() - prev_value; + VIntU128(blank_delta_end).serialize(writer)?; + prev_value = *value_range.end(); + } + + Ok(()) + } + + fn deserialize(reader: &mut R) -> io::Result { + let num_ranges = VInt::deserialize(reader)?.0; + let mut ranges_mapping: Vec = vec![]; + let mut value = 0u128; + let mut compact_start = 1u64; // 0 is reserved for `null` + for _ in 0..num_ranges { + let blank_delta_start = VIntU128::deserialize(reader)?.0; + value += blank_delta_start; + let blank_start = value; + + let blank_delta_end = VIntU128::deserialize(reader)?.0; + value += blank_delta_end; + let blank_end = value; + + let range_mapping = RangeMapping { + value_range: blank_start..=blank_end, + compact_start, + }; + let range_length = range_mapping.range_length(); + ranges_mapping.push(range_mapping); + compact_start += range_length; + } + + Ok(Self { ranges_mapping }) + } +} + +impl CompactSpace { + /// Amplitude is the value range of the compact space including the sentinel value used to + /// identify null values. The compact space is 0..=amplitude . + /// + /// It's only used to verify we don't exceed u64 number space, which would indicate a bug. + fn amplitude_compact_space(&self) -> u128 { + self.ranges_mapping + .last() + .map(|last_range| last_range.compact_end() as u128) + .unwrap_or(1) // compact space starts at 1, 0 == null + } + + fn get_range_mapping(&self, pos: usize) -> &RangeMapping { + &self.ranges_mapping[pos] + } + + /// Returns either Ok(the value in the compact space) or if it is outside the compact space the + /// Err(position where it would be inserted) + fn u128_to_compact(&self, value: u128) -> Result { + self.ranges_mapping + .binary_search_by(|probe| { + let value_range = &probe.value_range; + if value < *value_range.start() { + Ordering::Greater + } else if value > *value_range.end() { + Ordering::Less + } else { + Ordering::Equal + } + }) + .map(|pos| { + let range_mapping = &self.ranges_mapping[pos]; + let pos_in_range = (value - range_mapping.value_range.start()) as u64; + range_mapping.compact_start + pos_in_range + }) + } + + /// Unpacks a value from compact space u64 to u128 space + fn compact_to_u128(&self, compact: u64) -> u128 { + let pos = self + .ranges_mapping + .binary_search_by_key(&compact, |range_mapping| range_mapping.compact_start) + // Correctness: Overflow. The first range starts at compact space 0, the error from + // binary search can never be 0 + .map_or_else(|e| e - 1, |v| v); + + let range_mapping = &self.ranges_mapping[pos]; + let diff = compact - range_mapping.compact_start; + range_mapping.value_range.start() + diff as u128 + } +} + +pub struct CompactSpaceCompressor { + params: IPCodecParams, +} +#[derive(Debug, Clone)] +pub struct IPCodecParams { + compact_space: CompactSpace, + bit_unpacker: BitUnpacker, + min_value: u128, + max_value: u128, + num_vals: u32, + num_bits: u8, +} + +impl CompactSpaceCompressor { + /// Taking the vals as Vec may cost a lot of memory. It is used to sort the vals. + pub fn train_from(iter: impl Iterator, num_vals: u32) -> Self { + let mut values_sorted = BTreeSet::new(); + values_sorted.extend(iter); + let total_num_values = num_vals; + + let compact_space = + get_compact_space(&values_sorted, total_num_values, COST_PER_BLANK_IN_BITS); + let amplitude_compact_space = compact_space.amplitude_compact_space(); + + assert!( + amplitude_compact_space <= u64::MAX as u128, + "case unsupported." + ); + + let num_bits = tantivy_bitpacker::compute_num_bits(amplitude_compact_space as u64); + let min_value = *values_sorted.iter().next().unwrap_or(&0); + let max_value = *values_sorted.iter().last().unwrap_or(&0); + assert_eq!( + compact_space + .u128_to_compact(max_value) + .expect("could not convert max value to compact space"), + amplitude_compact_space as u64 + ); + CompactSpaceCompressor { + params: IPCodecParams { + compact_space, + bit_unpacker: BitUnpacker::new(num_bits), + min_value, + max_value, + num_vals: total_num_values, + num_bits, + }, + } + } + + fn write_footer(self, writer: &mut impl Write) -> io::Result<()> { + let writer = &mut CountingWriter::wrap(writer); + self.params.serialize(writer)?; + + let footer_len = writer.written_bytes() as u32; + footer_len.serialize(writer)?; + + Ok(()) + } + + pub fn compress_into( + self, + vals: impl Iterator, + write: &mut impl Write, + ) -> io::Result<()> { + let mut bitpacker = BitPacker::default(); + for val in vals { + let compact = self + .params + .compact_space + .u128_to_compact(val) + .map_err(|_| { + io::Error::new( + io::ErrorKind::InvalidData, + "Could not convert value to compact_space. This is a bug.", + ) + })?; + bitpacker.write(compact, self.params.num_bits, write)?; + } + bitpacker.close(write)?; + self.write_footer(write)?; + Ok(()) + } +} + +#[derive(Debug, Clone)] +pub struct CompactSpaceDecompressor { + data: OwnedBytes, + params: IPCodecParams, +} + +impl BinarySerializable for IPCodecParams { + fn serialize(&self, writer: &mut W) -> io::Result<()> { + // header flags for future optional dictionary encoding + let footer_flags = 0u64; + footer_flags.serialize(writer)?; + + VIntU128(self.min_value).serialize(writer)?; + VIntU128(self.max_value).serialize(writer)?; + VIntU128(self.num_vals as u128).serialize(writer)?; + self.num_bits.serialize(writer)?; + + self.compact_space.serialize(writer)?; + + Ok(()) + } + + fn deserialize(reader: &mut R) -> io::Result { + let _header_flags = u64::deserialize(reader)?; + let min_value = VIntU128::deserialize(reader)?.0; + let max_value = VIntU128::deserialize(reader)?.0; + let num_vals = VIntU128::deserialize(reader)?.0 as u32; + let num_bits = u8::deserialize(reader)?; + let compact_space = CompactSpace::deserialize(reader)?; + + Ok(Self { + compact_space, + bit_unpacker: BitUnpacker::new(num_bits), + min_value, + max_value, + num_vals, + num_bits, + }) + } +} + +impl ColumnValues for CompactSpaceDecompressor { + #[inline] + fn get_val(&self, doc: u32) -> u128 { + self.get(doc) + } + + fn min_value(&self) -> u128 { + self.min_value() + } + + fn max_value(&self) -> u128 { + self.max_value() + } + + fn num_vals(&self) -> u32 { + self.params.num_vals + } + + #[inline] + fn iter(&self) -> Box + '_> { + Box::new(self.iter()) + } + + #[inline] + fn get_docids_for_value_range( + &self, + value_range: RangeInclusive, + positions_range: Range, + positions: &mut Vec, + ) { + self.get_positions_for_value_range(value_range, positions_range, positions) + } +} + +impl CompactSpaceDecompressor { + pub fn open(data: OwnedBytes) -> io::Result { + let (data_slice, footer_len_bytes) = data.split_at(data.len() - 4); + let footer_len = u32::deserialize(&mut &footer_len_bytes[..])?; + + let data_footer = &data_slice[data_slice.len() - footer_len as usize..]; + let params = IPCodecParams::deserialize(&mut &data_footer[..])?; + let decompressor = CompactSpaceDecompressor { data, params }; + + Ok(decompressor) + } + + /// Converting to compact space for the decompressor is more complex, since we may get values + /// which are outside the compact space. e.g. if we map + /// 1000 => 5 + /// 2000 => 6 + /// + /// and we want a mapping for 1005, there is no equivalent compact space. We instead return an + /// error with the index of the next range. + fn u128_to_compact(&self, value: u128) -> Result { + self.params.compact_space.u128_to_compact(value) + } + + fn compact_to_u128(&self, compact: u64) -> u128 { + self.params.compact_space.compact_to_u128(compact) + } + + /// Comparing on compact space: Random dataset 0,24 (50% random hit) - 1.05 GElements/s + /// Comparing on compact space: Real dataset 1.08 GElements/s + /// + /// Comparing on original space: Real dataset .06 GElements/s (not completely optimized) + #[inline] + pub fn get_positions_for_value_range( + &self, + value_range: RangeInclusive, + position_range: Range, + positions: &mut Vec, + ) { + if value_range.start() > value_range.end() { + return; + } + let position_range = position_range.start..position_range.end.min(self.num_vals()); + let from_value = *value_range.start(); + let to_value = *value_range.end(); + assert!(to_value >= from_value); + let compact_from = self.u128_to_compact(from_value); + let compact_to = self.u128_to_compact(to_value); + + // Quick return, if both ranges fall into the same non-mapped space, the range can't cover + // any values, so we can early exit + match (compact_to, compact_from) { + (Err(pos1), Err(pos2)) if pos1 == pos2 => return, + _ => {} + } + + let compact_from = compact_from.unwrap_or_else(|pos| { + // Correctness: Out of bounds, if this value is Err(last_index + 1), we early exit, + // since the to_value also mapps into the same non-mapped space + let range_mapping = self.params.compact_space.get_range_mapping(pos); + range_mapping.compact_start + }); + // If there is no compact space, we go to the closest upperbound compact space + let compact_to = compact_to.unwrap_or_else(|pos| { + // Correctness: Overflow, if this value is Err(0), we early exit, + // since the from_value also mapps into the same non-mapped space + + // Get end of previous range + let pos = pos - 1; + let range_mapping = self.params.compact_space.get_range_mapping(pos); + range_mapping.compact_end() + }); + + let range = compact_from..=compact_to; + + let scan_num_docs = position_range.end - position_range.start; + + let step_size = 4; + let cutoff = position_range.start + scan_num_docs - scan_num_docs % step_size; + + let mut push_if_in_range = |idx, val| { + if range.contains(&val) { + positions.push(idx); + } + }; + let get_val = |idx| self.params.bit_unpacker.get(idx, &self.data); + // unrolled loop + for idx in (position_range.start..cutoff).step_by(step_size as usize) { + let idx1 = idx; + let idx2 = idx + 1; + let idx3 = idx + 2; + let idx4 = idx + 3; + let val1 = get_val(idx1); + let val2 = get_val(idx2); + let val3 = get_val(idx3); + let val4 = get_val(idx4); + push_if_in_range(idx1, val1); + push_if_in_range(idx2, val2); + push_if_in_range(idx3, val3); + push_if_in_range(idx4, val4); + } + + // handle rest + for idx in cutoff..position_range.end { + push_if_in_range(idx, get_val(idx)); + } + } + + #[inline] + fn iter_compact(&self) -> impl Iterator + '_ { + (0..self.params.num_vals).map(move |idx| self.params.bit_unpacker.get(idx, &self.data)) + } + + #[inline] + fn iter(&self) -> impl Iterator + '_ { + // TODO: Performance. It would be better to iterate on the ranges and check existence via + // the bit_unpacker. + self.iter_compact() + .map(|compact| self.compact_to_u128(compact)) + } + + #[inline] + pub fn get(&self, idx: u32) -> u128 { + let compact = self.params.bit_unpacker.get(idx, &self.data); + self.compact_to_u128(compact) + } + + pub fn min_value(&self) -> u128 { + self.params.min_value + } + + pub fn max_value(&self) -> u128 { + self.params.max_value + } +} + +// TODO reenable what can be reenabled. +// #[cfg(test)] +// mod tests { +// +// use super::*; +// use crate::column::format_version::read_format_version; +// use crate::column::column_footer::read_null_index_footer; +// use crate::column::serialize::U128Header; +// use crate::column::{open_u128, serialize_u128}; +// +// #[test] +// fn compact_space_test() { +// let ips = &[ +// 2u128, 4u128, 1000, 1001, 1002, 1003, 1004, 1005, 1008, 1010, 1012, 1260, +// ] +// .into_iter() +// .collect(); +// let compact_space = get_compact_space(ips, ips.len() as u32, 11); +// let amplitude = compact_space.amplitude_compact_space(); +// assert_eq!(amplitude, 17); +// assert_eq!(1, compact_space.u128_to_compact(2).unwrap()); +// assert_eq!(2, compact_space.u128_to_compact(3).unwrap()); +// assert_eq!(compact_space.u128_to_compact(100).unwrap_err(), 1); +// +// for (num1, num2) in (0..3).tuple_windows() { +// assert_eq!( +// compact_space.get_range_mapping(num1).compact_end() + 1, +// compact_space.get_range_mapping(num2).compact_start +// ); +// } +// +// let mut output: Vec = Vec::new(); +// compact_space.serialize(&mut output).unwrap(); +// +// assert_eq!( +// compact_space, +// CompactSpace::deserialize(&mut &output[..]).unwrap() +// ); +// +// for ip in ips { +// let compact = compact_space.u128_to_compact(*ip).unwrap(); +// assert_eq!(compact_space.compact_to_u128(compact), *ip); +// } +// } +// +// #[test] +// fn compact_space_amplitude_test() { +// let ips = &[100000u128, 1000000].into_iter().collect(); +// let compact_space = get_compact_space(ips, ips.len() as u32, 1); +// let amplitude = compact_space.amplitude_compact_space(); +// assert_eq!(amplitude, 2); +// } +// +// fn test_all(mut data: OwnedBytes, expected: &[u128]) { +// let _header = U128Header::deserialize(&mut data); +// let decompressor = CompactSpaceDecompressor::open(data).unwrap(); +// for (idx, expected_val) in expected.iter().cloned().enumerate() { +// let val = decompressor.get(idx as u32); +// assert_eq!(val, expected_val); +// +// let test_range = |range: RangeInclusive| { +// let expected_positions = expected +// .iter() +// .positions(|val| range.contains(val)) +// .map(|pos| pos as u32) +// .collect::>(); +// let mut positions = Vec::new(); +// decompressor.get_positions_for_value_range( +// range, +// 0..decompressor.num_vals(), +// &mut positions, +// ); +// assert_eq!(positions, expected_positions); +// }; +// +// test_range(expected_val.saturating_sub(1)..=expected_val); +// test_range(expected_val..=expected_val); +// test_range(expected_val..=expected_val.saturating_add(1)); +// test_range(expected_val.saturating_sub(1)..=expected_val.saturating_add(1)); +// } +// } +// +// fn test_aux_vals(u128_vals: &[u128]) -> OwnedBytes { +// let mut out = Vec::new(); +// serialize_u128( +// || u128_vals.iter().cloned(), +// u128_vals.len() as u32, +// &mut out, +// ) +// .unwrap(); +// +// let data = OwnedBytes::new(out); +// let (data, _format_version) = read_format_version(data).unwrap(); +// let (data, _null_index_footer) = read_null_index_footer(data).unwrap(); +// test_all(data.clone(), u128_vals); +// +// data +// } +// +// #[test] +// fn test_range_1() { +// let vals = &[ +// 1u128, +// 100u128, +// 3u128, +// 99999u128, +// 100000u128, +// 100001u128, +// 4_000_211_221u128, +// 4_000_211_222u128, +// 333u128, +// ]; +// let mut data = test_aux_vals(vals); +// +// let _header = U128Header::deserialize(&mut data); +// let decomp = CompactSpaceDecompressor::open(data).unwrap(); +// let complete_range = 0..vals.len() as u32; +// for (pos, val) in vals.iter().enumerate() { +// let val = *val; +// let pos = pos as u32; +// let mut positions = Vec::new(); +// decomp.get_positions_for_value_range(val..=val, pos..pos + 1, &mut positions); +// assert_eq!(positions, vec![pos]); +// } +// +// handle docid range out of bounds +// let positions: Vec = get_positions_for_value_range_helper(&decomp, 0..=1, 1..u32::MAX); +// assert!(positions.is_empty()); +// +// let positions = +// get_positions_for_value_range_helper(&decomp, 0..=1, complete_range.clone()); +// assert_eq!(positions, vec![0]); +// let positions = +// get_positions_for_value_range_helper(&decomp, 0..=2, complete_range.clone()); +// assert_eq!(positions, vec![0]); +// let positions = +// get_positions_for_value_range_helper(&decomp, 0..=3, complete_range.clone()); +// assert_eq!(positions, vec![0, 2]); +// assert_eq!( +// get_positions_for_value_range_helper( +// &decomp, +// 99999u128..=99999u128, +// complete_range.clone() +// ), +// vec![3] +// ); +// assert_eq!( +// get_positions_for_value_range_helper( +// &decomp, +// 99999u128..=100000u128, +// complete_range.clone() +// ), +// vec![3, 4] +// ); +// assert_eq!( +// get_positions_for_value_range_helper( +// &decomp, +// 99998u128..=100000u128, +// complete_range.clone() +// ), +// vec![3, 4] +// ); +// assert_eq!( +// &get_positions_for_value_range_helper( +// &decomp, +// 99998u128..=99999u128, +// complete_range.clone() +// ), +// &[3] +// ); +// assert!(get_positions_for_value_range_helper( +// &decomp, +// 99998u128..=99998u128, +// complete_range.clone() +// ) +// .is_empty()); +// assert_eq!( +// &get_positions_for_value_range_helper( +// &decomp, +// 333u128..=333u128, +// complete_range.clone() +// ), +// &[8] +// ); +// assert_eq!( +// &get_positions_for_value_range_helper( +// &decomp, +// 332u128..=333u128, +// complete_range.clone() +// ), +// &[8] +// ); +// assert_eq!( +// &get_positions_for_value_range_helper( +// &decomp, +// 332u128..=334u128, +// complete_range.clone() +// ), +// &[8] +// ); +// assert_eq!( +// &get_positions_for_value_range_helper( +// &decomp, +// 333u128..=334u128, +// complete_range.clone() +// ), +// &[8] +// ); +// +// assert_eq!( +// &get_positions_for_value_range_helper( +// &decomp, +// 4_000_211_221u128..=5_000_000_000u128, +// complete_range +// ), +// &[6, 7] +// ); +// } +// +// #[test] +// fn test_empty() { +// let vals = &[]; +// let data = test_aux_vals(vals); +// let _decomp = CompactSpaceDecompressor::open(data).unwrap(); +// } +// +// #[test] +// fn test_range_2() { +// let vals = &[ +// 100u128, +// 99999u128, +// 100000u128, +// 100001u128, +// 4_000_211_221u128, +// 4_000_211_222u128, +// 333u128, +// ]; +// let mut data = test_aux_vals(vals); +// let _header = U128Header::deserialize(&mut data); +// let decomp = CompactSpaceDecompressor::open(data).unwrap(); +// let complete_range = 0..vals.len() as u32; +// assert!( +// &get_positions_for_value_range_helper(&decomp, 0..=5, complete_range.clone()) +// .is_empty(), +// ); +// assert_eq!( +// &get_positions_for_value_range_helper(&decomp, 0..=100, complete_range.clone()), +// &[0] +// ); +// assert_eq!( +// &get_positions_for_value_range_helper(&decomp, 0..=105, complete_range), +// &[0] +// ); +// } +// +// fn get_positions_for_value_range_helper + ?Sized, T: PartialOrd>( +// column: &C, +// value_range: RangeInclusive, +// doc_id_range: Range, +// ) -> Vec { +// let mut positions = Vec::new(); +// column.get_docids_for_value_range(value_range, doc_id_range, &mut positions); +// positions +// } +// +// #[test] +// fn test_range_3() { +// let vals = &[ +// 200u128, +// 201, +// 202, +// 203, +// 204, +// 204, +// 206, +// 207, +// 208, +// 209, +// 210, +// 1_000_000, +// 5_000_000_000, +// ]; +// let mut out = Vec::new(); +// serialize_u128(|| vals.iter().cloned(), vals.len() as u32, &mut out).unwrap(); +// let decomp = open_u128::(OwnedBytes::new(out)).unwrap(); +// let complete_range = 0..vals.len() as u32; +// +// assert_eq!( +// get_positions_for_value_range_helper(&*decomp, 199..=200, complete_range.clone()), +// vec![0] +// ); +// +// assert_eq!( +// get_positions_for_value_range_helper(&*decomp, 199..=201, complete_range.clone()), +// vec![0, 1] +// ); +// +// assert_eq!( +// get_positions_for_value_range_helper(&*decomp, 200..=200, complete_range.clone()), +// vec![0] +// ); +// +// assert_eq!( +// get_positions_for_value_range_helper(&*decomp, 1_000_000..=1_000_000, complete_range), +// vec![11] +// ); +// } +// +// #[test] +// fn test_bug1() { +// let vals = &[9223372036854775806]; +// let _data = test_aux_vals(vals); +// } +// +// #[test] +// fn test_bug2() { +// let vals = &[340282366920938463463374607431768211455u128]; +// let _data = test_aux_vals(vals); +// } +// +// #[test] +// fn test_bug3() { +// let vals = &[340282366920938463463374607431768211454]; +// let _data = test_aux_vals(vals); +// } +// +// #[test] +// fn test_bug4() { +// let vals = &[340282366920938463463374607431768211455, 0]; +// let _data = test_aux_vals(vals); +// } +// +// #[test] +// fn test_first_large_gaps() { +// let vals = &[1_000_000_000u128; 100]; +// let _data = test_aux_vals(vals); +// } +// use itertools::Itertools; +// use proptest::prelude::*; +// +// fn num_strategy() -> impl Strategy { +// prop_oneof![ +// 1 => prop::num::u128::ANY.prop_map(|num| u128::MAX - (num % 10) ), +// 1 => prop::num::u128::ANY.prop_map(|num| i64::MAX as u128 + 5 - (num % 10) ), +// 1 => prop::num::u128::ANY.prop_map(|num| i128::MAX as u128 + 5 - (num % 10) ), +// 1 => prop::num::u128::ANY.prop_map(|num| num % 10 ), +// 20 => prop::num::u128::ANY, +// ] +// } +// +// proptest! { +// #![proptest_config(ProptestConfig::with_cases(10))] +// +// #[test] +// fn compress_decompress_random(vals in proptest::collection::vec(num_strategy() +// , 1..1000)) { +// let _data = test_aux_vals(&vals); +// } +// } +// } +// diff --git a/columnar/src/column_values/gcd.rs b/columnar/src/column_values/gcd.rs new file mode 100644 index 0000000000..2dc1b788e9 --- /dev/null +++ b/columnar/src/column_values/gcd.rs @@ -0,0 +1,75 @@ +use std::num::NonZeroU64; + +use fastdivide::DividerU64; + +/// Compute the gcd of two non null numbers. +/// +/// It is recommended, but not required, to feed values such that `large >= small`. +fn compute_gcd(mut large: NonZeroU64, mut small: NonZeroU64) -> NonZeroU64 { + loop { + let rem: u64 = large.get() % small; + if let Some(new_small) = NonZeroU64::new(rem) { + (large, small) = (small, new_small); + } else { + return small; + } + } +} + +// Find GCD for iterator of numbers +pub fn find_gcd(numbers: impl Iterator) -> Option { + let mut numbers = numbers.flat_map(NonZeroU64::new); + let mut gcd: NonZeroU64 = numbers.next()?; + if gcd.get() == 1 { + return Some(gcd); + } + + let mut gcd_divider = DividerU64::divide_by(gcd.get()); + for val in numbers { + let remainder = val.get() - (gcd_divider.divide(val.get())) * gcd.get(); + if remainder == 0 { + continue; + } + gcd = compute_gcd(val, gcd); + if gcd.get() == 1 { + return Some(gcd); + } + + gcd_divider = DividerU64::divide_by(gcd.get()); + } + Some(gcd) +} + +#[cfg(test)] +mod tests { + use std::num::NonZeroU64; + + use crate::column_values::gcd::{compute_gcd, find_gcd}; + + #[test] + fn test_compute_gcd() { + let test_compute_gcd_aux = |large, small, expected| { + let large = NonZeroU64::new(large).unwrap(); + let small = NonZeroU64::new(small).unwrap(); + let expected = NonZeroU64::new(expected).unwrap(); + assert_eq!(compute_gcd(small, large), expected); + assert_eq!(compute_gcd(large, small), expected); + }; + test_compute_gcd_aux(1, 4, 1); + test_compute_gcd_aux(2, 4, 2); + test_compute_gcd_aux(10, 25, 5); + test_compute_gcd_aux(25, 25, 25); + } + + #[test] + fn find_gcd_test() { + assert_eq!(find_gcd([0].into_iter()), None); + assert_eq!(find_gcd([0, 10].into_iter()), NonZeroU64::new(10)); + assert_eq!(find_gcd([10, 0].into_iter()), NonZeroU64::new(10)); + assert_eq!(find_gcd([].into_iter()), None); + assert_eq!(find_gcd([15, 30, 5, 10].into_iter()), NonZeroU64::new(5)); + assert_eq!(find_gcd([15, 16, 10].into_iter()), NonZeroU64::new(1)); + assert_eq!(find_gcd([0, 5, 5, 5].into_iter()), NonZeroU64::new(5)); + assert_eq!(find_gcd([0, 0].into_iter()), None); + } +} diff --git a/columnar/src/column_values/line.rs b/columnar/src/column_values/line.rs new file mode 100644 index 0000000000..3434ce824b --- /dev/null +++ b/columnar/src/column_values/line.rs @@ -0,0 +1,222 @@ +use std::io; +use std::num::NonZeroU32; + +use common::{BinarySerializable, VInt}; + +use crate::column_values::ColumnValues; + +const MID_POINT: u64 = (1u64 << 32) - 1u64; + +/// `Line` describes a line function `y: ax + b` using integer +/// arithmetics. +/// +/// The slope is in fact a decimal split into a 32 bit integer value, +/// and a 32-bit decimal value. +/// +/// The multiplication then becomes. +/// `y = m * x >> 32 + b` +#[derive(Debug, Clone, Copy, Default)] +pub struct Line { + slope: u64, + intercept: u64, +} + +/// Compute the line slope. +/// +/// This function has the nice property of being +/// invariant by translation. +/// ` +/// compute_slope(y0, y1) +/// = compute_slope(y0 + X % 2^64, y1 + X % 2^64) +/// ` +fn compute_slope(y0: u64, y1: u64, num_vals: NonZeroU32) -> u64 { + let dy = y1.wrapping_sub(y0); + let sign = dy <= (1 << 63); + let abs_dy = if sign { + y1.wrapping_sub(y0) + } else { + y0.wrapping_sub(y1) + }; + if abs_dy >= 1 << 32 { + // This is outside of realm we handle. + // Let's just bail. + return 0u64; + } + + let abs_slope = (abs_dy << 32) / num_vals.get() as u64; + if sign { + abs_slope + } else { + // The complement does indeed create the + // opposite decreasing slope... + // + // Intuitively (without the bitshifts and % u64::MAX) + // ``` + // (x + shift)*(u64::MAX - abs_slope) + // - (x * (u64::MAX - abs_slope)) + // = - shift * abs_slope + // ``` + u64::MAX - abs_slope + } +} + +impl Line { + #[inline(always)] + pub fn eval(&self, x: u32) -> u64 { + let linear_part = ((x as u64).wrapping_mul(self.slope) >> 32) as i32 as u64; + self.intercept.wrapping_add(linear_part) + } + + // Same as train, but the intercept is only estimated from provided sample positions + pub fn estimate(sample_positions_and_values: &[(u64, u64)]) -> Self { + let first_val = sample_positions_and_values[0].1; + let last_val = sample_positions_and_values[sample_positions_and_values.len() - 1].1; + let num_vals = sample_positions_and_values[sample_positions_and_values.len() - 1].0 + 1; + Self::train_from( + first_val, + last_val, + num_vals as u32, + sample_positions_and_values.iter().cloned(), + ) + } + + // Intercept is only computed from provided positions + fn train_from( + first_val: u64, + last_val: u64, + num_vals: u32, + positions_and_values: impl Iterator, + ) -> Self { + // TODO replace with let else + let idx_last_val = if let Some(idx_last_val) = NonZeroU32::new(num_vals - 1) { + idx_last_val + } else { + return Line::default(); + }; + + let y0 = first_val; + let y1 = last_val; + + // We first independently pick our slope. + let slope = compute_slope(y0, y1, idx_last_val); + + // We picked our slope. Note that it does not have to be perfect. + // Now we need to compute the best intercept. + // + // Intuitively, the best intercept is such that line passes through one of the + // `(i, ys[])`. + // + // The best intercept therefore has the form + // `y[i] - line.eval(i)` (using wrapping arithmetics). + // In other words, the best intercept is one of the `y - Line::eval(ys[i])` + // and our task is just to pick the one that minimizes our error. + // + // Without sorting our values, this is a difficult problem. + // We however rely on the following trick... + // + // We only focus on the case where the interpolation is half decent. + // If the line interpolation is doing its job on a dataset suited for it, + // we can hope that the maximum error won't be larger than `u64::MAX / 2`. + // + // In other words, even without the intercept the values `y - Line::eval(ys[i])` will all be + // within an interval that takes less than half of the modulo space of `u64`. + // + // Our task is therefore to identify this interval. + // Here we simply translate all of our values by `y0 - 2^63` and pick the min. + let mut line = Line { + slope, + intercept: 0, + }; + let heuristic_shift = y0.wrapping_sub(MID_POINT); + line.intercept = positions_and_values + .map(|(pos, y)| y.wrapping_sub(line.eval(pos as u32))) + .min_by_key(|&val| val.wrapping_sub(heuristic_shift)) + .unwrap_or(0u64); //< Never happens. + line + } + + /// Returns a line that attemps to approximate a function + /// f: i in 0..[ys.num_vals()) -> ys[i]. + /// + /// - The approximation is always lower than the actual value. + /// Or more rigorously, formally `f(i).wrapping_sub(ys[i])` is small + /// for any i in [0..ys.len()). + /// - It computes without panicking for any value of it. + /// + /// This function is only invariable by translation if all of the + /// `ys` are packaged into half of the space. (See heuristic below) + pub fn train(ys: &dyn ColumnValues) -> Self { + let first_val = ys.iter().next().unwrap(); + let last_val = ys.iter().nth(ys.num_vals() as usize - 1).unwrap(); + Self::train_from( + first_val, + last_val, + ys.num_vals(), + ys.iter().enumerate().map(|(pos, val)| (pos as u64, val)), + ) + } +} + +impl BinarySerializable for Line { + fn serialize(&self, writer: &mut W) -> io::Result<()> { + VInt(self.slope).serialize(writer)?; + VInt(self.intercept).serialize(writer)?; + Ok(()) + } + + fn deserialize(reader: &mut R) -> io::Result { + let slope = VInt::deserialize(reader)?.0; + let intercept = VInt::deserialize(reader)?.0; + Ok(Line { slope, intercept }) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::column_values::VecColumn; + + /// Test training a line and ensuring that the maximum difference between + /// the data points and the line is `expected`. + /// + /// This function operates translation over the data for better coverage. + #[track_caller] + fn test_line_interpol_with_translation(ys: &[u64], expected: Option) { + let mut translations = vec![0, 100, u64::MAX / 2, u64::MAX, u64::MAX - 1]; + translations.extend_from_slice(ys); + for translation in translations { + let translated_ys: Vec = ys + .iter() + .copied() + .map(|y| y.wrapping_add(translation)) + .collect(); + let largest_err = test_eval_max_err(&translated_ys); + assert_eq!(largest_err, expected); + } + } + + fn test_eval_max_err(ys: &[u64]) -> Option { + let line = Line::train(&VecColumn::from(&ys)); + ys.iter() + .enumerate() + .map(|(x, y)| y.wrapping_sub(line.eval(x as u32))) + .max() + } + + #[test] + fn test_train() { + test_line_interpol_with_translation(&[11, 11, 11, 12, 12, 13], Some(1)); + test_line_interpol_with_translation(&[13, 12, 12, 11, 11, 11], Some(1)); + test_line_interpol_with_translation(&[13, 13, 12, 11, 11, 11], Some(1)); + test_line_interpol_with_translation(&[13, 13, 12, 11, 11, 11], Some(1)); + test_line_interpol_with_translation(&[u64::MAX - 1, 0, 0, 1], Some(1)); + test_line_interpol_with_translation(&[u64::MAX - 1, u64::MAX, 0, 1], Some(0)); + test_line_interpol_with_translation(&[0, 1, 2, 3, 5], Some(0)); + test_line_interpol_with_translation(&[1, 2, 3, 4], Some(0)); + + let data: Vec = (0..255).collect(); + test_line_interpol_with_translation(&data, Some(0)); + let data: Vec = (0..255).map(|el| el * 2).collect(); + test_line_interpol_with_translation(&data, Some(0)); + } +} diff --git a/columnar/src/column_values/linear.rs b/columnar/src/column_values/linear.rs new file mode 100644 index 0000000000..9f4e2d38c3 --- /dev/null +++ b/columnar/src/column_values/linear.rs @@ -0,0 +1,230 @@ +use std::io::{self, Write}; + +use common::{BinarySerializable, OwnedBytes}; +use tantivy_bitpacker::{compute_num_bits, BitPacker, BitUnpacker}; + +use super::line::Line; +use super::serialize::NormalizedHeader; +use super::{ColumnValues, FastFieldCodec, FastFieldCodecType}; + +/// Depending on the field type, a different +/// fast field is required. +#[derive(Clone)] +pub struct LinearReader { + data: OwnedBytes, + linear_params: LinearParams, + header: NormalizedHeader, +} + +impl ColumnValues for LinearReader { + #[inline] + fn get_val(&self, doc: u32) -> u64 { + let interpoled_val: u64 = self.linear_params.line.eval(doc); + let bitpacked_diff = self.linear_params.bit_unpacker.get(doc, &self.data); + interpoled_val.wrapping_add(bitpacked_diff) + } + + #[inline(always)] + fn min_value(&self) -> u64 { + // The LinearReader assumes a normalized vector. + 0u64 + } + + #[inline(always)] + fn max_value(&self) -> u64 { + self.header.max_value + } + + #[inline] + fn num_vals(&self) -> u32 { + self.header.num_vals + } +} + +/// Fastfield serializer, which tries to guess values by linear interpolation +/// and stores the difference bitpacked. +pub struct LinearCodec; + +#[derive(Debug, Clone)] +struct LinearParams { + line: Line, + bit_unpacker: BitUnpacker, +} + +impl BinarySerializable for LinearParams { + fn serialize(&self, writer: &mut W) -> io::Result<()> { + self.line.serialize(writer)?; + self.bit_unpacker.bit_width().serialize(writer)?; + Ok(()) + } + + fn deserialize(reader: &mut R) -> io::Result { + let line = Line::deserialize(reader)?; + let bit_width = u8::deserialize(reader)?; + Ok(Self { + line, + bit_unpacker: BitUnpacker::new(bit_width), + }) + } +} + +impl FastFieldCodec for LinearCodec { + const CODEC_TYPE: FastFieldCodecType = FastFieldCodecType::Linear; + + type Reader = LinearReader; + + /// Opens a fast field given a file. + fn open_from_bytes(mut data: OwnedBytes, header: NormalizedHeader) -> io::Result { + let linear_params = LinearParams::deserialize(&mut data)?; + Ok(LinearReader { + data, + linear_params, + header, + }) + } + + /// Creates a new fast field serializer. + fn serialize(column: &dyn ColumnValues, write: &mut impl Write) -> io::Result<()> { + assert_eq!(column.min_value(), 0); + let line = Line::train(column); + + let max_offset_from_line = column + .iter() + .enumerate() + .map(|(pos, actual_value)| { + let calculated_value = line.eval(pos as u32); + actual_value.wrapping_sub(calculated_value) + }) + .max() + .unwrap(); + + let num_bits = compute_num_bits(max_offset_from_line); + let linear_params = LinearParams { + line, + bit_unpacker: BitUnpacker::new(num_bits), + }; + linear_params.serialize(write)?; + + let mut bit_packer = BitPacker::new(); + for (pos, actual_value) in column.iter().enumerate() { + let calculated_value = line.eval(pos as u32); + let offset = actual_value.wrapping_sub(calculated_value); + bit_packer.write(offset, num_bits, write)?; + } + bit_packer.close(write)?; + + Ok(()) + } + + /// estimation for linear interpolation is hard because, you don't know + /// where the local maxima for the deviation of the calculated value are and + /// the offset to shift all values to >=0 is also unknown. + #[allow(clippy::question_mark)] + fn estimate(column: &dyn ColumnValues) -> Option { + if column.num_vals() < 3 { + return None; // disable compressor for this case + } + + let limit_num_vals = column.num_vals().min(100_000); + + let num_samples = 100; + let step_size = (limit_num_vals / num_samples).max(1); // 20 samples + let mut sample_positions_and_values: Vec<_> = Vec::new(); + for (pos, val) in column.iter().enumerate().step_by(step_size as usize) { + sample_positions_and_values.push((pos as u64, val)); + } + + let line = Line::estimate(&sample_positions_and_values); + + let estimated_bit_width = sample_positions_and_values + .into_iter() + .map(|(pos, actual_value)| { + let interpolated_val = line.eval(pos as u32); + actual_value.wrapping_sub(interpolated_val) + }) + .map(|diff| ((diff as f32 * 1.5) * 2.0) as u64) + .map(compute_num_bits) + .max() + .unwrap_or(0); + + // Extrapolate to whole column + let num_bits = (estimated_bit_width as u64 * column.num_vals() as u64) + 64; + let num_bits_uncompressed = 64 * column.num_vals(); + Some(num_bits as f32 / num_bits_uncompressed as f32) + } +} + +#[cfg(test)] +mod tests { + use rand::RngCore; + + use super::*; + use crate::column_values::tests; + + fn create_and_validate(data: &[u64], name: &str) -> Option<(f32, f32)> { + tests::create_and_validate::(data, name) + } + + #[test] + fn test_compression() { + let data = (10..=6_000_u64).collect::>(); + let (estimate, actual_compression) = + create_and_validate(&data, "simple monotonically large").unwrap(); + + assert_le!(actual_compression, 0.001); + assert_le!(estimate, 0.02); + } + + #[test] + fn test_with_codec_datasets() { + let data_sets = tests::get_codec_test_datasets(); + for (mut data, name) in data_sets { + create_and_validate(&data, name); + data.reverse(); + create_and_validate(&data, name); + } + } + #[test] + fn linear_interpol_fast_field_test_large_amplitude() { + let data = vec![ + i64::MAX as u64 / 2, + i64::MAX as u64 / 3, + i64::MAX as u64 / 2, + ]; + + create_and_validate(&data, "large amplitude"); + } + + #[test] + fn overflow_error_test() { + let data = vec![1572656989877777, 1170935903116329, 720575940379279, 0]; + create_and_validate(&data, "overflow test"); + } + + #[test] + fn linear_interpol_fast_concave_data() { + let data = vec![0, 1, 2, 5, 8, 10, 20, 50]; + create_and_validate(&data, "concave data"); + } + #[test] + fn linear_interpol_fast_convex_data() { + let data = vec![0, 40, 60, 70, 75, 77]; + create_and_validate(&data, "convex data"); + } + #[test] + fn linear_interpol_fast_field_test_simple() { + let data = (10..=20_u64).collect::>(); + create_and_validate(&data, "simple monotonically"); + } + + #[test] + fn linear_interpol_fast_field_rand() { + let mut rng = rand::thread_rng(); + for _ in 0..50 { + let mut data = (0..10_000).map(|_| rng.next_u64()).collect::>(); + create_and_validate(&data, "random"); + data.reverse(); + create_and_validate(&data, "random"); + } + } +} diff --git a/columnar/src/column_values/main.rs b/columnar/src/column_values/main.rs new file mode 100644 index 0000000000..aad4218662 --- /dev/null +++ b/columnar/src/column_values/main.rs @@ -0,0 +1,222 @@ +#[macro_use] +extern crate prettytable; +use std::collections::HashSet; +use std::env; +use std::io::BufRead; +use std::net::{IpAddr, Ipv6Addr}; +use std::str::FromStr; + +use common::OwnedBytes; +use fastfield_codecs::{open_u128, serialize_u128, Column, FastFieldCodecType, VecColumn}; +use itertools::Itertools; +use measure_time::print_time; +use prettytable::{Cell, Row, Table}; + +fn print_set_stats(ip_addrs: &[u128]) { + println!("NumIps\t{}", ip_addrs.len()); + let ip_addr_set: HashSet = ip_addrs.iter().cloned().collect(); + println!("NumUniqueIps\t{}", ip_addr_set.len()); + let ratio_unique = ip_addr_set.len() as f64 / ip_addrs.len() as f64; + println!("RatioUniqueOverTotal\t{ratio_unique:.4}"); + + // histogram + let mut ip_addrs = ip_addrs.to_vec(); + ip_addrs.sort(); + let mut cnts: Vec = ip_addrs + .into_iter() + .dedup_with_count() + .map(|(cnt, _)| cnt) + .collect(); + cnts.sort(); + + let top_256_cnt: usize = cnts.iter().rev().take(256).sum(); + let top_128_cnt: usize = cnts.iter().rev().take(128).sum(); + let top_64_cnt: usize = cnts.iter().rev().take(64).sum(); + let top_8_cnt: usize = cnts.iter().rev().take(8).sum(); + let total: usize = cnts.iter().sum(); + + println!("{}", total); + println!("{}", top_256_cnt); + println!("{}", top_128_cnt); + println!("Percentage Top8 {:02}", top_8_cnt as f32 / total as f32); + println!("Percentage Top64 {:02}", top_64_cnt as f32 / total as f32); + println!("Percentage Top128 {:02}", top_128_cnt as f32 / total as f32); + println!("Percentage Top256 {:02}", top_256_cnt as f32 / total as f32); + + let mut cnts: Vec<(usize, usize)> = cnts.into_iter().dedup_with_count().collect(); + cnts.sort_by(|a, b| { + if a.1 == b.1 { + a.0.cmp(&b.0) + } else { + b.1.cmp(&a.1) + } + }); +} + +fn ip_dataset() -> Vec { + let mut ip_addr_v4 = 0; + + let stdin = std::io::stdin(); + let ip_addrs: Vec = stdin + .lock() + .lines() + .flat_map(|line| { + let line = line.unwrap(); + let line = line.trim(); + let ip_addr = IpAddr::from_str(line.trim()).ok()?; + if ip_addr.is_ipv4() { + ip_addr_v4 += 1; + } + let ip_addr_v6: Ipv6Addr = match ip_addr { + IpAddr::V4(v4) => v4.to_ipv6_mapped(), + IpAddr::V6(v6) => v6, + }; + Some(ip_addr_v6) + }) + .map(|ip_v6| u128::from_be_bytes(ip_v6.octets())) + .collect(); + + println!("IpAddrsAny\t{}", ip_addrs.len()); + println!("IpAddrsV4\t{}", ip_addr_v4); + + ip_addrs +} + +fn bench_ip() { + let dataset = ip_dataset(); + print_set_stats(&dataset); + + // Chunks + { + let mut data = vec![]; + for dataset in dataset.chunks(500_000) { + serialize_u128(|| dataset.iter().cloned(), dataset.len() as u32, &mut data).unwrap(); + } + let compression = data.len() as f64 / (dataset.len() * 16) as f64; + println!("Compression 50_000 chunks {:.4}", compression); + println!( + "Num Bits per elem {:.2}", + (data.len() * 8) as f32 / dataset.len() as f32 + ); + } + + let mut data = vec![]; + { + print_time!("creation"); + serialize_u128(|| dataset.iter().cloned(), dataset.len() as u32, &mut data).unwrap(); + } + + let compression = data.len() as f64 / (dataset.len() * 16) as f64; + println!("Compression {:.2}", compression); + println!( + "Num Bits per elem {:.2}", + (data.len() * 8) as f32 / dataset.len() as f32 + ); + + let decompressor = open_u128::(OwnedBytes::new(data)).unwrap(); + // Sample some ranges + let mut doc_values = Vec::new(); + for value in dataset.iter().take(1110).skip(1100).cloned() { + doc_values.clear(); + print_time!("get range"); + decompressor.get_docids_for_value_range( + value..=value, + 0..decompressor.num_vals(), + &mut doc_values, + ); + println!("{:?}", doc_values.len()); + } +} + +fn main() { + if env::args().nth(1).unwrap() == "bench_ip" { + bench_ip(); + return; + } + + let mut table = Table::new(); + + // Add a row per time + table.add_row(row!["", "Compression Ratio", "Compression Estimation"]); + + for (data, data_set_name) in get_codec_test_data_sets() { + let results: Vec<(f32, f32, FastFieldCodecType)> = [ + serialize_with_codec(&data, FastFieldCodecType::Bitpacked), + serialize_with_codec(&data, FastFieldCodecType::Linear), + serialize_with_codec(&data, FastFieldCodecType::BlockwiseLinear), + ] + .into_iter() + .flatten() + .collect(); + let best_compression_ratio_codec = results + .iter() + .min_by(|&res1, &res2| res1.partial_cmp(res2).unwrap()) + .cloned() + .unwrap(); + + table.add_row(Row::new(vec![Cell::new(data_set_name).style_spec("Bbb")])); + for (est, comp, codec_type) in results { + let est_cell = est.to_string(); + let ratio_cell = comp.to_string(); + let style = if comp == best_compression_ratio_codec.1 { + "Fb" + } else { + "" + }; + table.add_row(Row::new(vec![ + Cell::new(&format!("{codec_type:?}")).style_spec("bFg"), + Cell::new(&ratio_cell).style_spec(style), + Cell::new(&est_cell).style_spec(""), + ])); + } + } + + table.printstd(); +} + +pub fn get_codec_test_data_sets() -> Vec<(Vec, &'static str)> { + let mut data_and_names = vec![]; + + let data = (1000..=200_000_u64).collect::>(); + data_and_names.push((data, "Autoincrement")); + + let mut current_cumulative = 0; + let data = (1..=200_000_u64) + .map(|num| { + let num = (num as f32 + num as f32).log10() as u64; + current_cumulative += num; + current_cumulative + }) + .collect::>(); + // let data = (1..=200000_u64).map(|num| num + num).collect::>(); + data_and_names.push((data, "Monotonically increasing concave")); + + let mut current_cumulative = 0; + let data = (1..=200_000_u64) + .map(|num| { + let num = (200_000.0 - num as f32).log10() as u64; + current_cumulative += num; + current_cumulative + }) + .collect::>(); + data_and_names.push((data, "Monotonically increasing convex")); + + let data = (1000..=200_000_u64) + .map(|num| num + rand::random::() as u64) + .collect::>(); + data_and_names.push((data, "Almost monotonically increasing")); + + data_and_names +} + +pub fn serialize_with_codec( + data: &[u64], + codec_type: FastFieldCodecType, +) -> Option<(f32, f32, FastFieldCodecType)> { + let col = VecColumn::from(data); + let estimation = fastfield_codecs::estimate(&col, codec_type)?; + let mut out = Vec::new(); + fastfield_codecs::serialize(&col, &mut out, &[codec_type]).ok()?; + let actual_compression = out.len() as f32 / (col.num_vals() * 8) as f32; + Some((estimation, actual_compression, codec_type)) +} diff --git a/columnar/src/column_values/mod.rs b/columnar/src/column_values/mod.rs new file mode 100644 index 0000000000..c6e793b3aa --- /dev/null +++ b/columnar/src/column_values/mod.rs @@ -0,0 +1,333 @@ +#![warn(missing_docs)] +#![cfg_attr(all(feature = "unstable", test), feature(test))] + +//! # `fastfield_codecs` +//! +//! - Columnar storage of data for tantivy [`Column`]. +//! - Encode data in different codecs. +//! - Monotonically map values to u64/u128 + +#[cfg(test)] +mod tests; + +use std::io; +use std::io::Write; +use std::sync::Arc; + +use common::{BinarySerializable, OwnedBytes}; +use compact_space::CompactSpaceDecompressor; +use monotonic_mapping::{ + StrictlyMonotonicMappingInverter, StrictlyMonotonicMappingToInternal, + StrictlyMonotonicMappingToInternalBaseval, StrictlyMonotonicMappingToInternalGCDBaseval, +}; +use serialize::{Header, U128Header}; + +mod bitpacked; +mod blockwise_linear; +mod compact_space; +mod line; +mod linear; +pub(crate) mod monotonic_mapping; +// mod monotonic_mapping_u128; + +mod column; +mod column_with_cardinality; +mod gcd; +pub mod serialize; + +pub use self::column::{monotonic_map_column, ColumnValues, IterColumn, VecColumn}; +pub use self::monotonic_mapping::{MonotonicallyMappableToU64, StrictlyMonotonicFn}; +// pub use self::monotonic_mapping_u128::MonotonicallyMappableToU128; +pub use self::serialize::{serialize_and_load, serialize_column_values, NormalizedHeader}; +use crate::column_values::bitpacked::BitpackedCodec; +use crate::column_values::blockwise_linear::BlockwiseLinearCodec; +use crate::column_values::linear::LinearCodec; + +#[derive(PartialEq, Eq, PartialOrd, Ord, Debug, Clone, Copy)] +#[repr(u8)] +/// Available codecs to use to encode the u64 (via [`MonotonicallyMappableToU64`]) converted data. +pub enum FastFieldCodecType { + /// Bitpack all values in the value range. The number of bits is defined by the amplitude + /// `column.max_value() - column.min_value()` + Bitpacked = 1, + /// Linear interpolation puts a line between the first and last value and then bitpacks the + /// values by the offset from the line. The number of bits is defined by the max deviation from + /// the line. + Linear = 2, + /// Same as [`FastFieldCodecType::Linear`], but encodes in blocks of 512 elements. + BlockwiseLinear = 3, +} + +impl BinarySerializable for FastFieldCodecType { + fn serialize(&self, wrt: &mut W) -> io::Result<()> { + self.to_code().serialize(wrt) + } + + fn deserialize(reader: &mut R) -> io::Result { + let code = u8::deserialize(reader)?; + let codec_type: Self = Self::from_code(code) + .ok_or_else(|| io::Error::new(io::ErrorKind::InvalidData, "Unknown code `{code}.`"))?; + Ok(codec_type) + } +} + +impl FastFieldCodecType { + pub(crate) fn to_code(self) -> u8 { + self as u8 + } + + pub(crate) fn from_code(code: u8) -> Option { + match code { + 1 => Some(Self::Bitpacked), + 2 => Some(Self::Linear), + 3 => Some(Self::BlockwiseLinear), + _ => None, + } + } +} + +#[derive(PartialEq, Eq, PartialOrd, Ord, Debug, Clone, Copy)] +#[repr(u8)] +/// Available codecs to use to encode the u128 (via [`MonotonicallyMappableToU128`]) converted data. +pub enum U128FastFieldCodecType { + /// This codec takes a large number space (u128) and reduces it to a compact number space, by + /// removing the holes. + CompactSpace = 1, +} + +impl BinarySerializable for U128FastFieldCodecType { + fn serialize(&self, wrt: &mut W) -> io::Result<()> { + self.to_code().serialize(wrt) + } + + fn deserialize(reader: &mut R) -> io::Result { + let code = u8::deserialize(reader)?; + let codec_type: Self = Self::from_code(code) + .ok_or_else(|| io::Error::new(io::ErrorKind::InvalidData, "Unknown code `{code}.`"))?; + Ok(codec_type) + } +} + +impl U128FastFieldCodecType { + pub(crate) fn to_code(self) -> u8 { + self as u8 + } + + pub(crate) fn from_code(code: u8) -> Option { + match code { + 1 => Some(Self::CompactSpace), + _ => None, + } + } +} + +/// Returns the correct codec reader wrapped in the `Arc` for the data. +// pub fn open_u128( +// bytes: OwnedBytes, +// ) -> io::Result>> { +// todo!(); +// // let (bytes, _format_version) = read_format_version(bytes)?; +// // let (mut bytes, _null_index_footer) = read_null_index_footer(bytes)?; +// // let header = U128Header::deserialize(&mut bytes)?; +// // assert_eq!(header.codec_type, U128FastFieldCodecType::CompactSpace); +// // let reader = CompactSpaceDecompressor::open(bytes)?; +// // let inverted: StrictlyMonotonicMappingInverter> = +// // StrictlyMonotonicMappingToInternal::::new().into(); +// // Ok(Arc::new(monotonic_map_column(reader, inverted))) +// } + +/// Returns the correct codec reader wrapped in the `Arc` for the data. +pub fn open_u64_mapped( + mut bytes: OwnedBytes, +) -> io::Result>> { + let header = Header::deserialize(&mut bytes)?; + match header.codec_type { + FastFieldCodecType::Bitpacked => open_specific_codec::(bytes, &header), + FastFieldCodecType::Linear => open_specific_codec::(bytes, &header), + FastFieldCodecType::BlockwiseLinear => { + open_specific_codec::(bytes, &header) + } + } +} + +fn open_specific_codec( + bytes: OwnedBytes, + header: &Header, +) -> io::Result>> { + let normalized_header = header.normalized(); + let reader = C::open_from_bytes(bytes, normalized_header)?; + let min_value = header.min_value; + if let Some(gcd) = header.gcd { + let mapping = StrictlyMonotonicMappingInverter::from( + StrictlyMonotonicMappingToInternalGCDBaseval::new(gcd.get(), min_value), + ); + Ok(Arc::new(monotonic_map_column(reader, mapping))) + } else { + let mapping = StrictlyMonotonicMappingInverter::from( + StrictlyMonotonicMappingToInternalBaseval::new(min_value), + ); + Ok(Arc::new(monotonic_map_column(reader, mapping))) + } +} + +/// The FastFieldSerializerEstimate trait is required on all variants +/// of fast field compressions, to decide which one to choose. +pub(crate) trait FastFieldCodec: 'static { + /// A codex needs to provide a unique name and id, which is + /// used for debugging and de/serialization. + const CODEC_TYPE: FastFieldCodecType; + + type Reader: ColumnValues + 'static; + + /// Reads the metadata and returns the CodecReader + fn open_from_bytes(bytes: OwnedBytes, header: NormalizedHeader) -> io::Result; + + /// Serializes the data using the serializer into write. + /// + /// The column iterator should be preferred over using column `get_val` method for + /// performance reasons. + fn serialize(column: &dyn ColumnValues, write: &mut impl Write) -> io::Result<()>; + + /// Returns an estimate of the compression ratio. + /// If the codec is not applicable, returns `None`. + /// + /// The baseline is uncompressed 64bit data. + /// + /// It could make sense to also return a value representing + /// computational complexity. + fn estimate(column: &dyn ColumnValues) -> Option; +} + +/// The list of all available codecs for u64 convertible data. +pub const ALL_CODEC_TYPES: [FastFieldCodecType; 3] = [ + FastFieldCodecType::Bitpacked, + FastFieldCodecType::BlockwiseLinear, + FastFieldCodecType::Linear, +]; + +#[cfg(all(test, feature = "unstable"))] +mod bench { + use std::sync::Arc; + + use common::OwnedBytes; + use rand::rngs::StdRng; + use rand::{Rng, SeedableRng}; + use test::{self, Bencher}; + + use super::*; + + fn get_data() -> Vec { + let mut rng = StdRng::seed_from_u64(2u64); + let mut data: Vec<_> = (100..55000_u64) + .map(|num| num + rng.gen::() as u64) + .collect(); + data.push(99_000); + data.insert(1000, 2000); + data.insert(2000, 100); + data.insert(3000, 4100); + data.insert(4000, 100); + data.insert(5000, 800); + data + } + + #[inline(never)] + fn value_iter() -> impl Iterator { + 0..20_000 + } + fn get_reader_for_bench(data: &[u64]) -> Codec::Reader { + let mut bytes = Vec::new(); + let min_value = *data.iter().min().unwrap(); + let data = data.iter().map(|el| *el - min_value).collect::>(); + let col = VecColumn::from(&data); + let normalized_header = NormalizedHeader { + num_vals: col.num_vals(), + max_value: col.max_value(), + }; + Codec::serialize(&VecColumn::from(&data), &mut bytes).unwrap(); + Codec::open_from_bytes(OwnedBytes::new(bytes), normalized_header).unwrap() + } + fn bench_get(b: &mut Bencher, data: &[u64]) { + let col = get_reader_for_bench::(data); + b.iter(|| { + let mut sum = 0u64; + for pos in value_iter() { + let val = col.get_val(pos as u32); + sum = sum.wrapping_add(val); + } + sum + }); + } + + #[inline(never)] + fn bench_get_dynamic_helper(b: &mut Bencher, col: Arc) { + b.iter(|| { + let mut sum = 0u64; + for pos in value_iter() { + let val = col.get_val(pos as u32); + sum = sum.wrapping_add(val); + } + sum + }); + } + + fn bench_get_dynamic(b: &mut Bencher, data: &[u64]) { + let col = Arc::new(get_reader_for_bench::(data)); + bench_get_dynamic_helper(b, col); + } + fn bench_create(b: &mut Bencher, data: &[u64]) { + let min_value = *data.iter().min().unwrap(); + let data = data.iter().map(|el| *el - min_value).collect::>(); + + let mut bytes = Vec::new(); + b.iter(|| { + bytes.clear(); + Codec::serialize(&VecColumn::from(&data), &mut bytes).unwrap(); + }); + } + + #[bench] + fn bench_fastfield_bitpack_create(b: &mut Bencher) { + let data: Vec<_> = get_data(); + bench_create::(b, &data); + } + #[bench] + fn bench_fastfield_linearinterpol_create(b: &mut Bencher) { + let data: Vec<_> = get_data(); + bench_create::(b, &data); + } + #[bench] + fn bench_fastfield_multilinearinterpol_create(b: &mut Bencher) { + let data: Vec<_> = get_data(); + bench_create::(b, &data); + } + #[bench] + fn bench_fastfield_bitpack_get(b: &mut Bencher) { + let data: Vec<_> = get_data(); + bench_get::(b, &data); + } + #[bench] + fn bench_fastfield_bitpack_get_dynamic(b: &mut Bencher) { + let data: Vec<_> = get_data(); + bench_get_dynamic::(b, &data); + } + #[bench] + fn bench_fastfield_linearinterpol_get(b: &mut Bencher) { + let data: Vec<_> = get_data(); + bench_get::(b, &data); + } + #[bench] + fn bench_fastfield_linearinterpol_get_dynamic(b: &mut Bencher) { + let data: Vec<_> = get_data(); + bench_get_dynamic::(b, &data); + } + #[bench] + fn bench_fastfield_multilinearinterpol_get(b: &mut Bencher) { + let data: Vec<_> = get_data(); + bench_get::(b, &data); + } + #[bench] + fn bench_fastfield_multilinearinterpol_get_dynamic(b: &mut Bencher) { + let data: Vec<_> = get_data(); + bench_get_dynamic::(b, &data); + } +} diff --git a/columnar/src/column_values/monotonic_mapping.rs b/columnar/src/column_values/monotonic_mapping.rs new file mode 100644 index 0000000000..10bb27319d --- /dev/null +++ b/columnar/src/column_values/monotonic_mapping.rs @@ -0,0 +1,264 @@ +use std::marker::PhantomData; + +use fastdivide::DividerU64; + +use crate::RowId; + +/// Monotonic maps a value to u64 value space. +/// Monotonic mapping enables `PartialOrd` on u64 space without conversion to original space. +pub trait MonotonicallyMappableToU64: 'static + PartialOrd + Copy + Send + Sync { + /// Converts a value to u64. + /// + /// Internally all fast field values are encoded as u64. + fn to_u64(self) -> u64; + + /// Converts a value from u64 + /// + /// Internally all fast field values are encoded as u64. + /// **Note: To be used for converting encoded Term, Posting values.** + fn from_u64(val: u64) -> Self; +} + +/// Values need to be strictly monotonic mapped to a `Internal` value (u64 or u128) that can be +/// used in fast field codecs. +/// +/// The monotonic mapping is required so that `PartialOrd` can be used on `Internal` without +/// converting to `External`. +/// +/// All strictly monotonic functions are invertible because they are guaranteed to have a one-to-one +/// mapping from their range to their domain. The `inverse` method is required when opening a codec, +/// so a value can be converted back to its original domain (e.g. ip address or f64) from its +/// internal representation. +pub trait StrictlyMonotonicFn { + /// Strictly monotonically maps the value from External to Internal. + fn mapping(&self, inp: External) -> Internal; + /// Inverse of `mapping`. Maps the value from Internal to External. + fn inverse(&self, out: Internal) -> External; +} + +/// Inverts a strictly monotonic mapping from `StrictlyMonotonicFn` to +/// `StrictlyMonotonicFn`. +/// +/// # Warning +/// +/// This type comes with a footgun. A type being strictly monotonic does not impose that the inverse +/// mapping is strictly monotonic over the entire space External. e.g. a -> a * 2. Use at your own +/// risks. +pub(crate) struct StrictlyMonotonicMappingInverter { + orig_mapping: T, +} +impl From for StrictlyMonotonicMappingInverter { + fn from(orig_mapping: T) -> Self { + Self { orig_mapping } + } +} + +impl StrictlyMonotonicFn for StrictlyMonotonicMappingInverter +where T: StrictlyMonotonicFn +{ + #[inline(always)] + fn mapping(&self, val: To) -> From { + self.orig_mapping.inverse(val) + } + + #[inline(always)] + fn inverse(&self, val: From) -> To { + self.orig_mapping.mapping(val) + } +} + +/// Applies the strictly monotonic mapping from `T` without any additional changes. +pub(crate) struct StrictlyMonotonicMappingToInternal { + _phantom: PhantomData, +} + +impl StrictlyMonotonicMappingToInternal { + pub(crate) fn new() -> StrictlyMonotonicMappingToInternal { + Self { + _phantom: PhantomData, + } + } +} + +// TODO +// impl +// StrictlyMonotonicFn for StrictlyMonotonicMappingToInternal +// where T: MonotonicallyMappableToU128 +// { +// #[inline(always)] +// fn mapping(&self, inp: External) -> u128 { +// External::to_u128(inp) +// } + +// #[inline(always)] +// fn inverse(&self, out: u128) -> External { +// External::from_u128(out) +// } +// } + +impl + StrictlyMonotonicFn for StrictlyMonotonicMappingToInternal +where T: MonotonicallyMappableToU64 +{ + #[inline(always)] + fn mapping(&self, inp: External) -> u64 { + External::to_u64(inp) + } + + #[inline(always)] + fn inverse(&self, out: u64) -> External { + External::from_u64(out) + } +} + +/// Mapping dividing by gcd and a base value. +/// +/// The function is assumed to be only called on values divided by passed +/// gcd value. (It is necessary for the function to be monotonic.) +pub(crate) struct StrictlyMonotonicMappingToInternalGCDBaseval { + gcd_divider: DividerU64, + gcd: u64, + min_value: u64, +} +impl StrictlyMonotonicMappingToInternalGCDBaseval { + pub(crate) fn new(gcd: u64, min_value: u64) -> Self { + let gcd_divider = DividerU64::divide_by(gcd); + Self { + gcd_divider, + gcd, + min_value, + } + } +} +impl StrictlyMonotonicFn + for StrictlyMonotonicMappingToInternalGCDBaseval +{ + #[inline(always)] + fn mapping(&self, inp: External) -> u64 { + self.gcd_divider + .divide(External::to_u64(inp) - self.min_value) + } + + #[inline(always)] + fn inverse(&self, out: u64) -> External { + External::from_u64(self.min_value + out * self.gcd) + } +} + +/// Strictly monotonic mapping with a base value. +pub(crate) struct StrictlyMonotonicMappingToInternalBaseval { + min_value: u64, +} +impl StrictlyMonotonicMappingToInternalBaseval { + #[inline(always)] + pub(crate) fn new(min_value: u64) -> Self { + Self { min_value } + } +} + +impl StrictlyMonotonicFn + for StrictlyMonotonicMappingToInternalBaseval +{ + #[inline(always)] + fn mapping(&self, val: External) -> u64 { + External::to_u64(val) - self.min_value + } + + #[inline(always)] + fn inverse(&self, val: u64) -> External { + External::from_u64(self.min_value + val) + } +} + +impl MonotonicallyMappableToU64 for u64 { + #[inline(always)] + fn to_u64(self) -> u64 { + self + } + + #[inline(always)] + fn from_u64(val: u64) -> Self { + val + } +} + +impl MonotonicallyMappableToU64 for i64 { + #[inline(always)] + fn to_u64(self) -> u64 { + common::i64_to_u64(self) + } + + #[inline(always)] + fn from_u64(val: u64) -> Self { + common::u64_to_i64(val) + } +} + +impl MonotonicallyMappableToU64 for bool { + #[inline(always)] + fn to_u64(self) -> u64 { + u64::from(self) + } + + #[inline(always)] + fn from_u64(val: u64) -> Self { + val > 0 + } +} + +impl MonotonicallyMappableToU64 for RowId { + #[inline(always)] + fn to_u64(self) -> u64 { + u64::from(self) + } + + #[inline(always)] + fn from_u64(val: u64) -> RowId { + val as RowId + } +} + +// TODO remove me. +// Tantivy should refuse NaN values and work with NotNaN internally. +impl MonotonicallyMappableToU64 for f64 { + #[inline(always)] + fn to_u64(self) -> u64 { + common::f64_to_u64(self) + } + + #[inline(always)] + fn from_u64(val: u64) -> Self { + common::u64_to_f64(val) + } +} + +#[cfg(test)] +mod tests { + + use super::*; + + #[test] + fn strictly_monotonic_test() { + // identity mapping + test_round_trip(&StrictlyMonotonicMappingToInternal::::new(), 100u64); + // round trip to i64 + test_round_trip(&StrictlyMonotonicMappingToInternal::::new(), 100u64); + // TODO + // identity mapping + // test_round_trip(&StrictlyMonotonicMappingToInternal::::new(), 100u128); + + // base value to i64 round trip + let mapping = StrictlyMonotonicMappingToInternalBaseval::new(100); + test_round_trip::<_, _, u64>(&mapping, 100i64); + // base value and gcd to u64 round trip + let mapping = StrictlyMonotonicMappingToInternalGCDBaseval::new(10, 100); + test_round_trip::<_, _, u64>(&mapping, 100u64); + } + + fn test_round_trip, K: std::fmt::Debug + Eq + Copy, L>( + mapping: &T, + test_val: K, + ) { + assert_eq!(mapping.inverse(mapping.mapping(test_val)), test_val); + } +} diff --git a/columnar/src/column_values/monotonic_mapping_u128.rs b/columnar/src/column_values/monotonic_mapping_u128.rs new file mode 100644 index 0000000000..83758d2b15 --- /dev/null +++ b/columnar/src/column_values/monotonic_mapping_u128.rs @@ -0,0 +1,40 @@ +use std::net::Ipv6Addr; + +/// Montonic maps a value to u128 value space +/// Monotonic mapping enables `PartialOrd` on u128 space without conversion to original space. +pub trait MonotonicallyMappableToU128: 'static + PartialOrd + Copy + Send + Sync { + /// Converts a value to u128. + /// + /// Internally all fast field values are encoded as u64. + fn to_u128(self) -> u128; + + /// Converts a value from u128 + /// + /// Internally all fast field values are encoded as u64. + /// **Note: To be used for converting encoded Term, Posting values.** + fn from_u128(val: u128) -> Self; +} + +impl MonotonicallyMappableToU128 for u128 { + fn to_u128(self) -> u128 { + self + } + + fn from_u128(val: u128) -> Self { + val + } +} + +impl MonotonicallyMappableToU128 for Ipv6Addr { + fn to_u128(self) -> u128 { + ip_to_u128(self) + } + + fn from_u128(val: u128) -> Self { + Ipv6Addr::from(val.to_be_bytes()) + } +} + +fn ip_to_u128(ip_addr: Ipv6Addr) -> u128 { + u128::from_be_bytes(ip_addr.octets()) +} diff --git a/columnar/src/column_values/serialize.rs b/columnar/src/column_values/serialize.rs new file mode 100644 index 0000000000..57c108280b --- /dev/null +++ b/columnar/src/column_values/serialize.rs @@ -0,0 +1,343 @@ +// Copyright (C) 2022 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use std::io; +use std::num::NonZeroU64; +use std::sync::Arc; + +use common::{BinarySerializable, OwnedBytes, VInt}; +use log::warn; + +use super::bitpacked::BitpackedCodec; +use super::blockwise_linear::BlockwiseLinearCodec; +use super::linear::LinearCodec; +use super::monotonic_mapping::{ + StrictlyMonotonicFn, StrictlyMonotonicMappingToInternal, + StrictlyMonotonicMappingToInternalGCDBaseval, +}; +use super::{ + monotonic_map_column, ColumnValues, FastFieldCodec, FastFieldCodecType, + MonotonicallyMappableToU64, U128FastFieldCodecType, VecColumn, ALL_CODEC_TYPES, +}; + +/// The normalized header gives some parameters after applying the following +/// normalization of the vector: +/// `val -> (val - min_value) / gcd` +/// +/// By design, after normalization, `min_value = 0` and `gcd = 1`. +#[derive(Debug, Copy, Clone)] +pub struct NormalizedHeader { + /// The number of values in the underlying column. + pub num_vals: u32, + /// The max value of the underlying column. + pub max_value: u64, +} + +#[derive(Debug, Copy, Clone)] +pub(crate) struct Header { + pub num_vals: u32, + pub min_value: u64, + pub max_value: u64, + pub gcd: Option, + pub codec_type: FastFieldCodecType, +} + +impl Header { + pub fn normalized(self) -> NormalizedHeader { + let gcd = self.gcd.map(|gcd| gcd.get()).unwrap_or(1); + let gcd_min_val_mapping = + StrictlyMonotonicMappingToInternalGCDBaseval::new(gcd, self.min_value); + + let max_value = gcd_min_val_mapping.mapping(self.max_value); + NormalizedHeader { + num_vals: self.num_vals, + max_value, + } + } + + pub(crate) fn normalize_column(&self, from_column: C) -> impl ColumnValues { + normalize_column(from_column, self.min_value, self.gcd) + } + + pub fn compute_header( + column: impl ColumnValues, + codecs: &[FastFieldCodecType], + ) -> Option
{ + let num_vals = column.num_vals(); + let min_value = column.min_value(); + let max_value = column.max_value(); + let gcd = super::gcd::find_gcd(column.iter().map(|val| val - min_value)) + .filter(|gcd| gcd.get() > 1u64); + let normalized_column = normalize_column(column, min_value, gcd); + let codec_type = detect_codec(normalized_column, codecs)?; + Some(Header { + num_vals, + min_value, + max_value, + gcd, + codec_type, + }) + } +} + +#[derive(Debug, Copy, Clone, PartialEq, Eq)] +pub(crate) struct U128Header { + pub num_vals: u32, + pub codec_type: U128FastFieldCodecType, +} + +impl BinarySerializable for U128Header { + fn serialize(&self, writer: &mut W) -> io::Result<()> { + VInt(self.num_vals as u64).serialize(writer)?; + self.codec_type.serialize(writer)?; + Ok(()) + } + + fn deserialize(reader: &mut R) -> io::Result { + let num_vals = VInt::deserialize(reader)?.0 as u32; + let codec_type = U128FastFieldCodecType::deserialize(reader)?; + Ok(U128Header { + num_vals, + codec_type, + }) + } +} + +fn normalize_column( + from_column: C, + min_value: u64, + gcd: Option, +) -> impl ColumnValues { + let gcd = gcd.map(|gcd| gcd.get()).unwrap_or(1); + let mapping = StrictlyMonotonicMappingToInternalGCDBaseval::new(gcd, min_value); + monotonic_map_column(from_column, mapping) +} + +impl BinarySerializable for Header { + fn serialize(&self, writer: &mut W) -> io::Result<()> { + VInt(self.num_vals as u64).serialize(writer)?; + VInt(self.min_value).serialize(writer)?; + VInt(self.max_value - self.min_value).serialize(writer)?; + if let Some(gcd) = self.gcd { + VInt(gcd.get()).serialize(writer)?; + } else { + VInt(0u64).serialize(writer)?; + } + self.codec_type.serialize(writer)?; + Ok(()) + } + + fn deserialize(reader: &mut R) -> io::Result { + let num_vals = VInt::deserialize(reader)?.0 as u32; + let min_value = VInt::deserialize(reader)?.0; + let amplitude = VInt::deserialize(reader)?.0; + let max_value = min_value + amplitude; + let gcd_u64 = VInt::deserialize(reader)?.0; + let codec_type = FastFieldCodecType::deserialize(reader)?; + Ok(Header { + num_vals, + min_value, + max_value, + gcd: NonZeroU64::new(gcd_u64), + codec_type, + }) + } +} + +/// Return estimated compression for given codec in the value range [0.0..1.0], where 1.0 means no +/// compression. +pub(crate) fn estimate( + typed_column: impl ColumnValues, + codec_type: FastFieldCodecType, +) -> Option { + let column = monotonic_map_column(typed_column, StrictlyMonotonicMappingToInternal::::new()); + let min_value = column.min_value(); + let gcd = super::gcd::find_gcd(column.iter().map(|val| val - min_value)) + .filter(|gcd| gcd.get() > 1u64); + let mapping = StrictlyMonotonicMappingToInternalGCDBaseval::new( + gcd.map(|gcd| gcd.get()).unwrap_or(1u64), + min_value, + ); + let normalized_column = monotonic_map_column(&column, mapping); + match codec_type { + FastFieldCodecType::Bitpacked => BitpackedCodec::estimate(&normalized_column), + FastFieldCodecType::Linear => LinearCodec::estimate(&normalized_column), + FastFieldCodecType::BlockwiseLinear => BlockwiseLinearCodec::estimate(&normalized_column), + } +} + +// TODO +/// Serializes u128 values with the compact space codec. +// pub fn serialize_u128_new I, I: Iterator>( +// value_index: ColumnIndex, +// iter_gen: F, +// num_vals: u32, +// output: &mut impl io::Write, +// ) -> io::Result<()> { +// let header = U128Header { +// num_vals, +// codec_type: U128FastFieldCodecType::CompactSpace, +// }; +// header.serialize(output)?; +// let compressor = CompactSpaceCompressor::train_from(iter_gen(), num_vals); +// compressor.compress_into(iter_gen(), output).unwrap(); + +// let null_index_footer = ColumnFooter { +// cardinality: value_index.get_cardinality(), +// null_index_codec: NullIndexCodec::Full, +// null_index_byte_range: 0..0, +// }; +// append_null_index_footer(output, null_index_footer)?; +// append_format_version(output)?; + +// Ok(()) +// } + +/// Serializes the column with the codec with the best estimate on the data. +pub fn serialize_column_values( + typed_column: impl ColumnValues, + codecs: &[FastFieldCodecType], + output: &mut impl io::Write, +) -> io::Result<()> { + let column = monotonic_map_column(typed_column, StrictlyMonotonicMappingToInternal::::new()); + let header = Header::compute_header(&column, codecs).ok_or_else(|| { + io::Error::new( + io::ErrorKind::InvalidInput, + format!( + "Data cannot be serialized with this list of codec. {:?}", + codecs + ), + ) + })?; + header.serialize(output)?; + let normalized_column = header.normalize_column(column); + assert_eq!(normalized_column.min_value(), 0u64); + serialize_given_codec(normalized_column, header.codec_type, output)?; + Ok(()) +} + +fn detect_codec( + column: impl ColumnValues, + codecs: &[FastFieldCodecType], +) -> Option { + let mut estimations = Vec::new(); + for &codec in codecs { + let estimation_opt = match codec { + FastFieldCodecType::Bitpacked => BitpackedCodec::estimate(&column), + FastFieldCodecType::Linear => LinearCodec::estimate(&column), + FastFieldCodecType::BlockwiseLinear => BlockwiseLinearCodec::estimate(&column), + }; + if let Some(estimation) = estimation_opt { + estimations.push((estimation, codec)); + } + } + if let Some(broken_estimation) = estimations.iter().find(|estimation| estimation.0.is_nan()) { + warn!( + "broken estimation for fast field codec {:?}", + broken_estimation.1 + ); + } + // removing nan values for codecs with broken calculations, and max values which disables + // codecs + estimations.retain(|estimation| !estimation.0.is_nan() && estimation.0 != f32::MAX); + estimations.sort_by(|(score_left, _), (score_right, _)| score_left.total_cmp(score_right)); + Some(estimations.first()?.1) +} + +pub(crate) fn serialize_given_codec( + column: impl ColumnValues, + codec_type: FastFieldCodecType, + output: &mut impl io::Write, +) -> io::Result<()> { + match codec_type { + FastFieldCodecType::Bitpacked => { + BitpackedCodec::serialize(&column, output)?; + } + FastFieldCodecType::Linear => { + LinearCodec::serialize(&column, output)?; + } + FastFieldCodecType::BlockwiseLinear => { + BlockwiseLinearCodec::serialize(&column, output)?; + } + } + Ok(()) +} + +/// Helper function to serialize a column (autodetect from all codecs) and then open it +pub fn serialize_and_load( + column: &[T], +) -> Arc> { + let mut buffer = Vec::new(); + super::serialize_column_values(&VecColumn::from(&column), &ALL_CODEC_TYPES, &mut buffer) + .unwrap(); + super::open_u64_mapped(OwnedBytes::new(buffer)).unwrap() +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_serialize_deserialize_u128_header() { + let original = U128Header { + num_vals: 11, + codec_type: U128FastFieldCodecType::CompactSpace, + }; + let mut out = Vec::new(); + original.serialize(&mut out).unwrap(); + let restored = U128Header::deserialize(&mut &out[..]).unwrap(); + assert_eq!(restored, original); + } + + #[test] + fn test_serialize_deserialize() { + let original = [1u64, 5u64, 10u64]; + let restored: Vec = serialize_and_load(&original[..]).iter().collect(); + assert_eq!(&restored, &original[..]); + } + + #[test] + fn test_fastfield_bool_size_bitwidth_1() { + let mut buffer = Vec::new(); + let col = VecColumn::from(&[false, true][..]); + serialize_column_values(&col, &ALL_CODEC_TYPES, &mut buffer).unwrap(); + // TODO put the header as a footer so that it serves as a padding. + // 5 bytes of header, 1 byte of value, 7 bytes of padding. + assert_eq!(buffer.len(), 5 + 1 + 7); + } + + #[test] + fn test_fastfield_bool_bit_size_bitwidth_0() { + let mut buffer = Vec::new(); + let col = VecColumn::from(&[true][..]); + serialize_column_values(&col, &ALL_CODEC_TYPES, &mut buffer).unwrap(); + // 5 bytes of header, 0 bytes of value, 7 bytes of padding. + assert_eq!(buffer.len(), 5 + 7); + } + + #[test] + fn test_fastfield_gcd() { + let mut buffer = Vec::new(); + let vals: Vec = (0..80).map(|val| (val % 7) * 1_000u64).collect(); + let col = VecColumn::from(&vals[..]); + serialize_column_values(&col, &[FastFieldCodecType::Bitpacked], &mut buffer).unwrap(); + // Values are stored over 3 bits. + assert_eq!(buffer.len(), 7 + (3 * 80 / 8) + 7); + } +} diff --git a/columnar/src/column_values/tests.rs b/columnar/src/column_values/tests.rs new file mode 100644 index 0000000000..780921394c --- /dev/null +++ b/columnar/src/column_values/tests.rs @@ -0,0 +1,309 @@ +use proptest::prelude::*; +use proptest::strategy::Strategy; +use proptest::{prop_oneof, proptest}; + +use super::bitpacked::BitpackedCodec; +use super::blockwise_linear::BlockwiseLinearCodec; +use super::linear::LinearCodec; +use super::serialize::Header; + +pub(crate) fn create_and_validate( + data: &[u64], + name: &str, +) -> Option<(f32, f32)> { + let col = &VecColumn::from(data); + let header = Header::compute_header(col, &[Codec::CODEC_TYPE])?; + let normalized_col = header.normalize_column(col); + let estimation = Codec::estimate(&normalized_col)?; + + let mut out = Vec::new(); + let col = VecColumn::from(data); + serialize_column_values(&col, &[Codec::CODEC_TYPE], &mut out).unwrap(); + + let actual_compression = out.len() as f32 / (data.len() as f32 * 8.0); + + let reader = super::open_u64_mapped::(OwnedBytes::new(out)).unwrap(); + assert_eq!(reader.num_vals(), data.len() as u32); + for (doc, orig_val) in data.iter().copied().enumerate() { + let val = reader.get_val(doc as u32); + assert_eq!( + val, orig_val, + "val `{val}` does not match orig_val {orig_val:?}, in data set {name}, data `{data:?}`", + ); + } + + if !data.is_empty() { + let test_rand_idx = rand::thread_rng().gen_range(0..=data.len() - 1); + let expected_positions: Vec = data + .iter() + .enumerate() + .filter(|(_, el)| **el == data[test_rand_idx]) + .map(|(pos, _)| pos as u32) + .collect(); + let mut positions = Vec::new(); + reader.get_docids_for_value_range( + data[test_rand_idx]..=data[test_rand_idx], + 0..data.len() as u32, + &mut positions, + ); + assert_eq!(expected_positions, positions); + } + Some((estimation, actual_compression)) +} + +proptest! { + #![proptest_config(ProptestConfig::with_cases(100))] + + #[test] + fn test_proptest_small_bitpacked(data in proptest::collection::vec(num_strategy(), 1..10)) { + create_and_validate::(&data, "proptest bitpacked"); + } + + #[test] + fn test_proptest_small_linear(data in proptest::collection::vec(num_strategy(), 1..10)) { + create_and_validate::(&data, "proptest linearinterpol"); + } + + #[test] + fn test_proptest_small_blockwise_linear(data in proptest::collection::vec(num_strategy(), 1..10)) { + create_and_validate::(&data, "proptest multilinearinterpol"); + } +} + +proptest! { + #![proptest_config(ProptestConfig::with_cases(10))] + + #[test] + fn test_proptest_large_bitpacked(data in proptest::collection::vec(num_strategy(), 1..6000)) { + create_and_validate::(&data, "proptest bitpacked"); + } + + #[test] + fn test_proptest_large_linear(data in proptest::collection::vec(num_strategy(), 1..6000)) { + create_and_validate::(&data, "proptest linearinterpol"); + } + + #[test] + fn test_proptest_large_blockwise_linear(data in proptest::collection::vec(num_strategy(), 1..6000)) { + create_and_validate::(&data, "proptest multilinearinterpol"); + } +} + +fn num_strategy() -> impl Strategy { + prop_oneof![ + 1 => prop::num::u64::ANY.prop_map(|num| u64::MAX - (num % 10) ), + 1 => prop::num::u64::ANY.prop_map(|num| num % 10 ), + 20 => prop::num::u64::ANY, + ] +} + +pub fn get_codec_test_datasets() -> Vec<(Vec, &'static str)> { + let mut data_and_names = vec![]; + + let data = (10..=10_000_u64).collect::>(); + data_and_names.push((data, "simple monotonically increasing")); + + data_and_names.push(( + vec![5, 6, 7, 8, 9, 10, 99, 100], + "offset in linear interpol", + )); + data_and_names.push((vec![5, 50, 3, 13, 1, 1000, 35], "rand small")); + data_and_names.push((vec![10], "single value")); + + data_and_names.push(( + vec![1572656989877777, 1170935903116329, 720575940379279, 0], + "overflow error", + )); + + data_and_names +} + +fn test_codec() { + let codec_name = format!("{:?}", C::CODEC_TYPE); + for (data, dataset_name) in get_codec_test_datasets() { + let estimate_actual_opt: Option<(f32, f32)> = + tests::create_and_validate::(&data, dataset_name); + let result = if let Some((estimate, actual)) = estimate_actual_opt { + format!("Estimate `{estimate}` Actual `{actual}`") + } else { + "Disabled".to_string() + }; + println!("Codec {codec_name}, DataSet {dataset_name}, {result}"); + } +} +#[test] +fn test_codec_bitpacking() { + test_codec::(); +} +#[test] +fn test_codec_interpolation() { + test_codec::(); +} +#[test] +fn test_codec_multi_interpolation() { + test_codec::(); +} + +use super::*; + +#[test] +fn estimation_good_interpolation_case() { + let data = (10..=20000_u64).collect::>(); + let data: VecColumn = data.as_slice().into(); + + let linear_interpol_estimation = LinearCodec::estimate(&data).unwrap(); + assert_le!(linear_interpol_estimation, 0.01); + + let multi_linear_interpol_estimation = BlockwiseLinearCodec::estimate(&data).unwrap(); + assert_le!(multi_linear_interpol_estimation, 0.2); + assert_lt!(linear_interpol_estimation, multi_linear_interpol_estimation); + + let bitpacked_estimation = BitpackedCodec::estimate(&data).unwrap(); + assert_lt!(linear_interpol_estimation, bitpacked_estimation); +} +#[test] +fn estimation_test_bad_interpolation_case() { + let data: &[u64] = &[200, 10, 10, 10, 10, 1000, 20]; + + let data: VecColumn = data.into(); + let linear_interpol_estimation = LinearCodec::estimate(&data).unwrap(); + assert_le!(linear_interpol_estimation, 0.34); + + let bitpacked_estimation = BitpackedCodec::estimate(&data).unwrap(); + assert_lt!(bitpacked_estimation, linear_interpol_estimation); +} + +#[test] +fn estimation_prefer_bitpacked() { + let data = VecColumn::from(&[10, 10, 10, 10]); + let linear_interpol_estimation = LinearCodec::estimate(&data).unwrap(); + let bitpacked_estimation = BitpackedCodec::estimate(&data).unwrap(); + assert_lt!(bitpacked_estimation, linear_interpol_estimation); +} + +#[test] +fn estimation_test_bad_interpolation_case_monotonically_increasing() { + let mut data: Vec = (201..=20000_u64).collect(); + data.push(1_000_000); + let data: VecColumn = data.as_slice().into(); + + // in this case the linear interpolation can't in fact not be worse than bitpacking, + // but the estimator adds some threshold, which leads to estimated worse behavior + let linear_interpol_estimation = LinearCodec::estimate(&data).unwrap(); + assert_le!(linear_interpol_estimation, 0.35); + + let bitpacked_estimation = BitpackedCodec::estimate(&data).unwrap(); + assert_le!(bitpacked_estimation, 0.32); + assert_le!(bitpacked_estimation, linear_interpol_estimation); +} + +#[test] +fn test_fast_field_codec_type_to_code() { + let mut count_codec = 0; + for code in 0..=255 { + if let Some(codec_type) = FastFieldCodecType::from_code(code) { + assert_eq!(codec_type.to_code(), code); + count_codec += 1; + } + } + assert_eq!(count_codec, 3); +} + +fn test_fastfield_gcd_i64_with_codec( + codec_type: FastFieldCodecType, + num_vals: usize, +) -> io::Result<()> { + let mut vals: Vec = (-4..=(num_vals as i64) - 5).map(|val| val * 1000).collect(); + let mut buffer: Vec = Vec::new(); + crate::column_values::serialize_column_values( + &VecColumn::from(&vals), + &[codec_type], + &mut buffer, + )?; + let buffer = OwnedBytes::new(buffer); + let column = crate::column_values::open_u64_mapped::(buffer.clone())?; + assert_eq!(column.get_val(0), -4000i64); + assert_eq!(column.get_val(1), -3000i64); + assert_eq!(column.get_val(2), -2000i64); + assert_eq!(column.max_value(), (num_vals as i64 - 5) * 1000); + assert_eq!(column.min_value(), -4000i64); + + // Can't apply gcd + let mut buffer_without_gcd = Vec::new(); + vals.pop(); + vals.push(1001i64); + crate::column_values::serialize_column_values( + &VecColumn::from(&vals), + &[codec_type], + &mut buffer_without_gcd, + )?; + let buffer_without_gcd = OwnedBytes::new(buffer_without_gcd); + assert!(buffer_without_gcd.len() > buffer.len()); + + Ok(()) +} + +#[test] +fn test_fastfield_gcd_i64() -> io::Result<()> { + for &codec_type in &[ + FastFieldCodecType::Bitpacked, + FastFieldCodecType::BlockwiseLinear, + FastFieldCodecType::Linear, + ] { + test_fastfield_gcd_i64_with_codec(codec_type, 5500)?; + } + Ok(()) +} + +fn test_fastfield_gcd_u64_with_codec( + codec_type: FastFieldCodecType, + num_vals: usize, +) -> io::Result<()> { + let mut vals: Vec = (1..=num_vals).map(|i| i as u64 * 1000u64).collect(); + let mut buffer: Vec = Vec::new(); + crate::column_values::serialize_column_values( + &VecColumn::from(&vals), + &[codec_type], + &mut buffer, + )?; + let buffer = OwnedBytes::new(buffer); + let column = crate::column_values::open_u64_mapped::(buffer.clone())?; + assert_eq!(column.get_val(0), 1000u64); + assert_eq!(column.get_val(1), 2000u64); + assert_eq!(column.get_val(2), 3000u64); + assert_eq!(column.max_value(), num_vals as u64 * 1000); + assert_eq!(column.min_value(), 1000u64); + + // Can't apply gcd + let mut buffer_without_gcd = Vec::new(); + vals.pop(); + vals.push(1001u64); + crate::column_values::serialize_column_values( + &VecColumn::from(&vals), + &[codec_type], + &mut buffer_without_gcd, + )?; + let buffer_without_gcd = OwnedBytes::new(buffer_without_gcd); + assert!(buffer_without_gcd.len() > buffer.len()); + Ok(()) +} + +#[test] +fn test_fastfield_gcd_u64() -> io::Result<()> { + for &codec_type in &[ + FastFieldCodecType::Bitpacked, + FastFieldCodecType::BlockwiseLinear, + FastFieldCodecType::Linear, + ] { + test_fastfield_gcd_u64_with_codec(codec_type, 5500)?; + } + Ok(()) +} + +#[test] +pub fn test_fastfield2() { + let test_fastfield = crate::column_values::serialize_and_load(&[100u64, 200u64, 300u64]); + assert_eq!(test_fastfield.get_val(0), 100); + assert_eq!(test_fastfield.get_val(1), 200); + assert_eq!(test_fastfield.get_val(2), 300); +} diff --git a/columnar/src/column_type_header.rs b/columnar/src/columnar/column_type.rs similarity index 59% rename from columnar/src/column_type_header.rs rename to columnar/src/columnar/column_type.rs index 87add92049..6d851ed380 100644 --- a/columnar/src/column_type_header.rs +++ b/columnar/src/columnar/column_type.rs @@ -2,38 +2,6 @@ use crate::utils::{place_bits, select_bits}; use crate::value::NumericalType; use crate::InvalidData; -/// Enum describing the number of values that can exist per document -/// (or per row if you will). -/// -/// The cardinality must fit on 2 bits. -#[derive(Clone, Copy, Hash, Default, Debug, PartialEq, Eq, PartialOrd, Ord)] -#[repr(u8)] -pub enum Cardinality { - /// All documents contain exactly one value. - /// Required is the default for auto-detecting the Cardinality, since it is the most strict. - #[default] - Required = 0, - /// All documents contain at most one value. - Optional = 1, - /// All documents may contain any number of values. - Multivalued = 2, -} - -impl Cardinality { - pub(crate) fn to_code(self) -> u8 { - self as u8 - } - - pub(crate) fn try_from_code(code: u8) -> Result { - match code { - 0 => Ok(Cardinality::Required), - 1 => Ok(Cardinality::Optional), - 2 => Ok(Cardinality::Multivalued), - _ => Err(InvalidData), - } - } -} - /// The column type represents the column type and can fit on 6-bits. /// /// - bits[0..3]: Column category type. @@ -125,55 +93,12 @@ impl ColumnTypeCategory { } } -/// Represents the type and cardinality of a column. -/// This is encoded over one-byte and added to a column key in the -/// columnar sstable. -/// -/// - [0..6] bits: encodes the column type -/// - [6..8] bits: encodes the cardinality -#[derive(Eq, Hash, PartialEq, Debug, Copy, Clone)] -pub struct ColumnTypeAndCardinality { - pub typ: ColumnType, - pub cardinality: Cardinality, -} - -impl ColumnTypeAndCardinality { - pub fn to_code(self) -> u8 { - place_bits::<0, 6>(self.typ.to_code()) | place_bits::<6, 8>(self.cardinality.to_code()) - } - - pub fn try_from_code(code: u8) -> Result { - let typ_code = select_bits::<0, 6>(code); - let cardinality_code = select_bits::<6, 8>(code); - let cardinality = Cardinality::try_from_code(cardinality_code)?; - let typ = ColumnType::try_from_code(typ_code)?; - assert_eq!(typ.to_code(), typ_code); - Ok(ColumnTypeAndCardinality { cardinality, typ }) - } -} - #[cfg(test)] mod tests { use std::collections::HashSet; - use super::ColumnTypeAndCardinality; - use crate::column_type_header::{Cardinality, ColumnType}; - - #[test] - fn test_column_type_header_to_code() { - let mut column_type_header_set: HashSet = HashSet::new(); - for code in u8::MIN..=u8::MAX { - if let Ok(column_type_header) = ColumnTypeAndCardinality::try_from_code(code) { - assert_eq!(column_type_header.to_code(), code); - assert!(column_type_header_set.insert(column_type_header)); - } - } - assert_eq!( - column_type_header_set.len(), - 3 /* cardinality */ * - (1 + 1 + 3) // column_types (str, bool, numerical x 3) - ); - } + use super::*; + use crate::Cardinality; #[test] fn test_column_type_to_code() { diff --git a/columnar/src/columnar/format_version.rs b/columnar/src/columnar/format_version.rs new file mode 100644 index 0000000000..1e25e69f2d --- /dev/null +++ b/columnar/src/columnar/format_version.rs @@ -0,0 +1,73 @@ +use crate::InvalidData; + +pub const VERSION_FOOTER_NUM_BYTES: usize = MAGIC_BYTES.len() + std::mem::size_of::(); + +/// We end the file by these 4 bytes just to somewhat identify that +/// this is indeed a columnar file. +const MAGIC_BYTES: [u8; 4] = [2, 113, 119, 066]; + +pub fn footer() -> [u8; VERSION_FOOTER_NUM_BYTES] { + let mut footer_bytes = [0u8; VERSION_FOOTER_NUM_BYTES]; + footer_bytes[0..4].copy_from_slice(&Version::V1.to_bytes()); + footer_bytes[4..8].copy_from_slice(&MAGIC_BYTES[..]); + footer_bytes +} + +pub fn parse_footer(footer_bytes: [u8; VERSION_FOOTER_NUM_BYTES]) -> Result { + if footer_bytes[4..8] != MAGIC_BYTES { + return Err(InvalidData); + } + Version::try_from_bytes(footer_bytes[0..4].try_into().unwrap()) +} + +#[derive(Debug, Copy, Clone, Eq, PartialEq)] +#[repr(u32)] +pub enum Version { + V1 = 1u32, +} + +impl Version { + fn to_bytes(&self) -> [u8; 4] { + (*self as u32).to_le_bytes() + } + + fn try_from_bytes(bytes: [u8; 4]) -> Result { + let code = u32::from_le_bytes(bytes); + match code { + 1u32 => Ok(Version::V1), + _ => Err(InvalidData), + } + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashSet; + + use super::*; + + #[test] + fn test_footer_dserialization() { + let parsed_version: Version = parse_footer(footer()).unwrap(); + assert_eq!(Version::V1, parsed_version); + } + + #[test] + fn test_version_serialization() { + let version_to_tests: Vec = [0, 1 << 8, 1 << 16, 1 << 24] + .iter() + .copied() + .flat_map(|offset| (0..255).map(move |el| el + offset)) + .collect(); + let mut valid_versions: HashSet = HashSet::default(); + for &i in &version_to_tests { + let version_res = Version::try_from_bytes(i.to_le_bytes()); + if let Ok(version) = version_res { + assert_eq!(version, Version::V1); + assert_eq!(version.to_bytes(), i.to_le_bytes()); + valid_versions.insert(i); + } + } + assert_eq!(valid_versions.len(), 1); + } +} diff --git a/columnar/src/columnar/mod.rs b/columnar/src/columnar/mod.rs new file mode 100644 index 0000000000..37114c8d08 --- /dev/null +++ b/columnar/src/columnar/mod.rs @@ -0,0 +1,28 @@ +// Copyright (C) 2022 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . +// + +mod column_type; +mod format_version; +mod reader; +mod writer; + +pub use column_type::ColumnType; +pub use reader::ColumnarReader; +pub use writer::ColumnarWriter; diff --git a/columnar/src/reader/mod.rs b/columnar/src/columnar/reader/mod.rs similarity index 62% rename from columnar/src/reader/mod.rs rename to columnar/src/columnar/reader/mod.rs index 586b135078..80e49124c9 100644 --- a/columnar/src/reader/mod.rs +++ b/columnar/src/columnar/reader/mod.rs @@ -1,11 +1,11 @@ -use std::ops::Range; use std::{io, mem}; use common::file_slice::FileSlice; use common::BinarySerializable; use sstable::{Dictionary, RangeSSTable}; -use crate::column_type_header::ColumnTypeAndCardinality; +use crate::columnar::{format_version, ColumnType}; +use crate::dynamic_column::DynamicColumnHandle; fn io_invalid_data(msg: String) -> io::Error { io::Error::new(io::ErrorKind::InvalidData, msg) @@ -26,9 +26,14 @@ impl ColumnarReader { } fn open_inner(file_slice: FileSlice) -> io::Result { - let (file_slice_without_sstable_len, sstable_len_bytes) = - file_slice.split_from_end(mem::size_of::()); - let mut sstable_len_bytes = sstable_len_bytes.read_bytes()?; + let (file_slice_without_sstable_len, footer_slice) = file_slice + .split_from_end(mem::size_of::() + format_version::VERSION_FOOTER_NUM_BYTES); + let footer_bytes = footer_slice.read_bytes()?; + let (mut sstable_len_bytes, version_footer_bytes) = + footer_bytes.rsplit(format_version::VERSION_FOOTER_NUM_BYTES); + let version_footer_bytes: [u8; format_version::VERSION_FOOTER_NUM_BYTES] = + version_footer_bytes.as_slice().try_into().unwrap(); + let _version = format_version::parse_footer(version_footer_bytes)?; let sstable_len = u64::deserialize(&mut sstable_len_bytes)?; let (column_data, sstable) = file_slice_without_sstable_len.split_from_end(sstable_len as usize); @@ -40,25 +45,25 @@ impl ColumnarReader { } // TODO fix ugly API - pub fn list_columns( - &self, - ) -> io::Result, u64)>> { + pub fn list_columns(&self) -> io::Result> { let mut stream = self.column_dictionary.stream()?; let mut results = Vec::new(); while stream.advance() { let key_bytes: &[u8] = stream.key(); let column_code: u8 = key_bytes.last().cloned().unwrap(); - let column_type_and_cardinality = ColumnTypeAndCardinality::try_from_code(column_code) + let column_type: ColumnType = ColumnType::try_from_code(column_code) .map_err(|_| io_invalid_data(format!("Unknown column code `{column_code}`")))?; let range = stream.value().clone(); - let column_name = String::from_utf8_lossy(&key_bytes[..key_bytes.len() - 1]); - let range_len = range.end - range.start; - results.push(( - column_name.to_string(), - column_type_and_cardinality, - range, - range_len, - )); + let column_name = + String::from_utf8_lossy(&key_bytes[..key_bytes.len() - 1]).to_string(); + let file_slice = self + .column_data + .slice(range.start as usize..range.end as usize); + let column_handle = DynamicColumnHandle { + file_slice, + column_type, + }; + results.push((column_name, column_handle)); } Ok(results) } @@ -68,10 +73,7 @@ impl ColumnarReader { /// There can be more than one column associated to a given column name, provided they have /// different types. // TODO fix ugly API - pub fn read_columns( - &self, - column_name: &str, - ) -> io::Result)>> { + pub fn read_columns(&self, column_name: &str) -> io::Result> { // Each column is a associated to a given `column_key`, // that starts by `column_name\0column_header`. // @@ -80,6 +82,8 @@ impl ColumnarReader { // // This is in turn equivalent to searching for the range // `[column_name,\0`..column_name\1)`. + + // TODO can we get some more generic `prefix(..)` logic in the dictioanry. let mut start_key = column_name.to_string(); start_key.push('\0'); let mut end_key = column_name.to_string(); @@ -95,10 +99,17 @@ impl ColumnarReader { let key_bytes: &[u8] = stream.key(); assert!(key_bytes.starts_with(start_key.as_bytes())); let column_code: u8 = key_bytes.last().cloned().unwrap(); - let column_type_and_cardinality = ColumnTypeAndCardinality::try_from_code(column_code) + let column_type = ColumnType::try_from_code(column_code) .map_err(|_| io_invalid_data(format!("Unknown column code `{column_code}`")))?; let range = stream.value().clone(); - results.push((column_type_and_cardinality, range)); + let file_slice = self + .column_data + .slice(range.start as usize..range.end as usize); + let dynamic_column_handle = DynamicColumnHandle { + file_slice, + column_type, + }; + results.push(dynamic_column_handle); } Ok(results) } diff --git a/columnar/src/writer/column_operation.rs b/columnar/src/columnar/writer/column_operation.rs similarity index 99% rename from columnar/src/writer/column_operation.rs rename to columnar/src/columnar/writer/column_operation.rs index 9cb6bfe632..596bcf06e8 100644 --- a/columnar/src/writer/column_operation.rs +++ b/columnar/src/columnar/writer/column_operation.rs @@ -1,7 +1,7 @@ use crate::dictionary::UnorderedId; use crate::utils::{place_bits, pop_first_byte, select_bits}; use crate::value::NumericalValue; -use crate::{DocId, InvalidData, NumericalType}; +use crate::{InvalidData, NumericalType, RowId}; /// When we build a columnar dataframe, we first just group /// all mutations per column, and appends them in append-only buffer @@ -13,7 +13,7 @@ use crate::{DocId, InvalidData, NumericalType}; /// We represents all of these operations as `ColumnOperation`. #[derive(Eq, PartialEq, Debug, Clone, Copy)] pub(super) enum ColumnOperation { - NewDoc(DocId), + NewDoc(RowId), Value(T), } diff --git a/columnar/src/writer/column_writers.rs b/columnar/src/columnar/writer/column_writers.rs similarity index 96% rename from columnar/src/writer/column_writers.rs rename to columnar/src/columnar/writer/column_writers.rs index 6857138480..89f112f958 100644 --- a/columnar/src/writer/column_writers.rs +++ b/columnar/src/columnar/writer/column_writers.rs @@ -2,9 +2,9 @@ use std::cmp::Ordering; use stacker::{ExpUnrolledLinkedList, MemoryArena}; +use crate::columnar::writer::column_operation::{ColumnOperation, SymbolValue}; use crate::dictionary::{DictionaryBuilder, UnorderedId}; -use crate::writer::column_operation::{ColumnOperation, SymbolValue}; -use crate::{Cardinality, DocId, NumericalType, NumericalValue}; +use crate::{Cardinality, NumericalType, NumericalValue, RowId}; #[derive(Copy, Clone, Debug, Eq, PartialEq)] #[repr(u8)] @@ -53,7 +53,7 @@ impl ColumnWriter { /// /// This function will also update the cardinality of the column /// if necessary. - pub(super) fn record(&mut self, doc: DocId, value: S, arena: &mut MemoryArena) { + pub(super) fn record(&mut self, doc: RowId, value: S, arena: &mut MemoryArena) { // Difference between `doc` and the last doc. match delta_with_last_doc(self.last_doc_opt, doc) { DocumentStep::Same => { @@ -77,7 +77,7 @@ impl ColumnWriter { // The overall number of docs in the column is necessary to // deal with the case where the all docs contain 1 value, except some documents // at the end of the column. - pub(crate) fn get_cardinality(&self, num_docs: DocId) -> Cardinality { + pub(crate) fn get_cardinality(&self, num_docs: RowId) -> Cardinality { match delta_with_last_doc(self.last_doc_opt, num_docs) { DocumentStep::Same | DocumentStep::Next => self.cardinality, DocumentStep::Skipped => self.cardinality.max(Cardinality::Optional), @@ -150,7 +150,7 @@ impl CompatibleNumericalTypes { } impl NumericalColumnWriter { - pub fn column_type_and_cardinality(&self, num_docs: DocId) -> (NumericalType, Cardinality) { + pub fn column_type_and_cardinality(&self, num_docs: RowId) -> (NumericalType, Cardinality) { let numerical_type = self.compatible_numerical_types.to_numerical_type(); let cardinality = self.column_writer.get_cardinality(num_docs); (numerical_type, cardinality) @@ -158,7 +158,7 @@ impl NumericalColumnWriter { pub fn record_numerical_value( &mut self, - doc: DocId, + doc: RowId, value: NumericalValue, arena: &mut MemoryArena, ) { @@ -191,7 +191,7 @@ impl StrColumnWriter { pub(crate) fn record_bytes( &mut self, - doc: DocId, + doc: RowId, bytes: &[u8], dictionaries: &mut [DictionaryBuilder], arena: &mut MemoryArena, diff --git a/columnar/src/writer/mod.rs b/columnar/src/columnar/writer/mod.rs similarity index 83% rename from columnar/src/writer/mod.rs rename to columnar/src/columnar/writer/mod.rs index e53000baec..c3522448bb 100644 --- a/columnar/src/writer/mod.rs +++ b/columnar/src/columnar/writer/mod.rs @@ -7,23 +7,25 @@ use std::io; use column_operation::ColumnOperation; use common::CountingWriter; -use fastfield_codecs::serialize::ValueIndexInfo; -use fastfield_codecs::{Column, MonotonicallyMappableToU64, VecColumn}; use serializer::ColumnarSerializer; use stacker::{Addr, ArenaHashMap, MemoryArena}; -use crate::column_type_header::{ColumnType, ColumnTypeAndCardinality, ColumnTypeCategory}; +use crate::column_index::SerializableColumnIndex; +use crate::column_values::{ColumnValues, MonotonicallyMappableToU64, VecColumn}; +use crate::columnar::column_type::{ColumnType, ColumnTypeCategory}; +use crate::columnar::writer::column_writers::{ + ColumnWriter, NumericalColumnWriter, StrColumnWriter, +}; +use crate::columnar::writer::value_index::{IndexBuilder, PreallocatedIndexBuilders}; use crate::dictionary::{DictionaryBuilder, TermIdMapping, UnorderedId}; use crate::value::{Coerce, NumericalType, NumericalValue}; -use crate::writer::column_writers::{ColumnWriter, NumericalColumnWriter, StrColumnWriter}; -use crate::writer::value_index::{IndexBuilder, SpareIndexBuilders}; -use crate::{Cardinality, DocId}; +use crate::{Cardinality, RowId}; /// This is a set of buffers that are used to temporarily write the values into before passing them /// to the fast field codecs. #[derive(Default)] struct SpareBuffers { - value_index_builders: SpareIndexBuilders, + value_index_builders: PreallocatedIndexBuilders, i64_values: Vec, u64_values: Vec, f64_values: Vec, @@ -69,7 +71,7 @@ impl Default for ColumnarWriter { impl ColumnarWriter { pub fn record_numerical + Copy>( &mut self, - doc: DocId, + doc: RowId, column_name: &str, numerical_value: T, ) { @@ -88,7 +90,7 @@ impl ColumnarWriter { ); } - pub fn record_bool(&mut self, doc: DocId, column_name: &str, val: bool) { + pub fn record_bool(&mut self, doc: RowId, column_name: &str, val: bool) { assert!( !column_name.as_bytes().contains(&0u8), "key may not contain the 0 byte" @@ -104,7 +106,7 @@ impl ColumnarWriter { ); } - pub fn record_str(&mut self, doc: DocId, column_name: &str, value: &str) { + pub fn record_str(&mut self, doc: RowId, column_name: &str, value: &str) { assert!( !column_name.as_bytes().contains(&0u8), "key may not contain the 0 byte" @@ -129,7 +131,7 @@ impl ColumnarWriter { ); } - pub fn serialize(&mut self, num_docs: DocId, wrt: &mut dyn io::Write) -> io::Result<()> { + pub fn serialize(&mut self, num_docs: RowId, wrt: &mut dyn io::Write) -> io::Result<()> { let mut serializer = ColumnarSerializer::new(wrt); let mut field_columns: Vec<(&[u8], ColumnTypeCategory, Addr)> = self .numerical_field_hash_map @@ -154,12 +156,8 @@ impl ColumnarWriter { ColumnTypeCategory::Bool => { let column_writer: ColumnWriter = self.bool_field_hash_map.read(addr); let cardinality = column_writer.get_cardinality(num_docs); - let column_type_and_cardinality = ColumnTypeAndCardinality { - cardinality, - typ: ColumnType::Bool, - }; let mut column_serializer = - serializer.serialize_column(column_name, column_type_and_cardinality); + serializer.serialize_column(column_name, ColumnType::Bool); serialize_bool_column( cardinality, num_docs, @@ -173,12 +171,8 @@ impl ColumnarWriter { let dictionary_builder = &dictionaries[str_column_writer.dictionary_id as usize]; let cardinality = str_column_writer.column_writer.get_cardinality(num_docs); - let column_type_and_cardinality = ColumnTypeAndCardinality { - cardinality, - typ: ColumnType::Bytes, - }; let mut column_serializer = - serializer.serialize_column(column_name, column_type_and_cardinality); + serializer.serialize_column(column_name, ColumnType::Bytes); serialize_bytes_column( cardinality, num_docs, @@ -193,12 +187,8 @@ impl ColumnarWriter { self.numerical_field_hash_map.read(addr); let (numerical_type, cardinality) = numerical_column_writer.column_type_and_cardinality(num_docs); - let column_type_and_cardinality = ColumnTypeAndCardinality { - cardinality, - typ: ColumnType::Numerical(numerical_type), - }; - let mut column_serializer = - serializer.serialize_column(column_name, column_type_and_cardinality); + let mut column_serializer = serializer + .serialize_column(column_name, ColumnType::Numerical(numerical_type)); serialize_numerical_column( cardinality, num_docs, @@ -217,7 +207,7 @@ impl ColumnarWriter { fn serialize_bytes_column( cardinality: Cardinality, - num_docs: DocId, + num_docs: RowId, dictionary_builder: &DictionaryBuilder, operation_it: impl Iterator>, buffers: &mut SpareBuffers, @@ -256,7 +246,7 @@ fn serialize_bytes_column( fn serialize_numerical_column( cardinality: Cardinality, - num_docs: DocId, + num_docs: RowId, numerical_type: NumericalType, op_iterator: impl Iterator>, buffers: &mut SpareBuffers, @@ -306,7 +296,7 @@ fn serialize_numerical_column( fn serialize_bool_column( cardinality: Cardinality, - num_docs: DocId, + num_docs: RowId, column_operations_it: impl Iterator>, buffers: &mut SpareBuffers, wrt: &mut impl io::Write, @@ -332,51 +322,43 @@ fn serialize_column< >( op_iterator: impl Iterator>, cardinality: Cardinality, - num_docs: DocId, - value_index_builders: &mut SpareIndexBuilders, + num_docs: RowId, + value_index_builders: &mut PreallocatedIndexBuilders, values: &mut Vec, mut wrt: impl io::Write, ) -> io::Result<()> where - for<'a> VecColumn<'a, T>: Column, + for<'a> VecColumn<'a, T>: ColumnValues, { values.clear(); - match cardinality { - Cardinality::Required => { + let serializable_column_index = match cardinality { + Cardinality::Full => { consume_operation_iterator( op_iterator, value_index_builders.borrow_required_index_builder(), values, ); - fastfield_codecs::serialize( - VecColumn::from(&values[..]), - &mut wrt, - &fastfield_codecs::ALL_CODEC_TYPES[..], - )?; + SerializableColumnIndex::Full } Cardinality::Optional => { let optional_index_builder = value_index_builders.borrow_optional_index_builder(); consume_operation_iterator(op_iterator, optional_index_builder, values); let optional_index = optional_index_builder.finish(num_docs); - fastfield_codecs::serialize::serialize_new( - ValueIndexInfo::SingleValue(Box::new(optional_index)), - VecColumn::from(&values[..]), - &mut wrt, - &fastfield_codecs::ALL_CODEC_TYPES[..], - )?; + SerializableColumnIndex::Optional(Box::new(optional_index)) } Cardinality::Multivalued => { let multivalued_index_builder = value_index_builders.borrow_multivalued_index_builder(); consume_operation_iterator(op_iterator, multivalued_index_builder, values); let multivalued_index = multivalued_index_builder.finish(num_docs); - fastfield_codecs::serialize::serialize_new( - ValueIndexInfo::MultiValue(Box::new(multivalued_index)), - VecColumn::from(&values[..]), - &mut wrt, - &fastfield_codecs::ALL_CODEC_TYPES[..], - )?; + todo!(); + // SerializableColumnIndex::Multivalued(Box::new(multivalued_index)) } - } + }; + crate::column::serialize_column_u64( + serializable_column_index, + &VecColumn::from(&values[..]), + &mut wrt, + )?; Ok(()) } @@ -400,7 +382,7 @@ fn consume_operation_iterator( for symbol in operation_iterator { match symbol { ColumnOperation::NewDoc(doc) => { - index_builder.record_doc(doc); + index_builder.record_row(doc); } ColumnOperation::Value(value) => { index_builder.record_value(); @@ -410,6 +392,52 @@ fn consume_operation_iterator( } } +// /// Serializes the column with the codec with the best estimate on the data. +// fn serialize_numerical( +// value_index: ValueIndexInfo, +// typed_column: impl Column, +// output: &mut impl io::Write, +// codecs: &[FastFieldCodecType], +// ) -> io::Result<()> { + +// let counting_writer = CountingWriter::wrap(output); +// serialize_value_index(value_index, output)?; +// let value_index_len = counting_writer.written_bytes(); +// let output = counting_writer.finish(); + +// serialize_column(value_index, output)?; +// let column = monotonic_map_column( +// typed_column, +// crate::column::monotonic_mapping::StrictlyMonotonicMappingToInternal::::new(), +// ); +// let header = Header::compute_header(&column, codecs).ok_or_else(|| { +// io::Error::new( +// io::ErrorKind::InvalidInput, +// format!( +// "Data cannot be serialized with this list of codec. {:?}", +// codecs +// ), +// ) +// })?; +// header.serialize(output)?; +// let normalized_column = header.normalize_column(column); +// assert_eq!(normalized_column.min_value(), 0u64); +// serialize_given_codec(normalized_column, header.codec_type, output)?; + +// let column_header = ColumnFooter { +// value_index_len: todo!(), +// cardinality: todo!(), +// }; + +// let null_index_footer = NullIndexFooter { +// cardinality: value_index.get_cardinality(), +// null_index_codec: NullIndexCodec::Full, +// null_index_byte_range: 0..0, +// }; +// append_null_index_footer(output, null_index_footer)?; +// Ok(()) +// } + #[cfg(test)] mod tests { use column_operation::ColumnOperation; @@ -417,7 +445,6 @@ mod tests { use super::*; use crate::value::NumericalValue; - use crate::Cardinality; #[test] fn test_column_writer_required_simple() { @@ -426,7 +453,7 @@ mod tests { column_writer.record(0u32, NumericalValue::from(14i64), &mut arena); column_writer.record(1u32, NumericalValue::from(15i64), &mut arena); column_writer.record(2u32, NumericalValue::from(-16i64), &mut arena); - assert_eq!(column_writer.get_cardinality(3), Cardinality::Required); + assert_eq!(column_writer.get_cardinality(3), Cardinality::Full); let mut buffer = Vec::new(); let symbols: Vec> = column_writer .operation_iterator(&mut arena, &mut buffer) diff --git a/columnar/src/writer/serializer.rs b/columnar/src/columnar/writer/serializer.rs similarity index 77% rename from columnar/src/writer/serializer.rs rename to columnar/src/columnar/writer/serializer.rs index e1751da504..47364c3fd2 100644 --- a/columnar/src/writer/serializer.rs +++ b/columnar/src/columnar/writer/serializer.rs @@ -5,7 +5,7 @@ use common::CountingWriter; use sstable::value::RangeValueWriter; use sstable::RangeSSTable; -use crate::column_type_header::ColumnTypeAndCardinality; +use crate::columnar::ColumnType; pub struct ColumnarSerializer { wrt: CountingWriter, @@ -15,15 +15,11 @@ pub struct ColumnarSerializer { /// Returns a key consisting of the concatenation of the key and the column_type_and_cardinality /// code. -fn prepare_key( - key: &[u8], - column_type_cardinality: ColumnTypeAndCardinality, - buffer: &mut Vec, -) { +fn prepare_key(key: &[u8], column_type: ColumnType, buffer: &mut Vec) { buffer.clear(); buffer.extend_from_slice(key); buffer.push(0u8); - buffer.push(column_type_cardinality.to_code()); + buffer.push(column_type.to_code()); } impl ColumnarSerializer { @@ -40,14 +36,10 @@ impl ColumnarSerializer { pub fn serialize_column<'a>( &'a mut self, column_name: &[u8], - column_type_cardinality: ColumnTypeAndCardinality, + column_type: ColumnType, ) -> impl io::Write + 'a { let start_offset = self.wrt.written_bytes(); - prepare_key( - column_name, - column_type_cardinality, - &mut self.prepare_key_buffer, - ); + prepare_key(column_name, column_type, &mut self.prepare_key_buffer); ColumnSerializer { columnar_serializer: self, start_offset, @@ -59,6 +51,9 @@ impl ColumnarSerializer { let sstable_num_bytes: u64 = sstable_bytes.len() as u64; self.wrt.write_all(&sstable_bytes)?; self.wrt.write_all(&sstable_num_bytes.to_le_bytes()[..])?; + self.wrt + .write_all(&super::super::format_version::footer())?; + self.wrt.flush()?; Ok(()) } } @@ -97,20 +92,15 @@ impl<'a, W: io::Write> io::Write for ColumnSerializer<'a, W> { #[cfg(test)] mod tests { use super::*; - use crate::column_type_header::ColumnType; - use crate::Cardinality; + use crate::columnar::column_type::ColumnType; #[test] fn test_prepare_key_bytes() { let mut buffer: Vec = b"somegarbage".to_vec(); - let column_type_and_cardinality = ColumnTypeAndCardinality { - typ: ColumnType::Bytes, - cardinality: Cardinality::Optional, - }; - prepare_key(b"root\0child", column_type_and_cardinality, &mut buffer); + prepare_key(b"root\0child", ColumnType::Bytes, &mut buffer); assert_eq!(buffer.len(), 12); assert_eq!(&buffer[..10], b"root\0child"); assert_eq!(buffer[10], 0u8); - assert_eq!(buffer[11], column_type_and_cardinality.to_code()); + assert_eq!(buffer[11], ColumnType::Bytes.to_code()); } } diff --git a/columnar/src/writer/value_index.rs b/columnar/src/columnar/writer/value_index.rs similarity index 61% rename from columnar/src/writer/value_index.rs rename to columnar/src/columnar/writer/value_index.rs index 8d97877a1a..71f577b3da 100644 --- a/columnar/src/writer/value_index.rs +++ b/columnar/src/columnar/writer/value_index.rs @@ -1,6 +1,6 @@ -use fastfield_codecs::serialize::{MultiValueIndexInfo, SingleValueIndexInfo}; - -use crate::DocId; +use crate::column_index::SerializableOptionalIndex; +use crate::column_values::{ColumnValues, VecColumn}; +use crate::RowId; /// The `IndexBuilder` interprets a sequence of /// calls of the form: @@ -10,36 +10,42 @@ use crate::DocId; /// It has different implementation depending on whether the /// cardinality is required, optional, or multivalued. pub(crate) trait IndexBuilder { - fn record_doc(&mut self, doc: DocId); + fn record_row(&mut self, doc: RowId); #[inline] fn record_value(&mut self) {} } -/// The RequiredIndexBuilder does nothing. +/// The FullIndexBuilder does nothing. #[derive(Default)] -pub struct RequiredIndexBuilder; +pub struct FullIndexBuilder; -impl IndexBuilder for RequiredIndexBuilder { +impl IndexBuilder for FullIndexBuilder { #[inline(always)] - fn record_doc(&mut self, _doc: DocId) {} + fn record_row(&mut self, _doc: RowId) {} } #[derive(Default)] pub struct OptionalIndexBuilder { - docs: Vec, + docs: Vec, } struct SingleValueArrayIndex<'a> { - // DocIds with a value. DocIds are strictly increasing - docs: &'a [DocId], - num_docs: DocId, + // RowIds with a value, in a strictly increasing order + row_ids: &'a [RowId], + num_rows: RowId, } -impl<'a> SingleValueIndexInfo for SingleValueArrayIndex<'a> { - fn num_vals(&self) -> u32 { - self.num_docs as u32 +impl<'a> SerializableOptionalIndex<'a> for SingleValueArrayIndex<'a> { + fn num_rows(&self) -> RowId { + self.num_rows } + fn non_null_rows(&self) -> Box + 'a> { + Box::new(self.row_ids.iter().copied()) + } +} + +impl OptionalIndexBuilder { fn num_non_nulls(&self) -> u32 { self.docs.len() as u32 } @@ -50,16 +56,16 @@ impl<'a> SingleValueIndexInfo for SingleValueArrayIndex<'a> { } impl OptionalIndexBuilder { - pub fn finish(&mut self, num_docs: DocId) -> impl SingleValueIndexInfo + '_ { + pub fn finish<'a>(&'a mut self, num_rows: RowId) -> impl SerializableOptionalIndex + 'a { debug_assert!(self .docs .last() .copied() - .map(|last_doc| last_doc < num_docs) + .map(|last_doc| last_doc < num_rows) .unwrap_or(true)); SingleValueArrayIndex { - docs: &self.docs[..], - num_docs, + row_ids: &self.docs[..], + num_rows, } } @@ -70,7 +76,7 @@ impl OptionalIndexBuilder { impl IndexBuilder for OptionalIndexBuilder { #[inline(always)] - fn record_doc(&mut self, doc: DocId) { + fn record_row(&mut self, doc: RowId) { debug_assert!(self .docs .last() @@ -83,53 +89,32 @@ impl IndexBuilder for OptionalIndexBuilder { #[derive(Default)] pub struct MultivaluedIndexBuilder { - // TODO should we switch to `start_offset`? - // contains the num values so far for each `DocId`. - end_offsets: Vec, + start_offsets: Vec, total_num_vals_seen: u32, } -pub struct MultivaluedValueArrayIndex<'a> { - end_offsets: &'a [DocId], -} - -impl<'a> MultiValueIndexInfo for MultivaluedValueArrayIndex<'a> { - fn num_docs(&self) -> u32 { - self.end_offsets.len() as u32 - } - - fn num_vals(&self) -> u32 { - self.end_offsets.last().copied().unwrap_or(0u32) - } - - fn iter(&self) -> Box + '_> { - if self.end_offsets.is_empty() { - return Box::new(std::iter::empty()); - } - let n = self.end_offsets.len(); - Box::new(std::iter::once(0u32).chain(self.end_offsets[..n - 1].iter().copied())) - } -} - impl MultivaluedIndexBuilder { - pub fn finish(&mut self, num_docs: DocId) -> impl MultiValueIndexInfo + '_ { - self.end_offsets + pub fn finish(&mut self, num_docs: RowId) -> impl ColumnValues + '_ { + self.start_offsets .resize(num_docs as usize, self.total_num_vals_seen); - MultivaluedValueArrayIndex { - end_offsets: &self.end_offsets[..], + VecColumn { + values: &&self.start_offsets[..], + min_value: 0, + max_value: self.start_offsets.last().copied().unwrap_or(0), } } fn reset(&mut self) { - self.end_offsets.clear(); + self.start_offsets.clear(); + self.start_offsets.push(0u32); self.total_num_vals_seen = 0; } } impl IndexBuilder for MultivaluedIndexBuilder { - fn record_doc(&mut self, doc: DocId) { - self.end_offsets - .resize(doc as usize, self.total_num_vals_seen); + fn record_row(&mut self, row_id: RowId) { + self.start_offsets + .resize(row_id as usize + 1, self.total_num_vals_seen); } fn record_value(&mut self) { @@ -140,14 +125,14 @@ impl IndexBuilder for MultivaluedIndexBuilder { /// The `SpareIndexBuilders` is there to avoid allocating a /// new index builder for every single column. #[derive(Default)] -pub struct SpareIndexBuilders { - required_index_builder: RequiredIndexBuilder, +pub struct PreallocatedIndexBuilders { + required_index_builder: FullIndexBuilder, optional_index_builder: OptionalIndexBuilder, multivalued_index_builder: MultivaluedIndexBuilder, } -impl SpareIndexBuilders { - pub fn borrow_required_index_builder(&mut self) -> &mut RequiredIndexBuilder { +impl PreallocatedIndexBuilders { + pub fn borrow_required_index_builder(&mut self) -> &mut FullIndexBuilder { &mut self.required_index_builder } @@ -169,22 +154,22 @@ mod tests { #[test] fn test_optional_value_index_builder() { let mut opt_value_index_builder = OptionalIndexBuilder::default(); - opt_value_index_builder.record_doc(0u32); + opt_value_index_builder.record_row(0u32); opt_value_index_builder.record_value(); assert_eq!( &opt_value_index_builder .finish(1u32) - .iter() + .non_null_rows() .collect::>(), &[0] ); opt_value_index_builder.reset(); - opt_value_index_builder.record_doc(1u32); + opt_value_index_builder.record_row(1u32); opt_value_index_builder.record_value(); assert_eq!( &opt_value_index_builder .finish(2u32) - .iter() + .non_null_rows() .collect::>(), &[1] ); @@ -193,10 +178,10 @@ mod tests { #[test] fn test_multivalued_value_index_builder() { let mut multivalued_value_index_builder = MultivaluedIndexBuilder::default(); - multivalued_value_index_builder.record_doc(1u32); + multivalued_value_index_builder.record_row(1u32); multivalued_value_index_builder.record_value(); multivalued_value_index_builder.record_value(); - multivalued_value_index_builder.record_doc(2u32); + multivalued_value_index_builder.record_row(2u32); multivalued_value_index_builder.record_value(); assert_eq!( multivalued_value_index_builder @@ -206,7 +191,7 @@ mod tests { vec![0, 0, 2, 3] ); multivalued_value_index_builder.reset(); - multivalued_value_index_builder.record_doc(2u32); + multivalued_value_index_builder.record_row(2u32); multivalued_value_index_builder.record_value(); multivalued_value_index_builder.record_value(); assert_eq!( diff --git a/columnar/src/dynamic_column.rs b/columnar/src/dynamic_column.rs new file mode 100644 index 0000000000..fdfb7ad644 --- /dev/null +++ b/columnar/src/dynamic_column.rs @@ -0,0 +1,95 @@ +use std::io; +use std::net::IpAddr; + +use common::file_slice::FileSlice; +use common::{HasLen, OwnedBytes}; + +use crate::column::{BytesColumn, Column}; +use crate::columnar::ColumnType; +use crate::DateTime; + +#[derive(Clone)] +pub enum DynamicColumn { + Bool(Column), + I64(Column), + U64(Column), + F64(Column), + IpAddr(Column), + DateTime(Column), + Str(BytesColumn), +} + +impl From> for DynamicColumn { + fn from(column_i64: Column) -> Self { + DynamicColumn::I64(column_i64) + } +} + +impl From> for DynamicColumn { + fn from(column_u64: Column) -> Self { + DynamicColumn::U64(column_u64) + } +} + +impl From> for DynamicColumn { + fn from(column_f64: Column) -> Self { + DynamicColumn::F64(column_f64) + } +} + +impl From> for DynamicColumn { + fn from(bool_column: Column) -> Self { + DynamicColumn::Bool(bool_column) + } +} + +impl From for DynamicColumn { + fn from(dictionary_encoded_col: BytesColumn) -> Self { + DynamicColumn::Str(dictionary_encoded_col) + } +} + +#[derive(Clone)] +pub struct DynamicColumnHandle { + pub(crate) file_slice: FileSlice, + pub(crate) column_type: ColumnType, +} + +impl DynamicColumnHandle { + pub fn open(&self) -> io::Result { + let column_bytes: OwnedBytes = self.file_slice.read_bytes()?; + self.open_internal(column_bytes) + } + + pub async fn open_async(&self) -> io::Result { + let column_bytes: OwnedBytes = self.file_slice.read_bytes_async().await?; + self.open_internal(column_bytes) + } + + fn open_internal(&self, column_bytes: OwnedBytes) -> io::Result { + let dynamic_column: DynamicColumn = match self.column_type { + ColumnType::Bytes => crate::column::open_column_bytes(column_bytes)?.into(), + ColumnType::Numerical(numerical_type) => match numerical_type { + crate::NumericalType::I64 => { + crate::column::open_column_u64::(column_bytes)?.into() + } + crate::NumericalType::U64 => { + crate::column::open_column_u64::(column_bytes)?.into() + } + crate::NumericalType::F64 => { + crate::column::open_column_u64::(column_bytes)?.into() + } + }, + ColumnType::Bool => crate::column::open_column_u64::(column_bytes)?.into(), + }; + Ok(dynamic_column) + } + + pub fn num_bytes(&self) -> usize { + self.file_slice.len() + } + + pub fn column_type(&self) -> ColumnType { + self.column_type + } +} diff --git a/columnar/src/lib.rs b/columnar/src/lib.rs index 0e28de4da3..0c37a025a1 100644 --- a/columnar/src/lib.rs +++ b/columnar/src/lib.rs @@ -1,89 +1,75 @@ -mod column_type_header; +#![cfg_attr(all(feature = "unstable", test), feature(test))] + +#[cfg(test)] +#[macro_use] +extern crate more_asserts; + +#[cfg(all(test, feature = "unstable"))] +extern crate test; + +use std::io; + +mod column; +mod column_index; +mod column_values; +mod columnar; mod dictionary; -mod reader; +mod dynamic_column; pub(crate) mod utils; mod value; -mod writer; -pub use column_type_header::Cardinality; -pub use reader::ColumnarReader; +pub use columnar::{ColumnarReader, ColumnarWriter}; pub use value::{NumericalType, NumericalValue}; -pub use writer::ColumnarWriter; -pub type DocId = u32; +// pub use self::dynamic_column::DynamicColumnHandle; -#[derive(Copy, Clone, Debug)] -pub struct InvalidData; - -#[cfg(test)] -mod tests { - use std::ops::Range; +pub type RowId = u32; - use common::file_slice::FileSlice; +#[derive(Clone, Copy)] +pub struct DateTime { + timestamp_micros: i64, +} - use crate::column_type_header::{ColumnType, ColumnTypeAndCardinality}; - use crate::reader::ColumnarReader; - use crate::value::NumericalValue; - use crate::{Cardinality, ColumnarWriter}; +#[derive(Copy, Clone, Debug)] +pub struct InvalidData; - #[test] - fn test_dataframe_writer_bytes() { - let mut dataframe_writer = ColumnarWriter::default(); - dataframe_writer.record_str(1u32, "my_string", "hello"); - dataframe_writer.record_str(3u32, "my_string", "helloeee"); - let mut buffer: Vec = Vec::new(); - dataframe_writer.serialize(5, &mut buffer).unwrap(); - let columnar_fileslice = FileSlice::from(buffer); - let columnar = ColumnarReader::open(columnar_fileslice).unwrap(); - assert_eq!(columnar.num_columns(), 1); - let cols: Vec<(ColumnTypeAndCardinality, Range)> = - columnar.read_columns("my_string").unwrap(); - assert_eq!(cols.len(), 1); - assert_eq!(cols[0].1, 0..158); +impl From for io::Error { + fn from(_: InvalidData) -> Self { + io::Error::new(io::ErrorKind::InvalidData, "Invalid data") } +} - #[test] - fn test_dataframe_writer_bool() { - let mut dataframe_writer = ColumnarWriter::default(); - dataframe_writer.record_bool(1u32, "bool.value", false); - let mut buffer: Vec = Vec::new(); - dataframe_writer.serialize(5, &mut buffer).unwrap(); - let columnar_fileslice = FileSlice::from(buffer); - let columnar = ColumnarReader::open(columnar_fileslice).unwrap(); - assert_eq!(columnar.num_columns(), 1); - let cols: Vec<(ColumnTypeAndCardinality, Range)> = - columnar.read_columns("bool.value").unwrap(); - assert_eq!(cols.len(), 1); - assert_eq!( - cols[0].0, - ColumnTypeAndCardinality { - cardinality: Cardinality::Optional, - typ: ColumnType::Bool - } - ); - assert_eq!(cols[0].1, 0..21); +/// Enum describing the number of values that can exist per document +/// (or per row if you will). +/// +/// The cardinality must fit on 2 bits. +#[derive(Clone, Copy, Hash, Default, Debug, PartialEq, Eq, PartialOrd, Ord)] +#[repr(u8)] +pub enum Cardinality { + /// All documents contain exactly one value. + /// `Full` is the default for auto-detecting the Cardinality, since it is the most strict. + #[default] + Full = 0, + /// All documents contain at most one value. + Optional = 1, + /// All documents may contain any number of values. + Multivalued = 2, +} + +impl Cardinality { + pub(crate) fn to_code(self) -> u8 { + self as u8 } - #[test] - fn test_dataframe_writer_numerical() { - let mut dataframe_writer = ColumnarWriter::default(); - dataframe_writer.record_numerical(1u32, "srical.value", NumericalValue::U64(12u64)); - dataframe_writer.record_numerical(2u32, "srical.value", NumericalValue::U64(13u64)); - dataframe_writer.record_numerical(4u32, "srical.value", NumericalValue::U64(15u64)); - let mut buffer: Vec = Vec::new(); - dataframe_writer.serialize(5, &mut buffer).unwrap(); - let columnar_fileslice = FileSlice::from(buffer); - let columnar = ColumnarReader::open(columnar_fileslice).unwrap(); - assert_eq!(columnar.num_columns(), 1); - let cols: Vec<(ColumnTypeAndCardinality, Range)> = - columnar.read_columns("srical.value").unwrap(); - assert_eq!(cols.len(), 1); - // Right now this 31 bytes are spent as follows - // - // - header 14 bytes - // - vals 8 //< due to padding? could have been 1byte?. - // - null footer 6 bytes - // - version footer 3 bytes // Should be file-wide - assert_eq!(cols[0].1, 0..31); + pub(crate) fn try_from_code(code: u8) -> Result { + match code { + 0 => Ok(Cardinality::Full), + 1 => Ok(Cardinality::Optional), + 2 => Ok(Cardinality::Multivalued), + _ => Err(InvalidData), + } } } + +#[cfg(test)] +mod tests; diff --git a/columnar/src/tests.rs b/columnar/src/tests.rs new file mode 100644 index 0000000000..86ac3266ec --- /dev/null +++ b/columnar/src/tests.rs @@ -0,0 +1,84 @@ +use crate::columnar::ColumnType; +use crate::dynamic_column::{DynamicColumn, DynamicColumnHandle}; +use crate::value::NumericalValue; +use crate::{Cardinality, ColumnarReader, ColumnarWriter}; + +#[test] +fn test_dataframe_writer_bytes() { + let mut dataframe_writer = ColumnarWriter::default(); + dataframe_writer.record_str(1u32, "my_string", "hello"); + dataframe_writer.record_str(3u32, "my_string", "helloeee"); + let mut buffer: Vec = Vec::new(); + dataframe_writer.serialize(5, &mut buffer).unwrap(); + let columnar = ColumnarReader::open(buffer).unwrap(); + assert_eq!(columnar.num_columns(), 1); + let cols: Vec = columnar.read_columns("my_string").unwrap(); + assert_eq!(cols.len(), 1); + assert_eq!(cols[0].num_bytes(), 165); +} + +#[test] +fn test_dataframe_writer_bool() { + let mut dataframe_writer = ColumnarWriter::default(); + dataframe_writer.record_bool(1u32, "bool.value", false); + dataframe_writer.record_bool(3u32, "bool.value", true); + let mut buffer: Vec = Vec::new(); + dataframe_writer.serialize(5, &mut buffer).unwrap(); + let columnar = ColumnarReader::open(buffer).unwrap(); + assert_eq!(columnar.num_columns(), 1); + let cols: Vec = columnar.read_columns("bool.value").unwrap(); + assert_eq!(cols.len(), 1); + assert_eq!(cols[0].num_bytes(), 29); + assert_eq!(cols[0].column_type(), ColumnType::Bool); + let dyn_bool_col = cols[0].open().unwrap(); + let DynamicColumn::Bool(bool_col) = dyn_bool_col else { panic!(); }; + let vals: Vec> = (0..5).map(|row_id| bool_col.first(row_id)).collect(); + assert_eq!(&vals, &[None, Some(false), None, Some(true), None,]); +} + +#[test] +fn test_dataframe_writer_numerical() { + let mut dataframe_writer = ColumnarWriter::default(); + dataframe_writer.record_numerical(1u32, "srical.value", NumericalValue::U64(12u64)); + dataframe_writer.record_numerical(2u32, "srical.value", NumericalValue::U64(13u64)); + dataframe_writer.record_numerical(4u32, "srical.value", NumericalValue::U64(15u64)); + let mut buffer: Vec = Vec::new(); + dataframe_writer.serialize(6, &mut buffer).unwrap(); + let columnar = ColumnarReader::open(buffer).unwrap(); + assert_eq!(columnar.num_columns(), 1); + let cols: Vec = columnar.read_columns("srical.value").unwrap(); + assert_eq!(cols.len(), 1); + // Right now this 31 bytes are spent as follows + // + // - header 14 bytes + // - vals 8 //< due to padding? could have been 1byte?. + // - null footer 6 bytes + assert_eq!(cols[0].num_bytes(), 40); + let column = cols[0].open().unwrap(); + let DynamicColumn::I64(column_i64) = column else { panic!(); }; + assert_eq!(column_i64.idx.get_cardinality(), Cardinality::Optional); + assert_eq!(column_i64.first(0), None); + assert_eq!(column_i64.first(1), Some(12i64)); + assert_eq!(column_i64.first(2), Some(13i64)); + assert_eq!(column_i64.first(3), None); + assert_eq!(column_i64.first(4), Some(15i64)); + assert_eq!(column_i64.first(5), None); + assert_eq!(column_i64.first(6), None); //< we can change the spec for that one. +} + +#[test] +fn test_dictionary_encoded() { + let mut buffer = Vec::new(); + let mut columnar_writer = ColumnarWriter::default(); + columnar_writer.record_str(1, "my.column", "my.key"); + columnar_writer.record_str(3, "my.column", "my.key2"); + columnar_writer.record_str(3, "my.column2", "different_column!"); + columnar_writer.serialize(5, &mut buffer).unwrap(); + let columnar_reader = ColumnarReader::open(buffer).unwrap(); + assert_eq!(columnar_reader.num_columns(), 2); + let col_handles = columnar_reader.read_columns("my.column").unwrap(); + assert_eq!(col_handles.len(), 1); + let DynamicColumn::Str(str_col) = col_handles[0].open().unwrap() else { panic!(); }; + assert_eq!(str_col.num_rows(), 5); + // let term_ords = (0..) +}