Skip to content
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

feat(bloom-filter): bloom filter applier #5220

Merged
merged 17 commits into from
Dec 26, 2024
Merged
Show file tree
Hide file tree
Changes from 12 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions src/index/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ fst.workspace = true
futures.workspace = true
greptime-proto.workspace = true
mockall.workspace = true
parquet.workspace = true
pin-project.workspace = true
prost.workspace = true
regex.workspace = true
Expand Down
5 changes: 3 additions & 2 deletions src/index/src/bloom_filter.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@

use serde::{Deserialize, Serialize};

pub mod applier;
pub mod creator;
pub mod error;
pub mod reader;
Expand All @@ -25,7 +26,7 @@ pub type BytesRef<'a> = &'a [u8];
pub const SEED: u128 = 42;

/// The Meta information of the bloom filter stored in the file.
#[derive(Debug, Default, Serialize, Deserialize)]
#[derive(Debug, Default, Serialize, Deserialize, Clone)]
pub struct BloomFilterMeta {
/// The number of rows per segment.
pub rows_per_segment: usize,
Expand All @@ -44,7 +45,7 @@ pub struct BloomFilterMeta {
}

/// The location of the bloom filter segment in the file.
#[derive(Debug, Serialize, Deserialize)]
#[derive(Debug, Serialize, Deserialize, Clone, Hash, PartialEq, Eq)]
pub struct BloomFilterSegmentLocation {
/// The offset of the bloom filter segment in the file.
pub offset: u64,
Expand Down
115 changes: 115 additions & 0 deletions src/index/src/bloom_filter/applier.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::collections::{BTreeMap, HashSet};

use parquet::arrow::arrow_reader::RowSelection;
use parquet::file::metadata::RowGroupMetaData;

use crate::bloom_filter::error::Result;
use crate::bloom_filter::reader::BloomFilterReader;
use crate::bloom_filter::{BloomFilterMeta, BloomFilterSegmentLocation, Bytes};

/// Enumerates types of predicates for value filtering.
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum Predicate {
/// Predicate for matching values in a list.
InList(InListPredicate),
}

/// `InListPredicate` contains a list of acceptable values. A value needs to match at least
/// one of the elements (logical OR semantic) for the predicate to be satisfied.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct InListPredicate {
/// List of acceptable values.
pub list: HashSet<Bytes>,
}

pub struct BloomFilterApplier {
reader: Box<dyn BloomFilterReader + Send>,
meta: BloomFilterMeta,
}

impl BloomFilterApplier {
pub async fn new(mut reader: Box<dyn BloomFilterReader + Send>) -> Result<Self> {
let meta = reader.metadata().await?;

Ok(Self { reader, meta })
}

pub async fn search(
&mut self,
probes: &HashSet<Bytes>,
row_group_metas: &[RowGroupMetaData],
basement: &mut BTreeMap<usize, Option<RowSelection>>,
) -> Result<()> {
// 0. Fast path - if basement is empty return empty vec
if basement.is_empty() {
return Ok(());
}

// 1. Compute prefix sum for row counts
let mut sum = 0usize;
let mut prefix_sum = Vec::with_capacity(row_group_metas.len() + 1);
prefix_sum.push(0usize);
for meta in row_group_metas {
sum += meta.num_rows() as usize;
prefix_sum.push(sum);
}

// 2. Calculate bloom filter segment locations
let mut row_groups_to_remove = HashSet::new();
for &row_group_idx in basement.keys() {
// TODO(ruihang): support further filter over row selection

// todo: dedup & overlap
let rows_range_start = prefix_sum[row_group_idx] / self.meta.rows_per_segment;
let rows_range_end = (prefix_sum[row_group_idx + 1] as f64
/ self.meta.rows_per_segment as f64)
.ceil() as usize;

let mut is_any_range_hit = false;
for i in rows_range_start..rows_range_end {
// 3. Probe each bloom filter segment
let loc = BloomFilterSegmentLocation {
offset: self.meta.bloom_filter_segments[i].offset,
size: self.meta.bloom_filter_segments[i].size,
elem_count: self.meta.bloom_filter_segments[i].elem_count,
};
let bloom = self.reader.bloom_filter(&loc).await?;

// Check if all probes exist in bloom filter
let mut matches = true;
for probe in probes {
if !bloom.contains(probe) {
matches = false;
break;
}
}
waynexia marked this conversation as resolved.
Show resolved Hide resolved

is_any_range_hit |= matches;
}
if !is_any_range_hit {
row_groups_to_remove.insert(row_group_idx);
}
}

// 4. Remove row groups that do not match any bloom filter segment
for row_group_idx in row_groups_to_remove {
basement.remove(&row_group_idx);
}

Ok(())
}
}
14 changes: 9 additions & 5 deletions src/index/src/bloom_filter/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,15 @@ pub trait BloomFilterReader {
async fn range_read(&mut self, offset: u64, size: u32) -> Result<Bytes>;

/// Reads bunch of ranges from the file.
async fn read_vec(&mut self, ranges: &[Range<u64>]) -> Result<Vec<Bytes>>;
async fn read_vec(&mut self, ranges: &[Range<u64>]) -> Result<Vec<Bytes>> {
let mut results = Vec::with_capacity(ranges.len());
for range in ranges {
let size = (range.end - range.start) as u32;
let data = self.range_read(range.start, size).await?;
results.push(data);
}
Ok(results)
}

/// Reads the meta information of the bloom filter.
async fn metadata(&mut self) -> Result<BloomFilterMeta>;
Expand Down Expand Up @@ -79,10 +87,6 @@ impl<R: RangeReader> BloomFilterReader for BloomFilterReaderImpl<R> {
.context(IoSnafu)
}

async fn read_vec(&mut self, ranges: &[Range<u64>]) -> Result<Vec<Bytes>> {
self.reader.read_vec(ranges).await.context(IoSnafu)
}

async fn metadata(&mut self) -> Result<BloomFilterMeta> {
let metadata = self.reader.metadata().await.context(IoSnafu)?;
let file_size = metadata.content_length;
Expand Down
14 changes: 14 additions & 0 deletions src/mito2/src/cache.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ use std::sync::Arc;
use bytes::Bytes;
use datatypes::value::Value;
use datatypes::vectors::VectorRef;
use index::bloom_filter_index::{BloomFilterIndexCache, BloomFilterIndexCacheRef};
use moka::notification::RemovalCause;
use moka::sync::Cache;
use parquet::column::page::Page;
Expand Down Expand Up @@ -69,6 +70,8 @@ pub struct CacheManager {
write_cache: Option<WriteCacheRef>,
/// Cache for inverted index.
index_cache: Option<InvertedIndexCacheRef>,
/// Cache for bloom filter index.
bloom_filter_index_cache: Option<BloomFilterIndexCacheRef>,
/// Puffin metadata cache.
puffin_metadata_cache: Option<PuffinMetadataCacheRef>,
/// Cache for time series selectors.
Expand Down Expand Up @@ -221,6 +224,10 @@ impl CacheManager {
self.index_cache.as_ref()
}

pub(crate) fn bloom_filter_index_cache(&self) -> Option<&BloomFilterIndexCacheRef> {
self.bloom_filter_index_cache.as_ref()
}

pub(crate) fn puffin_metadata_cache(&self) -> Option<&PuffinMetadataCacheRef> {
self.puffin_metadata_cache.as_ref()
}
Expand Down Expand Up @@ -364,6 +371,12 @@ impl CacheManagerBuilder {
self.index_content_size,
self.index_content_page_size,
);
// TODO(ruihang): check if it's ok to reuse the same param with inverted index
let bloom_filter_index_cache = BloomFilterIndexCache::new(
self.index_metadata_size,
self.index_content_size,
self.index_content_page_size,
);
Comment on lines +375 to +379
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added BloomFilter dedicated cache size configuration in #5237

let puffin_metadata_cache =
PuffinMetadataCache::new(self.puffin_metadata_size, &CACHE_BYTES);
let selector_result_cache = (self.selector_result_cache_size != 0).then(|| {
Expand All @@ -387,6 +400,7 @@ impl CacheManagerBuilder {
page_cache,
write_cache: self.write_cache,
index_cache: Some(Arc::new(inverted_index_cache)),
bloom_filter_index_cache: Some(Arc::new(bloom_filter_index_cache)),
puffin_metadata_cache: Some(Arc::new(puffin_metadata_cache)),
selector_result_cache,
}
Expand Down
1 change: 1 addition & 0 deletions src/mito2/src/cache/index.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

pub mod bloom_filter_index;
pub mod inverted_index;

use std::future::Future;
Expand Down
Loading
Loading