index/bloom_filter/
creator.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
// 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.

mod finalize_segment;
mod intermediate_codec;

use std::collections::HashSet;
use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::Arc;

use finalize_segment::FinalizedBloomFilterStorage;
use futures::{AsyncWrite, AsyncWriteExt, StreamExt};
use greptime_proto::v1::index::{BloomFilterLoc, BloomFilterMeta};
use prost::Message;
use snafu::ResultExt;

use crate::bloom_filter::error::{IoSnafu, Result};
use crate::bloom_filter::SEED;
use crate::external_provider::ExternalTempFileProvider;
use crate::Bytes;

/// The false positive rate of the Bloom filter.
pub const FALSE_POSITIVE_RATE: f64 = 0.01;

/// `BloomFilterCreator` is responsible for creating and managing bloom filters
/// for a set of elements. It divides the rows into segments and creates
/// bloom filters for each segment.
///
/// # Format
///
/// The bloom filter creator writes the following format to the writer:
///
/// ```text
/// +--------------------+--------------------+-----+----------------------+----------------------+
/// | Bloom filter 0     | Bloom filter 1     | ... | BloomFilterMeta      | Meta size            |
/// +--------------------+--------------------+-----+----------------------+----------------------+
/// |<- bytes (size 0) ->|<- bytes (size 1) ->| ... |<- json (meta size) ->|<- u32 LE (4 bytes) ->|
/// ```
///
pub struct BloomFilterCreator {
    /// The number of rows per segment set by the user.
    rows_per_segment: usize,

    /// Row count that added to the bloom filter so far.
    accumulated_row_count: usize,

    /// A set of distinct elements in the current segment.
    cur_seg_distinct_elems: HashSet<Bytes>,

    /// The memory usage of the current segment's distinct elements.
    cur_seg_distinct_elems_mem_usage: usize,

    /// Storage for finalized Bloom filters.
    finalized_bloom_filters: FinalizedBloomFilterStorage,

    /// Row count that finalized so far.
    finalized_row_count: usize,

    /// Global memory usage of the bloom filter creator.
    global_memory_usage: Arc<AtomicUsize>,
}

impl BloomFilterCreator {
    /// Creates a new `BloomFilterCreator` with the specified number of rows per segment.
    ///
    /// # PANICS
    ///
    /// `rows_per_segment` <= 0
    pub fn new(
        rows_per_segment: usize,
        intermediate_provider: Arc<dyn ExternalTempFileProvider>,
        global_memory_usage: Arc<AtomicUsize>,
        global_memory_usage_threshold: Option<usize>,
    ) -> Self {
        assert!(
            rows_per_segment > 0,
            "rows_per_segment must be greater than 0"
        );

        Self {
            rows_per_segment,
            accumulated_row_count: 0,
            cur_seg_distinct_elems: HashSet::default(),
            cur_seg_distinct_elems_mem_usage: 0,
            global_memory_usage: global_memory_usage.clone(),
            finalized_bloom_filters: FinalizedBloomFilterStorage::new(
                intermediate_provider,
                global_memory_usage,
                global_memory_usage_threshold,
            ),
            finalized_row_count: 0,
        }
    }

    /// Adds multiple rows of elements to the bloom filter. If the number of accumulated rows
    /// reaches `rows_per_segment`, it finalizes the current segment.
    pub async fn push_n_row_elems(
        &mut self,
        mut nrows: usize,
        elems: impl IntoIterator<Item = Bytes>,
    ) -> Result<()> {
        if nrows == 0 {
            return Ok(());
        }
        if nrows == 1 {
            return self.push_row_elems(elems).await;
        }

        let elems = elems.into_iter().collect::<Vec<_>>();
        while nrows > 0 {
            let rows_to_seg_end =
                self.rows_per_segment - (self.accumulated_row_count % self.rows_per_segment);
            let rows_to_push = nrows.min(rows_to_seg_end);
            nrows -= rows_to_push;

            self.accumulated_row_count += rows_to_push;

            let mut mem_diff = 0;
            for elem in &elems {
                let len = elem.len();
                let is_new = self.cur_seg_distinct_elems.insert(elem.clone());
                if is_new {
                    mem_diff += len;
                }
            }
            self.cur_seg_distinct_elems_mem_usage += mem_diff;
            self.global_memory_usage
                .fetch_add(mem_diff, Ordering::Relaxed);

            if self.accumulated_row_count % self.rows_per_segment == 0 {
                self.finalize_segment().await?;
                self.finalized_row_count = self.accumulated_row_count;
            }
        }

        Ok(())
    }

    /// Adds a row of elements to the bloom filter. If the number of accumulated rows
    /// reaches `rows_per_segment`, it finalizes the current segment.
    pub async fn push_row_elems(&mut self, elems: impl IntoIterator<Item = Bytes>) -> Result<()> {
        self.accumulated_row_count += 1;

        let mut mem_diff = 0;
        for elem in elems.into_iter() {
            let len = elem.len();
            let is_new = self.cur_seg_distinct_elems.insert(elem);
            if is_new {
                mem_diff += len;
            }
        }
        self.cur_seg_distinct_elems_mem_usage += mem_diff;
        self.global_memory_usage
            .fetch_add(mem_diff, Ordering::Relaxed);

        if self.accumulated_row_count % self.rows_per_segment == 0 {
            self.finalize_segment().await?;
            self.finalized_row_count = self.accumulated_row_count;
        }

        Ok(())
    }

    /// Finalizes any remaining segments and writes the bloom filters and metadata to the provided writer.
    pub async fn finish(&mut self, mut writer: impl AsyncWrite + Unpin) -> Result<()> {
        if self.accumulated_row_count > self.finalized_row_count {
            self.finalize_segment().await?;
        }

        let mut meta = BloomFilterMeta {
            rows_per_segment: self.rows_per_segment as _,
            row_count: self.accumulated_row_count as _,
            ..Default::default()
        };

        let (indices, mut segs) = self.finalized_bloom_filters.drain().await?;
        meta.segment_loc_indices = indices.into_iter().map(|i| i as u64).collect();
        meta.segment_count = meta.segment_loc_indices.len() as _;

        while let Some(segment) = segs.next().await {
            let segment = segment?;
            writer
                .write_all(&segment.bloom_filter_bytes)
                .await
                .context(IoSnafu)?;

            let size = segment.bloom_filter_bytes.len() as u64;
            meta.bloom_filter_locs.push(BloomFilterLoc {
                offset: meta.bloom_filter_size as _,
                size,
                element_count: segment.element_count as _,
            });
            meta.bloom_filter_size += size;
        }

        let meta_bytes = meta.encode_to_vec();
        writer.write_all(&meta_bytes).await.context(IoSnafu)?;

        let meta_size = meta_bytes.len() as u32;
        writer
            .write_all(&meta_size.to_le_bytes())
            .await
            .context(IoSnafu)?;
        writer.flush().await.unwrap();

        Ok(())
    }

    /// Returns the memory usage of the creating bloom filter.
    pub fn memory_usage(&self) -> usize {
        self.cur_seg_distinct_elems_mem_usage + self.finalized_bloom_filters.memory_usage()
    }

    async fn finalize_segment(&mut self) -> Result<()> {
        let elem_count = self.cur_seg_distinct_elems.len();
        self.finalized_bloom_filters
            .add(self.cur_seg_distinct_elems.drain(), elem_count)
            .await?;

        self.global_memory_usage
            .fetch_sub(self.cur_seg_distinct_elems_mem_usage, Ordering::Relaxed);
        self.cur_seg_distinct_elems_mem_usage = 0;
        Ok(())
    }
}

impl Drop for BloomFilterCreator {
    fn drop(&mut self) {
        self.global_memory_usage
            .fetch_sub(self.cur_seg_distinct_elems_mem_usage, Ordering::Relaxed);
    }
}

#[cfg(test)]
mod tests {
    use fastbloom::BloomFilter;
    use futures::io::Cursor;

    use super::*;
    use crate::external_provider::MockExternalTempFileProvider;

    /// Converts a slice of bytes to a vector of `u64`.
    pub fn u64_vec_from_bytes(bytes: &[u8]) -> Vec<u64> {
        bytes
            .chunks_exact(std::mem::size_of::<u64>())
            .map(|chunk| u64::from_le_bytes(chunk.try_into().unwrap()))
            .collect()
    }

    #[tokio::test]
    async fn test_bloom_filter_creator() {
        let mut writer = Cursor::new(Vec::new());
        let mut creator = BloomFilterCreator::new(
            2,
            Arc::new(MockExternalTempFileProvider::new()),
            Arc::new(AtomicUsize::new(0)),
            None,
        );

        creator
            .push_row_elems(vec![b"a".to_vec(), b"b".to_vec()])
            .await
            .unwrap();
        assert!(creator.cur_seg_distinct_elems_mem_usage > 0);
        assert!(creator.memory_usage() > 0);

        creator
            .push_row_elems(vec![b"c".to_vec(), b"d".to_vec()])
            .await
            .unwrap();
        // Finalize the first segment
        assert_eq!(creator.cur_seg_distinct_elems_mem_usage, 0);
        assert!(creator.memory_usage() > 0);

        creator
            .push_row_elems(vec![b"e".to_vec(), b"f".to_vec()])
            .await
            .unwrap();
        assert!(creator.cur_seg_distinct_elems_mem_usage > 0);
        assert!(creator.memory_usage() > 0);

        creator.finish(&mut writer).await.unwrap();

        let bytes = writer.into_inner();
        let total_size = bytes.len();
        let meta_size_offset = total_size - 4;
        let meta_size = u32::from_le_bytes((&bytes[meta_size_offset..]).try_into().unwrap());

        let meta_bytes = &bytes[total_size - meta_size as usize - 4..total_size - 4];
        let meta = BloomFilterMeta::decode(meta_bytes).unwrap();

        assert_eq!(meta.rows_per_segment, 2);
        assert_eq!(meta.segment_count, 2);
        assert_eq!(meta.row_count, 3);
        assert_eq!(
            meta.bloom_filter_size as usize + meta_bytes.len() + 4,
            total_size
        );

        let mut bfs = Vec::new();
        for segment in meta.bloom_filter_locs {
            let bloom_filter_bytes =
                &bytes[segment.offset as usize..(segment.offset + segment.size) as usize];
            let v = u64_vec_from_bytes(bloom_filter_bytes);
            let bloom_filter = BloomFilter::from_vec(v)
                .seed(&SEED)
                .expected_items(segment.element_count as usize);
            bfs.push(bloom_filter);
        }

        assert_eq!(meta.segment_loc_indices.len(), 2);

        let bf0 = &bfs[meta.segment_loc_indices[0] as usize];
        assert!(bf0.contains(&b"a"));
        assert!(bf0.contains(&b"b"));
        assert!(bf0.contains(&b"c"));
        assert!(bf0.contains(&b"d"));

        let bf1 = &bfs[meta.segment_loc_indices[1] as usize];
        assert!(bf1.contains(&b"e"));
        assert!(bf1.contains(&b"f"));
    }

    #[tokio::test]
    async fn test_bloom_filter_creator_batch_push() {
        let mut writer = Cursor::new(Vec::new());
        let mut creator: BloomFilterCreator = BloomFilterCreator::new(
            2,
            Arc::new(MockExternalTempFileProvider::new()),
            Arc::new(AtomicUsize::new(0)),
            None,
        );

        creator
            .push_n_row_elems(5, vec![b"a".to_vec(), b"b".to_vec()])
            .await
            .unwrap();
        assert!(creator.cur_seg_distinct_elems_mem_usage > 0);
        assert!(creator.memory_usage() > 0);

        creator
            .push_n_row_elems(5, vec![b"c".to_vec(), b"d".to_vec()])
            .await
            .unwrap();
        assert_eq!(creator.cur_seg_distinct_elems_mem_usage, 0);
        assert!(creator.memory_usage() > 0);

        creator
            .push_n_row_elems(10, vec![b"e".to_vec(), b"f".to_vec()])
            .await
            .unwrap();
        assert_eq!(creator.cur_seg_distinct_elems_mem_usage, 0);
        assert!(creator.memory_usage() > 0);

        creator.finish(&mut writer).await.unwrap();

        let bytes = writer.into_inner();
        let total_size = bytes.len();
        let meta_size_offset = total_size - 4;
        let meta_size = u32::from_le_bytes((&bytes[meta_size_offset..]).try_into().unwrap());

        let meta_bytes = &bytes[total_size - meta_size as usize - 4..total_size - 4];
        let meta = BloomFilterMeta::decode(meta_bytes).unwrap();

        assert_eq!(meta.rows_per_segment, 2);
        assert_eq!(meta.segment_count, 10);
        assert_eq!(meta.row_count, 20);
        assert_eq!(
            meta.bloom_filter_size as usize + meta_bytes.len() + 4,
            total_size
        );

        let mut bfs = Vec::new();
        for segment in meta.bloom_filter_locs {
            let bloom_filter_bytes =
                &bytes[segment.offset as usize..(segment.offset + segment.size) as usize];
            let v = u64_vec_from_bytes(bloom_filter_bytes);
            let bloom_filter = BloomFilter::from_vec(v)
                .seed(&SEED)
                .expected_items(segment.element_count as _);
            bfs.push(bloom_filter);
        }

        // 4 bloom filters to serve 10 segments
        assert_eq!(bfs.len(), 4);
        assert_eq!(meta.segment_loc_indices.len(), 10);

        for idx in meta.segment_loc_indices.iter().take(3) {
            let bf = &bfs[*idx as usize];
            assert!(bf.contains(&b"a"));
            assert!(bf.contains(&b"b"));
        }
        for idx in meta.segment_loc_indices.iter().take(5).skip(2) {
            let bf = &bfs[*idx as usize];
            assert!(bf.contains(&b"c"));
            assert!(bf.contains(&b"d"));
        }
        for idx in meta.segment_loc_indices.iter().take(10).skip(5) {
            let bf = &bfs[*idx as usize];
            assert!(bf.contains(&b"e"));
            assert!(bf.contains(&b"f"));
        }
    }

    #[tokio::test]
    async fn test_final_seg_all_null() {
        let mut writer = Cursor::new(Vec::new());
        let mut creator = BloomFilterCreator::new(
            2,
            Arc::new(MockExternalTempFileProvider::new()),
            Arc::new(AtomicUsize::new(0)),
            None,
        );

        creator
            .push_n_row_elems(4, vec![b"a".to_vec(), b"b".to_vec()])
            .await
            .unwrap();
        creator.push_row_elems(Vec::new()).await.unwrap();

        creator.finish(&mut writer).await.unwrap();

        let bytes = writer.into_inner();
        let total_size = bytes.len();
        let meta_size_offset = total_size - 4;
        let meta_size = u32::from_le_bytes((&bytes[meta_size_offset..]).try_into().unwrap());

        let meta_bytes = &bytes[total_size - meta_size as usize - 4..total_size - 4];
        let meta = BloomFilterMeta::decode(meta_bytes).unwrap();

        assert_eq!(meta.rows_per_segment, 2);
        assert_eq!(meta.segment_count, 3);
        assert_eq!(meta.row_count, 5);
    }
}