index/fulltext_index/create/
bloom_filter.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
// 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::HashMap;
use std::sync::atomic::AtomicUsize;
use std::sync::Arc;

use async_trait::async_trait;
use common_error::ext::BoxedError;
use puffin::puffin_manager::{PuffinWriter, PutOptions};
use snafu::{OptionExt, ResultExt};
use tokio_util::compat::{TokioAsyncReadCompatExt, TokioAsyncWriteCompatExt};

use crate::bloom_filter::creator::BloomFilterCreator;
use crate::external_provider::ExternalTempFileProvider;
use crate::fulltext_index::create::FulltextIndexCreator;
use crate::fulltext_index::error::{
    AbortedSnafu, BiErrorsSnafu, BloomFilterFinishSnafu, ExternalSnafu, PuffinAddBlobSnafu, Result,
    SerializeToJsonSnafu,
};
use crate::fulltext_index::tokenizer::{Analyzer, ChineseTokenizer, EnglishTokenizer};
use crate::fulltext_index::Config;

const PIPE_BUFFER_SIZE_FOR_SENDING_BLOB: usize = 8192;

pub const KEY_FULLTEXT_CONFIG: &str = "fulltext_config";

/// `BloomFilterFulltextIndexCreator` is for creating a fulltext index using a bloom filter.
pub struct BloomFilterFulltextIndexCreator {
    inner: Option<BloomFilterCreator>,
    analyzer: Analyzer,
    config: Config,
}

impl BloomFilterFulltextIndexCreator {
    pub fn new(
        config: Config,
        rows_per_segment: usize,
        intermediate_provider: Arc<dyn ExternalTempFileProvider>,
        global_memory_usage: Arc<AtomicUsize>,
        global_memory_usage_threshold: Option<usize>,
    ) -> Self {
        let tokenizer = match config.analyzer {
            crate::fulltext_index::Analyzer::English => Box::new(EnglishTokenizer) as _,
            crate::fulltext_index::Analyzer::Chinese => Box::new(ChineseTokenizer) as _,
        };
        let analyzer = Analyzer::new(tokenizer, config.case_sensitive);

        let inner = BloomFilterCreator::new(
            rows_per_segment,
            intermediate_provider,
            global_memory_usage,
            global_memory_usage_threshold,
        );
        Self {
            inner: Some(inner),
            analyzer,
            config,
        }
    }
}

#[async_trait]
impl FulltextIndexCreator for BloomFilterFulltextIndexCreator {
    async fn push_text(&mut self, text: &str) -> Result<()> {
        let tokens = self.analyzer.analyze_text(text)?;
        self.inner
            .as_mut()
            .context(AbortedSnafu)?
            .push_row_elems(tokens)
            .await
            .map_err(BoxedError::new)
            .context(ExternalSnafu)?;
        Ok(())
    }

    async fn finish(
        &mut self,
        puffin_writer: &mut (impl PuffinWriter + Send),
        blob_key: &str,
        put_options: PutOptions,
    ) -> Result<u64> {
        let creator = self.inner.as_mut().context(AbortedSnafu)?;

        let (tx, rx) = tokio::io::duplex(PIPE_BUFFER_SIZE_FOR_SENDING_BLOB);

        let property_key = KEY_FULLTEXT_CONFIG.to_string();
        let property_value = serde_json::to_string(&self.config).context(SerializeToJsonSnafu)?;

        let (index_finish, puffin_add_blob) = futures::join!(
            creator.finish(tx.compat_write()),
            puffin_writer.put_blob(
                blob_key,
                rx.compat(),
                put_options,
                HashMap::from([(property_key, property_value)]),
            )
        );

        match (
            puffin_add_blob.context(PuffinAddBlobSnafu),
            index_finish.context(BloomFilterFinishSnafu),
        ) {
            (Err(e1), Err(e2)) => BiErrorsSnafu {
                first: Box::new(e1),
                second: Box::new(e2),
            }
            .fail()?,

            (Ok(_), e @ Err(_)) => e?,
            (e @ Err(_), Ok(_)) => e.map(|_| ())?,
            (Ok(written_bytes), Ok(_)) => {
                return Ok(written_bytes);
            }
        }
        Ok(0)
    }

    async fn abort(&mut self) -> Result<()> {
        self.inner.take().context(AbortedSnafu)?;
        Ok(())
    }

    fn memory_usage(&self) -> usize {
        self.inner
            .as_ref()
            .map(|i| i.memory_usage())
            .unwrap_or_default()
    }
}