mito2/sst/index/bloom_filter/
creator.rs1use std::collections::HashMap;
16use std::sync::atomic::AtomicUsize;
17use std::sync::Arc;
18
19use common_telemetry::{debug, warn};
20use datatypes::schema::SkippingIndexType;
21use index::bloom_filter::creator::BloomFilterCreator;
22use mito_codec::index::{IndexValueCodec, IndexValuesCodec};
23use mito_codec::row_converter::SortField;
24use puffin::puffin_manager::{PuffinWriter, PutOptions};
25use snafu::{ensure, ResultExt};
26use store_api::metadata::RegionMetadataRef;
27use store_api::storage::ColumnId;
28use tokio_util::compat::{TokioAsyncReadCompatExt, TokioAsyncWriteCompatExt};
29
30use crate::error::{
31 BiErrorsSnafu, BloomFilterFinishSnafu, EncodeSnafu, IndexOptionsSnafu,
32 OperateAbortedIndexSnafu, PuffinAddBlobSnafu, PushBloomFilterValueSnafu, Result,
33};
34use crate::read::Batch;
35use crate::sst::file::FileId;
36use crate::sst::index::bloom_filter::INDEX_BLOB_TYPE;
37use crate::sst::index::intermediate::{
38 IntermediateLocation, IntermediateManager, TempFileProvider,
39};
40use crate::sst::index::puffin_manager::SstPuffinWriter;
41use crate::sst::index::statistics::{ByteCount, RowCount, Statistics};
42use crate::sst::index::TYPE_BLOOM_FILTER_INDEX;
43
44const PIPE_BUFFER_SIZE_FOR_SENDING_BLOB: usize = 8192;
46
47pub struct BloomFilterIndexer {
49 creators: HashMap<ColumnId, BloomFilterCreator>,
51
52 temp_file_provider: Arc<TempFileProvider>,
54
55 codec: IndexValuesCodec,
57
58 aborted: bool,
60
61 stats: Statistics,
63
64 global_memory_usage: Arc<AtomicUsize>,
66}
67
68impl BloomFilterIndexer {
69 pub fn new(
71 sst_file_id: FileId,
72 metadata: &RegionMetadataRef,
73 intermediate_manager: IntermediateManager,
74 memory_usage_threshold: Option<usize>,
75 ) -> Result<Option<Self>> {
76 let mut creators = HashMap::new();
77
78 let temp_file_provider = Arc::new(TempFileProvider::new(
79 IntermediateLocation::new(&metadata.region_id, &sst_file_id),
80 intermediate_manager,
81 ));
82 let global_memory_usage = Arc::new(AtomicUsize::new(0));
83
84 for column in &metadata.column_metadatas {
85 let options =
86 column
87 .column_schema
88 .skipping_index_options()
89 .context(IndexOptionsSnafu {
90 column_name: &column.column_schema.name,
91 })?;
92
93 let options = match options {
94 Some(options) if options.index_type == SkippingIndexType::BloomFilter => options,
95 _ => continue,
96 };
97
98 let creator = BloomFilterCreator::new(
99 options.granularity as _,
100 temp_file_provider.clone(),
101 global_memory_usage.clone(),
102 memory_usage_threshold,
103 );
104 creators.insert(column.column_id, creator);
105 }
106
107 if creators.is_empty() {
108 return Ok(None);
109 }
110
111 let codec = IndexValuesCodec::from_tag_columns(
112 metadata.primary_key_encoding,
113 metadata.primary_key_columns(),
114 );
115 let indexer = Self {
116 creators,
117 temp_file_provider,
118 codec,
119 aborted: false,
120 stats: Statistics::new(TYPE_BLOOM_FILTER_INDEX),
121 global_memory_usage,
122 };
123 Ok(Some(indexer))
124 }
125
126 pub async fn update(&mut self, batch: &mut Batch) -> Result<()> {
131 ensure!(!self.aborted, OperateAbortedIndexSnafu);
132
133 if self.creators.is_empty() {
134 return Ok(());
135 }
136
137 if let Err(update_err) = self.do_update(batch).await {
138 if let Err(err) = self.do_cleanup().await {
140 if cfg!(any(test, feature = "test")) {
141 panic!("Failed to clean up index creator, err: {err:?}",);
142 } else {
143 warn!(err; "Failed to clean up index creator");
144 }
145 }
146 return Err(update_err);
147 }
148
149 Ok(())
150 }
151
152 pub async fn finish(
157 &mut self,
158 puffin_writer: &mut SstPuffinWriter,
159 ) -> Result<(RowCount, ByteCount)> {
160 ensure!(!self.aborted, OperateAbortedIndexSnafu);
161
162 if self.stats.row_count() == 0 {
163 return Ok((0, 0));
165 }
166
167 let finish_res = self.do_finish(puffin_writer).await;
168 if let Err(err) = self.do_cleanup().await {
170 if cfg!(any(test, feature = "test")) {
171 panic!("Failed to clean up index creator, err: {err:?}",);
172 } else {
173 warn!(err; "Failed to clean up index creator");
174 }
175 }
176
177 finish_res.map(|_| (self.stats.row_count(), self.stats.byte_count()))
178 }
179
180 pub async fn abort(&mut self) -> Result<()> {
184 if self.aborted {
185 return Ok(());
186 }
187 self.aborted = true;
188
189 self.do_cleanup().await
190 }
191
192 async fn do_update(&mut self, batch: &mut Batch) -> Result<()> {
193 let mut guard = self.stats.record_update();
194
195 let n = batch.num_rows();
196 guard.inc_row_count(n);
197
198 for (col_id, creator) in &mut self.creators {
199 match self.codec.pk_col_info(*col_id) {
200 Some(col_info) => {
202 let pk_idx = col_info.idx;
203 let field = &col_info.field;
204 let elems = batch
205 .pk_col_value(self.codec.decoder(), pk_idx, *col_id)?
206 .filter(|v| !v.is_null())
207 .map(|v| {
208 let mut buf = vec![];
209 IndexValueCodec::encode_nonnull_value(
210 v.as_value_ref(),
211 field,
212 &mut buf,
213 )
214 .context(EncodeSnafu)?;
215 Ok(buf)
216 })
217 .transpose()?;
218 creator
219 .push_n_row_elems(n, elems)
220 .await
221 .context(PushBloomFilterValueSnafu)?;
222 }
223 None => {
225 let Some(values) = batch.field_col_value(*col_id) else {
226 debug!(
227 "Column {} not found in the batch during building bloom filter index",
228 col_id
229 );
230 continue;
231 };
232 let sort_field = SortField::new(values.data.data_type());
233 for i in 0..n {
234 let value = values.data.get_ref(i);
235 let elems = (!value.is_null())
236 .then(|| {
237 let mut buf = vec![];
238 IndexValueCodec::encode_nonnull_value(value, &sort_field, &mut buf)
239 .context(EncodeSnafu)?;
240 Ok(buf)
241 })
242 .transpose()?;
243
244 creator
245 .push_row_elems(elems)
246 .await
247 .context(PushBloomFilterValueSnafu)?;
248 }
249 }
250 }
251 }
252
253 Ok(())
254 }
255
256 async fn do_finish(&mut self, puffin_writer: &mut SstPuffinWriter) -> Result<()> {
258 let mut guard = self.stats.record_finish();
259
260 for (id, creator) in &mut self.creators {
261 let written_bytes = Self::do_finish_single_creator(id, creator, puffin_writer).await?;
262 guard.inc_byte_count(written_bytes);
263 }
264
265 Ok(())
266 }
267
268 async fn do_cleanup(&mut self) -> Result<()> {
269 let mut _guard = self.stats.record_cleanup();
270
271 self.creators.clear();
272 self.temp_file_provider.cleanup().await
273 }
274
275 async fn do_finish_single_creator(
296 col_id: &ColumnId,
297 creator: &mut BloomFilterCreator,
298 puffin_writer: &mut SstPuffinWriter,
299 ) -> Result<ByteCount> {
300 let (tx, rx) = tokio::io::duplex(PIPE_BUFFER_SIZE_FOR_SENDING_BLOB);
301
302 let blob_name = format!("{}-{}", INDEX_BLOB_TYPE, col_id);
303 let (index_finish, puffin_add_blob) = futures::join!(
304 creator.finish(tx.compat_write()),
305 puffin_writer.put_blob(
306 &blob_name,
307 rx.compat(),
308 PutOptions::default(),
309 Default::default(),
310 )
311 );
312
313 match (
314 puffin_add_blob.context(PuffinAddBlobSnafu),
315 index_finish.context(BloomFilterFinishSnafu),
316 ) {
317 (Err(e1), Err(e2)) => BiErrorsSnafu {
318 first: Box::new(e1),
319 second: Box::new(e2),
320 }
321 .fail()?,
322
323 (Ok(_), e @ Err(_)) => e?,
324 (e @ Err(_), Ok(_)) => e.map(|_| ())?,
325 (Ok(written_bytes), Ok(_)) => {
326 return Ok(written_bytes);
327 }
328 }
329
330 Ok(0)
331 }
332
333 pub fn memory_usage(&self) -> usize {
335 self.global_memory_usage
336 .load(std::sync::atomic::Ordering::Relaxed)
337 }
338
339 pub fn column_ids(&self) -> impl Iterator<Item = ColumnId> + use<'_> {
341 self.creators.keys().copied()
342 }
343}
344
345#[cfg(test)]
346pub(crate) mod tests {
347
348 use api::v1::SemanticType;
349 use datatypes::data_type::ConcreteDataType;
350 use datatypes::schema::{ColumnSchema, SkippingIndexOptions};
351 use datatypes::value::ValueRef;
352 use datatypes::vectors::{UInt64Vector, UInt8Vector};
353 use index::bloom_filter::reader::{BloomFilterReader, BloomFilterReaderImpl};
354 use mito_codec::row_converter::{DensePrimaryKeyCodec, PrimaryKeyCodecExt};
355 use object_store::services::Memory;
356 use object_store::ObjectStore;
357 use puffin::puffin_manager::{PuffinManager, PuffinReader};
358 use store_api::metadata::{ColumnMetadata, RegionMetadataBuilder};
359 use store_api::storage::RegionId;
360
361 use super::*;
362 use crate::access_layer::FilePathProvider;
363 use crate::read::BatchColumn;
364 use crate::sst::index::puffin_manager::PuffinManagerFactory;
365
366 pub fn mock_object_store() -> ObjectStore {
367 ObjectStore::new(Memory::default()).unwrap().finish()
368 }
369
370 pub async fn new_intm_mgr(path: impl AsRef<str>) -> IntermediateManager {
371 IntermediateManager::init_fs(path).await.unwrap()
372 }
373
374 pub struct TestPathProvider;
375
376 impl FilePathProvider for TestPathProvider {
377 fn build_index_file_path(&self, file_id: FileId) -> String {
378 file_id.to_string()
379 }
380
381 fn build_sst_file_path(&self, file_id: FileId) -> String {
382 file_id.to_string()
383 }
384 }
385
386 pub fn mock_region_metadata() -> RegionMetadataRef {
403 let mut builder = RegionMetadataBuilder::new(RegionId::new(1, 2));
404 builder
405 .push_column_metadata(ColumnMetadata {
406 column_schema: ColumnSchema::new(
407 "tag_str",
408 ConcreteDataType::string_datatype(),
409 false,
410 )
411 .with_skipping_options(SkippingIndexOptions {
412 index_type: SkippingIndexType::BloomFilter,
413 granularity: 2,
414 })
415 .unwrap(),
416 semantic_type: SemanticType::Tag,
417 column_id: 1,
418 })
419 .push_column_metadata(ColumnMetadata {
420 column_schema: ColumnSchema::new(
421 "ts",
422 ConcreteDataType::timestamp_millisecond_datatype(),
423 false,
424 ),
425 semantic_type: SemanticType::Timestamp,
426 column_id: 2,
427 })
428 .push_column_metadata(ColumnMetadata {
429 column_schema: ColumnSchema::new(
430 "field_u64",
431 ConcreteDataType::uint64_datatype(),
432 false,
433 )
434 .with_skipping_options(SkippingIndexOptions {
435 index_type: SkippingIndexType::BloomFilter,
436 granularity: 4,
437 })
438 .unwrap(),
439 semantic_type: SemanticType::Field,
440 column_id: 3,
441 })
442 .primary_key(vec![1]);
443
444 Arc::new(builder.build().unwrap())
445 }
446
447 pub fn new_batch(str_tag: impl AsRef<str>, u64_field: impl IntoIterator<Item = u64>) -> Batch {
448 let fields = vec![(0, SortField::new(ConcreteDataType::string_datatype()))];
449 let codec = DensePrimaryKeyCodec::with_fields(fields);
450 let row: [ValueRef; 1] = [str_tag.as_ref().into()];
451 let primary_key = codec.encode(row.into_iter()).unwrap();
452
453 let u64_field = BatchColumn {
454 column_id: 3,
455 data: Arc::new(UInt64Vector::from_iter_values(u64_field)),
456 };
457 let num_rows = u64_field.data.len();
458
459 Batch::new(
460 primary_key,
461 Arc::new(UInt64Vector::from_iter_values(std::iter::repeat_n(
462 0, num_rows,
463 ))),
464 Arc::new(UInt64Vector::from_iter_values(std::iter::repeat_n(
465 0, num_rows,
466 ))),
467 Arc::new(UInt8Vector::from_iter_values(std::iter::repeat_n(
468 1, num_rows,
469 ))),
470 vec![u64_field],
471 )
472 .unwrap()
473 }
474
475 #[tokio::test]
476 async fn test_bloom_filter_indexer() {
477 let prefix = "test_bloom_filter_indexer_";
478 let tempdir = common_test_util::temp_dir::create_temp_dir(prefix);
479 let object_store = mock_object_store();
480 let intm_mgr = new_intm_mgr(tempdir.path().to_string_lossy()).await;
481 let region_metadata = mock_region_metadata();
482 let memory_usage_threshold = Some(1024);
483
484 let mut indexer = BloomFilterIndexer::new(
485 FileId::random(),
486 ®ion_metadata,
487 intm_mgr,
488 memory_usage_threshold,
489 )
490 .unwrap()
491 .unwrap();
492
493 let mut batch = new_batch("tag1", 0..10);
495 indexer.update(&mut batch).await.unwrap();
496
497 let mut batch = new_batch("tag2", 10..20);
498 indexer.update(&mut batch).await.unwrap();
499
500 let (_d, factory) = PuffinManagerFactory::new_for_test_async(prefix).await;
501 let puffin_manager = factory.build(object_store, TestPathProvider);
502
503 let file_id = FileId::random();
504 let mut puffin_writer = puffin_manager.writer(&file_id).await.unwrap();
505 let (row_count, byte_count) = indexer.finish(&mut puffin_writer).await.unwrap();
506 assert_eq!(row_count, 20);
507 assert!(byte_count > 0);
508 puffin_writer.finish().await.unwrap();
509
510 let puffin_reader = puffin_manager.reader(&file_id).await.unwrap();
511
512 {
514 let blob_guard = puffin_reader
515 .blob("greptime-bloom-filter-v1-1")
516 .await
517 .unwrap();
518 let reader = blob_guard.reader().await.unwrap();
519 let bloom_filter = BloomFilterReaderImpl::new(reader);
520 let metadata = bloom_filter.metadata().await.unwrap();
521
522 assert_eq!(metadata.segment_count, 10);
523 for i in 0..5 {
524 let loc = &metadata.bloom_filter_locs[metadata.segment_loc_indices[i] as usize];
525 let bf = bloom_filter.bloom_filter(loc).await.unwrap();
526 assert!(bf.contains(b"tag1"));
527 }
528 for i in 5..10 {
529 let loc = &metadata.bloom_filter_locs[metadata.segment_loc_indices[i] as usize];
530 let bf = bloom_filter.bloom_filter(loc).await.unwrap();
531 assert!(bf.contains(b"tag2"));
532 }
533 }
534
535 {
537 let sort_field = SortField::new(ConcreteDataType::uint64_datatype());
538
539 let blob_guard = puffin_reader
540 .blob("greptime-bloom-filter-v1-3")
541 .await
542 .unwrap();
543 let reader = blob_guard.reader().await.unwrap();
544 let bloom_filter = BloomFilterReaderImpl::new(reader);
545 let metadata = bloom_filter.metadata().await.unwrap();
546
547 assert_eq!(metadata.segment_count, 5);
548 for i in 0u64..20 {
549 let idx = i as usize / 4;
550 let loc = &metadata.bloom_filter_locs[metadata.segment_loc_indices[idx] as usize];
551 let bf = bloom_filter.bloom_filter(loc).await.unwrap();
552 let mut buf = vec![];
553 IndexValueCodec::encode_nonnull_value(ValueRef::UInt64(i), &sort_field, &mut buf)
554 .unwrap();
555
556 assert!(bf.contains(&buf));
557 }
558 }
559 }
560}