Skip to main content

mito2/memtable/bulk/
context.rs

1// Copyright 2023 Greptime Team
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15//! Context for iterating bulk memtable.
16
17use std::collections::VecDeque;
18use std::sync::Arc;
19
20use common_recordbatch::filter::SimpleFilterEvaluator;
21use mito_codec::row_converter::build_primary_key_codec;
22use parquet::file::metadata::ParquetMetaData;
23use store_api::metadata::RegionMetadataRef;
24use store_api::storage::ColumnId;
25use table::predicate::Predicate;
26
27use crate::error::Result;
28use crate::sst::parquet::file_range::{PreFilterMode, RangeBase};
29use crate::sst::parquet::format::ReadFormat;
30use crate::sst::parquet::prefilter::CachedPrimaryKeyFilter;
31use crate::sst::parquet::reader::SimpleFilterContext;
32use crate::sst::parquet::stats::RowGroupPruningStats;
33
34pub(crate) type BulkIterContextRef = Arc<BulkIterContext>;
35
36pub struct BulkIterContext {
37    pub(crate) base: RangeBase,
38    pub(crate) predicate: Option<Predicate>,
39    /// Pre-extracted primary key filters for PK prefiltering.
40    /// `None` if PK prefiltering is not applicable.
41    pk_filters: Option<Arc<Vec<SimpleFilterEvaluator>>>,
42}
43
44impl BulkIterContext {
45    pub fn new(
46        region_metadata: RegionMetadataRef,
47        projection: Option<&[ColumnId]>,
48        predicate: Option<Predicate>,
49        skip_auto_convert: bool,
50    ) -> Result<Self> {
51        Self::new_with_pre_filter_mode(
52            region_metadata,
53            projection,
54            predicate,
55            skip_auto_convert,
56            PreFilterMode::All,
57        )
58    }
59
60    pub fn new_with_pre_filter_mode(
61        region_metadata: RegionMetadataRef,
62        projection: Option<&[ColumnId]>,
63        predicate: Option<Predicate>,
64        skip_auto_convert: bool,
65        pre_filter_mode: PreFilterMode,
66    ) -> Result<Self> {
67        let codec = build_primary_key_codec(&region_metadata);
68
69        let simple_filters: Vec<SimpleFilterContext> = predicate
70            .as_ref()
71            .iter()
72            .flat_map(|predicate| {
73                predicate
74                    .exprs()
75                    .iter()
76                    .filter_map(|expr| SimpleFilterContext::new_opt(&region_metadata, None, expr))
77            })
78            .collect();
79
80        let read_format = ReadFormat::new(
81            region_metadata.clone(),
82            projection,
83            true,
84            None,
85            "memtable",
86            skip_auto_convert,
87        )?;
88
89        let dyn_filters = predicate
90            .as_ref()
91            .map(|pred| pred.dyn_filters().as_ref().clone())
92            .unwrap_or_default();
93
94        // Pre-extract PK filters if applicable.
95        let pk_filters = Self::extract_pk_filters(&read_format, &simple_filters);
96
97        Ok(Self {
98            base: RangeBase {
99                filters: simple_filters,
100                dyn_filters,
101                read_format,
102                prune_schema: region_metadata.schema.clone(),
103                expected_metadata: Some(region_metadata),
104                codec,
105                // we don't need to compat batch since all batch in memtable have the same schema.
106                compat_batch: None,
107                compaction_projection_mapper: None,
108                pre_filter_mode,
109                partition_filter: None,
110            },
111            predicate,
112            pk_filters,
113        })
114    }
115
116    /// Prunes row groups by stats.
117    pub(crate) fn row_groups_to_read(
118        &self,
119        file_meta: &Arc<ParquetMetaData>,
120        skip_fields: bool,
121    ) -> VecDeque<usize> {
122        let region_meta = self.base.read_format.metadata();
123        let row_groups = file_meta.row_groups();
124        // expected_metadata is set to None since we always expect region metadata of memtable is up-to-date.
125        let stats =
126            RowGroupPruningStats::new(row_groups, &self.base.read_format, None, skip_fields);
127        if let Some(predicate) = self.predicate.as_ref() {
128            predicate
129                .prune_with_stats(&stats, region_meta.schema.arrow_schema())
130                .iter()
131                .zip(0..file_meta.num_row_groups())
132                .filter_map(|(selected, row_group)| {
133                    if !*selected {
134                        return None;
135                    }
136                    Some(row_group)
137                })
138                .collect::<VecDeque<_>>()
139        } else {
140            (0..file_meta.num_row_groups()).collect()
141        }
142    }
143
144    /// Extracts PK filters if flat format with dictionary-encoded PKs is used.
145    fn extract_pk_filters(
146        read_format: &ReadFormat,
147        filters: &[SimpleFilterContext],
148    ) -> Option<Arc<Vec<SimpleFilterEvaluator>>> {
149        let flat_format = read_format.as_flat()?;
150        if flat_format.batch_has_raw_pk_columns() {
151            return None;
152        }
153        let metadata = read_format.metadata();
154        if metadata.primary_key.is_empty() {
155            return None;
156        }
157
158        let pk_filters: Vec<_> = filters
159            .iter()
160            .filter_map(|f| f.primary_key_prefilter())
161            .collect();
162        if pk_filters.is_empty() {
163            return None;
164        }
165
166        Some(Arc::new(pk_filters))
167    }
168
169    /// Builds a fresh PK filter for a new iterator. Returns `None` if PK
170    /// prefiltering is not applicable.
171    pub(crate) fn build_pk_filter(&self) -> Option<CachedPrimaryKeyFilter> {
172        let pk_filters = self.pk_filters.as_ref()?;
173        let metadata = self.base.read_format.metadata();
174        // Parquet PK prefilter always supports the partition column.
175        let inner = self
176            .base
177            .codec
178            .primary_key_filter(metadata, Arc::clone(pk_filters), false);
179        Some(CachedPrimaryKeyFilter::new(inner))
180    }
181
182    pub(crate) fn read_format(&self) -> &ReadFormat {
183        &self.base.read_format
184    }
185
186    /// Returns the pre-filter mode.
187    pub(crate) fn pre_filter_mode(&self) -> PreFilterMode {
188        self.base.pre_filter_mode
189    }
190
191    /// Returns the region id.
192    pub(crate) fn region_id(&self) -> store_api::storage::RegionId {
193        self.base.read_format.metadata().region_id
194    }
195}