-
Notifications
You must be signed in to change notification settings - Fork 728
fix: handle logical rows deletion properly for zonemap and bloomfilter #5140
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 3 commits
a28f159
ad300de
bac5a00
d8b6d26
88ecb64
edf263f
b41d03e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -52,10 +52,11 @@ const BLOOMFILTER_INDEX_VERSION: u32 = 0; | |
| #[derive(Debug, Clone)] | ||
| struct BloomFilterStatistics { | ||
| fragment_id: u64, | ||
| // zone_start is the start row of the zone in the fragment, also known | ||
| // as local row offset | ||
| // zone_start is the actual first row address (local offset within fragment) | ||
| zone_start: u64, | ||
| zone_length: usize, | ||
| // zone_length is the address span: (last_row_addr - first_row_addr + 1) | ||
| // AKA offset in the fragment, which allows handling non-contiguous addresses after deletions | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't understand this comment. What does it mean? How are deletions handled with respect to this?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Rewrote the whole part and added two examples |
||
| zone_length: u64, | ||
| // Whether this zone contains any null values | ||
| has_null: bool, | ||
| // The actual bloom filter (SBBF) for efficient querying | ||
|
|
@@ -231,7 +232,7 @@ impl BloomFilterIndex { | |
| blocks.push(BloomFilterStatistics { | ||
| fragment_id: fragment_id_col.value(i), | ||
| zone_start: zone_start_col.value(i), | ||
| zone_length: zone_length_col.value(i) as usize, | ||
| zone_length: zone_length_col.value(i), | ||
| has_null: has_null_col.value(i), | ||
| bloom_filter, | ||
| }); | ||
|
|
@@ -470,9 +471,10 @@ impl ScalarIndex for BloomFilterIndex { | |
| for block in self.zones.iter() { | ||
| if self.evaluate_block_against_query(block, query)? { | ||
| // Calculate the range of row addresses for this zone | ||
| // Row addresses are: (fragment_id << 32) + zone_start | ||
| // zone_length is the address span (not row count), so we can directly use it | ||
| // This handles non-contiguous addresses from deletions correctly | ||
| let zone_start_addr = (block.fragment_id << 32) + block.zone_start; | ||
| let zone_end_addr = zone_start_addr + (block.zone_length as u64); | ||
| let zone_end_addr = zone_start_addr + block.zone_length; | ||
|
|
||
| // Add all row addresses in this zone to the result | ||
| row_id_tree_map.insert_range(zone_start_addr..zone_end_addr); | ||
|
|
@@ -619,6 +621,10 @@ pub struct BloomFilterIndexBuilder { | |
| // The local offset within the current zones | ||
| cur_zone_offset: usize, | ||
| cur_fragment_id: u64, | ||
| // Track the actual first and last row addresses in the current zone | ||
| // This handles non-contiguous addresses after deletions | ||
| cur_zone_first_row_addr: Option<u64>, | ||
| cur_zone_last_row_addr: Option<u64>, | ||
| cur_zone_has_null: bool, | ||
| sbbf: Option<Sbbf>, | ||
| } | ||
|
|
@@ -639,6 +645,8 @@ impl BloomFilterIndexBuilder { | |
| blocks: Vec::new(), | ||
| cur_zone_offset: 0, | ||
| cur_fragment_id: 0, | ||
| cur_zone_first_row_addr: None, | ||
| cur_zone_last_row_addr: None, | ||
| cur_zone_has_null: false, | ||
| sbbf: Some(sbbf), | ||
| }) | ||
|
|
@@ -922,13 +930,14 @@ impl BloomFilterIndexBuilder { | |
| } | ||
|
|
||
| fn new_block(&mut self, fragment_id: u64) -> Result<()> { | ||
| // Calculate zone_start based on existing zones in the same fragment | ||
| let zone_start = self | ||
| .blocks | ||
| .iter() | ||
| .filter(|block| block.fragment_id == fragment_id) | ||
| .map(|block| block.zone_length as u64) | ||
| .sum::<u64>(); | ||
| // Use the actual first and last row addresses we tracked | ||
| // zone_length is the address span (last - first + 1), not row count | ||
| // This correctly handles non-contiguous addresses after deletions | ||
| let zone_start = self.cur_zone_first_row_addr.unwrap_or(0); | ||
| let zone_length = self | ||
| .cur_zone_last_row_addr | ||
| .map(|last_addr| last_addr - zone_start + 1) | ||
| .unwrap_or(self.cur_zone_offset as u64); | ||
|
|
||
| // Store the current bloom filter directly | ||
| let bloom_filter = if let Some(ref sbbf) = self.sbbf { | ||
|
|
@@ -948,13 +957,15 @@ impl BloomFilterIndexBuilder { | |
| let new_block = BloomFilterStatistics { | ||
| fragment_id, | ||
| zone_start, | ||
| zone_length: self.cur_zone_offset, | ||
| zone_length, | ||
| has_null: self.cur_zone_has_null, | ||
| bloom_filter, | ||
| }; | ||
|
|
||
| self.blocks.push(new_block); | ||
| self.cur_zone_offset = 0; | ||
| self.cur_zone_first_row_addr = None; | ||
| self.cur_zone_last_row_addr = None; | ||
| self.cur_zone_has_null = false; | ||
|
|
||
| // Reset sbbf for the next block | ||
|
|
@@ -1023,11 +1034,30 @@ impl BloomFilterIndexBuilder { | |
| if desired > remaining { | ||
| // Not enough data to fill a map, just increment counts | ||
| self.update_stats(&data_array.slice(array_offset, remaining))?; | ||
|
|
||
| // Track first and last row addresses (local offsets within fragment) | ||
| let first_addr = row_addrs_array.value(array_offset) & 0xFFFFFFFF; | ||
| let last_addr = | ||
| row_addrs_array.value(array_offset + remaining - 1) & 0xFFFFFFFF; | ||
| if self.cur_zone_first_row_addr.is_none() { | ||
| self.cur_zone_first_row_addr = Some(first_addr); | ||
| } | ||
| self.cur_zone_last_row_addr = Some(last_addr); | ||
|
|
||
| self.cur_zone_offset += remaining; | ||
| break; | ||
| } else if desired > 0 { | ||
| // There is enough data, create a new zone | ||
| self.update_stats(&data_array.slice(array_offset, desired))?; | ||
|
|
||
| // Track first and last row addresses (local offsets within fragment) | ||
| let first_addr = row_addrs_array.value(array_offset) & 0xFFFFFFFF; | ||
| let last_addr = row_addrs_array.value(array_offset + desired - 1) & 0xFFFFFFFF; | ||
| if self.cur_zone_first_row_addr.is_none() { | ||
| self.cur_zone_first_row_addr = Some(first_addr); | ||
| } | ||
| self.cur_zone_last_row_addr = Some(last_addr); | ||
|
|
||
| self.cur_zone_offset += desired; | ||
| self.new_block(row_addrs_array.value(array_offset) >> 32)?; | ||
| } else if desired == 0 { | ||
|
|
@@ -1059,7 +1089,7 @@ impl BloomFilterIndexBuilder { | |
| UInt64Array::from_iter_values(self.blocks.iter().map(|block| block.zone_start)); | ||
|
|
||
| let zone_lengths = | ||
| UInt64Array::from_iter_values(self.blocks.iter().map(|block| block.zone_length as u64)); | ||
| UInt64Array::from_iter_values(self.blocks.iter().map(|block| block.zone_length)); | ||
|
|
||
| let has_nulls = arrow_array::BooleanArray::from( | ||
| self.blocks | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Local offset and row address are not the same thing.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
true, will fix the wrong wording