index/bloom_filter/creator/
finalize_segment.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
// 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::pin::Pin;
use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::Arc;

use asynchronous_codec::{FramedRead, FramedWrite};
use fastbloom::BloomFilter;
use futures::stream::StreamExt;
use futures::{stream, AsyncWriteExt, Stream};
use snafu::ResultExt;

use super::intermediate_codec::IntermediateBloomFilterCodecV1;
use crate::bloom_filter::creator::{FALSE_POSITIVE_RATE, SEED};
use crate::bloom_filter::error::{IntermediateSnafu, IoSnafu, Result};
use crate::external_provider::ExternalTempFileProvider;
use crate::Bytes;

/// The minimum memory usage threshold for flushing in-memory Bloom filters to disk.
const MIN_MEMORY_USAGE_THRESHOLD: usize = 1024 * 1024; // 1MB

/// Storage for finalized Bloom filters.
pub struct FinalizedBloomFilterStorage {
    /// Indices of the segments in the sequence of finalized Bloom filters.
    segment_indices: Vec<usize>,

    /// Bloom filters that are stored in memory.
    in_memory: Vec<FinalizedBloomFilterSegment>,

    /// Used to generate unique file IDs for intermediate Bloom filters.
    intermediate_file_id_counter: usize,

    /// Prefix for intermediate Bloom filter files.
    intermediate_prefix: String,

    /// The provider for intermediate Bloom filter files.
    intermediate_provider: Arc<dyn ExternalTempFileProvider>,

    /// The memory usage of the in-memory Bloom filters.
    memory_usage: usize,

    /// The global memory usage provided by the user to track the
    /// total memory usage of the creating Bloom filters.
    global_memory_usage: Arc<AtomicUsize>,

    /// The threshold of the global memory usage of the creating Bloom filters.
    global_memory_usage_threshold: Option<usize>,

    /// Records the number of flushed segments.
    flushed_seg_count: usize,
}

impl FinalizedBloomFilterStorage {
    /// Creates a new `FinalizedBloomFilterStorage`.
    pub fn new(
        intermediate_provider: Arc<dyn ExternalTempFileProvider>,
        global_memory_usage: Arc<AtomicUsize>,
        global_memory_usage_threshold: Option<usize>,
    ) -> Self {
        let external_prefix = format!("intm-bloom-filters-{}", uuid::Uuid::new_v4());
        Self {
            segment_indices: Vec::new(),
            in_memory: Vec::new(),
            intermediate_file_id_counter: 0,
            intermediate_prefix: external_prefix,
            intermediate_provider,
            memory_usage: 0,
            global_memory_usage,
            global_memory_usage_threshold,
            flushed_seg_count: 0,
        }
    }

    /// Returns the memory usage of the storage.
    pub fn memory_usage(&self) -> usize {
        self.memory_usage
    }

    /// Adds a new finalized Bloom filter to the storage.
    ///
    /// If the memory usage exceeds the threshold, flushes the in-memory Bloom filters to disk.
    pub async fn add(
        &mut self,
        elems: impl IntoIterator<Item = Bytes>,
        element_count: usize,
    ) -> Result<()> {
        let mut bf = BloomFilter::with_false_pos(FALSE_POSITIVE_RATE)
            .seed(&SEED)
            .expected_items(element_count);
        for elem in elems.into_iter() {
            bf.insert(&elem);
        }

        let fbf = FinalizedBloomFilterSegment::from(bf, element_count);

        // Reuse the last segment if it is the same as the current one.
        if self.in_memory.last() == Some(&fbf) {
            self.segment_indices
                .push(self.flushed_seg_count + self.in_memory.len() - 1);
            return Ok(());
        }

        // Update memory usage.
        let memory_diff = fbf.bloom_filter_bytes.len();
        self.memory_usage += memory_diff;
        self.global_memory_usage
            .fetch_add(memory_diff, Ordering::Relaxed);

        // Add the finalized Bloom filter to the in-memory storage.
        self.in_memory.push(fbf);
        self.segment_indices
            .push(self.flushed_seg_count + self.in_memory.len() - 1);

        // Flush to disk if necessary.

        // Do not flush if memory usage is too low.
        if self.memory_usage < MIN_MEMORY_USAGE_THRESHOLD {
            return Ok(());
        }

        // Check if the global memory usage exceeds the threshold and flush to disk if necessary.
        if let Some(threshold) = self.global_memory_usage_threshold {
            let global = self.global_memory_usage.load(Ordering::Relaxed);

            if global > threshold {
                self.flush_in_memory_to_disk().await?;

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

        Ok(())
    }

    /// Drains the storage and returns indieces of the segments and a stream of finalized Bloom filters.
    pub async fn drain(
        &mut self,
    ) -> Result<(
        Vec<usize>,
        Pin<Box<dyn Stream<Item = Result<FinalizedBloomFilterSegment>> + Send + '_>>,
    )> {
        // FAST PATH: memory only
        if self.intermediate_file_id_counter == 0 {
            return Ok((
                std::mem::take(&mut self.segment_indices),
                Box::pin(stream::iter(self.in_memory.drain(..).map(Ok))),
            ));
        }

        // SLOW PATH: memory + disk
        let mut on_disk = self
            .intermediate_provider
            .read_all(&self.intermediate_prefix)
            .await
            .context(IntermediateSnafu)?;
        on_disk.sort_unstable_by(|x, y| x.0.cmp(&y.0));

        let streams = on_disk
            .into_iter()
            .map(|(_, reader)| FramedRead::new(reader, IntermediateBloomFilterCodecV1::default()));

        let in_memory_stream = stream::iter(self.in_memory.drain(..)).map(Ok);
        Ok((
            std::mem::take(&mut self.segment_indices),
            Box::pin(stream::iter(streams).flatten().chain(in_memory_stream)),
        ))
    }

    /// Flushes the in-memory Bloom filters to disk.
    async fn flush_in_memory_to_disk(&mut self) -> Result<()> {
        let file_id = self.intermediate_file_id_counter;
        self.intermediate_file_id_counter += 1;
        self.flushed_seg_count += self.in_memory.len();

        let file_id = format!("{:08}", file_id);
        let mut writer = self
            .intermediate_provider
            .create(&self.intermediate_prefix, &file_id)
            .await
            .context(IntermediateSnafu)?;

        let fw = FramedWrite::new(&mut writer, IntermediateBloomFilterCodecV1::default());
        // `forward()` will flush and close the writer when the stream ends
        if let Err(e) = stream::iter(self.in_memory.drain(..).map(Ok))
            .forward(fw)
            .await
        {
            writer.close().await.context(IoSnafu)?;
            writer.flush().await.context(IoSnafu)?;
            return Err(e);
        }

        Ok(())
    }
}

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

/// A finalized Bloom filter segment.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct FinalizedBloomFilterSegment {
    /// The underlying Bloom filter bytes.
    pub bloom_filter_bytes: Vec<u8>,

    /// The number of elements in the Bloom filter.
    pub element_count: usize,
}

impl FinalizedBloomFilterSegment {
    fn from(bf: BloomFilter, elem_count: usize) -> Self {
        let bf_slice = bf.as_slice();
        let mut bloom_filter_bytes = Vec::with_capacity(std::mem::size_of_val(bf_slice));
        for &x in bf_slice {
            bloom_filter_bytes.extend_from_slice(&x.to_le_bytes());
        }

        Self {
            bloom_filter_bytes,
            element_count: elem_count,
        }
    }
}

#[cfg(test)]
mod tests {
    use std::collections::HashMap;
    use std::sync::Mutex;

    use futures::AsyncRead;
    use tokio::io::duplex;
    use tokio_util::compat::{TokioAsyncReadCompatExt, TokioAsyncWriteCompatExt};

    use super::*;
    use crate::bloom_filter::creator::tests::u64_vec_from_bytes;
    use crate::external_provider::MockExternalTempFileProvider;

    #[tokio::test]
    async fn test_finalized_bloom_filter_storage() {
        let mut mock_provider = MockExternalTempFileProvider::new();

        let mock_files: Arc<Mutex<HashMap<String, Box<dyn AsyncRead + Unpin + Send>>>> =
            Arc::new(Mutex::new(HashMap::new()));

        mock_provider.expect_create().returning({
            let files = Arc::clone(&mock_files);
            move |file_group, file_id| {
                assert!(file_group.starts_with("intm-bloom-filters-"));
                let mut files = files.lock().unwrap();
                let (writer, reader) = duplex(2 * 1024 * 1024);
                files.insert(file_id.to_string(), Box::new(reader.compat()));
                Ok(Box::new(writer.compat_write()))
            }
        });

        mock_provider.expect_read_all().returning({
            let files = Arc::clone(&mock_files);
            move |file_group| {
                assert!(file_group.starts_with("intm-bloom-filters-"));
                let mut files = files.lock().unwrap();
                Ok(files.drain().collect::<Vec<_>>())
            }
        });

        let global_memory_usage = Arc::new(AtomicUsize::new(0));
        let global_memory_usage_threshold = Some(1024 * 1024); // 1MB
        let provider = Arc::new(mock_provider);
        let mut storage = FinalizedBloomFilterStorage::new(
            provider,
            global_memory_usage.clone(),
            global_memory_usage_threshold,
        );

        let elem_count = 2000;
        let batch = 1000;
        let dup_batch = 200;

        for i in 0..(batch - dup_batch) {
            let elems = (elem_count * i..elem_count * (i + 1)).map(|x| x.to_string().into_bytes());
            storage.add(elems, elem_count).await.unwrap();
        }
        for _ in 0..dup_batch {
            storage.add(Some(vec![]), 1).await.unwrap();
        }

        // Flush happens.
        assert!(storage.intermediate_file_id_counter > 0);

        // Drain the storage.
        let (indices, mut stream) = storage.drain().await.unwrap();
        assert_eq!(indices.len(), batch);

        for (i, idx) in indices.iter().enumerate().take(batch - dup_batch) {
            let segment = stream.next().await.unwrap().unwrap();
            assert_eq!(segment.element_count, elem_count);

            let v = u64_vec_from_bytes(&segment.bloom_filter_bytes);

            // Check the correctness of the Bloom filter.
            let bf = BloomFilter::from_vec(v)
                .seed(&SEED)
                .expected_items(segment.element_count);
            for elem in (elem_count * i..elem_count * (i + 1)).map(|x| x.to_string().into_bytes()) {
                assert!(bf.contains(&elem));
            }
            assert_eq!(indices[i], *idx);
        }

        // Check the correctness of the duplicated segments.
        let dup_seg = stream.next().await.unwrap().unwrap();
        assert_eq!(dup_seg.element_count, 1);
        assert!(stream.next().await.is_none());
        assert!(indices[(batch - dup_batch)..batch]
            .iter()
            .all(|&x| x == batch - dup_batch));
    }

    #[tokio::test]
    async fn test_finalized_bloom_filter_storage_all_dup() {
        let mock_provider = MockExternalTempFileProvider::new();
        let global_memory_usage = Arc::new(AtomicUsize::new(0));
        let global_memory_usage_threshold = Some(1024 * 1024); // 1MB
        let provider = Arc::new(mock_provider);
        let mut storage = FinalizedBloomFilterStorage::new(
            provider,
            global_memory_usage.clone(),
            global_memory_usage_threshold,
        );

        let batch = 1000;
        for _ in 0..batch {
            storage.add(Some(vec![]), 1).await.unwrap();
        }

        // Drain the storage.
        let (indices, mut stream) = storage.drain().await.unwrap();

        let bf = stream.next().await.unwrap().unwrap();
        assert_eq!(bf.element_count, 1);

        assert!(stream.next().await.is_none());

        assert_eq!(indices.len(), batch);
        assert!(indices.iter().all(|&x| x == 0));
    }
}