From ecdc91fd4652d913d5d1fef2cd3c1700741f6548 Mon Sep 17 00:00:00 2001 From: zhyass Date: Mon, 30 Jun 2025 22:05:25 +0800 Subject: [PATCH 01/11] improve stream write --- src/query/settings/src/settings_default.rs | 7 - .../settings/src/settings_getter_setter.rs | 4 - .../storages/common/index/src/bloom_index.rs | 92 ++++-- src/query/storages/fuse/src/fuse_table.rs | 7 + .../fuse/src/io/write/block_writer.rs | 8 + .../fuse/src/io/write/bloom_index_writer.rs | 1 + .../src/io/write/inverted_index_writer.rs | 1 + .../fuse/src/io/write/stream/block_builder.rs | 26 +- .../io/write/stream/column_ndv_estimator.rs | 143 +++++++++ .../src/io/write/stream/column_statistics.rs | 253 --------------- .../write/stream/column_statistics_builder.rs | 297 ++++++++++++++++++ .../write/stream/column_statistics_state.rs | 116 +++++++ .../storages/fuse/src/io/write/stream/mod.rs | 9 +- .../storages/fuse/src/operations/append.rs | 26 +- .../src/operations/common/processors/mod.rs | 1 + .../processors/transform_block_writer.rs | 164 ++++++---- .../fuse/src/statistics/column_statistic.rs | 36 ++- .../09_0004_remote_insert_into_select.test | 3 - .../09_0008_fuse_optimize_table.test | 3 - .../suites/base/issues/issue_18275.test | 3 - 20 files changed, 804 insertions(+), 396 deletions(-) create mode 100644 src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs delete mode 100644 src/query/storages/fuse/src/io/write/stream/column_statistics.rs create mode 100644 src/query/storages/fuse/src/io/write/stream/column_statistics_builder.rs create mode 100644 src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index aec6cfc7288a1..2f60028b130bf 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -1361,13 +1361,6 @@ impl DefaultSettings { scope: SettingScope::Both, range: None, }), - ("enable_block_stream_write", DefaultSettingValue { - value: UserSettingValue::UInt64(0), - desc: "Enables block stream write", - mode: SettingMode::Both, - scope: SettingScope::Both, - range: Some(SettingRange::Numeric(0..=1)), - }), ("trace_sample_rate", DefaultSettingValue { value: UserSettingValue::UInt64(1), desc: "Setting the trace sample rate. The value should be between '0' and '100'", diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index 98023b4356390..7c3e35d83ed8c 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -998,10 +998,6 @@ impl Settings { self.set_setting("optimizer_skip_list".to_string(), v) } - pub fn get_enable_block_stream_write(&self) -> Result { - Ok(self.try_get_u64("enable_block_stream_write")? == 1) - } - pub fn get_statement_queue_ttl_in_seconds(&self) -> Result { self.try_get_u64("statement_queue_ttl_in_seconds") } diff --git a/src/query/storages/common/index/src/bloom_index.rs b/src/query/storages/common/index/src/bloom_index.rs index bf5ec35035e64..59d06f6298a49 100644 --- a/src/query/storages/common/index/src/bloom_index.rs +++ b/src/query/storages/common/index/src/bloom_index.rs @@ -14,6 +14,7 @@ use std::collections::BTreeMap; use std::collections::HashMap; +use std::hash::DefaultHasher; use std::hash::Hasher; use std::ops::ControlFlow; use std::ops::Deref; @@ -35,12 +36,18 @@ use databend_common_expression::types::BinaryType; use databend_common_expression::types::Bitmap; use databend_common_expression::types::Buffer; use databend_common_expression::types::DataType; +use databend_common_expression::types::DateType; use databend_common_expression::types::MapType; use databend_common_expression::types::NullableType; use databend_common_expression::types::Number; use databend_common_expression::types::NumberDataType; +use databend_common_expression::types::NumberType; +use databend_common_expression::types::StringType; +use databend_common_expression::types::TimestampType; use databend_common_expression::types::UInt64Type; +use databend_common_expression::types::ValueType; use databend_common_expression::visit_expr; +use databend_common_expression::with_number_mapped_type; use databend_common_expression::BlockEntry; use databend_common_expression::Column; use databend_common_expression::ColumnBuilder; @@ -349,6 +356,71 @@ impl BloomIndex { Ok(column) } + pub fn calculate_digest_by_type(data_type: &DataType, column: &Column) -> Result> { + let inner_type = data_type.remove_nullable(); + with_number_mapped_type!(|NUM_TYPE| match inner_type { + DataType::Number(NumberDataType::NUM_TYPE) => { + Self::calculate_nullable_column_digests::>(column) + } + DataType::String => { + Self::calculate_nullable_column_digests::(column) + } + DataType::Date => { + Self::calculate_nullable_column_digests::(column) + } + DataType::Timestamp => { + Self::calculate_nullable_column_digests::(column) + } + _ => Err(ErrorCode::Internal(format!( + "Unsupported data type: {:?}", + data_type + ))), + }) + } + + #[inline(always)] + fn hash_one(v: &T) -> u64 { + let mut hasher = DefaultHasher::default(); + DFHash::hash(v, &mut hasher); + hasher.finish() + } + + fn calculate_nullable_column_digests(column: &Column) -> Result> + where for<'a> T::ScalarRef<'a>: DFHash { + let (column, validity) = if let Column::Nullable(box inner) = column { + let validity = if inner.validity.null_count() == 0 { + None + } else { + Some(&inner.validity) + }; + (&inner.column, validity) + } else { + (column, None) + }; + + let capacity = validity.map_or(column.len(), |v| v.true_count() + 1); + let mut result = Vec::with_capacity(capacity); + if validity.is_some() { + result.push(0); + } + let column = T::try_downcast_column(column).unwrap(); + if let Some(validity) = validity { + let column_iter = T::iter_column(&column); + let value_iter = column_iter + .zip(validity.iter()) + .filter(|(_, v)| *v) + .map(|(v, _)| v); + for value in value_iter { + result.push(Self::hash_one(&value)); + } + } else { + for value in T::iter_column(&column) { + result.push(Self::hash_one(&value)); + } + } + Ok(result) + } + /// calculate digest for column that may have null values /// /// returns (column, validity) where column is the digest of the column @@ -734,24 +806,8 @@ impl BloomIndexBuilder { } }; - let (column, validity) = - BloomIndex::calculate_nullable_column_digest(&self.func_ctx, &column, &data_type)?; - // create filter per column - if validity.as_ref().map(|v| v.null_count()).unwrap_or(0) > 0 { - let validity = validity.unwrap(); - let it = column.deref().iter().zip(validity.iter()).map( - |(v, b)| { - if !b { - &0 - } else { - v - } - }, - ); - index_column.builder.add_digests(it); - } else { - index_column.builder.add_digests(column.deref()); - } + let column = BloomIndex::calculate_digest_by_type(&data_type, &column)?; + index_column.builder.add_digests(column.deref()); } for index_column in self.ngram_columns.iter_mut() { let field_type = &block.data_type(index_column.index); diff --git a/src/query/storages/fuse/src/fuse_table.rs b/src/query/storages/fuse/src/fuse_table.rs index c0af05a4fa6ad..ad9a99afc3666 100644 --- a/src/query/storages/fuse/src/fuse_table.rs +++ b/src/query/storages/fuse/src/fuse_table.rs @@ -745,6 +745,13 @@ impl FuseTable { ) }) } + + pub fn enable_stream_block_write(&self) -> bool { + matches!(self.storage_format, FuseStorageFormat::Parquet) + && self + .cluster_type() + .is_none_or(|v| matches!(v, ClusterType::Hilbert)) + } } #[async_trait::async_trait] diff --git a/src/query/storages/fuse/src/io/write/block_writer.rs b/src/query/storages/fuse/src/io/write/block_writer.rs index 1f1b6a5579e25..fdfbd02320568 100644 --- a/src/query/storages/fuse/src/io/write/block_writer.rs +++ b/src/query/storages/fuse/src/io/write/block_writer.rs @@ -20,6 +20,8 @@ use std::time::Instant; use chrono::Utc; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; +use databend_common_expression::local_block_meta_serde; +use databend_common_expression::BlockMetaInfo; use databend_common_expression::Column; use databend_common_expression::ColumnId; use databend_common_expression::DataBlock; @@ -129,6 +131,7 @@ pub async fn write_data(data: Vec, data_accessor: &Operator, location: &str) Ok(()) } +#[derive(Debug)] pub struct BlockSerialization { pub block_raw_data: Vec, pub block_meta: BlockMeta, @@ -138,6 +141,11 @@ pub struct BlockSerialization { pub vector_index_state: Option, } +local_block_meta_serde!(BlockSerialization); + +#[typetag::serde(name = "block_serialization_meta")] +impl BlockMetaInfo for BlockSerialization {} + #[derive(Clone)] pub struct BlockBuilder { pub ctx: Arc, diff --git a/src/query/storages/fuse/src/io/write/bloom_index_writer.rs b/src/query/storages/fuse/src/io/write/bloom_index_writer.rs index ec49070a6f08f..738c33ac2f2c3 100644 --- a/src/query/storages/fuse/src/io/write/bloom_index_writer.rs +++ b/src/query/storages/fuse/src/io/write/bloom_index_writer.rs @@ -40,6 +40,7 @@ use opendal::Operator; use crate::io::BlockReader; use crate::FuseStorageFormat; +#[derive(Debug)] pub struct BloomIndexState { pub(crate) data: Vec, pub(crate) size: u64, diff --git a/src/query/storages/fuse/src/io/write/inverted_index_writer.rs b/src/query/storages/fuse/src/io/write/inverted_index_writer.rs index 74377a86108cb..8cf0b5f2355f0 100644 --- a/src/query/storages/fuse/src/io/write/inverted_index_writer.rs +++ b/src/query/storages/fuse/src/io/write/inverted_index_writer.rs @@ -121,6 +121,7 @@ pub fn create_inverted_index_builders(table_meta: &TableMeta) -> Vec, pub(crate) size: u64, diff --git a/src/query/storages/fuse/src/io/write/stream/block_builder.rs b/src/query/storages/fuse/src/io/write/stream/block_builder.rs index 73ee3c1ded29e..cca3c1cc0da6c 100644 --- a/src/query/storages/fuse/src/io/write/stream/block_builder.rs +++ b/src/query/storages/fuse/src/io/write/stream/block_builder.rs @@ -36,6 +36,7 @@ use databend_common_expression::ORIGIN_BLOCK_ROW_NUM_COLUMN_ID; use databend_common_io::constants::DEFAULT_BLOCK_BUFFER_SIZE; use databend_common_meta_app::schema::TableIndex; use databend_common_native::write::NativeWriter; +use databend_common_sql::executor::physical_plans::MutationKind; use databend_storages_common_index::BloomIndex; use databend_storages_common_index::BloomIndexBuilder; use databend_storages_common_index::Index; @@ -53,7 +54,7 @@ use parquet::file::properties::WriterProperties; use crate::io::create_inverted_index_builders; use crate::io::write::stream::cluster_statistics::ClusterStatisticsBuilder; use crate::io::write::stream::cluster_statistics::ClusterStatisticsState; -use crate::io::write::stream::column_statistics::ColumnStatisticsState; +use crate::io::write::stream::ColumnStatisticsState; use crate::io::write::InvertedIndexState; use crate::io::BlockSerialization; use crate::io::BloomIndexState; @@ -263,7 +264,7 @@ impl StreamBlockBuilder { pub fn need_flush(&self) -> bool { let file_size = self.block_writer.compressed_size(); self.row_count >= self.properties.block_thresholds.min_rows_per_block - || self.block_size >= self.properties.block_thresholds.max_bytes_per_block + || self.block_size >= self.properties.block_thresholds.min_bytes_per_block * 2 || (file_size >= self.properties.block_thresholds.min_compressed_per_block && self.block_size >= self.properties.block_thresholds.min_bytes_per_block) } @@ -410,8 +411,8 @@ pub struct StreamBlockProperties { source_schema: TableSchemaRef, cluster_stats_builder: Arc, - stats_columns: Vec, - distinct_columns: Vec, + stats_columns: Vec<(ColumnId, DataType)>, + distinct_columns: Vec<(ColumnId, DataType)>, bloom_columns_map: BTreeMap, ngram_args: Vec, inverted_index_builders: Vec, @@ -424,16 +425,23 @@ impl StreamBlockProperties { pub fn try_create( ctx: Arc, table: &FuseTable, + kind: MutationKind, table_meta_timestamps: TableMetaTimestamps, ) -> Result> { // remove virtual computed fields. - let fields = table + let mut fields = table .schema() .fields() .iter() .filter(|f| !matches!(f.computed_expr(), Some(ComputedExpr::Virtual(_)))) .cloned() .collect::>(); + if !matches!(kind, MutationKind::Insert | MutationKind::Replace) { + // add stream fields. + for stream_column in table.stream_columns().iter() { + fields.push(stream_column.table_field()); + } + } let source_schema = Arc::new(TableSchema { fields, @@ -464,12 +472,12 @@ impl StreamBlockProperties { let leaf_fields = source_schema.leaf_fields(); for field in leaf_fields.iter() { let column_id = field.column_id(); - if RangeIndex::supported_type(&DataType::from(field.data_type())) - && column_id != ORIGIN_BLOCK_ROW_NUM_COLUMN_ID + let data_type = DataType::from(field.data_type()); + if RangeIndex::supported_type(&data_type) && column_id != ORIGIN_BLOCK_ROW_NUM_COLUMN_ID { - stats_columns.push(column_id); + stats_columns.push((column_id, data_type.clone())); if !bloom_column_ids.contains(&column_id) { - distinct_columns.push(column_id); + distinct_columns.push((column_id, data_type)); } } } diff --git a/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs b/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs new file mode 100644 index 0000000000000..cb138c53a1d8c --- /dev/null +++ b/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs @@ -0,0 +1,143 @@ +// Copyright 2021 Datafuse Labs +// +// 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::hash::Hash; +use std::marker::PhantomData; + +use databend_common_expression::types::boolean::TrueIdxIter; +use databend_common_expression::types::DataType; +use databend_common_expression::types::DateType; +use databend_common_expression::types::Decimal128Type; +use databend_common_expression::types::Decimal256Type; +use databend_common_expression::types::Decimal64Type; +use databend_common_expression::types::NumberDataType; +use databend_common_expression::types::NumberType; +use databend_common_expression::types::StringType; +use databend_common_expression::types::TimestampType; +use databend_common_expression::types::ValueType; +use databend_common_expression::with_number_mapped_type; +use databend_common_expression::Column; +use databend_common_expression::ScalarRef; +use databend_common_expression::SELECTIVITY_THRESHOLD; +use databend_storages_common_table_meta::meta::ColumnDistinctHLL; + +pub trait ColumnNDVEstimator: Send + Sync { + fn update_column(&mut self, column: &Column); + fn update_scalar(&mut self, scalar: &ScalarRef); + fn finalize(&self) -> u64; +} + +pub fn create_column_ndv_estimator(data_type: &DataType) -> Box { + let inner_type = data_type.remove_nullable(); + with_number_mapped_type!(|NUM_TYPE| match inner_type { + DataType::Number(NumberDataType::NUM_TYPE) => { + ColumnNDVEstimatorImpl::>::create() + } + DataType::String => { + ColumnNDVEstimatorImpl::::create() + } + DataType::Date => { + ColumnNDVEstimatorImpl::::create() + } + DataType::Timestamp => { + ColumnNDVEstimatorImpl::::create() + } + DataType::Decimal(size) => { + if size.can_carried_by_64() { + ColumnNDVEstimatorImpl::::create() + } else if size.can_carried_by_128() { + ColumnNDVEstimatorImpl::::create() + } else { + ColumnNDVEstimatorImpl::::create() + } + } + _ => unreachable!("Unsupported data type: {:?}", data_type), + }) +} + +pub struct ColumnNDVEstimatorImpl +where + T: ValueType + Send + Sync, + for<'a> T::ScalarRef<'a>: Hash, +{ + hll: ColumnDistinctHLL, + _phantom: PhantomData, +} + +impl ColumnNDVEstimatorImpl +where + T: ValueType + Send + Sync, + for<'a> T::ScalarRef<'a>: Hash, +{ + pub fn create() -> Box { + Box::new(Self { + hll: ColumnDistinctHLL::new(), + _phantom: Default::default(), + }) + } +} + +impl ColumnNDVEstimator for ColumnNDVEstimatorImpl +where + T: ValueType + Send + Sync, + for<'a> T::ScalarRef<'a>: Hash, +{ + fn update_column(&mut self, column: &Column) { + let (column, validity) = match column { + Column::Nullable(box inner) => { + let validity = if inner.validity.null_count() == 0 { + None + } else { + Some(&inner.validity) + }; + (&inner.column, validity) + } + Column::Null { .. } => return, + column => (column, None), + }; + + let column = T::try_downcast_column(column).unwrap(); + if let Some(v) = validity { + if v.true_count() as f64 / v.len() as f64 >= SELECTIVITY_THRESHOLD { + for (data, valid) in T::iter_column(&column).zip(v.iter()) { + if valid { + self.hll.add_object(&data); + } + } + } else { + TrueIdxIter::new(v.len(), Some(v)).for_each(|idx| { + let val = unsafe { T::index_column_unchecked(&column, idx) }; + self.hll.add_object(&val); + }) + } + } else { + for value in T::iter_column(&column) { + self.hll.add_object(&value); + } + } + } + + fn update_scalar(&mut self, scalar: &ScalarRef) { + if matches!(scalar, ScalarRef::Null) { + return; + } + + let val = T::try_downcast_scalar(scalar).unwrap(); + self.hll.add_object(&val); + } + + fn finalize(&self) -> u64 { + self.hll.count() as u64 + } +} diff --git a/src/query/storages/fuse/src/io/write/stream/column_statistics.rs b/src/query/storages/fuse/src/io/write/stream/column_statistics.rs deleted file mode 100644 index 402504e3a207e..0000000000000 --- a/src/query/storages/fuse/src/io/write/stream/column_statistics.rs +++ /dev/null @@ -1,253 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// 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 databend_common_exception::Result; -use databend_common_expression::types::AccessType; -use databend_common_expression::types::DataType; -use databend_common_expression::types::DateType; -use databend_common_expression::types::DecimalColumn; -use databend_common_expression::types::DecimalScalar; -use databend_common_expression::types::NumberDataType; -use databend_common_expression::types::NumberType; -use databend_common_expression::types::StringType; -use databend_common_expression::types::TimestampType; -use databend_common_expression::with_number_mapped_type; -use databend_common_expression::Column; -use databend_common_expression::ColumnId; -use databend_common_expression::DataBlock; -use databend_common_expression::Scalar; -use databend_common_expression::ScalarRef; -use databend_common_expression::TableSchemaRef; -use databend_common_expression::Value; -use databend_common_functions::aggregates::eval_aggr; -use databend_storages_common_table_meta::meta::ColumnDistinctHLL; -use databend_storages_common_table_meta::meta::ColumnStatistics; -use databend_storages_common_table_meta::meta::StatisticsOfColumns; - -use crate::statistics::reducers::reduce_column_statistics; -use crate::statistics::traverse_values_dfs; -use crate::statistics::Trim; - -pub struct ColumnStatisticsState { - col_stats: HashMap>, - distinct_columns: HashMap, -} - -impl ColumnStatisticsState { - pub fn new(stats_columns: &[ColumnId], distinct_columns: &[ColumnId]) -> Self { - let col_stats = stats_columns - .iter() - .map(|&col_id| (col_id, Vec::new())) - .collect(); - - let distinct_columns = distinct_columns - .iter() - .map(|&col_id| (col_id, ColumnDistinctHLL::default())) - .collect(); - - Self { - col_stats, - distinct_columns, - } - } - - pub fn add_block(&mut self, schema: &TableSchemaRef, data_block: &DataBlock) -> Result<()> { - let rows = data_block.num_rows(); - let leaves = traverse_values_dfs(data_block.columns(), schema.fields())?; - for (column_id, col, data_type) in leaves { - match col { - Value::Scalar(s) => { - let unset_bits = if s == Scalar::Null { rows } else { 0 }; - // when we read it back from parquet, it is a Column instead of Scalar - let in_memory_size = s.as_ref().estimated_scalar_repeat_size(rows, &data_type); - let col_stats = ColumnStatistics::new( - s.clone(), - s.clone(), - unset_bits as u64, - in_memory_size as u64, - None, - ); - if let Some(hll) = self.distinct_columns.get_mut(&column_id) { - scalar_update_hll_cardinality(&s.as_ref(), &data_type, hll); - } - self.col_stats.get_mut(&column_id).unwrap().push(col_stats); - } - Value::Column(col) => { - // later, during the evaluation of expressions, name of field does not matter - let mut min = Scalar::Null; - let mut max = Scalar::Null; - - let (mins, _) = eval_aggr("min", vec![], &[col.clone().into()], rows, vec![])?; - if mins.len() > 0 { - min = if let Some(v) = mins.index(0) { - // safe upwrap. - v.to_owned().trim_min().unwrap() - } else { - self.col_stats.remove(&column_id); - continue; - } - } - - let (maxs, _) = eval_aggr("max", vec![], &[col.clone().into()], rows, vec![])?; - if maxs.len() > 0 { - max = if let Some(v) = maxs.index(0) { - if let Some(v) = v.to_owned().trim_max() { - v - } else { - self.col_stats.remove(&column_id); - continue; - } - } else { - self.col_stats.remove(&column_id); - continue; - } - } - - let (is_all_null, bitmap) = col.validity(); - let unset_bits = match (is_all_null, bitmap) { - (true, _) => rows, - (false, Some(bitmap)) => bitmap.null_count(), - (false, None) => 0, - }; - let in_memory_size = col.memory_size() as u64; - let col_stats = - ColumnStatistics::new(min, max, unset_bits as u64, in_memory_size, None); - self.col_stats.get_mut(&column_id).unwrap().push(col_stats); - - // use distinct count calculated by the xor hash function to avoid repetitive operation. - if let Some(hll) = self.distinct_columns.get_mut(&column_id) { - column_update_hll_cardinality(&col, &data_type, hll); - } - } - } - } - Ok(()) - } - - pub fn finalize( - self, - column_distinct_count: HashMap, - ) -> Result { - let mut statistics = StatisticsOfColumns::with_capacity(self.col_stats.len()); - for (id, stats) in &self.col_stats { - let mut col_stats = reduce_column_statistics(stats); - if let Some(count) = column_distinct_count.get(id) { - col_stats.distinct_of_values = Some(*count as u64); - } else if let Some(hll) = self.distinct_columns.get(id) { - col_stats.distinct_of_values = Some(hll.count() as u64); - } - statistics.insert(*id, col_stats); - } - Ok(statistics) - } -} - -fn column_update_hll_cardinality(col: &Column, ty: &DataType, hll: &mut ColumnDistinctHLL) { - if let DataType::Nullable(inner) = ty { - let col = col.as_nullable().unwrap(); - for (i, v) in col.validity.iter().enumerate() { - if v { - let scalar = col.column.index(i).unwrap(); - scalar_update_hll_cardinality(&scalar, inner, hll); - } - } - return; - } - - with_number_mapped_type!(|NUM_TYPE| match ty { - DataType::Number(NumberDataType::NUM_TYPE) => { - let col = NumberType::::try_downcast_column(col).unwrap(); - for v in col.iter() { - hll.add_object(v); - } - } - DataType::String => { - let col = StringType::try_downcast_column(col).unwrap(); - for v in col.iter() { - hll.add_object(&v); - } - } - DataType::Date => { - let col = DateType::try_downcast_column(col).unwrap(); - for v in col.iter() { - hll.add_object(v); - } - } - DataType::Timestamp => { - let col = TimestampType::try_downcast_column(col).unwrap(); - for v in col.iter() { - hll.add_object(v); - } - } - DataType::Decimal(_) => { - match col { - Column::Decimal(DecimalColumn::Decimal64(col, _)) => { - for v in col.iter() { - hll.add_object(v); - } - } - Column::Decimal(DecimalColumn::Decimal128(col, _)) => { - for v in col.iter() { - hll.add_object(v); - } - } - Column::Decimal(DecimalColumn::Decimal256(col, _)) => { - for v in col.iter() { - hll.add_object(v); - } - } - _ => unreachable!(), - }; - } - _ => unreachable!("Unsupported data type: {:?}", ty), - }); -} - -fn scalar_update_hll_cardinality(scalar: &ScalarRef, ty: &DataType, hll: &mut ColumnDistinctHLL) { - if matches!(scalar, ScalarRef::Null) { - return; - } - - let ty = ty.remove_nullable(); - - with_number_mapped_type!(|NUM_TYPE| match ty { - DataType::Number(NumberDataType::NUM_TYPE) => { - let val = NumberType::::try_downcast_scalar(scalar).unwrap(); - hll.add_object(&val); - } - DataType::String => { - let val = StringType::try_downcast_scalar(scalar).unwrap(); - hll.add_object(&val); - } - DataType::Date => { - let val = DateType::try_downcast_scalar(scalar).unwrap(); - hll.add_object(&val); - } - DataType::Timestamp => { - let val = TimestampType::try_downcast_scalar(scalar).unwrap(); - hll.add_object(&val); - } - DataType::Decimal(_) => { - match scalar { - ScalarRef::Decimal(DecimalScalar::Decimal64(v, _)) => hll.add_object(&v), - ScalarRef::Decimal(DecimalScalar::Decimal128(v, _)) => hll.add_object(&v), - ScalarRef::Decimal(DecimalScalar::Decimal256(v, _)) => hll.add_object(&v), - _ => unreachable!(), - } - } - _ => unreachable!("Unsupported data type: {:?}", ty), - }); -} diff --git a/src/query/storages/fuse/src/io/write/stream/column_statistics_builder.rs b/src/query/storages/fuse/src/io/write/stream/column_statistics_builder.rs new file mode 100644 index 0000000000000..3178e5da00ef3 --- /dev/null +++ b/src/query/storages/fuse/src/io/write/stream/column_statistics_builder.rs @@ -0,0 +1,297 @@ +// Copyright 2021 Datafuse Labs +// +// 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::cmp::Ordering; +use std::marker::PhantomData; + +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::types::boolean::TrueIdxIter; +use databend_common_expression::types::DataType; +use databend_common_expression::types::DateType; +use databend_common_expression::types::Decimal; +use databend_common_expression::types::Decimal128Type; +use databend_common_expression::types::Decimal256Type; +use databend_common_expression::types::Decimal64Type; +use databend_common_expression::types::NumberDataType; +use databend_common_expression::types::NumberType; +use databend_common_expression::types::StringType; +use databend_common_expression::types::TimestampType; +use databend_common_expression::types::ValueType; +use databend_common_expression::with_number_mapped_type; +use databend_common_expression::Column; +use databend_common_expression::Scalar; +use databend_common_expression::ScalarRef; +use databend_common_expression::SELECTIVITY_THRESHOLD; +use databend_storages_common_table_meta::meta::ColumnStatistics; + +use crate::statistics::Trim; + +pub trait ColumnStatisticsBuilder: Send + Sync { + fn update_column(&mut self, column: &Column); + + fn update_scalar(&mut self, scalar: &ScalarRef, num_rows: usize, data_type: &DataType); + + fn finalize(self: Box) -> Result; +} + +pub fn create_column_stats_builder(data_type: &DataType) -> Box { + let inner_type = data_type.remove_nullable(); + with_number_mapped_type!(|NUM_TYPE| match inner_type { + DataType::Number(NumberDataType::NUM_TYPE) => { + GenericColumnStatisticsBuilder::, CommonAdapter>::create( + inner_type, + ) + } + DataType::String => { + GenericColumnStatisticsBuilder::::create(inner_type) + } + DataType::Date => { + GenericColumnStatisticsBuilder::::create(inner_type) + } + DataType::Timestamp => { + GenericColumnStatisticsBuilder::::create(inner_type) + } + DataType::Decimal(size) => { + if size.can_carried_by_64() { + GenericColumnStatisticsBuilder::::create(inner_type) + } else if size.can_carried_by_128() { + GenericColumnStatisticsBuilder::::create(inner_type) + } else { + GenericColumnStatisticsBuilder::::create(inner_type) + } + } + _ => unreachable!("Unsupported data type: {:?}", data_type), + }) +} + +pub trait ColumnStatisticsAdapter: Send + Sync { + type Value: Clone + Send + Sync; + + fn scalar_to_value(val: T::ScalarRef<'_>) -> Self::Value; + + fn value_to_scalar(val: Self::Value) -> T::Scalar; + + fn update_value(value: &mut Self::Value, scalar: T::ScalarRef<'_>, ordering: Ordering); +} + +struct CommonAdapter; + +impl ColumnStatisticsAdapter for CommonAdapter +where + T: ValueType, + T::Scalar: Send + Sync, + for<'a, 'b> T::ScalarRef<'a>: PartialOrd>, +{ + type Value = T::Scalar; + + fn scalar_to_value(val: T::ScalarRef<'_>) -> Self::Value { + T::to_owned_scalar(val) + } + + fn value_to_scalar(val: Self::Value) -> T::Scalar { + val + } + + fn update_value(value: &mut Self::Value, scalar: T::ScalarRef<'_>, ordering: Ordering) { + if scalar.partial_cmp(&T::to_scalar_ref(value)) == Some(ordering) { + *value = T::to_owned_scalar(scalar); + } + } +} + +struct DecimalAdapter; + +impl ColumnStatisticsAdapter for DecimalAdapter +where + T: ValueType, + T::Scalar: Decimal + Send + Sync, + for<'a, 'b> T::ScalarRef<'a>: PartialOrd>, +{ + type Value = ::U64Array; + + fn scalar_to_value(val: T::ScalarRef<'_>) -> Self::Value { + T::Scalar::to_u64_array(T::to_owned_scalar(val)) + } + + fn value_to_scalar(val: Self::Value) -> T::Scalar { + T::Scalar::from_u64_array(val) + } + + fn update_value(value: &mut Self::Value, scalar: T::ScalarRef<'_>, ordering: Ordering) { + let val = T::Scalar::from_u64_array(*value); + if scalar.partial_cmp(&T::to_scalar_ref(&val)) == Some(ordering) { + *value = T::Scalar::to_u64_array(T::to_owned_scalar(scalar)); + } + } +} + +struct GenericColumnStatisticsBuilder +where + T: ValueType, + A: ColumnStatisticsAdapter, +{ + min: Option, + max: Option, + null_count: usize, + in_memory_size: usize, + data_type: DataType, + + _phantom: PhantomData<(T, A)>, +} + +impl GenericColumnStatisticsBuilder +where + T: ValueType + Send + Sync, + T::Scalar: Send + Sync, + A: ColumnStatisticsAdapter + 'static, + for<'a, 'b> T::ScalarRef<'a>: PartialOrd>, +{ + fn create(data_type: DataType) -> Box { + Box::new(Self { + min: None, + max: None, + null_count: 0, + in_memory_size: 0, + data_type, + _phantom: PhantomData, + }) + } + + fn add_batch<'a, I>(&mut self, mut iter: I) + where I: Iterator> { + let first = iter.next().unwrap(); + let mut min = first.clone(); + let mut max = first; + for v in iter { + if matches!(min.partial_cmp(&v), Some(Ordering::Greater)) { + min = v; + continue; + } + + if matches!(max.partial_cmp(&v), Some(Ordering::Less)) { + max = v; + } + } + + self.add(min, max); + } + + fn add(&mut self, min: T::ScalarRef<'_>, max: T::ScalarRef<'_>) { + if let Some(val) = self.min.as_mut() { + A::update_value(val, min, Ordering::Less); + } else { + self.min = Some(A::scalar_to_value(min)); + } + + if let Some(val) = self.max.as_mut() { + A::update_value(val, max, Ordering::Greater); + } else { + self.max = Some(A::scalar_to_value(max)); + } + } +} + +impl ColumnStatisticsBuilder for GenericColumnStatisticsBuilder +where + T: ValueType + Send + Sync, + T::Scalar: Send + Sync, + A: ColumnStatisticsAdapter + 'static, + for<'a, 'b> T::ScalarRef<'a>: PartialOrd>, +{ + fn update_column(&mut self, column: &Column) { + self.in_memory_size += column.memory_size(); + if column.len() == 0 { + return; + } + let (column, validity) = match column { + Column::Nullable(box inner) => { + let validity = if inner.validity.null_count() == 0 { + None + } else { + Some(&inner.validity) + }; + (&inner.column, validity) + } + Column::Null { len } => { + self.null_count += *len; + return; + } + col => (col, None), + }; + self.null_count += validity.map_or(0, |v| v.null_count()); + + let column = T::try_downcast_column(column).unwrap(); + if let Some(v) = validity { + if v.true_count() as f64 / v.len() as f64 >= SELECTIVITY_THRESHOLD { + let column_iter = T::iter_column(&column); + let value_iter = column_iter + .zip(v.iter()) + .filter(|(_, v)| *v) + .map(|(v, _)| v); + self.add_batch(value_iter); + } else { + for idx in TrueIdxIter::new(v.len(), Some(v)) { + let v = unsafe { T::index_column_unchecked(&column, idx) }; + self.add(v.clone(), v); + } + } + } else { + let column_iter = T::iter_column(&column); + self.add_batch(column_iter); + } + } + + fn update_scalar(&mut self, scalar: &ScalarRef, num_rows: usize, data_type: &DataType) { + // when we read it back from parquet, it is a Column instead of Scalar + self.in_memory_size += scalar.estimated_scalar_repeat_size(num_rows, data_type); + if scalar.is_null() { + self.null_count += num_rows; + return; + } + + let val = T::try_downcast_scalar(scalar).unwrap(); + self.add(val.clone(), val); + } + + fn finalize(self: Box) -> Result { + let min = if let Some(v) = self.min { + let v = A::value_to_scalar(v); + // safe upwrap. + T::upcast_scalar_with_type(v, &self.data_type) + .trim_min() + .unwrap() + } else { + Scalar::Null + }; + let max = if let Some(v) = self.max { + let v = A::value_to_scalar(v); + if let Some(v) = T::upcast_scalar_with_type(v, &self.data_type).trim_max() { + v + } else { + return Err(ErrorCode::Internal("Unable to trim string")); + } + } else { + Scalar::Null + }; + + Ok(ColumnStatistics::new( + min, + max, + self.null_count as u64, + self.in_memory_size as u64, + None, + )) + } +} diff --git a/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs b/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs new file mode 100644 index 0000000000000..0e246911ea542 --- /dev/null +++ b/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs @@ -0,0 +1,116 @@ +// Copyright 2021 Datafuse Labs +// +// 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 databend_common_exception::Result; +use databend_common_expression::types::DataType; +use databend_common_expression::ColumnId; +use databend_common_expression::DataBlock; +use databend_common_expression::TableSchemaRef; +use databend_common_expression::Value; +use databend_storages_common_table_meta::meta::StatisticsOfColumns; + +use crate::io::write::stream::create_column_ndv_estimator; +use crate::io::write::stream::create_column_stats_builder; +use crate::io::write::stream::ColumnNDVEstimator; +use crate::io::write::stream::ColumnStatisticsBuilder; +use crate::statistics::traverse_values_dfs; + +pub struct ColumnStatisticsState { + col_stats: HashMap>, + distinct_columns: HashMap>, +} + +impl ColumnStatisticsState { + pub fn new( + stats_columns: &[(ColumnId, DataType)], + distinct_columns: &[(ColumnId, DataType)], + ) -> Self { + let col_stats = stats_columns + .iter() + .map(|(col_id, data_type)| (*col_id, create_column_stats_builder(data_type))) + .collect(); + + let distinct_columns = distinct_columns + .iter() + .map(|(col_id, data_type)| (*col_id, create_column_ndv_estimator(data_type))) + .collect(); + + Self { + col_stats, + distinct_columns, + } + } + + pub fn add_block(&mut self, schema: &TableSchemaRef, data_block: &DataBlock) -> Result<()> { + let rows = data_block.num_rows(); + let leaves = traverse_values_dfs(data_block.columns(), schema.fields())?; + for (column_id, col, data_type) in leaves { + match col { + Value::Scalar(s) => { + self.col_stats.get_mut(&column_id).unwrap().update_scalar( + &s.as_ref(), + rows, + &data_type, + ); + if let Some(estimator) = self.distinct_columns.get_mut(&column_id) { + estimator.update_scalar(&s.as_ref()); + } + } + Value::Column(col) => { + self.col_stats + .get_mut(&column_id) + .unwrap() + .update_column(&col); + // use distinct count calculated by the xor hash function to avoid repetitive operation. + if let Some(estimator) = self.distinct_columns.get_mut(&column_id) { + estimator.update_column(&col); + } + } + } + } + Ok(()) + } + + pub fn finalize( + self, + column_distinct_count: HashMap, + ) -> Result { + let mut statistics = StatisticsOfColumns::with_capacity(self.col_stats.len()); + for (id, stats) in self.col_stats { + let mut col_stats = stats.finalize()?; + if let Some(count) = column_distinct_count.get(&id) { + // value calculated by xor hash function include NULL, need to subtract one. + let distinct_of_values = if col_stats.null_count > 0 { + *count as u64 - 1 + } else { + *count as u64 + }; + col_stats.distinct_of_values = Some(distinct_of_values); + } else if let Some(estimator) = self.distinct_columns.get(&id) { + col_stats.distinct_of_values = Some(estimator.finalize()); + } else if col_stats.min == col_stats.max { + // Bloom index will skip the large string column, it also no need to calc distinct values. + if col_stats.min.is_null() { + col_stats.distinct_of_values = Some(0); + } else { + col_stats.distinct_of_values = Some(1); + } + } + statistics.insert(id, col_stats); + } + Ok(statistics) + } +} diff --git a/src/query/storages/fuse/src/io/write/stream/mod.rs b/src/query/storages/fuse/src/io/write/stream/mod.rs index 26d32ee679582..3eda792ec7433 100644 --- a/src/query/storages/fuse/src/io/write/stream/mod.rs +++ b/src/query/storages/fuse/src/io/write/stream/mod.rs @@ -14,7 +14,14 @@ mod block_builder; mod cluster_statistics; -mod column_statistics; +mod column_ndv_estimator; +mod column_statistics_builder; +mod column_statistics_state; pub(crate) use block_builder::StreamBlockBuilder; pub(crate) use block_builder::StreamBlockProperties; +pub(crate) use column_ndv_estimator::create_column_ndv_estimator; +pub(crate) use column_ndv_estimator::ColumnNDVEstimator; +pub(crate) use column_statistics_builder::create_column_stats_builder; +pub(crate) use column_statistics_builder::ColumnStatisticsBuilder; +pub(crate) use column_statistics_state::ColumnStatisticsState; diff --git a/src/query/storages/fuse/src/operations/append.rs b/src/query/storages/fuse/src/operations/append.rs index 9316374128528..d1ca44fd41d4c 100644 --- a/src/query/storages/fuse/src/operations/append.rs +++ b/src/query/storages/fuse/src/operations/append.rs @@ -37,10 +37,11 @@ use databend_common_sql::executor::physical_plans::MutationKind; use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::table::ClusterType; +use crate::io::StreamBlockProperties; +use crate::operations::TransformBlockBuilder; use crate::operations::TransformBlockWriter; use crate::operations::TransformSerializeBlock; use crate::statistics::ClusterStatsGenerator; -use crate::FuseStorageFormat; use crate::FuseTable; impl FuseTable { @@ -50,19 +51,22 @@ impl FuseTable { pipeline: &mut Pipeline, table_meta_timestamps: TableMetaTimestamps, ) -> Result<()> { - let enable_stream_block_write = ctx.get_settings().get_enable_block_stream_write()? - && matches!(self.storage_format, FuseStorageFormat::Parquet); + let enable_stream_block_write = self.enable_stream_block_write(); if enable_stream_block_write { + let properties = StreamBlockProperties::try_create( + ctx.clone(), + self, + MutationKind::Insert, + table_meta_timestamps, + )?; + pipeline.add_transform(|input, output| { - TransformBlockWriter::try_create( - ctx.clone(), - input, - output, - self, - table_meta_timestamps, - false, - ) + TransformBlockBuilder::try_create(input, output, properties.clone()) })?; + + pipeline.add_async_accumulating_transformer(|| { + TransformBlockWriter::create(ctx.clone(), MutationKind::Insert, self, false) + }); } else { let block_thresholds = self.get_block_thresholds(); build_compact_block_pipeline(pipeline, block_thresholds)?; diff --git a/src/query/storages/fuse/src/operations/common/processors/mod.rs b/src/query/storages/fuse/src/operations/common/processors/mod.rs index e0e3d3b25f25a..d43c569c14016 100644 --- a/src/query/storages/fuse/src/operations/common/processors/mod.rs +++ b/src/query/storages/fuse/src/operations/common/processors/mod.rs @@ -22,6 +22,7 @@ mod transform_serialize_segment; pub use multi_table_insert_commit::CommitMultiTableInsert; pub use sink_commit::CommitSink; +pub use transform_block_writer::TransformBlockBuilder; pub use transform_block_writer::TransformBlockWriter; pub use transform_merge_commit_meta::TransformMergeCommitMeta; pub use transform_mutation_aggregator::TableMutationAggregator; diff --git a/src/query/storages/fuse/src/operations/common/processors/transform_block_writer.rs b/src/query/storages/fuse/src/operations/common/processors/transform_block_writer.rs index ce11a4834c7fb..895f5c7a0ebc5 100644 --- a/src/query/storages/fuse/src/operations/common/processors/transform_block_writer.rs +++ b/src/query/storages/fuse/src/operations/common/processors/transform_block_writer.rs @@ -22,35 +22,36 @@ use databend_common_catalog::table::Table; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; -use databend_common_io::constants::DEFAULT_BLOCK_ROW_COUNT; +use databend_common_metrics::storage::metrics_inc_recluster_write_block_nums; use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_pipeline_transforms::AsyncAccumulatingTransform; +use databend_common_sql::executor::physical_plans::MutationKind; use databend_common_storage::MutationStatus; -use databend_storages_common_table_meta::meta::TableMetaTimestamps; use opendal::Operator; use crate::io::BlockSerialization; use crate::io::BlockWriter; use crate::io::StreamBlockBuilder; use crate::io::StreamBlockProperties; +use crate::operations::MutationLogEntry; +use crate::operations::MutationLogs; use crate::FuseTable; -use crate::FUSE_OPT_KEY_ROW_PER_BLOCK; -#[allow(clippy::large_enum_variant)] enum State { Consume, Collect(DataBlock), Serialize, Finalize, Flush, - Write(BlockSerialization), } -pub struct TransformBlockWriter { +pub struct TransformBlockBuilder { state: State, input: Arc, output: Arc, @@ -62,43 +63,27 @@ pub struct TransformBlockWriter { input_data_size: usize, input_num_rows: usize, - dal: Operator, - // Only used in multi table insert - table_id: Option, - - max_block_rows: usize, - input_data: VecDeque, + input_data: VecDeque<(usize, DataBlock)>, output_data: Option, } -impl TransformBlockWriter { +impl TransformBlockBuilder { pub fn try_create( - ctx: Arc, input: Arc, output: Arc, - table: &FuseTable, - table_meta_timestamps: TableMetaTimestamps, - with_tid: bool, + properties: Arc, ) -> Result { - let max_block_rows = std::cmp::min( - ctx.get_settings().get_max_block_size()? as usize, - table.get_option(FUSE_OPT_KEY_ROW_PER_BLOCK, DEFAULT_BLOCK_ROW_COUNT), - ); - let properties = StreamBlockProperties::try_create(ctx, table, table_meta_timestamps)?; - Ok(ProcessorPtr::create(Box::new(TransformBlockWriter { + Ok(ProcessorPtr::create(Box::new(TransformBlockBuilder { state: State::Consume, input, output, properties, builder: None, - dal: table.get_operator(), need_flush: false, - table_id: if with_tid { Some(table.get_id()) } else { None }, input_data: VecDeque::new(), input_data_size: 0, input_num_rows: 0, output_data: None, - max_block_rows, }))) } @@ -111,23 +96,24 @@ impl TransformBlockWriter { Ok(self.builder.as_mut().unwrap()) } - fn calc_max_block_rows(&self, block: &DataBlock) -> usize { - let min_bytes_per_block = self.properties.block_thresholds.min_bytes_per_block; - let block_size = block.estimate_block_size(); - if block_size < min_bytes_per_block { - return self.max_block_rows; - } - let num_rows = block.num_rows(); + fn split_input(&self, input: DataBlock) -> Vec { + let block_size = input.estimate_block_size(); + let num_rows = input.num_rows(); let average_row_size = block_size.div_ceil(num_rows); - let max_rows = min_bytes_per_block.div_ceil(average_row_size); - self.max_block_rows.min(max_rows) + let max_rows = self + .properties + .block_thresholds + .min_bytes_per_block + .div_ceil(average_row_size) + .min(self.properties.block_thresholds.max_rows_per_block); + input.split_by_rows_no_tail(max_rows) } } #[async_trait] -impl Processor for TransformBlockWriter { +impl Processor for TransformBlockBuilder { fn name(&self) -> String { - "TransformBlockWriter".to_string() + "TransformBlockBuilder".to_string() } fn as_any(&mut self) -> &mut dyn Any { @@ -135,15 +121,15 @@ impl Processor for TransformBlockWriter { } fn event(&mut self) -> Result { - match &self.state { - State::Collect(_) | State::Serialize | State::Flush | State::Finalize => { - return Ok(Event::Sync) - } - State::Write(_) => return Ok(Event::Async), - _ => {} + if matches!( + self.state, + State::Collect(_) | State::Serialize | State::Flush | State::Finalize + ) { + return Ok(Event::Sync); } if self.output.is_finished() { + self.input.finish(); return Ok(Event::Finished); } @@ -196,15 +182,16 @@ impl Processor for TransformBlockWriter { State::Collect(block) => { // Check if the datablock is valid, this is needed to ensure data is correct block.check_valid()?; - self.input_data_size += block.estimate_block_size(); self.input_num_rows += block.num_rows(); - let max_rows_per_block = self.calc_max_block_rows(&block); - let blocks = block.split_by_rows_no_tail(max_rows_per_block); - self.input_data.extend(blocks); + for block in self.split_input(block) { + let block_size = block.estimate_block_size(); + self.input_data_size += block_size; + self.input_data.push_back((block_size, block)); + } } State::Serialize => { - while let Some(b) = self.input_data.pop_front() { - self.input_data_size -= b.estimate_block_size(); + while let Some((block_size, b)) = self.input_data.pop_front() { + self.input_data_size -= block_size; self.input_num_rows -= b.num_rows(); let builder = self.get_or_create_builder()?; @@ -217,7 +204,7 @@ impl Processor for TransformBlockWriter { } } State::Finalize => { - while let Some(b) = self.input_data.pop_front() { + while let Some((_, b)) = self.input_data.pop_front() { let builder = self.get_or_create_builder()?; builder.write(b)?; } @@ -227,7 +214,7 @@ impl Processor for TransformBlockWriter { let builder = self.builder.take().unwrap(); if !builder.is_empty() { let serialized = builder.finish()?; - self.state = State::Write(serialized); + self.output_data = Some(DataBlock::empty_with_meta(Box::new(serialized))); } self.need_flush = false; } @@ -235,11 +222,41 @@ impl Processor for TransformBlockWriter { } Ok(()) } +} - #[async_backtrace::framed] - async fn async_process(&mut self) -> Result<()> { - match std::mem::replace(&mut self.state, State::Consume) { - State::Write(serialized) => { +pub struct TransformBlockWriter { + kind: MutationKind, + dal: Operator, + ctx: Arc, + // Only used in multi table insert + table_id: Option, +} + +impl TransformBlockWriter { + pub fn create( + ctx: Arc, + kind: MutationKind, + table: &FuseTable, + with_tid: bool, + ) -> Self { + Self { + ctx, + dal: table.get_operator(), + table_id: if with_tid { Some(table.get_id()) } else { None }, + kind, + } + } +} + +#[async_trait::async_trait] +impl AsyncAccumulatingTransform for TransformBlockWriter { + const NAME: &'static str = "TransformBlockWriter"; + + async fn transform(&mut self, data: DataBlock) -> Result> { + debug_assert!(data.is_empty()); + + if let Some(ptr) = data.get_owned_meta() { + if let Some(serialized) = BlockSerialization::downcast_from(ptr) { let extended_block_meta = BlockWriter::write_down(&self.dal, serialized).await?; let bytes = if let Some(draft_virtual_block_meta) = @@ -251,32 +268,45 @@ impl Processor for TransformBlockWriter { extended_block_meta.block_meta.block_size as usize }; - self.properties - .ctx - .get_write_progress() - .incr(&ProgressValues { - rows: extended_block_meta.block_meta.row_count as usize, - bytes, - }); + self.ctx.get_write_progress().incr(&ProgressValues { + rows: extended_block_meta.block_meta.row_count as usize, + bytes, + }); // appending new data block if let Some(tid) = self.table_id { - self.properties.ctx.update_multi_table_insert_status( + self.ctx.update_multi_table_insert_status( tid, extended_block_meta.block_meta.row_count, ); } else { - self.properties.ctx.add_mutation_status(MutationStatus { + self.ctx.add_mutation_status(MutationStatus { insert_rows: extended_block_meta.block_meta.row_count, update_rows: 0, deleted_rows: 0, }); } - self.output_data = Some(DataBlock::empty_with_meta(Box::new(extended_block_meta))); + let output = if matches!(self.kind, MutationKind::Insert) { + DataBlock::empty_with_meta(Box::new(extended_block_meta)) + } else { + if matches!(self.kind, MutationKind::Recluster) { + metrics_inc_recluster_write_block_nums(); + } + + DataBlock::empty_with_meta(Box::new(MutationLogs { + entries: vec![MutationLogEntry::AppendBlock { + block_meta: Arc::new(extended_block_meta), + }], + })) + }; + + return Ok(Some(output)); } - _ => return Err(ErrorCode::Internal("It's a bug.")), } - Ok(()) + + Err(ErrorCode::Internal( + "Cannot downcast meta to BlockSerialization", + )) } } diff --git a/src/query/storages/fuse/src/statistics/column_statistic.rs b/src/query/storages/fuse/src/statistics/column_statistic.rs index 36737dd9e7a62..d02cf23a96883 100644 --- a/src/query/storages/fuse/src/statistics/column_statistic.rs +++ b/src/query/storages/fuse/src/statistics/column_statistic.rs @@ -83,37 +83,39 @@ pub fn gen_columns_statistics( let mut min = Scalar::Null; let mut max = Scalar::Null; - let (mins, _) = eval_aggr("min", vec![], &[col.clone().into()], rows, vec![])?; - let (maxs, _) = eval_aggr("max", vec![], &[col.clone().into()], rows, vec![])?; + if col.len() > 0 { + let (mins, _) = eval_aggr("min", vec![], &[col.clone().into()], rows, vec![])?; + let (maxs, _) = eval_aggr("max", vec![], &[col.clone().into()], rows, vec![])?; - if mins.len() > 0 { - min = if let Some(v) = mins.index(0) { - if let Some(v) = v.to_owned().trim_min() { - v + if mins.len() > 0 { + min = if let Some(v) = mins.index(0) { + if let Some(v) = v.to_owned().trim_min() { + v + } else { + continue; + } } else { continue; } - } else { - continue; } - } - if maxs.len() > 0 { - max = if let Some(v) = maxs.index(0) { - if let Some(v) = v.to_owned().trim_max() { - v + if maxs.len() > 0 { + max = if let Some(v) = maxs.index(0) { + if let Some(v) = v.to_owned().trim_max() { + v + } else { + continue; + } } else { continue; } - } else { - continue; } } let (is_all_null, bitmap) = col.validity(); let unset_bits = match (is_all_null, bitmap) { - (true, _) => rows, - (false, Some(bitmap)) => bitmap.null_count(), + (_, Some(bitmap)) => bitmap.null_count(), + (true, None) => rows, (false, None) => 0, }; diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0004_remote_insert_into_select.test b/tests/sqllogictests/suites/base/09_fuse_engine/09_0004_remote_insert_into_select.test index 5e399dc1b406f..5ad4e316896d0 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0004_remote_insert_into_select.test +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0004_remote_insert_into_select.test @@ -7,9 +7,6 @@ CREATE DATABASE db_09_004 statement ok USE db_09_004 -statement ok -set enable_block_stream_write = 1 - statement ok CREATE TABLE IF NOT EXISTS t1(a UInt8 not null, b UInt64 not null, c Int8 not null, d Int64 not null, e Date not null, f Date not null, g DateTime not null, h String not null) Engine = Fuse diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0008_fuse_optimize_table.test b/tests/sqllogictests/suites/base/09_fuse_engine/09_0008_fuse_optimize_table.test index 43a2b262ca2f9..c19a27a9e8890 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0008_fuse_optimize_table.test +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0008_fuse_optimize_table.test @@ -7,9 +7,6 @@ CREATE DATABASE db_09_0008 statement ok USE db_09_0008 -statement ok -set enable_block_stream_write = 1 - statement ok create table t(a uint64 not null) diff --git a/tests/sqllogictests/suites/base/issues/issue_18275.test b/tests/sqllogictests/suites/base/issues/issue_18275.test index 36217cda4b3ca..ce895d228329e 100644 --- a/tests/sqllogictests/suites/base/issues/issue_18275.test +++ b/tests/sqllogictests/suites/base/issues/issue_18275.test @@ -14,9 +14,6 @@ CREATE OR REPLACE TABLE product_test ( stock INT ); -statement ok -set enable_block_stream_write = 1; - statement ok INSERT INTO product_test (id, name, category, price, stock) VALUES(6, 'Keyboard', 'Electronics', 79.99, 25), From 6cc6e4b87bdfc2df9dcf5469ca8feb3408de55cb Mon Sep 17 00:00:00 2001 From: zhyass Date: Sat, 5 Jul 2025 03:43:45 +0800 Subject: [PATCH 02/11] fix virtual column builder --- Cargo.lock | 2 + src/query/storages/fuse/Cargo.toml | 2 + .../fuse/src/io/write/stream/block_builder.rs | 40 +++--- .../io/write/stream/column_ndv_estimator.rs | 83 +++++++++--- .../write/stream/column_statistics_builder.rs | 126 +++++++++++++----- .../write/stream/column_statistics_state.rs | 6 +- .../storages/fuse/src/io/write/stream/mod.rs | 2 + .../01_ee_system/01_0002_virtual_column.test | 6 - ...5_01_02_load_compact_copy_row_per_block.sh | 2 +- 9 files changed, 184 insertions(+), 85 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index be86c5270fa2b..d4a188e61ad82 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4354,6 +4354,7 @@ dependencies = [ "databend-storages-common-table-meta", "divan", "enum-as-inner", + "enum_dispatch", "fastrace", "futures", "futures-util", @@ -4365,6 +4366,7 @@ dependencies = [ "opendal", "parking_lot 0.12.3", "parquet", + "paste", "rand 0.8.5", "serde", "serde_json", diff --git a/src/query/storages/fuse/Cargo.toml b/src/query/storages/fuse/Cargo.toml index 811821ec8088d..0601548e6e4bf 100644 --- a/src/query/storages/fuse/Cargo.toml +++ b/src/query/storages/fuse/Cargo.toml @@ -50,6 +50,7 @@ backoff = { workspace = true, features = ["futures", "tokio"] } bytes = { workspace = true } chrono = { workspace = true } enum-as-inner = { workspace = true } +enum_dispatch = { workspace = true } fastrace = { workspace = true } futures = { workspace = true } futures-util = { workspace = true } @@ -61,6 +62,7 @@ match-template = { workspace = true } opendal = { workspace = true } parking_lot = { workspace = true } parquet = { workspace = true } +paste = { workspace = true } rand = { workspace = true } serde = { workspace = true } serde_json = { workspace = true } diff --git a/src/query/storages/fuse/src/io/write/stream/block_builder.rs b/src/query/storages/fuse/src/io/write/stream/block_builder.rs index cca3c1cc0da6c..e049f53c646b9 100644 --- a/src/query/storages/fuse/src/io/write/stream/block_builder.rs +++ b/src/query/storages/fuse/src/io/write/stream/block_builder.rs @@ -36,6 +36,7 @@ use databend_common_expression::ORIGIN_BLOCK_ROW_NUM_COLUMN_ID; use databend_common_io::constants::DEFAULT_BLOCK_BUFFER_SIZE; use databend_common_meta_app::schema::TableIndex; use databend_common_native::write::NativeWriter; +use databend_common_native::write::WriteOptions; use databend_common_sql::executor::physical_plans::MutationKind; use databend_storages_common_index::BloomIndex; use databend_storages_common_index::BloomIndexBuilder; @@ -192,7 +193,7 @@ impl StreamBlockBuilder { let writer = NativeWriter::new( buffer, properties.source_schema.as_ref().clone(), - databend_common_native::write::WriteOptions { + WriteOptions { default_compression: properties.write_settings.table_compression.into(), max_page_size: Some(properties.write_settings.max_page_size), default_compress_ratio, @@ -217,27 +218,12 @@ impl StreamBlockBuilder { &properties.ngram_args, )?; - let virtual_column_builder = if properties - .ctx - .get_settings() - .get_enable_refresh_virtual_column_after_write() - .unwrap_or_default() - && properties.support_virtual_columns - { - VirtualColumnBuilder::try_create( - properties.ctx.clone(), - properties.source_schema.clone(), - ) - .ok() - } else { - None - }; + let virtual_column_builder = properties.virtual_column_builder.clone(); let vector_index_builder = VectorIndexBuilder::try_create( properties.ctx.clone(), &properties.table_indexes, properties.source_schema.clone(), ); - let cluster_stats_state = ClusterStatisticsState::new(properties.cluster_stats_builder.clone()); let column_stats_state = @@ -387,7 +373,10 @@ impl StreamBlockBuilder { vector_index_size, vector_index_location, create_on: Some(Utc::now()), - ngram_filter_index_size: None, + ngram_filter_index_size: bloom_index_state + .as_ref() + .map(|v| v.ngram_size) + .unwrap_or_default(), virtual_block_meta: None, }; let serialized = BlockSerialization { @@ -416,8 +405,8 @@ pub struct StreamBlockProperties { bloom_columns_map: BTreeMap, ngram_args: Vec, inverted_index_builders: Vec, + virtual_column_builder: Option, table_meta_timestamps: TableMetaTimestamps, - support_virtual_columns: bool, table_indexes: BTreeMap, } @@ -463,6 +452,16 @@ impl StreamBlockProperties { .collect::>(); let inverted_index_builders = create_inverted_index_builders(&table.table_info.meta); + let virtual_column_builder = if ctx + .get_settings() + .get_enable_refresh_virtual_column_after_write() + .unwrap_or_default() + && table.support_virtual_columns() + { + VirtualColumnBuilder::try_create(ctx.clone(), source_schema.clone()).ok() + } else { + None + }; let cluster_stats_builder = ClusterStatisticsBuilder::try_create(table, ctx.clone(), &source_schema)?; @@ -481,7 +480,6 @@ impl StreamBlockProperties { } } } - let support_virtual_columns = table.support_virtual_columns(); let table_indexes = table.table_info.meta.indexes.clone(); Ok(Arc::new(StreamBlockProperties { ctx, @@ -490,13 +488,13 @@ impl StreamBlockProperties { source_schema, write_settings, cluster_stats_builder, + virtual_column_builder, stats_columns, distinct_columns, bloom_columns_map, ngram_args, inverted_index_builders, table_meta_timestamps, - support_virtual_columns, table_indexes, })) } diff --git a/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs b/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs index cb138c53a1d8c..4410f06feba42 100644 --- a/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs +++ b/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs @@ -21,49 +21,88 @@ use databend_common_expression::types::DateType; use databend_common_expression::types::Decimal128Type; use databend_common_expression::types::Decimal256Type; use databend_common_expression::types::Decimal64Type; +use databend_common_expression::types::Float32Type; +use databend_common_expression::types::Float64Type; +use databend_common_expression::types::Int16Type; +use databend_common_expression::types::Int32Type; +use databend_common_expression::types::Int64Type; +use databend_common_expression::types::Int8Type; use databend_common_expression::types::NumberDataType; -use databend_common_expression::types::NumberType; use databend_common_expression::types::StringType; use databend_common_expression::types::TimestampType; +use databend_common_expression::types::UInt16Type; +use databend_common_expression::types::UInt32Type; +use databend_common_expression::types::UInt64Type; +use databend_common_expression::types::UInt8Type; use databend_common_expression::types::ValueType; -use databend_common_expression::with_number_mapped_type; +use databend_common_expression::with_number_type; use databend_common_expression::Column; use databend_common_expression::ScalarRef; use databend_common_expression::SELECTIVITY_THRESHOLD; use databend_storages_common_table_meta::meta::ColumnDistinctHLL; +use enum_dispatch::enum_dispatch; -pub trait ColumnNDVEstimator: Send + Sync { +#[enum_dispatch] +pub trait ColumnNDVEstimatorOps: Send + Sync { fn update_column(&mut self, column: &Column); fn update_scalar(&mut self, scalar: &ScalarRef); fn finalize(&self) -> u64; } -pub fn create_column_ndv_estimator(data_type: &DataType) -> Box { +#[enum_dispatch(ColumnNDVEstimatorOps)] +pub enum ColumnNDVEstimator { + Int8(ColumnNDVEstimatorImpl), + Int16(ColumnNDVEstimatorImpl), + Int32(ColumnNDVEstimatorImpl), + Int64(ColumnNDVEstimatorImpl), + UInt8(ColumnNDVEstimatorImpl), + UInt16(ColumnNDVEstimatorImpl), + UInt32(ColumnNDVEstimatorImpl), + UInt64(ColumnNDVEstimatorImpl), + Float32(ColumnNDVEstimatorImpl), + Float64(ColumnNDVEstimatorImpl), + String(ColumnNDVEstimatorImpl), + Date(ColumnNDVEstimatorImpl), + Timestamp(ColumnNDVEstimatorImpl), + Decimal64(ColumnNDVEstimatorImpl), + Decimal128(ColumnNDVEstimatorImpl), + Decimal256(ColumnNDVEstimatorImpl), +} + +pub fn create_column_ndv_estimator(data_type: &DataType) -> ColumnNDVEstimator { + macro_rules! match_number_type_create { + ($inner_type:expr) => {{ + with_number_type!(|NUM_TYPE| match $inner_type { + NumberDataType::NUM_TYPE => { + paste::paste! { + ColumnNDVEstimator::NUM_TYPE(ColumnNDVEstimatorImpl::<[]>::new()) + } + } + }) + }}; + } + let inner_type = data_type.remove_nullable(); - with_number_mapped_type!(|NUM_TYPE| match inner_type { - DataType::Number(NumberDataType::NUM_TYPE) => { - ColumnNDVEstimatorImpl::>::create() - } - DataType::String => { - ColumnNDVEstimatorImpl::::create() - } - DataType::Date => { - ColumnNDVEstimatorImpl::::create() + match inner_type { + DataType::Number(num_type) => { + match_number_type_create!(num_type) } + DataType::String => ColumnNDVEstimator::String(ColumnNDVEstimatorImpl::::new()), + DataType::Date => ColumnNDVEstimator::Date(ColumnNDVEstimatorImpl::::new()), DataType::Timestamp => { - ColumnNDVEstimatorImpl::::create() + ColumnNDVEstimator::Timestamp(ColumnNDVEstimatorImpl::::new()) } DataType::Decimal(size) => { if size.can_carried_by_64() { - ColumnNDVEstimatorImpl::::create() + ColumnNDVEstimator::Decimal64(ColumnNDVEstimatorImpl::::new()) } else if size.can_carried_by_128() { - ColumnNDVEstimatorImpl::::create() + ColumnNDVEstimator::Decimal128(ColumnNDVEstimatorImpl::::new()) } else { - ColumnNDVEstimatorImpl::::create() + ColumnNDVEstimator::Decimal256(ColumnNDVEstimatorImpl::::new()) } } _ => unreachable!("Unsupported data type: {:?}", data_type), - }) + } } pub struct ColumnNDVEstimatorImpl @@ -80,15 +119,15 @@ where T: ValueType + Send + Sync, for<'a> T::ScalarRef<'a>: Hash, { - pub fn create() -> Box { - Box::new(Self { + pub fn new() -> Self { + Self { hll: ColumnDistinctHLL::new(), _phantom: Default::default(), - }) + } } } -impl ColumnNDVEstimator for ColumnNDVEstimatorImpl +impl ColumnNDVEstimatorOps for ColumnNDVEstimatorImpl where T: ValueType + Send + Sync, for<'a> T::ScalarRef<'a>: Hash, diff --git a/src/query/storages/fuse/src/io/write/stream/column_statistics_builder.rs b/src/query/storages/fuse/src/io/write/stream/column_statistics_builder.rs index 3178e5da00ef3..e9278b4b1e71a 100644 --- a/src/query/storages/fuse/src/io/write/stream/column_statistics_builder.rs +++ b/src/query/storages/fuse/src/io/write/stream/column_statistics_builder.rs @@ -24,56 +24,124 @@ use databend_common_expression::types::Decimal; use databend_common_expression::types::Decimal128Type; use databend_common_expression::types::Decimal256Type; use databend_common_expression::types::Decimal64Type; +use databend_common_expression::types::Float32Type; +use databend_common_expression::types::Float64Type; +use databend_common_expression::types::Int16Type; +use databend_common_expression::types::Int32Type; +use databend_common_expression::types::Int64Type; +use databend_common_expression::types::Int8Type; use databend_common_expression::types::NumberDataType; -use databend_common_expression::types::NumberType; use databend_common_expression::types::StringType; use databend_common_expression::types::TimestampType; +use databend_common_expression::types::UInt16Type; +use databend_common_expression::types::UInt32Type; +use databend_common_expression::types::UInt64Type; +use databend_common_expression::types::UInt8Type; use databend_common_expression::types::ValueType; -use databend_common_expression::with_number_mapped_type; +use databend_common_expression::with_number_type; use databend_common_expression::Column; use databend_common_expression::Scalar; use databend_common_expression::ScalarRef; use databend_common_expression::SELECTIVITY_THRESHOLD; use databend_storages_common_table_meta::meta::ColumnStatistics; +use enum_dispatch::enum_dispatch; use crate::statistics::Trim; -pub trait ColumnStatisticsBuilder: Send + Sync { - fn update_column(&mut self, column: &Column); +pub type CommonBuilder = GenericColumnStatisticsBuilder; +pub type DecimalBuilder = GenericColumnStatisticsBuilder; + +#[enum_dispatch(ColumnStatsOps)] +pub enum ColumnStatisticsBuilder { + Int8(CommonBuilder), + Int16(CommonBuilder), + Int32(CommonBuilder), + Int64(CommonBuilder), + UInt8(CommonBuilder), + UInt16(CommonBuilder), + UInt32(CommonBuilder), + UInt64(CommonBuilder), + Float32(CommonBuilder), + Float64(CommonBuilder), + String(CommonBuilder), + Date(CommonBuilder), + Timestamp(CommonBuilder), + Decimal64(DecimalBuilder), + Decimal128(DecimalBuilder), + Decimal256(DecimalBuilder), +} +#[enum_dispatch] +pub trait ColumnStatsOps { + fn update_column(&mut self, column: &Column); fn update_scalar(&mut self, scalar: &ScalarRef, num_rows: usize, data_type: &DataType); + fn finalize(self) -> Result; +} - fn finalize(self: Box) -> Result; +impl ColumnStatsOps for GenericColumnStatisticsBuilder +where + T: ValueType + Send + Sync, + T::Scalar: Send + Sync, + A: ColumnStatisticsAdapter + 'static, + for<'a, 'b> T::ScalarRef<'a>: PartialOrd>, +{ + fn update_column(&mut self, column: &Column) { + GenericColumnStatisticsBuilder::update_column(self, column); + } + + fn update_scalar(&mut self, scalar: &ScalarRef, num_rows: usize, data_type: &DataType) { + GenericColumnStatisticsBuilder::update_scalar(self, scalar, num_rows, data_type); + } + + fn finalize(self) -> Result { + GenericColumnStatisticsBuilder::finalize(self) + } } -pub fn create_column_stats_builder(data_type: &DataType) -> Box { +pub fn create_column_stats_builder(data_type: &DataType) -> ColumnStatisticsBuilder { let inner_type = data_type.remove_nullable(); - with_number_mapped_type!(|NUM_TYPE| match inner_type { - DataType::Number(NumberDataType::NUM_TYPE) => { - GenericColumnStatisticsBuilder::, CommonAdapter>::create( - inner_type, - ) + macro_rules! match_number_type_create { + ($inner_type:expr) => {{ + with_number_type!(|NUM_TYPE| match $inner_type { + NumberDataType::NUM_TYPE => { + paste::paste! { + ColumnStatisticsBuilder::NUM_TYPE(CommonBuilder::<[]>::create(inner_type)) + } + } + }) + }}; + } + + match inner_type { + DataType::Number(num_type) => { + match_number_type_create!(num_type) } DataType::String => { - GenericColumnStatisticsBuilder::::create(inner_type) + ColumnStatisticsBuilder::String(CommonBuilder::::create(inner_type)) } DataType::Date => { - GenericColumnStatisticsBuilder::::create(inner_type) + ColumnStatisticsBuilder::Date(CommonBuilder::::create(inner_type)) } DataType::Timestamp => { - GenericColumnStatisticsBuilder::::create(inner_type) + ColumnStatisticsBuilder::Timestamp(CommonBuilder::::create(inner_type)) } DataType::Decimal(size) => { if size.can_carried_by_64() { - GenericColumnStatisticsBuilder::::create(inner_type) + ColumnStatisticsBuilder::Decimal64(DecimalBuilder::::create( + inner_type, + )) } else if size.can_carried_by_128() { - GenericColumnStatisticsBuilder::::create(inner_type) + ColumnStatisticsBuilder::Decimal128(DecimalBuilder::::create( + inner_type, + )) } else { - GenericColumnStatisticsBuilder::::create(inner_type) + ColumnStatisticsBuilder::Decimal256(DecimalBuilder::::create( + inner_type, + )) } } _ => unreachable!("Unsupported data type: {:?}", data_type), - }) + } } pub trait ColumnStatisticsAdapter: Send + Sync { @@ -86,7 +154,7 @@ pub trait ColumnStatisticsAdapter: Send + Sync { fn update_value(value: &mut Self::Value, scalar: T::ScalarRef<'_>, ordering: Ordering); } -struct CommonAdapter; +pub struct CommonAdapter; impl ColumnStatisticsAdapter for CommonAdapter where @@ -111,7 +179,7 @@ where } } -struct DecimalAdapter; +pub struct DecimalAdapter; impl ColumnStatisticsAdapter for DecimalAdapter where @@ -137,7 +205,7 @@ where } } -struct GenericColumnStatisticsBuilder +pub struct GenericColumnStatisticsBuilder where T: ValueType, A: ColumnStatisticsAdapter, @@ -158,15 +226,15 @@ where A: ColumnStatisticsAdapter + 'static, for<'a, 'b> T::ScalarRef<'a>: PartialOrd>, { - fn create(data_type: DataType) -> Box { - Box::new(Self { + fn create(data_type: DataType) -> Self { + Self { min: None, max: None, null_count: 0, in_memory_size: 0, data_type, _phantom: PhantomData, - }) + } } fn add_batch<'a, I>(&mut self, mut iter: I) @@ -201,15 +269,7 @@ where self.max = Some(A::scalar_to_value(max)); } } -} -impl ColumnStatisticsBuilder for GenericColumnStatisticsBuilder -where - T: ValueType + Send + Sync, - T::Scalar: Send + Sync, - A: ColumnStatisticsAdapter + 'static, - for<'a, 'b> T::ScalarRef<'a>: PartialOrd>, -{ fn update_column(&mut self, column: &Column) { self.in_memory_size += column.memory_size(); if column.len() == 0 { @@ -265,7 +325,7 @@ where self.add(val.clone(), val); } - fn finalize(self: Box) -> Result { + fn finalize(self) -> Result { let min = if let Some(v) = self.min { let v = A::value_to_scalar(v); // safe upwrap. diff --git a/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs b/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs index 0e246911ea542..6e7b5d0f87704 100644 --- a/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs +++ b/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs @@ -25,12 +25,14 @@ use databend_storages_common_table_meta::meta::StatisticsOfColumns; use crate::io::write::stream::create_column_ndv_estimator; use crate::io::write::stream::create_column_stats_builder; use crate::io::write::stream::ColumnNDVEstimator; +use crate::io::write::stream::ColumnNDVEstimatorOps; use crate::io::write::stream::ColumnStatisticsBuilder; +use crate::io::write::stream::ColumnStatsOps; use crate::statistics::traverse_values_dfs; pub struct ColumnStatisticsState { - col_stats: HashMap>, - distinct_columns: HashMap>, + col_stats: HashMap, + distinct_columns: HashMap, } impl ColumnStatisticsState { diff --git a/src/query/storages/fuse/src/io/write/stream/mod.rs b/src/query/storages/fuse/src/io/write/stream/mod.rs index 3eda792ec7433..f0c7365b5ba01 100644 --- a/src/query/storages/fuse/src/io/write/stream/mod.rs +++ b/src/query/storages/fuse/src/io/write/stream/mod.rs @@ -22,6 +22,8 @@ pub(crate) use block_builder::StreamBlockBuilder; pub(crate) use block_builder::StreamBlockProperties; pub(crate) use column_ndv_estimator::create_column_ndv_estimator; pub(crate) use column_ndv_estimator::ColumnNDVEstimator; +pub(crate) use column_ndv_estimator::ColumnNDVEstimatorOps; pub(crate) use column_statistics_builder::create_column_stats_builder; pub(crate) use column_statistics_builder::ColumnStatisticsBuilder; +pub(crate) use column_statistics_builder::ColumnStatsOps; pub(crate) use column_statistics_state::ColumnStatisticsState; diff --git a/tests/sqllogictests/suites/ee/01_ee_system/01_0002_virtual_column.test b/tests/sqllogictests/suites/ee/01_ee_system/01_0002_virtual_column.test index fe93e45352b5b..9f4d1601f21d0 100644 --- a/tests/sqllogictests/suites/ee/01_ee_system/01_0002_virtual_column.test +++ b/tests/sqllogictests/suites/ee/01_ee_system/01_0002_virtual_column.test @@ -623,9 +623,6 @@ S001 ST001 A Excellent Y S002 ST002 B Good Y S003 ST003 C Average N -statement ok -set enable_block_stream_write = 1 - statement ok CREATE OR REPLACE TABLE test_stream ( id INT, @@ -680,9 +677,6 @@ FROM test_stream; 9 "Richard" 33 "Austin" "hiking" "cycling" 10 "Lisa" 26 "Chicago" "gaming" "reading" -statement ok -set enable_block_stream_write = 0 - statement ok set enable_experimental_virtual_column = 0; diff --git a/tests/suites/1_stateful/05_formats/05_01_compact/05_01_02_load_compact_copy_row_per_block.sh b/tests/suites/1_stateful/05_formats/05_01_compact/05_01_02_load_compact_copy_row_per_block.sh index 4d6cafcb184f9..862680eab447d 100755 --- a/tests/suites/1_stateful/05_formats/05_01_compact/05_01_02_load_compact_copy_row_per_block.sh +++ b/tests/suites/1_stateful/05_formats/05_01_compact/05_01_02_load_compact_copy_row_per_block.sh @@ -15,7 +15,7 @@ echo "drop table if exists t1 all" | $BENDSQL_CLIENT_CONNECT echo "CREATE TABLE t1 ( c0 string -) engine=fuse row_per_block=800; +) engine=fuse row_per_block=500; " | $BENDSQL_CLIENT_CONNECT From a7d9b3cab37fe47b6e65d29dbcdf3176b9ce4242 Mon Sep 17 00:00:00 2001 From: zhyass Date: Mon, 14 Jul 2025 01:27:00 +0800 Subject: [PATCH 03/11] add block statistics --- src/common/metrics/src/metrics/storage.rs | 21 +++ .../storages/fuse/operations/vacuum_table.rs | 58 ++++++- .../fuse/operations/vacuum_table_v2.rs | 3 + .../it/storages/fuse/operations/vacuum.rs | 3 + .../common/table_option_validation.rs | 14 ++ .../interpreters/interpreter_table_create.rs | 4 +- .../interpreter_table_set_options.rs | 2 + .../service/src/test_kits/block_writer.rs | 33 ++++ src/query/service/src/test_kits/check.rs | 12 ++ .../it/storages/fuse/bloom_index_meta_size.rs | 2 + .../it/storages/fuse/meta/column_oriented.rs | 41 ++++- .../it/storages/fuse/operations/analyze.rs | 22 ++- .../tests/it/storages/fuse/operations/gc.rs | 18 ++- .../operations/mutation/recluster_mutator.rs | 2 + .../mutation/segments_compact_mutator.rs | 2 + .../it/storages/fuse/operations/optimize.rs | 4 +- .../it/storages/fuse/operations/purge_drop.rs | 15 ++ .../it/storages/fuse/operations/read_plan.rs | 2 + .../tests/it/storages/fuse/statistics.rs | 2 + .../service/tests/it/storages/fuse/utils.rs | 2 + src/query/settings/src/settings_default.rs | 7 + .../settings/src/settings_getter_setter.rs | 4 + .../sql/src/planner/metadata/hll_columns.rs | 148 ++++++++++++++++++ src/query/sql/src/planner/metadata/mod.rs | 2 + .../storages/common/cache/src/manager.rs | 2 + .../storages/common/index/src/bloom_index.rs | 2 + .../storages/common/index/src/range_index.rs | 6 + .../src/meta/column_oriented_segment/mod.rs | 2 + .../meta/column_oriented_segment/schema.rs | 6 + .../segment_builder.rs | 75 ++++++--- .../common/table_meta/src/meta/current/mod.rs | 1 + .../common/table_meta/src/meta/mod.rs | 1 + .../common/table_meta/src/meta/v2/segment.rs | 12 ++ .../src/meta/v3/frozen/block_meta.rs | 2 + .../src/meta/v3/table_snapshot_statistics.rs | 2 +- .../src/meta/v4/block_statistics.rs | 84 ++++++++++ .../common/table_meta/src/meta/v4/mod.rs | 2 + .../common/table_meta/src/meta/versions.rs | 34 +++- .../src/readers/versioned_reader.rs | 14 ++ .../common/table_meta/src/table/table_keys.rs | 1 + src/query/storages/fuse/src/constants.rs | 1 + src/query/storages/fuse/src/fuse_table.rs | 21 ++- src/query/storages/fuse/src/io/locations.rs | 37 +++++ src/query/storages/fuse/src/io/mod.rs | 1 + .../src/io/write/block_statistics_writer.rs | 116 ++++++++++++++ .../fuse/src/io/write/block_writer.rs | 52 +++++- src/query/storages/fuse/src/io/write/mod.rs | 3 + .../fuse/src/io/write/stream/block_builder.rs | 39 ++++- .../io/write/stream/column_ndv_estimator.rs | 11 +- .../write/stream/column_statistics_state.rs | 92 ++++++++++- .../storages/fuse/src/operations/append.rs | 2 +- .../processors/transform_serialize_block.rs | 5 + src/query/storages/fuse/src/operations/gc.rs | 67 ++++++-- .../storages/fuse/src/operations/merge.rs | 5 + .../fuse/src/statistics/column_statistic.rs | 8 +- .../storages/fuse/src/statistics/traverse.rs | 9 +- .../fuse/src/table_functions/fuse_block.rs | 7 + .../09_0006_func_fuse_history.test | 30 ++++ .../09_http_handler/09_0007_session.py | 26 ++- 59 files changed, 1104 insertions(+), 97 deletions(-) create mode 100644 src/query/sql/src/planner/metadata/hll_columns.rs create mode 100644 src/query/storages/common/table_meta/src/meta/v4/block_statistics.rs create mode 100644 src/query/storages/fuse/src/io/write/block_statistics_writer.rs diff --git a/src/common/metrics/src/metrics/storage.rs b/src/common/metrics/src/metrics/storage.rs index 8059be0b39645..d7ae34d57f9ec 100644 --- a/src/common/metrics/src/metrics/storage.rs +++ b/src/common/metrics/src/metrics/storage.rs @@ -337,6 +337,14 @@ static BLOCK_VIRTUAL_COLUMN_WRITE_MILLISECONDS: LazyLock = LazyLock:: register_histogram_in_milliseconds("fuse_block_virtual_column_write_milliseconds") }); +// Block statistics metrics. +static BLOCK_STATS_WRITE_NUMS: LazyLock = + LazyLock::new(|| register_counter("fuse_block_stats_write_nums")); +static BLOCK_STATS_WRITE_BYTES: LazyLock = + LazyLock::new(|| register_counter("fuse_block_stats_write_bytes")); +static BLOCK_STATS_WRITE_MILLISECONDS: LazyLock = + LazyLock::new(|| register_histogram_in_milliseconds("fuse_block_stats_write_milliseconds")); + /// Common metrics. pub fn metrics_inc_omit_filter_rowgroups(c: u64) { OMIT_FILTER_ROWGROUPS.inc_by(c); @@ -907,3 +915,16 @@ pub fn metrics_inc_block_virtual_column_write_bytes(c: u64) { pub fn metrics_inc_block_virtual_column_write_milliseconds(c: u64) { BLOCK_VIRTUAL_COLUMN_WRITE_MILLISECONDS.observe(c as f64); } + +/// Block stats metrics. +pub fn metrics_inc_block_stats_write_nums(c: u64) { + BLOCK_STATS_WRITE_NUMS.inc_by(c); +} + +pub fn metrics_inc_block_stats_write_bytes(c: u64) { + BLOCK_STATS_WRITE_BYTES.inc_by(c); +} + +pub fn metrics_inc_block_stats_write_milliseconds(c: u64) { + BLOCK_STATS_WRITE_MILLISECONDS.observe(c as f64); +} diff --git a/src/query/ee/src/storages/fuse/operations/vacuum_table.rs b/src/query/ee/src/storages/fuse/operations/vacuum_table.rs index 141f987a8b3e3..19ee34c627efa 100644 --- a/src/query/ee/src/storages/fuse/operations/vacuum_table.rs +++ b/src/query/ee/src/storages/fuse/operations/vacuum_table.rs @@ -40,6 +40,7 @@ pub struct SnapshotReferencedFiles { pub segments: HashSet, pub blocks: HashSet, pub blocks_index: HashSet, + pub blocks_stats: HashSet, } impl SnapshotReferencedFiles { @@ -54,6 +55,9 @@ impl SnapshotReferencedFiles { for file in &self.blocks_index { files.push(file.clone()); } + for file in &self.blocks_stats { + files.push(file.clone()); + } files } } @@ -132,6 +136,7 @@ pub async fn get_snapshot_referenced_files( segments, blocks: locations_referenced.block_location, blocks_index: locations_referenced.bloom_location, + blocks_stats: locations_referenced.stats_location, })) } @@ -164,10 +169,11 @@ pub async fn do_gc_orphan_files( None => return Ok(()), }; let status = format!( - "gc orphan: read referenced files:{},{},{}, cost:{:?}", + "gc orphan: read referenced files:{},{},{},{}, cost:{:?}", referenced_files.segments.len(), referenced_files.blocks.len(), referenced_files.blocks_index.len(), + referenced_files.blocks_stats.len(), start.elapsed() ); ctx.set_status_info(&status); @@ -268,6 +274,36 @@ pub async fn do_gc_orphan_files( ); ctx.set_status_info(&status); + // 5. Purge orphan block stats files. + // 5.1 Get orphan block stats files to be purged + let stats_locations_to_be_purged = get_orphan_files_to_be_purged( + fuse_table, + location_gen.block_statistics_location_prefix(), + referenced_files.blocks_stats, + retention_time, + ) + .await?; + let status = format!( + "gc orphan: read stats_locations_to_be_purged:{}, cost:{:?}", + stats_locations_to_be_purged.len(), + start.elapsed() + ); + ctx.set_status_info(&status); + + // 5.2 Delete all the orphan block stats files to be purged + let purged_file_num = stats_locations_to_be_purged.len(); + fuse_table + .try_purge_location_files( + ctx.clone(), + HashSet::from_iter(stats_locations_to_be_purged.into_iter()), + ) + .await?; + let status = format!( + "gc orphan: purged block stats files:{}, cost:{:?}", + purged_file_num, + start.elapsed() + ); + ctx.set_status_info(&status); Ok(()) } @@ -286,10 +322,11 @@ pub async fn do_dry_run_orphan_files( None => return Ok(()), }; let status = format!( - "dry_run orphan: read referenced files:{},{},{}, cost:{:?}", + "dry_run orphan: read referenced files:{},{},{},{}, cost:{:?}", referenced_files.segments.len(), referenced_files.blocks.len(), referenced_files.blocks_index.len(), + referenced_files.blocks_stats.len(), start.elapsed() ); ctx.set_status_info(&status); @@ -351,6 +388,23 @@ pub async fn do_dry_run_orphan_files( purge_files.extend(index_locations_to_be_purged); + // 5. Get purge orphan block stats files. + let stats_locations_to_be_purged = get_orphan_files_to_be_purged( + fuse_table, + location_gen.block_statistics_location_prefix(), + referenced_files.blocks_stats, + retention_time, + ) + .await?; + let status = format!( + "dry_run orphan: read stats_locations_to_be_purged:{}, cost:{:?}", + stats_locations_to_be_purged.len(), + start.elapsed() + ); + ctx.set_status_info(&status); + + purge_files.extend(stats_locations_to_be_purged); + Ok(()) } diff --git a/src/query/ee/src/storages/fuse/operations/vacuum_table_v2.rs b/src/query/ee/src/storages/fuse/operations/vacuum_table_v2.rs index e03ce27f8466b..da2d43f7f708e 100644 --- a/src/query/ee/src/storages/fuse/operations/vacuum_table_v2.rs +++ b/src/query/ee/src/storages/fuse/operations/vacuum_table_v2.rs @@ -336,6 +336,9 @@ pub async fn do_vacuum2( } indexes_to_gc .push(TableMetaLocationGenerator::gen_bloom_index_location_from_block_location(loc)); + + indexes_to_gc + .push(TableMetaLocationGenerator::gen_block_stats_location_from_block_location(loc)); } ctx.set_status_info(&format!( diff --git a/src/query/ee/tests/it/storages/fuse/operations/vacuum.rs b/src/query/ee/tests/it/storages/fuse/operations/vacuum.rs index 87c2268ceee0d..dfdbbf35266e9 100644 --- a/src/query/ee/tests/it/storages/fuse/operations/vacuum.rs +++ b/src/query/ee/tests/it/storages/fuse/operations/vacuum.rs @@ -66,6 +66,7 @@ async fn test_fuse_do_vacuum_drop_tables() -> Result<()> { 1, 1, 1, + 1, None, None, ) @@ -90,6 +91,7 @@ async fn test_fuse_do_vacuum_drop_tables() -> Result<()> { 1, 1, 1, + 1, None, None, ) @@ -108,6 +110,7 @@ async fn test_fuse_do_vacuum_drop_tables() -> Result<()> { 0, 0, 0, + 0, None, None, ) diff --git a/src/query/service/src/interpreters/common/table_option_validation.rs b/src/query/service/src/interpreters/common/table_option_validation.rs index ab332a1dce642..1a967a1c469a2 100644 --- a/src/query/service/src/interpreters/common/table_option_validation.rs +++ b/src/query/service/src/interpreters/common/table_option_validation.rs @@ -24,6 +24,7 @@ use databend_common_exception::ErrorCode; use databend_common_expression::TableSchemaRef; use databend_common_io::constants::DEFAULT_BLOCK_ROW_COUNT; use databend_common_settings::Settings; +use databend_common_sql::ApproxDistinctColumns; use databend_common_sql::BloomIndexColumns; use databend_common_storages_fuse::FUSE_OPT_KEY_BLOCK_IN_MEM_SIZE_THRESHOLD; use databend_common_storages_fuse::FUSE_OPT_KEY_BLOCK_PER_SEGMENT; @@ -35,6 +36,8 @@ use databend_common_storages_fuse::FUSE_OPT_KEY_ROW_AVG_DEPTH_THRESHOLD; use databend_common_storages_fuse::FUSE_OPT_KEY_ROW_PER_BLOCK; use databend_common_storages_fuse::FUSE_OPT_KEY_ROW_PER_PAGE; use databend_storages_common_index::BloomIndex; +use databend_storages_common_index::RangeIndex; +use databend_storages_common_table_meta::table::OPT_KEY_APPROX_DISTINCT_COLUMNS; use databend_storages_common_table_meta::table::OPT_KEY_BLOOM_INDEX_COLUMNS; use databend_storages_common_table_meta::table::OPT_KEY_CHANGE_TRACKING; use databend_storages_common_table_meta::table::OPT_KEY_CLUSTER_TYPE; @@ -68,6 +71,7 @@ pub static CREATE_FUSE_OPTIONS: LazyLock> = LazyLock::new( r.insert(FUSE_OPT_KEY_ENABLE_AUTO_VACUUM); r.insert(OPT_KEY_BLOOM_INDEX_COLUMNS); + r.insert(OPT_KEY_APPROX_DISTINCT_COLUMNS); r.insert(OPT_KEY_TABLE_COMPRESSION); r.insert(OPT_KEY_STORAGE_FORMAT); r.insert(OPT_KEY_DATABASE_ID); @@ -213,6 +217,16 @@ pub fn is_valid_bloom_index_columns( Ok(()) } +pub fn is_valid_approx_distinct_columns( + options: &BTreeMap, + schema: TableSchemaRef, +) -> databend_common_exception::Result<()> { + if let Some(value) = options.get(OPT_KEY_APPROX_DISTINCT_COLUMNS) { + ApproxDistinctColumns::verify_definition(value, schema, RangeIndex::supported_table_type)?; + } + Ok(()) +} + pub fn is_valid_change_tracking( options: &BTreeMap, ) -> databend_common_exception::Result<()> { diff --git a/src/query/service/src/interpreters/interpreter_table_create.rs b/src/query/service/src/interpreters/interpreter_table_create.rs index 2961b9aae3851..a0d054ee037ed 100644 --- a/src/query/service/src/interpreters/interpreter_table_create.rs +++ b/src/query/service/src/interpreters/interpreter_table_create.rs @@ -66,6 +66,7 @@ use databend_storages_common_table_meta::table::OPT_KEY_TEMP_PREFIX; use log::error; use log::info; +use crate::interpreters::common::table_option_validation::is_valid_approx_distinct_columns; use crate::interpreters::common::table_option_validation::is_valid_block_per_segment; use crate::interpreters::common::table_option_validation::is_valid_bloom_index_columns; use crate::interpreters::common::table_option_validation::is_valid_change_tracking; @@ -470,7 +471,8 @@ impl CreateTableInterpreter { is_valid_block_per_segment(&table_meta.options)?; is_valid_row_per_block(&table_meta.options)?; // check bloom_index_columns. - is_valid_bloom_index_columns(&table_meta.options, schema)?; + is_valid_bloom_index_columns(&table_meta.options, schema.clone())?; + is_valid_approx_distinct_columns(&table_meta.options, schema)?; is_valid_change_tracking(&table_meta.options)?; // check random seed is_valid_random_seed(&table_meta.options)?; diff --git a/src/query/service/src/interpreters/interpreter_table_set_options.rs b/src/query/service/src/interpreters/interpreter_table_set_options.rs index d75f94772cddf..93185f25feeb8 100644 --- a/src/query/service/src/interpreters/interpreter_table_set_options.rs +++ b/src/query/service/src/interpreters/interpreter_table_set_options.rs @@ -49,6 +49,7 @@ use databend_storages_common_table_meta::table::OPT_KEY_STORAGE_FORMAT; use databend_storages_common_table_meta::table::OPT_KEY_TEMP_PREFIX; use log::error; +use crate::interpreters::common::table_option_validation::is_valid_approx_distinct_columns; use crate::interpreters::common::table_option_validation::is_valid_block_per_segment; use crate::interpreters::common::table_option_validation::is_valid_bloom_index_columns; use crate::interpreters::common::table_option_validation::is_valid_create_opt; @@ -163,6 +164,7 @@ impl Interpreter for SetOptionsInterpreter { // check bloom_index_columns. is_valid_bloom_index_columns(&self.plan.set_options, table.schema())?; + is_valid_approx_distinct_columns(&self.plan.set_options, table.schema())?; if let Some(new_snapshot_location) = set_segment_format(self.ctx.clone(), table.clone(), &self.plan.set_options).await? diff --git a/src/query/service/src/test_kits/block_writer.rs b/src/query/service/src/test_kits/block_writer.rs index 6f81b9a8f1dbe..75914348ed1f2 100644 --- a/src/query/service/src/test_kits/block_writer.rs +++ b/src/query/service/src/test_kits/block_writer.rs @@ -19,14 +19,17 @@ use databend_common_expression::FunctionContext; use databend_common_expression::TableSchemaRef; use databend_common_io::constants::DEFAULT_BLOCK_BUFFER_SIZE; use databend_common_io::constants::DEFAULT_BLOCK_INDEX_BUFFER_SIZE; +use databend_common_sql::ApproxDistinctColumns; use databend_common_sql::BloomIndexColumns; use databend_common_storages_fuse::io::serialize_block; +use databend_common_storages_fuse::io::BlockStatisticsState; use databend_common_storages_fuse::io::TableMetaLocationGenerator; use databend_common_storages_fuse::io::WriteSettings; use databend_common_storages_fuse::FuseStorageFormat; use databend_storages_common_blocks::blocks_to_parquet; use databend_storages_common_index::BloomIndex; use databend_storages_common_index::BloomIndexBuilder; +use databend_storages_common_index::RangeIndex; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::ClusterStatistics; use databend_storages_common_table_meta::meta::Compression; @@ -85,6 +88,9 @@ impl<'a> BlockWriter<'a> { let (bloom_filter_index_size, bloom_filter_index_location, meta) = self .build_block_index(data_accessor, schema.clone(), &block, block_id) .await?; + let (block_stats_size, block_stats_location) = self + .build_block_stats(data_accessor, schema.clone(), &block, block_id) + .await?; let write_settings = WriteSettings { storage_format, @@ -112,6 +118,8 @@ impl<'a> BlockWriter<'a> { None, None, None, + block_stats_location, + block_stats_size, Compression::Lz4Raw, Some(Utc::now()), ); @@ -154,4 +162,29 @@ impl<'a> BlockWriter<'a> { Ok((0u64, None, None)) } } + + pub async fn build_block_stats( + &self, + data_accessor: &Operator, + schema: TableSchemaRef, + block: &DataBlock, + block_id: Uuid, + ) -> Result<(u64, Option)> { + let location = self.location_generator.block_stats_location(&block_id); + + let hll_columns = ApproxDistinctColumns::All; + let ndv_columns_map = + hll_columns.distinct_column_fields(schema.clone(), RangeIndex::supported_table_type)?; + let maybe_block_stats = + BlockStatisticsState::from_data_block(location, block, &ndv_columns_map)?; + if let Some(block_stats) = maybe_block_stats { + let size = block_stats.block_stats_size(); + data_accessor + .write(&block_stats.location.0, block_stats.data) + .await?; + Ok((size, Some(block_stats.location))) + } else { + Ok((0u64, None)) + } + } } diff --git a/src/query/service/src/test_kits/check.rs b/src/query/service/src/test_kits/check.rs index a8a82cf60727a..6943810c7d949 100644 --- a/src/query/service/src/test_kits/check.rs +++ b/src/query/service/src/test_kits/check.rs @@ -24,6 +24,7 @@ use databend_common_meta_app::storage::StorageParams; use databend_common_storages_fuse::operations::load_last_snapshot_hint; use databend_common_storages_fuse::FuseTable; use databend_common_storages_fuse::FUSE_TBL_BLOCK_PREFIX; +use databend_common_storages_fuse::FUSE_TBL_BLOCK_STATISTICS_PREFIX; use databend_common_storages_fuse::FUSE_TBL_SEGMENT_PREFIX; use databend_common_storages_fuse::FUSE_TBL_SNAPSHOT_PREFIX; use databend_common_storages_fuse::FUSE_TBL_SNAPSHOT_STATISTICS_PREFIX; @@ -79,6 +80,7 @@ pub async fn check_data_dir( segment_count: u32, block_count: u32, index_count: u32, + block_stat_count: u32, check_last_snapshot: Option<()>, check_table_statistic_file: Option<()>, ) -> Result<()> { @@ -92,12 +94,14 @@ pub async fn check_data_dir( let mut sg_count = 0; let mut b_count = 0; let mut i_count = 0; + let mut b_stat_count = 0; let mut table_statistic_files = vec![]; let prefix_snapshot = FUSE_TBL_SNAPSHOT_PREFIX; let prefix_snapshot_statistics = FUSE_TBL_SNAPSHOT_STATISTICS_PREFIX; let prefix_segment = FUSE_TBL_SEGMENT_PREFIX; let prefix_block = FUSE_TBL_BLOCK_PREFIX; let prefix_index = FUSE_TBL_XOR_BLOOM_INDEX_PREFIX; + let prefix_block_stats = FUSE_TBL_BLOCK_STATISTICS_PREFIX; for entry in WalkDir::new(root) { let entry = entry.unwrap(); if entry.file_type().is_file() { @@ -109,6 +113,8 @@ pub async fn check_data_dir( ss_count += 1; } else if path.starts_with(prefix_segment) { sg_count += 1; + } else if path.starts_with(prefix_block_stats) { + b_stat_count += 1; } else if path.starts_with(prefix_block) { b_count += 1; } else if path.starts_with(prefix_index) { @@ -142,6 +148,12 @@ pub async fn check_data_dir( case_name ); + assert_eq!( + b_stat_count, block_stat_count, + "case [{}], check block statistics count", + case_name + ); + assert_eq!( i_count, index_count, "case [{}], check index count", diff --git a/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs b/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs index 217f171356d3f..bc5ecc4e76978 100644 --- a/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs +++ b/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs @@ -340,6 +340,8 @@ fn build_test_segment_info( vector_index_size: None, vector_index_location: None, virtual_block_meta: None, + block_stats_location: None, + block_stats_size: 0, compression: Compression::Lz4, create_on: Some(Utc::now()), }; diff --git a/src/query/service/tests/it/storages/fuse/meta/column_oriented.rs b/src/query/service/tests/it/storages/fuse/meta/column_oriented.rs index 0a4d46b8c6f40..dc50928f2b4e1 100644 --- a/src/query/service/tests/it/storages/fuse/meta/column_oriented.rs +++ b/src/query/service/tests/it/storages/fuse/meta/column_oriented.rs @@ -288,6 +288,39 @@ fn check_block_level_meta( assert!(is_null); } + // check block stats location + let block_stats_location = column_oriented_segment + .col_by_name(&[BLOCK_STATS_LOCATION]) + .unwrap(); + for (block_stats_location, block_meta) in block_stats_location.iter().zip(block_metas.iter()) { + let block_stats_location = block_stats_location.as_tuple(); + if let Some(block_stats_location) = block_stats_location { + assert_eq!( + block_stats_location[0].as_string().unwrap(), + &block_meta.block_stats_location.as_ref().unwrap().0 + ); + assert_eq!( + block_stats_location[1] + .as_number() + .unwrap() + .as_u_int64() + .unwrap(), + &block_meta.block_stats_location.as_ref().unwrap().1 + ); + } else { + assert!(block_meta.block_stats_location.is_none()); + } + } + + // check block stats size + let block_stats_size = column_oriented_segment + .col_by_name(&[BLOCK_STATS_SIZE]) + .unwrap(); + for (block_stats_size, block_meta) in block_stats_size.iter().zip(block_metas.iter()) { + let block_stats_size = block_stats_size.as_number().unwrap().as_u_int64().unwrap(); + assert_eq!(block_stats_size, &block_meta.block_stats_size); + } + // check compression let compression = column_oriented_segment.col_by_name(&[COMPRESSION]).unwrap(); for (compression, block_meta) in compression.iter().zip(block_metas.iter()) { @@ -372,7 +405,7 @@ async fn test_segment_cache() -> Result<()> { ) .await?; let cached = cache.get(&location).unwrap(); - assert_eq!(cached.segment_schema.fields.len(), 10); + assert_eq!(cached.segment_schema.fields.len(), 12); assert_eq!(cached.segment_schema, segment_schema(&TableSchema::empty())); check_summary(&block_metas, &cached); check_block_level_meta(&block_metas, &cached); @@ -385,7 +418,7 @@ async fn test_segment_cache() -> Result<()> { let _column_oriented_segment = read_column_oriented_segment(operator.clone(), &location, &projection, true).await?; let cached = cache.get(&location).unwrap(); - assert_eq!(cached.segment_schema.fields.len(), 12); + assert_eq!(cached.segment_schema.fields.len(), 14); let column_1 = table_schema.field_of_column_id(col_id).unwrap(); let stat_1 = column_oriented_segment @@ -409,7 +442,7 @@ async fn test_segment_cache() -> Result<()> { read_column_oriented_segment(operator.clone(), &location, &projection, true).await?; let cached = cache.get(&location).unwrap(); // column 2 does not have stats - assert_eq!(cached.segment_schema.fields.len(), 13); + assert_eq!(cached.segment_schema.fields.len(), 15); check_summary(&block_metas, &cached); check_block_level_meta(&block_metas, &cached); check_column_stats_and_meta(&block_metas, &cached, &[1, 2]); @@ -423,7 +456,7 @@ async fn test_segment_cache() -> Result<()> { read_column_oriented_segment(operator.clone(), &location, &projection, true).await?; let cached = cache.get(&location).unwrap(); // column 2 does not have stats - assert_eq!(cached.segment_schema.fields.len(), 13); + assert_eq!(cached.segment_schema.fields.len(), 15); check_summary(&block_metas, &cached); check_block_level_meta(&block_metas, &cached); check_column_stats_and_meta(&block_metas, &cached, &[1, 2]); diff --git a/src/query/service/tests/it/storages/fuse/operations/analyze.rs b/src/query/service/tests/it/storages/fuse/operations/analyze.rs index 2a23d26433b5b..0b5ee07035bc5 100644 --- a/src/query/service/tests/it/storages/fuse/operations/analyze.rs +++ b/src/query/service/tests/it/storages/fuse/operations/analyze.rs @@ -32,7 +32,7 @@ async fn test_fuse_snapshot_analyze() -> Result<()> { do_insertions(&fixture).await?; analyze_table(&fixture).await?; - check_data_dir(&fixture, case_name, 3, 1, 2, 2, 2, Some(()), None).await?; + check_data_dir(&fixture, case_name, 3, 1, 2, 2, 2, 2, Some(()), None).await?; // Purge will keep at least two snapshots. let table = fixture.latest_default_table().await?; @@ -42,7 +42,7 @@ async fn test_fuse_snapshot_analyze() -> Result<()> { fuse_table .do_purge(&table_ctx, snapshot_files, None, true, false) .await?; - check_data_dir(&fixture, case_name, 1, 1, 1, 1, 1, Some(()), Some(())).await?; + check_data_dir(&fixture, case_name, 1, 1, 1, 1, 1, 1, Some(()), Some(())).await?; Ok(()) } @@ -65,7 +65,7 @@ async fn test_fuse_snapshot_analyze_and_truncate() -> Result<()> { fixture.execute_command(&qry).await?; - check_data_dir(&fixture, case_name, 3, 1, 2, 2, 2, None, Some(())).await?; + check_data_dir(&fixture, case_name, 3, 1, 2, 2, 2, 2, None, Some(())).await?; } // truncate table @@ -103,7 +103,19 @@ async fn test_fuse_snapshot_analyze_purge() -> Result<()> { // optimize statistics three times for i in 0..3 { analyze_table(&fixture).await?; - check_data_dir(&fixture, case_name, 3 + i, 1 + i, 2, 2, 2, Some(()), None).await?; + check_data_dir( + &fixture, + case_name, + 3 + i, + 1 + i, + 2, + 2, + 2, + 2, + Some(()), + None, + ) + .await?; } // Purge will keep at least two snapshots. @@ -114,7 +126,7 @@ async fn test_fuse_snapshot_analyze_purge() -> Result<()> { fuse_table .do_purge(&table_ctx, snapshot_files, None, true, false) .await?; - check_data_dir(&fixture, case_name, 1, 1, 1, 1, 1, Some(()), Some(())).await?; + check_data_dir(&fixture, case_name, 1, 1, 1, 1, 1, 1, Some(()), Some(())).await?; Ok(()) } diff --git a/src/query/service/tests/it/storages/fuse/operations/gc.rs b/src/query/service/tests/it/storages/fuse/operations/gc.rs index 24202424686f6..bb60f0dbf240f 100644 --- a/src/query/service/tests/it/storages/fuse/operations/gc.rs +++ b/src/query/service/tests/it/storages/fuse/operations/gc.rs @@ -59,6 +59,7 @@ async fn test_fuse_purge_normal_case() -> Result<()> { 1, // 1 segments 1, // 1 blocks 1, // 1 index + 1, // 1 block statistic Some(()), None, ) @@ -118,9 +119,10 @@ async fn test_fuse_purge_normal_orphan_snapshot() -> Result<()> { "do_gc: there should be 1 snapshot, 0 segment/block", expected_num_of_snapshot, 0, // 0 snapshot statistic - 1, // 0 segments - 1, // 0 blocks - 1, // 0 index + 1, // 1 segments + 1, // 1 blocks + 1, // 1 index + 1, // 1 block statistic Some(()), None, ) @@ -250,6 +252,7 @@ async fn test_fuse_purge_orphan_retention() -> Result<()> { let expected_num_of_segment = 3; let expected_num_of_blocks = 3; let expected_num_of_index = expected_num_of_blocks; + let expected_num_of_block_stats = expected_num_of_blocks; check_data_dir( &fixture, "do_gc: verify retention period", @@ -258,6 +261,7 @@ async fn test_fuse_purge_orphan_retention() -> Result<()> { expected_num_of_segment, expected_num_of_blocks, expected_num_of_index, + expected_num_of_block_stats, Some(()), None, ) @@ -296,6 +300,7 @@ async fn test_fuse_purge_older_version() -> Result<()> { let expected_num_of_segment = 3; let expected_num_of_blocks = 6; let expected_num_of_index = expected_num_of_blocks; + let expected_num_of_block_stats = expected_num_of_blocks; check_data_dir( &fixture, "do_gc: navigate to time point", @@ -304,6 +309,7 @@ async fn test_fuse_purge_older_version() -> Result<()> { expected_num_of_segment, expected_num_of_blocks, expected_num_of_index, + expected_num_of_block_stats, Some(()), None, ) @@ -317,7 +323,7 @@ async fn test_fuse_purge_older_version() -> Result<()> { { let table = fixture.latest_default_table().await?; compact_segment(ctx.clone(), &table).await?; - check_data_dir(&fixture, "", 4, 0, 5, 7, 7, Some(()), None).await?; + check_data_dir(&fixture, "", 4, 0, 5, 7, 7, 7, Some(()), None).await?; } let table = fixture.latest_default_table().await?; @@ -333,6 +339,7 @@ async fn test_fuse_purge_older_version() -> Result<()> { let expected_num_of_segment = 1; let expected_num_of_blocks = 7; let expected_num_of_index = expected_num_of_blocks; + let expected_num_of_block_stats = expected_num_of_blocks; check_data_dir( &fixture, "do_gc: with older version", @@ -341,6 +348,7 @@ async fn test_fuse_purge_older_version() -> Result<()> { expected_num_of_segment, expected_num_of_blocks, expected_num_of_index, + expected_num_of_block_stats, Some(()), None, ) @@ -357,6 +365,7 @@ async fn test_fuse_purge_older_version() -> Result<()> { let expected_num_of_segment = 0; let expected_num_of_blocks = 0; let expected_num_of_index = expected_num_of_blocks; + let expected_num_of_block_stats = expected_num_of_blocks; check_data_dir( &fixture, "do_gc: purge last snapshot", @@ -365,6 +374,7 @@ async fn test_fuse_purge_older_version() -> Result<()> { expected_num_of_segment, expected_num_of_blocks, expected_num_of_index, + expected_num_of_block_stats, Some(()), None, ) diff --git a/src/query/service/tests/it/storages/fuse/operations/mutation/recluster_mutator.rs b/src/query/service/tests/it/storages/fuse/operations/mutation/recluster_mutator.rs index 97c3e9cf85f0d..8d0ec47d7f19e 100644 --- a/src/query/service/tests/it/storages/fuse/operations/mutation/recluster_mutator.rs +++ b/src/query/service/tests/it/storages/fuse/operations/mutation/recluster_mutator.rs @@ -81,6 +81,8 @@ async fn test_recluster_mutator_block_select() -> Result<()> { None, None, None, + None, + 0, meta::Compression::Lz4Raw, Some(Utc::now()), )); diff --git a/src/query/service/tests/it/storages/fuse/operations/mutation/segments_compact_mutator.rs b/src/query/service/tests/it/storages/fuse/operations/mutation/segments_compact_mutator.rs index ebdcabfd13662..8387d3c7a2ace 100644 --- a/src/query/service/tests/it/storages/fuse/operations/mutation/segments_compact_mutator.rs +++ b/src/query/service/tests/it/storages/fuse/operations/mutation/segments_compact_mutator.rs @@ -781,6 +781,8 @@ impl CompactSegmentTestFixture { None, None, None, + None, + 0, Compression::Lz4Raw, Some(Utc::now()), ); diff --git a/src/query/service/tests/it/storages/fuse/operations/optimize.rs b/src/query/service/tests/it/storages/fuse/operations/optimize.rs index 02f300cf3680d..5470aa4853b8d 100644 --- a/src/query/service/tests/it/storages/fuse/operations/optimize.rs +++ b/src/query/service/tests/it/storages/fuse/operations/optimize.rs @@ -24,12 +24,12 @@ use crate::storages::fuse::utils::do_purge_test; #[tokio::test(flavor = "multi_thread")] async fn test_fuse_snapshot_optimize_purge() -> Result<()> { - do_purge_test("test_fuse_snapshot_optimize_purge", 1, 0, 1, 1, 1).await + do_purge_test("test_fuse_snapshot_optimize_purge", 1, 0, 1, 1, 1, 1).await } #[tokio::test(flavor = "multi_thread")] async fn test_fuse_snapshot_optimize_all() -> Result<()> { - do_purge_test("test_fuse_snapshot_optimize_all", 1, 0, 1, 1, 1).await + do_purge_test("test_fuse_snapshot_optimize_all", 1, 0, 1, 1, 1, 1).await } #[tokio::test(flavor = "multi_thread")] diff --git a/src/query/service/tests/it/storages/fuse/operations/purge_drop.rs b/src/query/service/tests/it/storages/fuse/operations/purge_drop.rs index 75c8bb405a844..54b4a535c9946 100644 --- a/src/query/service/tests/it/storages/fuse/operations/purge_drop.rs +++ b/src/query/service/tests/it/storages/fuse/operations/purge_drop.rs @@ -46,6 +46,20 @@ async fn test_fuse_snapshot_truncate_in_drop_all_stmt() -> Result<()> { // ingests some test data append_sample_data(1, &fixture).await?; + check_data_dir( + &fixture, + "drop table: there should be 1 snapshot, 0 segment/block", + 1, // 1 snapshot + 0, // 0 snapshot statistic + 1, // 0 segments + 1, // 0 blocks + 1, // 0 index + 1, // 0 block statistic + None, + None, + ) + .await?; + // let's Drop let qry = format!("drop table {}.{} all", db, tbl); fixture.execute_command(qry.as_str()).await?; @@ -58,6 +72,7 @@ async fn test_fuse_snapshot_truncate_in_drop_all_stmt() -> Result<()> { 0, // 0 segments 0, // 0 blocks 0, // 0 index + 0, // 0 block statistic None, None, ) diff --git a/src/query/service/tests/it/storages/fuse/operations/read_plan.rs b/src/query/service/tests/it/storages/fuse/operations/read_plan.rs index 09dccdb11816e..eb2d64ad67465 100644 --- a/src/query/service/tests/it/storages/fuse/operations/read_plan.rs +++ b/src/query/service/tests/it/storages/fuse/operations/read_plan.rs @@ -107,6 +107,8 @@ fn test_to_partitions() -> Result<()> { None, None, None, + None, + 0, meta::Compression::Lz4Raw, Some(Utc::now()), )); diff --git a/src/query/service/tests/it/storages/fuse/statistics.rs b/src/query/service/tests/it/storages/fuse/statistics.rs index c87b524f9217f..4bcfc5bf9cf6d 100644 --- a/src/query/service/tests/it/storages/fuse/statistics.rs +++ b/src/query/service/tests/it/storages/fuse/statistics.rs @@ -637,6 +637,8 @@ fn test_reduce_block_meta() -> databend_common_exception::Result<()> { None, None, None, + None, + 0, Compression::Lz4Raw, Some(Utc::now()), ); diff --git a/src/query/service/tests/it/storages/fuse/utils.rs b/src/query/service/tests/it/storages/fuse/utils.rs index 4e29ed6594705..2dbf9d36de9dc 100644 --- a/src/query/service/tests/it/storages/fuse/utils.rs +++ b/src/query/service/tests/it/storages/fuse/utils.rs @@ -54,6 +54,7 @@ pub async fn do_purge_test( segment_count: u32, block_count: u32, index_count: u32, + block_stat_count: u32, ) -> Result<()> { let fixture = TestFixture::setup().await?; fixture.create_default_database().await?; @@ -81,6 +82,7 @@ pub async fn do_purge_test( segment_count, block_count, index_count, + block_stat_count, Some(()), None, ) diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 2f60028b130bf..c00553fb3a397 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -1361,6 +1361,13 @@ impl DefaultSettings { scope: SettingScope::Both, range: None, }), + ("enable_block_stream_write", DefaultSettingValue { + value: UserSettingValue::UInt64(1), + desc: "Enables block stream write", + mode: SettingMode::Both, + scope: SettingScope::Both, + range: Some(SettingRange::Numeric(0..=1)), + }), ("trace_sample_rate", DefaultSettingValue { value: UserSettingValue::UInt64(1), desc: "Setting the trace sample rate. The value should be between '0' and '100'", diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index 7c3e35d83ed8c..98023b4356390 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -998,6 +998,10 @@ impl Settings { self.set_setting("optimizer_skip_list".to_string(), v) } + pub fn get_enable_block_stream_write(&self) -> Result { + Ok(self.try_get_u64("enable_block_stream_write")? == 1) + } + pub fn get_statement_queue_ttl_in_seconds(&self) -> Result { self.try_get_u64("statement_queue_ttl_in_seconds") } diff --git a/src/query/sql/src/planner/metadata/hll_columns.rs b/src/query/sql/src/planner/metadata/hll_columns.rs new file mode 100644 index 0000000000000..ec708a5744153 --- /dev/null +++ b/src/query/sql/src/planner/metadata/hll_columns.rs @@ -0,0 +1,148 @@ +// Copyright 2021 Datafuse Labs +// +// 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::BTreeMap; +use std::str::FromStr; + +use databend_common_ast::parser::parse_comma_separated_idents; +use databend_common_ast::parser::tokenize_sql; +use databend_common_ast::parser::Dialect; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::ComputedExpr; +use databend_common_expression::FieldIndex; +use databend_common_expression::TableDataType; +use databend_common_expression::TableField; +use databend_common_expression::TableSchemaRef; +use databend_common_meta_app::tenant::Tenant; +use databend_common_settings::Settings; + +use crate::normalize_identifier; +use crate::NameResolutionContext; + +#[derive(Clone)] +pub enum ApproxDistinctColumns { + /// Default, all columns that support distinct columns. + All, + /// Specify with column names. + Specify(Vec), + /// The column of distinct is empty. + None, +} + +impl FromStr for ApproxDistinctColumns { + type Err = ErrorCode; + + fn from_str(s: &str) -> std::result::Result { + let s = s.trim(); + if s.is_empty() { + return Ok(ApproxDistinctColumns::None); + } + + let sql_dialect = Dialect::default(); + let tokens = tokenize_sql(s)?; + let idents = parse_comma_separated_idents(&tokens, sql_dialect)?; + + let settings = Settings::create(Tenant::new_literal("dummy")); + let name_resolution_ctx = NameResolutionContext::try_from(settings.as_ref())?; + + let mut cols = Vec::with_capacity(idents.len()); + idents + .into_iter() + .for_each(|ident| cols.push(normalize_identifier(&ident, &name_resolution_ctx).name)); + + Ok(ApproxDistinctColumns::Specify(cols)) + } +} + +impl ApproxDistinctColumns { + /// Verify the definition based on schema. + pub fn verify_definition( + definition: &str, + schema: TableSchemaRef, + verify_type: F, + ) -> Result<()> + where + F: Fn(&TableDataType) -> bool, + { + let definition = definition.trim(); + if definition.is_empty() { + return Ok(()); + } + + let settings = Settings::create(Tenant::new_literal("dummy")); + let name_resolution_ctx = NameResolutionContext::try_from(settings.as_ref())?; + + let sql_dialect = Dialect::default(); + let tokens = tokenize_sql(definition)?; + let idents = parse_comma_separated_idents(&tokens, sql_dialect)?; + for ident in idents.iter() { + let name = &normalize_identifier(ident, &name_resolution_ctx).name; + let field = schema.field_with_name(name)?; + + if matches!(field.computed_expr(), Some(ComputedExpr::Virtual(_))) { + return Err(ErrorCode::TableOptionInvalid(format!( + "The value specified for computed column '{}' is not allowed for distinct columns", + name + ))); + } + + let data_type = field.data_type(); + if !verify_type(data_type) { + return Err(ErrorCode::TableOptionInvalid(format!( + "Unsupported data type '{}' for distinct columns", + data_type + ))); + } + } + Ok(()) + } + + pub fn distinct_column_fields( + &self, + schema: TableSchemaRef, + verify_type: F, + ) -> Result> + where + F: Fn(&TableDataType) -> bool, + { + let source_schema = schema.remove_virtual_computed_fields(); + let mut fields_map = BTreeMap::new(); + match self { + ApproxDistinctColumns::All => { + for (i, field) in source_schema.fields.into_iter().enumerate() { + if verify_type(field.data_type()) { + fields_map.insert(i, field); + } + } + } + ApproxDistinctColumns::Specify(cols) => { + for col in cols { + let field_index = source_schema.index_of(col)?; + let field = source_schema.fields[field_index].clone(); + let data_type = field.data_type(); + if !verify_type(data_type) { + return Err(ErrorCode::BadArguments(format!( + "Unsupported data type for distinct columns: {:?}", + data_type + ))); + } + fields_map.insert(field_index, field); + } + } + ApproxDistinctColumns::None => (), + } + Ok(fields_map) + } +} diff --git a/src/query/sql/src/planner/metadata/mod.rs b/src/query/sql/src/planner/metadata/mod.rs index f3de756113350..e37aa7b078812 100644 --- a/src/query/sql/src/planner/metadata/mod.rs +++ b/src/query/sql/src/planner/metadata/mod.rs @@ -13,8 +13,10 @@ // limitations under the License. mod bloom_index; +mod hll_columns; #[allow(clippy::module_inception)] mod metadata; pub use bloom_index::BloomIndexColumns; +pub use hll_columns::ApproxDistinctColumns; pub use metadata::*; diff --git a/src/query/storages/common/cache/src/manager.rs b/src/query/storages/common/cache/src/manager.rs index 2e4aeb6b0fb6b..1b8432a52ce98 100644 --- a/src/query/storages/common/cache/src/manager.rs +++ b/src/query/storages/common/cache/src/manager.rs @@ -1033,6 +1033,8 @@ mod tests { vector_index_location: None, vector_index_size: None, virtual_block_meta: None, + block_stats_location: None, + block_stats_size: 0, compression: Compression::Lz4, create_on: None, }); diff --git a/src/query/storages/common/index/src/bloom_index.rs b/src/query/storages/common/index/src/bloom_index.rs index 59d06f6298a49..3909071a03540 100644 --- a/src/query/storages/common/index/src/bloom_index.rs +++ b/src/query/storages/common/index/src/bloom_index.rs @@ -827,6 +827,8 @@ impl BloomIndexBuilder { index_column.builder.add_digests(digests.iter()) } } + // reverse sorting. + bloom_keys_to_remove.sort_by(|a, b| b.cmp(a)); for k in bloom_keys_to_remove { self.bloom_columns.remove(k); } diff --git a/src/query/storages/common/index/src/range_index.rs b/src/query/storages/common/index/src/range_index.rs index 4f23fcef61bee..60d4da9655cc4 100644 --- a/src/query/storages/common/index/src/range_index.rs +++ b/src/query/storages/common/index/src/range_index.rs @@ -42,6 +42,7 @@ use databend_common_expression::Domain; use databend_common_expression::Expr; use databend_common_expression::FunctionContext; use databend_common_expression::Scalar; +use databend_common_expression::TableDataType; use databend_common_expression::TableSchemaRef; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_storages_common_table_meta::meta::ColumnStatistics; @@ -169,6 +170,11 @@ impl RangeIndex { } .apply(stats, |_| false) } + + pub fn supported_table_type(data_type: &TableDataType) -> bool { + let data_type = DataType::from(data_type); + Self::supported_type(&data_type) + } } pub fn statistics_to_domain(mut stats: Vec<&ColumnStatistics>, data_type: &DataType) -> Domain { diff --git a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/mod.rs b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/mod.rs index 86453a71543fc..21787350577ee 100644 --- a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/mod.rs +++ b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/mod.rs @@ -26,6 +26,8 @@ pub use schema::meta_name; pub use schema::segment_schema; pub use schema::stat_name; pub use schema::BLOCK_SIZE; +pub use schema::BLOCK_STATS_LOCATION; +pub use schema::BLOCK_STATS_SIZE; pub use schema::BLOOM_FILTER_INDEX_LOCATION; pub use schema::BLOOM_FILTER_INDEX_SIZE; pub use schema::CLUSTER_STATS; diff --git a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/schema.rs b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/schema.rs index dbba08ce7c622..daea76a037897 100644 --- a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/schema.rs +++ b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/schema.rs @@ -34,6 +34,8 @@ pub const COMPRESSION: &str = "compression"; pub const CREATE_ON: &str = "create_on"; pub const LOCATION_PATH: &str = "path"; pub const LOCATION_FORMAT_VERSION: &str = "format_version"; +pub const BLOCK_STATS_LOCATION: &str = "block_stats_location"; +pub const BLOCK_STATS_SIZE: &str = "block_stats_size"; pub fn block_level_field_names() -> HashSet { let mut set = HashSet::new(); @@ -45,6 +47,8 @@ pub fn block_level_field_names() -> HashSet { set.insert(BLOOM_FILTER_INDEX_LOCATION.to_string()); set.insert(BLOOM_FILTER_INDEX_SIZE.to_string()); set.insert(INVERTED_INDEX_SIZE.to_string()); + set.insert(BLOCK_STATS_LOCATION.to_string()); + set.insert(BLOCK_STATS_SIZE.to_string()); set.insert(COMPRESSION.to_string()); set.insert(CREATE_ON.to_string()); set @@ -130,6 +134,8 @@ pub fn segment_schema(table_schema: &TableSchema) -> TableSchema { TableField::new(BLOOM_FILTER_INDEX_LOCATION, nullable_location_type()), TableField::new(BLOOM_FILTER_INDEX_SIZE, u64_t.clone()), TableField::new(INVERTED_INDEX_SIZE, nullable_u64_t.clone()), + TableField::new(BLOCK_STATS_LOCATION, nullable_location_type()), + TableField::new(BLOCK_STATS_SIZE, u64_t.clone()), TableField::new(COMPRESSION, u8_t.clone()), TableField::new(CREATE_ON, i64_t.clone()), ]; diff --git a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/segment_builder.rs b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/segment_builder.rs index 657f4796cd3a8..917fc6e935de6 100644 --- a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/segment_builder.rs +++ b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/segment_builder.rs @@ -45,6 +45,7 @@ use crate::meta::supported_stat_type; use crate::meta::BlockMeta; use crate::meta::ClusterStatistics; use crate::meta::ColumnStatistics; +use crate::meta::Location; use crate::meta::MetaEncoding; use crate::meta::Statistics; use crate::meta::VirtualBlockMeta; @@ -67,10 +68,12 @@ pub struct ColumnOrientedSegmentBuilder { file_size: Vec, cluster_stats: Vec>, location: (Vec, Vec), - bloom_filter_index_location: (Vec, Vec, MutableBitmap), + bloom_filter_index_location: LocationsWithOption, bloom_filter_index_size: Vec, inverted_index_size: Vec>, virtual_block_meta: Vec>, + block_stats_location: LocationsWithOption, + block_stats_size: Vec, compression: Vec, create_on: Vec>, column_stats: HashMap, @@ -127,28 +130,16 @@ impl SegmentBuilder for ColumnOrientedSegmentBuilder { self.cluster_stats.push(block_meta.cluster_stats); self.location.0.push(block_meta.location.0); self.location.1.push(block_meta.location.1); - self.bloom_filter_index_location.0.push( - block_meta - .bloom_filter_index_location - .as_ref() - .map(|l| l.0.clone()) - .unwrap_or_default(), - ); - self.bloom_filter_index_location.1.push( - block_meta - .bloom_filter_index_location - .as_ref() - .map(|l| l.1) - .unwrap_or_default(), - ); self.bloom_filter_index_location - .2 - .push(block_meta.bloom_filter_index_location.is_some()); + .add_location(block_meta.bloom_filter_index_location.as_ref()); self.bloom_filter_index_size .push(block_meta.bloom_filter_index_size); self.inverted_index_size .push(block_meta.inverted_index_size); self.virtual_block_meta.push(block_meta.virtual_block_meta); + self.block_stats_location + .add_location(block_meta.block_stats_location.as_ref()); + self.block_stats_size.push(block_meta.block_stats_size); self.compression.push(block_meta.compression.to_u8()); self.create_on .push(block_meta.create_on.map(|t| t.timestamp())); @@ -199,13 +190,21 @@ impl SegmentBuilder for ColumnOrientedSegmentBuilder { ]), Column::Nullable(Box::new(NullableColumn::new( Column::Tuple(vec![ - StringType::from_data(this.bloom_filter_index_location.0), - UInt64Type::from_data(this.bloom_filter_index_location.1), + StringType::from_data(this.bloom_filter_index_location.locations), + UInt64Type::from_data(this.bloom_filter_index_location.versions), ]), - this.bloom_filter_index_location.2.into(), + this.bloom_filter_index_location.validity.into(), ))), UInt64Type::from_data(this.bloom_filter_index_size), UInt64Type::from_opt_data(this.inverted_index_size), + Column::Nullable(Box::new(NullableColumn::new( + Column::Tuple(vec![ + StringType::from_data(this.block_stats_location.locations), + UInt64Type::from_data(this.block_stats_location.versions), + ]), + this.block_stats_location.validity.into(), + ))), + UInt64Type::from_data(this.block_stats_size), UInt8Type::from_data(this.compression), Int64Type::from_opt_data(this.create_on), ]; @@ -264,14 +263,12 @@ impl SegmentBuilder for ColumnOrientedSegmentBuilder { Vec::with_capacity(block_per_segment), Vec::with_capacity(block_per_segment), ), - bloom_filter_index_location: ( - Vec::with_capacity(block_per_segment), - Vec::with_capacity(block_per_segment), - MutableBitmap::with_capacity(block_per_segment), - ), + bloom_filter_index_location: LocationsWithOption::new_with_capacity(block_per_segment), bloom_filter_index_size: Vec::with_capacity(block_per_segment), inverted_index_size: Vec::with_capacity(block_per_segment), virtual_block_meta: Vec::with_capacity(block_per_segment), + block_stats_location: LocationsWithOption::new_with_capacity(block_per_segment), + block_stats_size: Vec::with_capacity(block_per_segment), compression: Vec::with_capacity(block_per_segment), create_on: Vec::with_capacity(block_per_segment), column_stats, @@ -439,3 +436,31 @@ fn cmp_with_null(v1: &Scalar, v2: &Scalar) -> Ordering { (false, false) => v1.cmp(v2), } } + +struct LocationsWithOption { + locations: Vec, + versions: Vec, + validity: MutableBitmap, +} + +impl LocationsWithOption { + fn new_with_capacity(capacity: usize) -> Self { + Self { + locations: Vec::with_capacity(capacity), + versions: Vec::with_capacity(capacity), + validity: MutableBitmap::with_capacity(capacity), + } + } + + fn add_location(&mut self, location: Option<&Location>) { + if let Some(location) = location { + self.locations.push(location.0.clone()); + self.versions.push(location.1); + self.validity.push(true); + } else { + self.locations.push(String::new()); + self.versions.push(0); + self.validity.push(false); + } + } +} diff --git a/src/query/storages/common/table_meta/src/meta/current/mod.rs b/src/query/storages/common/table_meta/src/meta/current/mod.rs index 2bb5544e0ace9..cc254a566e671 100644 --- a/src/query/storages/common/table_meta/src/meta/current/mod.rs +++ b/src/query/storages/common/table_meta/src/meta/current/mod.rs @@ -25,6 +25,7 @@ pub use v2::Statistics; pub use v2::VirtualBlockMeta; pub use v2::VirtualColumnMeta; pub use v3::TableSnapshotStatistics; +pub use v4::BlockStatistics; pub use v4::CompactSegmentInfo; pub use v4::RawBlockMeta; pub use v4::SegmentInfo; diff --git a/src/query/storages/common/table_meta/src/meta/mod.rs b/src/query/storages/common/table_meta/src/meta/mod.rs index 68e3bfd3e0bce..3c9133ec4fdca 100644 --- a/src/query/storages/common/table_meta/src/meta/mod.rs +++ b/src/query/storages/common/table_meta/src/meta/mod.rs @@ -49,6 +49,7 @@ pub use utils::VACUUM2_OBJECT_KEY_PREFIX; pub(crate) use utils::*; pub use v0::ColumnMeta as ColumnMetaV0; pub use versions::testify_version; +pub use versions::BlockStatisticsVersion; pub use versions::SegmentInfoVersion; pub use versions::SnapshotVersion; pub use versions::TableSnapshotStatisticsVersion; diff --git a/src/query/storages/common/table_meta/src/meta/v2/segment.rs b/src/query/storages/common/table_meta/src/meta/v2/segment.rs index 47c2127145d60..f9b0bcbd403bd 100644 --- a/src/query/storages/common/table_meta/src/meta/v2/segment.rs +++ b/src/query/storages/common/table_meta/src/meta/v2/segment.rs @@ -176,6 +176,10 @@ pub struct BlockMeta { pub virtual_block_meta: Option, pub compression: Compression, + pub block_stats_location: Option, + #[serde(default)] + pub block_stats_size: u64, + // block create_on pub create_on: Option>, } @@ -197,6 +201,8 @@ impl BlockMeta { vector_index_size: Option, vector_index_location: Option, virtual_block_meta: Option, + block_stats_location: Option, + block_stats_size: u64, compression: Compression, create_on: Option>, ) -> Self { @@ -215,6 +221,8 @@ impl BlockMeta { vector_index_size, vector_index_location, virtual_block_meta, + block_stats_location, + block_stats_size, compression, create_on, } @@ -377,6 +385,8 @@ impl BlockMeta { vector_index_size: None, vector_index_location: None, virtual_block_meta: None, + block_stats_location: None, + block_stats_size: 0, create_on: None, ngram_filter_index_size: None, } @@ -405,6 +415,8 @@ impl BlockMeta { vector_index_size: None, vector_index_location: None, virtual_block_meta: None, + block_stats_location: None, + block_stats_size: 0, create_on: None, ngram_filter_index_size: None, } diff --git a/src/query/storages/common/table_meta/src/meta/v3/frozen/block_meta.rs b/src/query/storages/common/table_meta/src/meta/v3/frozen/block_meta.rs index ccc22fdbdb600..a02b2761848f6 100644 --- a/src/query/storages/common/table_meta/src/meta/v3/frozen/block_meta.rs +++ b/src/query/storages/common/table_meta/src/meta/v3/frozen/block_meta.rs @@ -66,6 +66,8 @@ impl From for crate::meta::BlockMeta { vector_index_size: None, vector_index_location: None, virtual_block_meta: None, + block_stats_location: None, + block_stats_size: 0, compression: value.compression.into(), create_on: None, } diff --git a/src/query/storages/common/table_meta/src/meta/v3/table_snapshot_statistics.rs b/src/query/storages/common/table_meta/src/meta/v3/table_snapshot_statistics.rs index 55841a886e03c..0689e4a687dfd 100644 --- a/src/query/storages/common/table_meta/src/meta/v3/table_snapshot_statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/v3/table_snapshot_statistics.rs @@ -29,7 +29,7 @@ pub type MetaHLL = simple_hll::HyperLogLog<12>; #[derive(Serialize, Deserialize, Clone, Debug)] pub struct TableSnapshotStatistics { - /// format version of snapshot + /// format version of statistics pub format_version: FormatVersion, /// id of snapshot diff --git a/src/query/storages/common/table_meta/src/meta/v4/block_statistics.rs b/src/query/storages/common/table_meta/src/meta/v4/block_statistics.rs new file mode 100644 index 0000000000000..1ac9703376d5c --- /dev/null +++ b/src/query/storages/common/table_meta/src/meta/v4/block_statistics.rs @@ -0,0 +1,84 @@ +// Copyright 2021 Datafuse Labs +// +// 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::io::Cursor; +use std::io::Read; + +use databend_common_exception::Result; +use databend_common_expression::ColumnId; +use databend_common_io::prelude::BinaryRead; +use serde::Deserialize; +use serde::Serialize; + +use crate::meta::format::compress; +use crate::meta::format::encode; +use crate::meta::format::read_and_deserialize; +use crate::meta::versions::Versioned; +use crate::meta::ColumnDistinctHLL; +use crate::meta::FormatVersion; +use crate::meta::MetaCompression; +use crate::meta::MetaEncoding; + +#[derive(Serialize, Deserialize, Clone, Debug)] +pub struct BlockStatistics { + pub format_version: FormatVersion, + + pub hll: HashMap, +} + +impl BlockStatistics { + pub fn new(hll: HashMap) -> Self { + Self { + format_version: BlockStatistics::VERSION, + hll, + } + } + + pub fn to_bytes(&self) -> Result> { + let encoding = MetaEncoding::MessagePack; + let compression = MetaCompression::default(); + + let data = encode(&encoding, &self)?; + let data_compress = compress(&compression, data)?; + + let data_size = self.format_version.to_le_bytes().len() + + 2 + + data_compress.len().to_le_bytes().len() + + data_compress.len(); + let mut buf = Vec::with_capacity(data_size); + + buf.extend_from_slice(&self.format_version.to_le_bytes()); + buf.push(encoding as u8); + buf.push(compression as u8); + buf.extend_from_slice(&data_compress.len().to_le_bytes()); + + buf.extend(data_compress); + + Ok(buf) + } + + pub fn from_slice(buffer: &[u8]) -> Result { + Self::from_read(Cursor::new(buffer)) + } + + pub fn from_read(mut r: impl Read) -> Result { + let version = r.read_scalar::()?; + assert_eq!(version, BlockStatistics::VERSION); + let encoding = MetaEncoding::try_from(r.read_scalar::()?)?; + let compression = MetaCompression::try_from(r.read_scalar::()?)?; + let statistics_size: u64 = r.read_scalar::()?; + read_and_deserialize(&mut r, statistics_size, &encoding, &compression) + } +} diff --git a/src/query/storages/common/table_meta/src/meta/v4/mod.rs b/src/query/storages/common/table_meta/src/meta/v4/mod.rs index 6a596b9ec8807..cf73718ea7c29 100644 --- a/src/query/storages/common/table_meta/src/meta/v4/mod.rs +++ b/src/query/storages/common/table_meta/src/meta/v4/mod.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod block_statistics; mod segment; mod snapshot; +pub use block_statistics::BlockStatistics; pub use segment::CompactSegmentInfo; pub use segment::RawBlockMeta; pub use segment::SegmentInfo; diff --git a/src/query/storages/common/table_meta/src/meta/versions.rs b/src/query/storages/common/table_meta/src/meta/versions.rs index 71493750481b3..670ce5c73805c 100644 --- a/src/query/storages/common/table_meta/src/meta/versions.rs +++ b/src/query/storages/common/table_meta/src/meta/versions.rs @@ -128,6 +128,24 @@ impl TableSnapshotStatisticsVersion { } } +impl Versioned<0> for v4::BlockStatistics {} + +pub enum BlockStatisticsVersion { + V0(PhantomData), +} + +impl BlockStatisticsVersion { + pub fn version(&self) -> u64 { + match self { + BlockStatisticsVersion::V0(a) => Self::ver(a), + } + } + + fn ver>(_v: &PhantomData) -> u64 { + V + } +} + /// Statically check that if T implements Versioned where U equals V #[inline] pub fn testify_version(t: PhantomData) -> PhantomData @@ -189,7 +207,21 @@ mod converters { PhantomData, ))), _ => Err(ErrorCode::Internal(format!( - "unknown table snapshot statistics version {value}, versions supported: 0" + "unknown table snapshot statistics version {value}, versions supported: 0, 2, 3" + ))), + } + } + } + + impl TryFrom for BlockStatisticsVersion { + type Error = ErrorCode; + fn try_from(value: u64) -> Result { + match value { + 0 => Ok(BlockStatisticsVersion::V0(testify_version::<_, 0>( + PhantomData, + ))), + _ => Err(ErrorCode::Internal(format!( + "unknown block statistics version {value}, versions supported: 0" ))), } } diff --git a/src/query/storages/common/table_meta/src/readers/versioned_reader.rs b/src/query/storages/common/table_meta/src/readers/versioned_reader.rs index a2ef7efec9558..a67c796d7eda1 100644 --- a/src/query/storages/common/table_meta/src/readers/versioned_reader.rs +++ b/src/query/storages/common/table_meta/src/readers/versioned_reader.rs @@ -17,6 +17,8 @@ use std::io::Read; use databend_common_exception::Result; use crate::meta::load_json; +use crate::meta::BlockStatistics; +use crate::meta::BlockStatisticsVersion; use crate::meta::TableSnapshotStatistics; use crate::meta::TableSnapshotStatisticsVersion; @@ -46,3 +48,15 @@ impl VersionedReader for TableSnapshotStatisticsVersion Ok(r) } } + +impl VersionedReader for BlockStatisticsVersion { + type TargetType = BlockStatistics; + + fn read(&self, reader: R) -> Result + where R: Read + Unpin + Send { + let r = match self { + BlockStatisticsVersion::V0(_) => BlockStatistics::from_read(reader)?, + }; + Ok(r) + } +} diff --git a/src/query/storages/common/table_meta/src/table/table_keys.rs b/src/query/storages/common/table_meta/src/table/table_keys.rs index 9a75763f47080..fddc120058b74 100644 --- a/src/query/storages/common/table_meta/src/table/table_keys.rs +++ b/src/query/storages/common/table_meta/src/table/table_keys.rs @@ -27,6 +27,7 @@ pub const OPT_KEY_TABLE_COMPRESSION: &str = "compression"; pub const OPT_KEY_COMMENT: &str = "comment"; pub const OPT_KEY_ENGINE: &str = "engine"; pub const OPT_KEY_BLOOM_INDEX_COLUMNS: &str = "bloom_index_columns"; +pub const OPT_KEY_APPROX_DISTINCT_COLUMNS: &str = "approx_distinct_columns"; pub const OPT_KEY_CHANGE_TRACKING: &str = "change_tracking"; pub const OPT_KEY_CHANGE_TRACKING_BEGIN_VER: &str = "begin_version"; diff --git a/src/query/storages/fuse/src/constants.rs b/src/query/storages/fuse/src/constants.rs index f16b4975939ad..bc95d3fe14a48 100644 --- a/src/query/storages/fuse/src/constants.rs +++ b/src/query/storages/fuse/src/constants.rs @@ -31,6 +31,7 @@ pub const FUSE_TBL_XOR_BLOOM_INDEX_PREFIX: &str = "_i_b_v2"; pub const FUSE_TBL_SEGMENT_PREFIX: &str = "_sg"; pub const FUSE_TBL_SNAPSHOT_PREFIX: &str = "_ss"; pub const FUSE_TBL_SNAPSHOT_STATISTICS_PREFIX: &str = "_ts"; +pub const FUSE_TBL_BLOCK_STATISTICS_PREFIX: &str = "_bs"; pub const FUSE_TBL_LAST_SNAPSHOT_HINT: &str = "last_snapshot_location_hint"; pub const FUSE_TBL_LAST_SNAPSHOT_HINT_V2: &str = "last_snapshot_location_hint_v2"; pub const FUSE_TBL_VIRTUAL_BLOCK_PREFIX: &str = "_vb"; diff --git a/src/query/storages/fuse/src/fuse_table.rs b/src/query/storages/fuse/src/fuse_table.rs index ad9a99afc3666..b8d105c4dd45a 100644 --- a/src/query/storages/fuse/src/fuse_table.rs +++ b/src/query/storages/fuse/src/fuse_table.rs @@ -69,6 +69,7 @@ use databend_common_pipeline_core::Pipeline; use databend_common_sql::binder::STREAM_COLUMN_FACTORY; use databend_common_sql::parse_cluster_keys; use databend_common_sql::plans::TruncateMode; +use databend_common_sql::ApproxDistinctColumns; use databend_common_sql::BloomIndexColumns; use databend_common_storage::init_operator; use databend_common_storage::DataOperator; @@ -87,6 +88,7 @@ use databend_storages_common_table_meta::meta::Versioned; use databend_storages_common_table_meta::table::ChangeType; use databend_storages_common_table_meta::table::ClusterType; use databend_storages_common_table_meta::table::TableCompression; +use databend_storages_common_table_meta::table::OPT_KEY_APPROX_DISTINCT_COLUMNS; use databend_storages_common_table_meta::table::OPT_KEY_BLOOM_INDEX_COLUMNS; use databend_storages_common_table_meta::table::OPT_KEY_CHANGE_TRACKING; use databend_storages_common_table_meta::table::OPT_KEY_CLUSTER_TYPE; @@ -141,6 +143,7 @@ pub struct FuseTable { pub(crate) segment_format: FuseSegmentFormat, pub(crate) table_compression: TableCompression, pub(crate) bloom_index_cols: BloomIndexColumns, + pub(crate) approx_distinct_cols: ApproxDistinctColumns, pub(crate) operator: Operator, pub(crate) data_metrics: Arc, @@ -234,6 +237,12 @@ impl FuseTable { .and_then(|s| s.parse::().ok()) .unwrap_or(BloomIndexColumns::All); + let approx_distinct_cols = table_info + .options() + .get(OPT_KEY_APPROX_DISTINCT_COLUMNS) + .and_then(|s| s.parse::().ok()) + .unwrap_or(ApproxDistinctColumns::All); + let meta_location_generator = TableMetaLocationGenerator::new(storage_prefix); if !table_info.meta.part_prefix.is_empty() { return Err(ErrorCode::StorageOther( @@ -246,6 +255,7 @@ impl FuseTable { meta_location_generator, cluster_key_meta, bloom_index_cols, + approx_distinct_cols, operator, data_metrics, storage_format: FuseStorageFormat::from_str(storage_format.as_str())?, @@ -460,6 +470,10 @@ impl FuseTable { self.bloom_index_cols.clone() } + pub fn approx_distinct_cols(&self) -> ApproxDistinctColumns { + self.approx_distinct_cols.clone() + } + // Check if table is attached. pub fn is_table_attached(table_meta_options: &BTreeMap) -> bool { table_meta_options @@ -746,11 +760,12 @@ impl FuseTable { }) } - pub fn enable_stream_block_write(&self) -> bool { - matches!(self.storage_format, FuseStorageFormat::Parquet) + pub fn enable_stream_block_write(&self, ctx: Arc) -> Result { + Ok(ctx.get_settings().get_enable_block_stream_write()? + && matches!(self.storage_format, FuseStorageFormat::Parquet) && self .cluster_type() - .is_none_or(|v| matches!(v, ClusterType::Hilbert)) + .is_none_or(|v| matches!(v, ClusterType::Hilbert))) } } diff --git a/src/query/storages/fuse/src/io/locations.rs b/src/query/storages/fuse/src/io/locations.rs index 13272ee77706c..76de95a868b9c 100644 --- a/src/query/storages/fuse/src/io/locations.rs +++ b/src/query/storages/fuse/src/io/locations.rs @@ -18,6 +18,7 @@ use databend_common_exception::Result; use databend_common_expression::DataBlock; use databend_storages_common_table_meta::meta::trim_object_prefix; use databend_storages_common_table_meta::meta::uuid_from_date_time; +use databend_storages_common_table_meta::meta::BlockStatistics; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::SnapshotVersion; @@ -36,6 +37,7 @@ use crate::constants::FUSE_TBL_VIRTUAL_BLOCK_PREFIX; use crate::index::filters::BlockFilter; use crate::index::InvertedIndexFile; use crate::FUSE_TBL_AGG_INDEX_PREFIX; +use crate::FUSE_TBL_BLOCK_STATISTICS_PREFIX; use crate::FUSE_TBL_INVERTED_INDEX_PREFIX; use crate::FUSE_TBL_LAST_SNAPSHOT_HINT_V2; use crate::FUSE_TBL_VECTOR_INDEX_PREFIX; @@ -65,6 +67,7 @@ pub struct TableMetaLocationGenerator { agg_index_location_prefix: String, inverted_index_location_prefix: String, vector_index_location_prefix: String, + block_statistics_location_prefix: String, } impl TableMetaLocationGenerator { @@ -78,6 +81,8 @@ impl TableMetaLocationGenerator { let inverted_index_location_prefix = format!("{}/{}/", &prefix, FUSE_TBL_INVERTED_INDEX_PREFIX); let vector_index_location_prefix = format!("{}/{}/", &prefix, FUSE_TBL_VECTOR_INDEX_PREFIX); + let block_statistics_location_prefix = + format!("{}/{}/", &prefix, FUSE_TBL_BLOCK_STATISTICS_PREFIX); Self { prefix, block_location_prefix, @@ -87,6 +92,7 @@ impl TableMetaLocationGenerator { agg_index_location_prefix, inverted_index_location_prefix, vector_index_location_prefix, + block_statistics_location_prefix, } } @@ -114,6 +120,10 @@ impl TableMetaLocationGenerator { &self.snapshot_location_prefix } + pub fn block_statistics_location_prefix(&self) -> &str { + &self.block_statistics_location_prefix + } + pub fn gen_block_location( &self, table_meta_timestamps: TableMetaTimestamps, @@ -142,6 +152,18 @@ impl TableMetaLocationGenerator { ) } + pub fn block_stats_location(&self, block_id: &Uuid) -> Location { + ( + format!( + "{}{}_v{}.parquet", + self.block_statistics_location_prefix(), + block_id.as_simple(), + BlockStatistics::VERSION, + ), + BlockStatistics::VERSION, + ) + } + pub fn block_vector_index_location(&self) -> Location { let uuid = Uuid::now_v7(); ( @@ -296,6 +318,21 @@ impl TableMetaLocationGenerator { BlockFilter::VERSION, ) } + + pub fn gen_block_stats_location_from_block_location(loc: &str) -> String { + let splits = loc.split('/').collect::>(); + let len = splits.len(); + let prefix = splits[..len - 2].join("/"); + let block_name = trim_object_prefix(splits[len - 1]); + let id: String = block_name.chars().take(32).collect(); + format!( + "{}/{}/{}_v{}.mpk", + prefix, + FUSE_TBL_BLOCK_STATISTICS_PREFIX, + id, + BlockStatistics::VERSION, + ) + } } trait SnapshotLocationCreator { diff --git a/src/query/storages/fuse/src/io/mod.rs b/src/query/storages/fuse/src/io/mod.rs index 94d22a40bbb90..24fc70297a0cd 100644 --- a/src/query/storages/fuse/src/io/mod.rs +++ b/src/query/storages/fuse/src/io/mod.rs @@ -43,6 +43,7 @@ pub use write::serialize_block; pub use write::write_data; pub use write::BlockBuilder; pub use write::BlockSerialization; +pub use write::BlockStatisticsState; pub use write::BlockWriter; pub use write::BloomIndexRebuilder; pub use write::BloomIndexState; diff --git a/src/query/storages/fuse/src/io/write/block_statistics_writer.rs b/src/query/storages/fuse/src/io/write/block_statistics_writer.rs new file mode 100644 index 0000000000000..d6f2fb777cadc --- /dev/null +++ b/src/query/storages/fuse/src/io/write/block_statistics_writer.rs @@ -0,0 +1,116 @@ +// Copyright 2021 Datafuse Labs +// +// 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::BTreeMap; +use std::collections::HashMap; + +use databend_common_exception::Result; +use databend_common_expression::BlockEntry; +use databend_common_expression::ColumnId; +use databend_common_expression::DataBlock; +use databend_common_expression::FieldIndex; +use databend_common_expression::TableField; +use databend_storages_common_table_meta::meta::BlockStatistics; +use databend_storages_common_table_meta::meta::Location; + +use crate::io::write::stream::create_column_ndv_estimator; +use crate::io::write::stream::ColumnNDVEstimator; +use crate::io::write::stream::ColumnNDVEstimatorOps; + +#[derive(Debug)] +pub struct BlockStatisticsState { + pub data: Vec, + pub location: Location, + pub column_distinct_count: HashMap, +} + +impl BlockStatisticsState { + pub fn from_data_block( + location: Location, + block: &DataBlock, + ndv_columns_map: &BTreeMap, + ) -> Result> { + let mut builder = BlockStatsBuilder::new(ndv_columns_map); + builder.add_block(block)?; + builder.finalize(location) + } + + pub fn block_stats_size(&self) -> u64 { + self.data.len() as u64 + } +} + +pub struct BlockStatsBuilder { + builders: Vec, +} + +pub struct ColumnNDVBuilder { + index: FieldIndex, + field: TableField, + builder: ColumnNDVEstimator, +} + +impl BlockStatsBuilder { + pub fn new(ndv_columns_map: &BTreeMap) -> BlockStatsBuilder { + let mut builders = Vec::with_capacity(ndv_columns_map.len()); + for (index, field) in ndv_columns_map { + let builder = create_column_ndv_estimator(&field.data_type().into()); + builders.push(ColumnNDVBuilder { + index: *index, + field: field.clone(), + builder, + }); + } + BlockStatsBuilder { builders } + } + + pub fn add_block(&mut self, block: &DataBlock) -> Result<()> { + for column_builder in self.builders.iter_mut() { + let entry = block.get_by_offset(column_builder.index); + match entry { + BlockEntry::Const(s, ..) => { + column_builder.builder.update_scalar(&s.as_ref()); + } + BlockEntry::Column(col) => { + column_builder.builder.update_column(col); + } + } + } + Ok(()) + } + + pub fn finalize(self, location: Location) -> Result> { + if self.builders.is_empty() { + return Ok(None); + } + + let mut hlls = HashMap::with_capacity(self.builders.len()); + let mut column_distinct_count = HashMap::with_capacity(self.builders.len()); + for column_builder in self.builders { + let column_id = column_builder.field.column_id(); + let distinct_count = column_builder.builder.finalize(); + let hll = column_builder.builder.hll(); + hlls.insert(column_id, hll); + column_distinct_count.insert(column_id, distinct_count); + } + + let block_stats = BlockStatistics::new(hlls); + let data = block_stats.to_bytes()?; + Ok(Some(BlockStatisticsState { + data, + location, + column_distinct_count, + })) + } +} diff --git a/src/query/storages/fuse/src/io/write/block_writer.rs b/src/query/storages/fuse/src/io/write/block_writer.rs index fdfbd02320568..007285f691530 100644 --- a/src/query/storages/fuse/src/io/write/block_writer.rs +++ b/src/query/storages/fuse/src/io/write/block_writer.rs @@ -34,6 +34,9 @@ use databend_common_metrics::storage::metrics_inc_block_index_write_nums; use databend_common_metrics::storage::metrics_inc_block_inverted_index_write_bytes; use databend_common_metrics::storage::metrics_inc_block_inverted_index_write_milliseconds; use databend_common_metrics::storage::metrics_inc_block_inverted_index_write_nums; +use databend_common_metrics::storage::metrics_inc_block_stats_write_bytes; +use databend_common_metrics::storage::metrics_inc_block_stats_write_milliseconds; +use databend_common_metrics::storage::metrics_inc_block_stats_write_nums; use databend_common_metrics::storage::metrics_inc_block_vector_index_write_bytes; use databend_common_metrics::storage::metrics_inc_block_vector_index_write_milliseconds; use databend_common_metrics::storage::metrics_inc_block_vector_index_write_nums; @@ -55,6 +58,7 @@ use opendal::Operator; use crate::io::write::virtual_column_builder::VirtualColumnBuilder; use crate::io::write::virtual_column_builder::VirtualColumnState; +use crate::io::write::BlockStatisticsState; use crate::io::write::InvertedIndexBuilder; use crate::io::write::InvertedIndexState; use crate::io::write::VectorIndexBuilder; @@ -139,6 +143,7 @@ pub struct BlockSerialization { pub inverted_index_states: Vec, pub virtual_column_state: Option, pub vector_index_state: Option, + pub block_stats_state: Option, } local_block_meta_serde!(BlockSerialization); @@ -154,6 +159,7 @@ pub struct BlockBuilder { pub write_settings: WriteSettings, pub cluster_stats_gen: ClusterStatsGenerator, pub bloom_columns_map: BTreeMap, + pub ndv_columns_map: BTreeMap, pub ngram_args: Vec, pub inverted_index_builders: Vec, pub virtual_column_builder: Option, @@ -178,9 +184,20 @@ impl BlockBuilder { self.bloom_columns_map.clone(), &self.ngram_args, )?; - let column_distinct_count = bloom_index_state + let mut column_distinct_count = bloom_index_state .as_ref() - .map(|i| i.column_distinct_count.clone()); + .map(|i| i.column_distinct_count.clone()) + .unwrap_or_default(); + + let block_stats_location = self.meta_locations.block_stats_location(&block_id); + let block_stats_state = BlockStatisticsState::from_data_block( + block_stats_location, + &data_block, + &self.ndv_columns_map, + )?; + if let Some(block_stats_state) = &block_stats_state { + column_distinct_count.extend(block_stats_state.column_distinct_count.clone()); + } let mut inverted_index_states = Vec::with_capacity(self.inverted_index_builders.len()); for inverted_index_builder in &self.inverted_index_builders { @@ -214,8 +231,11 @@ impl BlockBuilder { }; let row_count = data_block.num_rows() as u64; - let col_stats = - gen_columns_statistics(&data_block, column_distinct_count, &self.source_schema)?; + let col_stats = gen_columns_statistics( + &data_block, + Some(column_distinct_count), + &self.source_schema, + )?; let mut buffer = Vec::with_capacity(DEFAULT_BLOCK_BUFFER_SIZE); let block_size = data_block.estimate_block_size() as u64; @@ -254,6 +274,10 @@ impl BlockBuilder { compression: self.write_settings.table_compression.into(), inverted_index_size, virtual_block_meta: None, + block_stats_location: block_stats_state.as_ref().map(|v| v.location.clone()), + block_stats_size: block_stats_state + .as_ref() + .map_or(0, |v| v.block_stats_size()), create_on: Some(Utc::now()), }; @@ -264,6 +288,7 @@ impl BlockBuilder { inverted_index_states, virtual_column_state, vector_index_state, + block_stats_state, }; Ok(serialized) } @@ -298,6 +323,7 @@ impl BlockWriter { Self::write_down_vector_index_state(dal, serialized.vector_index_state).await?; Self::write_down_inverted_index_state(dal, serialized.inverted_index_states).await?; Self::write_down_virtual_column_state(dal, serialized.virtual_column_state).await?; + Self::write_down_block_stats_state(dal, serialized.block_stats_state).await?; Ok(extended_block_meta) } @@ -399,4 +425,22 @@ impl BlockWriter { } Ok(()) } + + pub async fn write_down_block_stats_state( + dal: &Operator, + block_stats_state: Option, + ) -> Result<()> { + if let Some(block_stats_state) = block_stats_state { + let start = Instant::now(); + + let stats_size = block_stats_state.block_stats_size(); + let location = &block_stats_state.location.0; + write_data(block_stats_state.data, dal, location).await?; + + metrics_inc_block_stats_write_nums(1); + metrics_inc_block_stats_write_bytes(stats_size); + metrics_inc_block_stats_write_milliseconds(start.elapsed().as_millis() as u64); + } + Ok(()) + } } diff --git a/src/query/storages/fuse/src/io/write/mod.rs b/src/query/storages/fuse/src/io/write/mod.rs index e7f3bfbe82c2f..544d32d988988 100644 --- a/src/query/storages/fuse/src/io/write/mod.rs +++ b/src/query/storages/fuse/src/io/write/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod block_statistics_writer; mod block_writer; mod bloom_index_writer; mod inverted_index_writer; @@ -21,6 +22,8 @@ mod vector_index_writer; mod virtual_column_builder; mod write_settings; +pub use block_statistics_writer::BlockStatisticsState; +pub use block_statistics_writer::BlockStatsBuilder; pub use block_writer::serialize_block; pub use block_writer::write_data; pub use block_writer::BlockBuilder; diff --git a/src/query/storages/fuse/src/io/write/stream/block_builder.rs b/src/query/storages/fuse/src/io/write/stream/block_builder.rs index e049f53c646b9..ea9c3ca42ed00 100644 --- a/src/query/storages/fuse/src/io/write/stream/block_builder.rs +++ b/src/query/storages/fuse/src/io/write/stream/block_builder.rs @@ -32,7 +32,6 @@ use databend_common_expression::FieldIndex; use databend_common_expression::TableField; use databend_common_expression::TableSchema; use databend_common_expression::TableSchemaRef; -use databend_common_expression::ORIGIN_BLOCK_ROW_NUM_COLUMN_ID; use databend_common_io::constants::DEFAULT_BLOCK_BUFFER_SIZE; use databend_common_meta_app::schema::TableIndex; use databend_common_native::write::NativeWriter; @@ -56,6 +55,7 @@ use crate::io::create_inverted_index_builders; use crate::io::write::stream::cluster_statistics::ClusterStatisticsBuilder; use crate::io::write::stream::cluster_statistics::ClusterStatisticsState; use crate::io::write::stream::ColumnStatisticsState; +use crate::io::write::BlockStatsBuilder; use crate::io::write::InvertedIndexState; use crate::io::BlockSerialization; use crate::io::BloomIndexState; @@ -71,6 +71,7 @@ use crate::FuseTable; pub enum BlockWriterImpl { Arrow(ArrowWriter>), + // Native format doesnot support stream write. Native(NativeWriter>), } @@ -155,6 +156,7 @@ pub struct StreamBlockBuilder { bloom_index_builder: BloomIndexBuilder, virtual_column_builder: Option, vector_index_builder: Option, + block_stats_builder: BlockStatsBuilder, cluster_stats_state: ClusterStatisticsState, column_stats_state: ColumnStatisticsState, @@ -224,6 +226,7 @@ impl StreamBlockBuilder { &properties.table_indexes, properties.source_schema.clone(), ); + let block_stats_builder = BlockStatsBuilder::new(&properties.ndv_columns_map); let cluster_stats_state = ClusterStatisticsState::new(properties.cluster_stats_builder.clone()); let column_stats_state = @@ -236,6 +239,7 @@ impl StreamBlockBuilder { bloom_index_builder, virtual_column_builder, vector_index_builder, + block_stats_builder, row_count: 0, block_size: 0, column_stats_state, @@ -268,6 +272,7 @@ impl StreamBlockBuilder { self.column_stats_state .add_block(&self.properties.source_schema, &block)?; self.bloom_index_builder.add_block(&block)?; + self.block_stats_builder.add_block(&block)?; for writer in self.inverted_index_writers.iter_mut() { writer.add_block(&self.properties.source_schema, &block)?; } @@ -303,11 +308,21 @@ impl StreamBlockBuilder { } else { None }; - let column_distinct_count = bloom_index_state + let bloom_distinct_count = bloom_index_state .as_ref() .map(|i| i.column_distinct_count.clone()) .unwrap_or_default(); - let col_stats = self.column_stats_state.finalize(column_distinct_count)?; + let block_stats_location = self + .properties + .meta_locations + .block_stats_location(&block_id); + let block_stats_state = self.block_stats_builder.finalize(block_stats_location)?; + let hll_distinct_count = block_stats_state + .as_ref() + .map_or(HashMap::new(), |i| i.column_distinct_count.clone()); + let col_stats = self + .column_stats_state + .finalize(bloom_distinct_count, hll_distinct_count)?; let mut inverted_index_states = Vec::with_capacity(self.inverted_index_writers.len()); for (i, inverted_index_writer) in std::mem::take(&mut self.inverted_index_writers) @@ -378,6 +393,10 @@ impl StreamBlockBuilder { .map(|v| v.ngram_size) .unwrap_or_default(), virtual_block_meta: None, + block_stats_location: block_stats_state.as_ref().map(|v| v.location.clone()), + block_stats_size: block_stats_state + .as_ref() + .map_or(0, |v| v.block_stats_size()), }; let serialized = BlockSerialization { block_raw_data, @@ -386,6 +405,7 @@ impl StreamBlockBuilder { inverted_index_states, virtual_column_state, vector_index_state, + block_stats_state, }; Ok(serialized) } @@ -403,6 +423,7 @@ pub struct StreamBlockProperties { stats_columns: Vec<(ColumnId, DataType)>, distinct_columns: Vec<(ColumnId, DataType)>, bloom_columns_map: BTreeMap, + ndv_columns_map: BTreeMap, ngram_args: Vec, inverted_index_builders: Vec, virtual_column_builder: Option, @@ -446,8 +467,12 @@ impl StreamBlockProperties { &table.table_info.meta, &table.table_info.meta.schema, )?; - let bloom_column_ids = bloom_columns_map + let ndv_columns_map = table + .approx_distinct_cols + .distinct_column_fields(source_schema.clone(), RangeIndex::supported_table_type)?; + let bloom_ndv_columns = bloom_columns_map .values() + .chain(ndv_columns_map.values()) .map(|v| v.column_id()) .collect::>(); @@ -472,10 +497,9 @@ impl StreamBlockProperties { for field in leaf_fields.iter() { let column_id = field.column_id(); let data_type = DataType::from(field.data_type()); - if RangeIndex::supported_type(&data_type) && column_id != ORIGIN_BLOCK_ROW_NUM_COLUMN_ID - { + if RangeIndex::supported_type(&data_type) { stats_columns.push((column_id, data_type.clone())); - if !bloom_column_ids.contains(&column_id) { + if !bloom_ndv_columns.contains(&column_id) { distinct_columns.push((column_id, data_type)); } } @@ -496,6 +520,7 @@ impl StreamBlockProperties { inverted_index_builders, table_meta_timestamps, table_indexes, + ndv_columns_map, })) } } diff --git a/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs b/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs index 4410f06feba42..e2fa21f07786c 100644 --- a/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs +++ b/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs @@ -46,7 +46,8 @@ use enum_dispatch::enum_dispatch; pub trait ColumnNDVEstimatorOps: Send + Sync { fn update_column(&mut self, column: &Column); fn update_scalar(&mut self, scalar: &ScalarRef); - fn finalize(&self) -> u64; + fn finalize(&self) -> usize; + fn hll(self) -> ColumnDistinctHLL; } #[enum_dispatch(ColumnNDVEstimatorOps)] @@ -176,7 +177,11 @@ where self.hll.add_object(&val); } - fn finalize(&self) -> u64 { - self.hll.count() as u64 + fn finalize(&self) -> usize { + self.hll.count() + } + + fn hll(self) -> ColumnDistinctHLL { + self.hll } } diff --git a/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs b/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs index 6e7b5d0f87704..ff7895a83353f 100644 --- a/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs +++ b/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs @@ -88,21 +88,29 @@ impl ColumnStatisticsState { pub fn finalize( self, - column_distinct_count: HashMap, + bloom_distinct_count: HashMap, + hll_distinct_count: HashMap, ) -> Result { + let mut distinct_count = + HashMap::with_capacity(self.distinct_columns.len() + hll_distinct_count.len()); + for (column_id, estimator) in &self.distinct_columns { + distinct_count.insert(*column_id, estimator.finalize()); + } + distinct_count.extend(hll_distinct_count); + let mut statistics = StatisticsOfColumns::with_capacity(self.col_stats.len()); for (id, stats) in self.col_stats { let mut col_stats = stats.finalize()?; - if let Some(count) = column_distinct_count.get(&id) { + if let Some(count) = distinct_count.get(&id) { + col_stats.distinct_of_values = Some(*count as u64); + } else if let Some(&count) = bloom_distinct_count.get(&id) { // value calculated by xor hash function include NULL, need to subtract one. - let distinct_of_values = if col_stats.null_count > 0 { - *count as u64 - 1 + let distinct_of_values = if col_stats.null_count > 0 && count > 0 { + count as u64 - 1 } else { - *count as u64 + count as u64 }; col_stats.distinct_of_values = Some(distinct_of_values); - } else if let Some(estimator) = self.distinct_columns.get(&id) { - col_stats.distinct_of_values = Some(estimator.finalize()); } else if col_stats.min == col_stats.max { // Bloom index will skip the large string column, it also no need to calc distinct values. if col_stats.min.is_null() { @@ -116,3 +124,73 @@ impl ColumnStatisticsState { Ok(statistics) } } + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use databend_common_expression::types::BinaryType; + use databend_common_expression::types::Int64Type; + use databend_common_expression::types::NumberDataType; + use databend_common_expression::types::StringType; + use databend_common_expression::types::UInt64Type; + use databend_common_expression::Column; + use databend_common_expression::FromData; + use databend_common_expression::TableDataType; + use databend_common_expression::TableField; + use databend_common_expression::TableSchema; + use databend_storages_common_index::Index; + use databend_storages_common_index::RangeIndex; + + use super::*; + use crate::statistics::gen_columns_statistics; + + #[test] + fn test_column_stats_state() -> Result<()> { + let field1 = TableField::new( + "a", + TableDataType::Nullable(Box::new(TableDataType::Number(NumberDataType::Int64))), + ); + let field2 = TableField::new("b", TableDataType::String); + let field3 = TableField::new("c", TableDataType::Tuple { + fields_name: vec!["d".to_string(), "e".to_string()], + fields_type: vec![ + TableDataType::Number(NumberDataType::UInt64), + TableDataType::Binary, + ], + }); + let schema = Arc::new(TableSchema::new(vec![field1, field2, field3])); + let block = DataBlock::new_from_columns(vec![ + Int64Type::from_opt_data(vec![Some(1), Some(2), None, Some(4), Some(5)]), + StringType::from_data(vec!["a", "b", "c", "d", "e"]), + Column::Tuple(vec![ + UInt64Type::from_data(vec![11, 12, 13, 14, 15]), + BinaryType::from_data(vec![ + "hello".as_bytes().to_vec(), + "world".as_bytes().to_vec(), + "".as_bytes().to_vec(), + "foo".as_bytes().to_vec(), + "bar".as_bytes().to_vec(), + ]), + ]), + ]); + + let stats_0 = gen_columns_statistics(&block, None, &schema)?; + + let mut stats_columns = vec![]; + let leaf_fields = schema.leaf_fields(); + for field in leaf_fields.iter() { + let column_id = field.column_id(); + let data_type = DataType::from(field.data_type()); + if RangeIndex::supported_type(&data_type) { + stats_columns.push((column_id, data_type.clone())); + } + } + let mut column_stats_state = ColumnStatisticsState::new(&stats_columns, &stats_columns); + column_stats_state.add_block(&schema, &block)?; + let stats_1 = column_stats_state.finalize(HashMap::new(), HashMap::new())?; + + assert_eq!(stats_0, stats_1); + Ok(()) + } +} diff --git a/src/query/storages/fuse/src/operations/append.rs b/src/query/storages/fuse/src/operations/append.rs index d1ca44fd41d4c..b6a10fb38e06e 100644 --- a/src/query/storages/fuse/src/operations/append.rs +++ b/src/query/storages/fuse/src/operations/append.rs @@ -51,7 +51,7 @@ impl FuseTable { pipeline: &mut Pipeline, table_meta_timestamps: TableMetaTimestamps, ) -> Result<()> { - let enable_stream_block_write = self.enable_stream_block_write(); + let enable_stream_block_write = self.enable_stream_block_write(ctx.clone())?; if enable_stream_block_write { let properties = StreamBlockProperties::try_create( ctx.clone(), diff --git a/src/query/storages/fuse/src/operations/common/processors/transform_serialize_block.rs b/src/query/storages/fuse/src/operations/common/processors/transform_serialize_block.rs index f13ed6701482d..9997b7dd5a893 100644 --- a/src/query/storages/fuse/src/operations/common/processors/transform_serialize_block.rs +++ b/src/query/storages/fuse/src/operations/common/processors/transform_serialize_block.rs @@ -34,6 +34,7 @@ use databend_common_pipeline_core::PipeItem; use databend_common_sql::executor::physical_plans::MutationKind; use databend_common_storage::MutationStatus; use databend_storages_common_index::BloomIndex; +use databend_storages_common_index::RangeIndex; use databend_storages_common_table_meta::meta::TableMetaTimestamps; use opendal::Operator; @@ -152,6 +153,9 @@ impl TransformSerializeBlock { let bloom_columns_map = table .bloom_index_cols .bloom_index_fields(source_schema.clone(), BloomIndex::supported_type)?; + let ndv_columns_map = table + .approx_distinct_cols + .distinct_column_fields(source_schema.clone(), RangeIndex::supported_table_type)?; let ngram_args = FuseTable::create_ngram_index_args( &table.table_info.meta, &table.table_info.meta.schema, @@ -182,6 +186,7 @@ impl TransformSerializeBlock { write_settings: table.get_write_settings(), cluster_stats_gen, bloom_columns_map, + ndv_columns_map, ngram_args, inverted_index_builders, virtual_column_builder, diff --git a/src/query/storages/fuse/src/operations/gc.rs b/src/query/storages/fuse/src/operations/gc.rs index 9ddc1da9d1133..7e3bfee8e566e 100644 --- a/src/query/storages/fuse/src/operations/gc.rs +++ b/src/query/storages/fuse/src/operations/gc.rs @@ -32,6 +32,7 @@ use databend_storages_common_index::InvertedIndexFile; use databend_storages_common_index::InvertedIndexMeta; use databend_storages_common_io::Files; use databend_storages_common_table_meta::meta::column_oriented_segment::ColumnOrientedSegment; +use databend_storages_common_table_meta::meta::column_oriented_segment::BLOCK_STATS_LOCATION; use databend_storages_common_table_meta::meta::column_oriented_segment::BLOOM_FILTER_INDEX_LOCATION; use databend_storages_common_table_meta::meta::column_oriented_segment::LOCATION; use databend_storages_common_table_meta::meta::CompactSegmentInfo; @@ -390,6 +391,13 @@ impl FuseTable { purge_files.push(loc.to_string()) } + for loc in &locations.stats_location { + if locations_referenced_by_root.stats_location.contains(loc) { + continue; + } + purge_files.push(loc.to_string()) + } + purge_files.extend(chunk.iter().map(|loc| loc.0.clone())); } purge_files.extend(ts_to_be_purged.iter().map(|loc| loc.to_string())); @@ -455,6 +463,14 @@ impl FuseTable { blooms_to_be_purged.insert(loc.to_string()); } + let mut stats_to_be_purged = HashSet::new(); + for loc in &locations.stats_location { + if locations_referenced_by_root.stats_location.contains(loc) { + continue; + } + stats_to_be_purged.insert(loc.to_string()); + } + let segment_locations_to_be_purged = HashSet::from_iter( chunk .iter() @@ -481,6 +497,7 @@ impl FuseTable { agg_indexes_to_be_purged, inverted_indexes_to_be_purged, blooms_to_be_purged, + stats_to_be_purged, segment_locations_to_be_purged, ) .await?; @@ -541,6 +558,7 @@ impl FuseTable { agg_indexes_to_be_purged, inverted_indexes_to_be_purged, root_location_tuple.bloom_location, + root_location_tuple.stats_location, segment_locations_to_be_purged, ) .await?; @@ -566,6 +584,7 @@ impl FuseTable { agg_indexes_to_be_purged: HashSet, inverted_indexes_to_be_purged: HashSet, blooms_to_be_purged: HashSet, + stats_to_be_purged: HashSet, segments_to_be_purged: HashSet, ) -> Result<()> { // 1. Try to purge block file chunks. @@ -616,7 +635,15 @@ impl FuseTable { .await?; } - // 3. Try to purge segment file chunks. + // 3. Try to purge block statistic file chunks. + let stats_count = stats_to_be_purged.len(); + if stats_count > 0 { + counter.block_stats += stats_count; + self.try_purge_location_files(ctx.clone(), stats_to_be_purged) + .await?; + } + + // 4. Try to purge segment file chunks. let segments_count = segments_to_be_purged.len(); if segments_count > 0 { counter.segments += segments_count; @@ -661,9 +688,10 @@ impl FuseTable { // 5. Refresh status. { let status = format!( - "gc: block files purged:{}, bloom files purged:{}, segment files purged:{}, table statistic files purged:{}, snapshots purged:{}, take:{:?}", + "gc: block files purged:{}, bloom files purged:{}, block stats files purged:{}, segment files purged:{}, table statistic files purged:{}, snapshots purged:{}, take:{:?}", counter.blocks, counter.blooms, + counter.block_stats, counter.segments, counter.table_statistics, counter.snapshots, @@ -714,6 +742,7 @@ impl FuseTable { ) -> Result { let mut blocks = HashSet::new(); let mut blooms = HashSet::new(); + let mut stats = HashSet::new(); let fuse_segments = SegmentsIO::create(ctx.clone(), self.operator.clone(), self.schema()); let chunk_size = ctx.get_settings().get_max_threads()? as usize * 4; @@ -779,12 +808,14 @@ impl FuseTable { }; blocks.extend(location_tuple.block_location.into_iter()); blooms.extend(location_tuple.bloom_location.into_iter()); + stats.extend(location_tuple.stats_location.into_iter()); } } Ok(LocationTuple { block_location: blocks, bloom_location: blooms, + stats_location: stats, }) } @@ -808,6 +839,7 @@ struct RootSnapshotInfo { pub struct LocationTuple { pub block_location: HashSet, pub bloom_location: HashSet, + pub stats_location: HashSet, } impl TryFrom> for LocationTuple { @@ -815,16 +847,21 @@ impl TryFrom> for LocationTuple { fn try_from(value: Arc) -> Result { let mut block_location = HashSet::new(); let mut bloom_location = HashSet::new(); + let mut stats_location = HashSet::new(); let block_metas = value.block_metas()?; for block_meta in block_metas.into_iter() { block_location.insert(block_meta.location.0.clone()); if let Some(bloom_loc) = &block_meta.bloom_filter_index_location { bloom_location.insert(bloom_loc.0.clone()); } + if let Some(stats_loc) = &block_meta.block_stats_location { + stats_location.insert(stats_loc.0.clone()); + } } Ok(Self { block_location, bloom_location, + stats_location, }) } } @@ -834,6 +871,7 @@ impl TryFrom> for LocationTuple { fn try_from(value: Arc) -> Result { let mut block_location = HashSet::new(); let mut bloom_location = HashSet::new(); + let mut stats_location = HashSet::new(); let location_path = value.location_path_col(); for path in location_path.iter() { @@ -846,19 +884,28 @@ impl TryFrom> for LocationTuple { .unwrap(); let column = value.block_metas.get_by_offset(index).to_column(); for value in column.iter() { - match value { - ScalarRef::Null => {} - ScalarRef::Tuple(values) => { - let path = values[0].as_string().unwrap(); - bloom_location.insert(path.to_string()); - } - _ => unreachable!(), + if let ScalarRef::Tuple(values) = value { + let path = values[0].as_string().unwrap(); + bloom_location.insert(path.to_string()); + } + } + + let (index, _) = value + .segment_schema + .column_with_name(BLOCK_STATS_LOCATION) + .unwrap(); + let column = value.block_metas.get_by_offset(index).to_column(); + for value in column.iter() { + if let ScalarRef::Tuple(values) = value { + let path = values[0].as_string().unwrap(); + stats_location.insert(path.to_string()); } } Ok(Self { block_location, bloom_location, + stats_location, }) } } @@ -870,6 +917,7 @@ struct PurgeCounter { agg_indexes: usize, inverted_indexes: usize, blooms: usize, + block_stats: usize, segments: usize, table_statistics: usize, snapshots: usize, @@ -883,6 +931,7 @@ impl PurgeCounter { agg_indexes: 0, inverted_indexes: 0, blooms: 0, + block_stats: 0, segments: 0, table_statistics: 0, snapshots: 0, diff --git a/src/query/storages/fuse/src/operations/merge.rs b/src/query/storages/fuse/src/operations/merge.rs index e149196075dcd..6ec54cd993899 100644 --- a/src/query/storages/fuse/src/operations/merge.rs +++ b/src/query/storages/fuse/src/operations/merge.rs @@ -21,6 +21,7 @@ use databend_common_exception::Result; use databend_common_expression::TableSchemaRef; use databend_common_pipeline_core::PipeItem; use databend_storages_common_index::BloomIndex; +use databend_storages_common_index::RangeIndex; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::TableMetaTimestamps; @@ -93,6 +94,9 @@ impl FuseTable { let bloom_columns_map = self .bloom_index_cols() .bloom_index_fields(new_schema.clone(), BloomIndex::supported_type)?; + let ndv_columns_map = self + .approx_distinct_cols() + .distinct_column_fields(new_schema.clone(), RangeIndex::supported_table_type)?; let ngram_args = FuseTable::create_ngram_index_args( &self.table_info.meta, &self.table_info.meta.schema, @@ -111,6 +115,7 @@ impl FuseTable { write_settings: self.get_write_settings(), cluster_stats_gen, bloom_columns_map, + ndv_columns_map, ngram_args, inverted_index_builders, vector_index_builder, diff --git a/src/query/storages/fuse/src/statistics/column_statistic.rs b/src/query/storages/fuse/src/statistics/column_statistic.rs index d02cf23a96883..d178dd496dab1 100644 --- a/src/query/storages/fuse/src/statistics/column_statistic.rs +++ b/src/query/storages/fuse/src/statistics/column_statistic.rs @@ -120,15 +120,15 @@ pub fn gen_columns_statistics( }; // use distinct count calculated by the xor hash function to avoid repetitive operation. - let distinct_of_values = if let Some(value) = column_distinct_count + let distinct_of_values = if let Some(&value) = column_distinct_count .as_ref() .and_then(|v| v.get(&column_id)) { // value calculated by xor hash function include NULL, need to subtract one. - if unset_bits > 0 { - *value as u64 - 1 + if unset_bits > 0 && value > 0 { + value as u64 - 1 } else { - *value as u64 + value as u64 } } else { calc_column_distinct_of_values(&col, rows)? diff --git a/src/query/storages/fuse/src/statistics/traverse.rs b/src/query/storages/fuse/src/statistics/traverse.rs index b42375eff5030..5681c281df8ae 100644 --- a/src/query/storages/fuse/src/statistics/traverse.rs +++ b/src/query/storages/fuse/src/statistics/traverse.rs @@ -23,7 +23,6 @@ use databend_common_expression::ColumnId; use databend_common_expression::Scalar; use databend_common_expression::TableField; use databend_common_expression::Value; -use databend_common_expression::ORIGIN_BLOCK_ROW_NUM_COLUMN_ID; use databend_storages_common_index::Index; use databend_storages_common_index::RangeIndex; @@ -107,9 +106,7 @@ fn traverse_column_recursive( _ => unreachable!(), }, _ => { - if RangeIndex::supported_type(data_type) - && *next_column_id != ORIGIN_BLOCK_ROW_NUM_COLUMN_ID - { + if RangeIndex::supported_type(data_type) { leaves.push(( *next_column_id, Value::Column(column.clone()), @@ -169,9 +166,7 @@ fn traverse_scalar_recursive( }, _ => { // Ignore the range index does not supported type. - if RangeIndex::supported_type(data_type) - && *next_column_id != ORIGIN_BLOCK_ROW_NUM_COLUMN_ID - { + if RangeIndex::supported_type(data_type) { leaves.push(( *next_column_id, Value::Scalar(scalar.clone()), diff --git a/src/query/storages/fuse/src/table_functions/fuse_block.rs b/src/query/storages/fuse/src/table_functions/fuse_block.rs index eaecb5425c809..11c31d1227e0b 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_block.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_block.rs @@ -75,6 +75,10 @@ impl TableMetaFunc for FuseBlock { "virtual_column_size", TableDataType::Nullable(Box::new(TableDataType::Number(NumberDataType::UInt64))), ), + TableField::new( + "block_stats_size", + TableDataType::Number(NumberDataType::UInt64), + ), ]) } @@ -99,6 +103,7 @@ impl TableMetaFunc for FuseBlock { let mut ngram_index_size = Vec::with_capacity(len); let mut vector_index_size = Vec::with_capacity(len); let mut virtual_column_size = Vec::with_capacity(len); + let mut block_stats_size = Vec::with_capacity(len); let segments_io = SegmentsIO::create(ctx.clone(), tbl.operator.clone(), tbl.schema()); @@ -134,6 +139,7 @@ impl TableMetaFunc for FuseBlock { .as_ref() .map(|m| m.virtual_column_size), ); + block_stats_size.push(block.block_stats_size); num_rows += 1; if num_rows >= limit { @@ -157,6 +163,7 @@ impl TableMetaFunc for FuseBlock { UInt64Type::from_opt_data(ngram_index_size).into(), UInt64Type::from_opt_data(vector_index_size).into(), UInt64Type::from_opt_data(virtual_column_size).into(), + UInt64Type::from_data(block_stats_size).into(), ], num_rows, )) diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history.test b/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history.test index 6f55970c81103..cad96b3804842 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history.test +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history.test @@ -122,12 +122,42 @@ create table t_in_memory(a uint64 not null) engine = Memory statement error 4000 select * from fuse_snapshot('db_09_0006', 't_in_memory') +statement ok +CREATE TABLE t2(a int, b string) approx_distinct_columns = 'a,b'; + +query I +insert into t2 values(1, 'a'), (2, 'b'); +---- +2 + +query I +select block_stats_size > 0 from fuse_block('db_09_0006', 't2'); +---- +1 + +statement ok +alter table t2 set options(approx_distinct_columns = ''); + +query I +insert into t2 values(3, 'c'), (4, 'd'); +---- +2 + +query I +select block_stats_size > 0 from fuse_block('db_09_0006', 't2') order by block_location; +---- +1 +0 + statement ok DROP TABLE t statement ok DROP TABLE t1 +statement ok +DROP TABLE t2 + statement ok DROP TABLE t_in_memory diff --git a/tests/suites/1_stateful/09_http_handler/09_0007_session.py b/tests/suites/1_stateful/09_http_handler/09_0007_session.py index 3f3c96959d59a..5860548afa3ea 100755 --- a/tests/suites/1_stateful/09_http_handler/09_0007_session.py +++ b/tests/suites/1_stateful/09_http_handler/09_0007_session.py @@ -16,21 +16,31 @@ logout_url = "http://localhost:8000/v1/session/logout" auth = ("root", "") + def check(func): def wrapper(self, *args, **kwargs): print(f"---- {func.__name__}{args[:1]}") - resp : Response = func(self, *args, **kwargs) + resp: Response = func(self, *args, **kwargs) self.session_header = resp.headers.get(HEADER_SESSION) last = self.session_header_json - self.session_header_json = json.loads(base64.urlsafe_b64decode(self.session_header)) + self.session_header_json = json.loads( + base64.urlsafe_b64decode(self.session_header) + ) if last: if last["id"] != self.session_header_json["id"]: - print("error: session id should not change", last, self.session_header_json) + print( + "error: session id should not change", + last, + self.session_header_json, + ) if last["last_refresh_time"] < time.time() - 100: if last["last_refresh_time"] > time.time() - 2: print("error: last_refresh_time should not change") else: - if last["last_refresh_time"] != self.session_header_json["last_refresh_time"]: + if ( + last["last_refresh_time"] + != self.session_header_json["last_refresh_time"] + ): print("error: last_refresh_time should not change") # print("get header: ", self.session_header_json) @@ -44,11 +54,11 @@ def wrapper(self, *args, **kwargs): if err: pprint(err) return resp + return wrapper class Client(object): - def __init__(self): self.client = requests.session() self.session_header = "" @@ -82,7 +92,7 @@ def do_query(self, query, url=query_url): auth=auth, headers={ "Content-Type": "application/json", - HEADER_SESSION: self.session_header + HEADER_SESSION: self.session_header, }, json=query_payload, ) @@ -91,7 +101,9 @@ def do_query(self, query, url=query_url): def set_fake_last_refresh_time(self): j = self.session_header_json j["last_refresh_time"] = int(time.time()) - 10 * 60 - self.session_header = base64.urlsafe_b64encode(json.dumps(j).encode('utf-8')).decode('ascii') + self.session_header = base64.urlsafe_b64encode( + json.dumps(j).encode("utf-8") + ).decode("ascii") def main(): From 46dda49cf7f466e2dbb65a233491fb7f0e1c43d1 Mon Sep 17 00:00:00 2001 From: zhyass Date: Thu, 17 Jul 2025 14:51:02 +0800 Subject: [PATCH 04/11] remove null in bloom index builder --- .../storages/common/index/src/bloom_index.rs | 5 +---- .../fuse/src/io/write/stream/block_builder.rs | 12 +++++------ .../write/stream/column_statistics_state.rs | 20 ++++--------------- .../fuse/src/statistics/column_statistic.rs | 7 +------ 4 files changed, 11 insertions(+), 33 deletions(-) diff --git a/src/query/storages/common/index/src/bloom_index.rs b/src/query/storages/common/index/src/bloom_index.rs index 3909071a03540..a23ead502c54b 100644 --- a/src/query/storages/common/index/src/bloom_index.rs +++ b/src/query/storages/common/index/src/bloom_index.rs @@ -398,11 +398,8 @@ impl BloomIndex { (column, None) }; - let capacity = validity.map_or(column.len(), |v| v.true_count() + 1); + let capacity = validity.map_or(column.len(), |v| v.true_count()); let mut result = Vec::with_capacity(capacity); - if validity.is_some() { - result.push(0); - } let column = T::try_downcast_column(column).unwrap(); if let Some(validity) = validity { let column_iter = T::iter_column(&column); diff --git a/src/query/storages/fuse/src/io/write/stream/block_builder.rs b/src/query/storages/fuse/src/io/write/stream/block_builder.rs index ea9c3ca42ed00..46833c3956e39 100644 --- a/src/query/storages/fuse/src/io/write/stream/block_builder.rs +++ b/src/query/storages/fuse/src/io/write/stream/block_builder.rs @@ -308,7 +308,7 @@ impl StreamBlockBuilder { } else { None }; - let bloom_distinct_count = bloom_index_state + let mut column_distinct_count = bloom_index_state .as_ref() .map(|i| i.column_distinct_count.clone()) .unwrap_or_default(); @@ -317,12 +317,10 @@ impl StreamBlockBuilder { .meta_locations .block_stats_location(&block_id); let block_stats_state = self.block_stats_builder.finalize(block_stats_location)?; - let hll_distinct_count = block_stats_state - .as_ref() - .map_or(HashMap::new(), |i| i.column_distinct_count.clone()); - let col_stats = self - .column_stats_state - .finalize(bloom_distinct_count, hll_distinct_count)?; + if let Some(state) = &block_stats_state { + column_distinct_count.extend(state.column_distinct_count.clone()); + } + let col_stats = self.column_stats_state.finalize(column_distinct_count)?; let mut inverted_index_states = Vec::with_capacity(self.inverted_index_writers.len()); for (i, inverted_index_writer) in std::mem::take(&mut self.inverted_index_writers) diff --git a/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs b/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs index ff7895a83353f..ac65378b20d22 100644 --- a/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs +++ b/src/query/storages/fuse/src/io/write/stream/column_statistics_state.rs @@ -88,29 +88,17 @@ impl ColumnStatisticsState { pub fn finalize( self, - bloom_distinct_count: HashMap, - hll_distinct_count: HashMap, + mut column_distinct_count: HashMap, ) -> Result { - let mut distinct_count = - HashMap::with_capacity(self.distinct_columns.len() + hll_distinct_count.len()); for (column_id, estimator) in &self.distinct_columns { - distinct_count.insert(*column_id, estimator.finalize()); + column_distinct_count.insert(*column_id, estimator.finalize()); } - distinct_count.extend(hll_distinct_count); let mut statistics = StatisticsOfColumns::with_capacity(self.col_stats.len()); for (id, stats) in self.col_stats { let mut col_stats = stats.finalize()?; - if let Some(count) = distinct_count.get(&id) { + if let Some(count) = column_distinct_count.get(&id) { col_stats.distinct_of_values = Some(*count as u64); - } else if let Some(&count) = bloom_distinct_count.get(&id) { - // value calculated by xor hash function include NULL, need to subtract one. - let distinct_of_values = if col_stats.null_count > 0 && count > 0 { - count as u64 - 1 - } else { - count as u64 - }; - col_stats.distinct_of_values = Some(distinct_of_values); } else if col_stats.min == col_stats.max { // Bloom index will skip the large string column, it also no need to calc distinct values. if col_stats.min.is_null() { @@ -188,7 +176,7 @@ mod tests { } let mut column_stats_state = ColumnStatisticsState::new(&stats_columns, &stats_columns); column_stats_state.add_block(&schema, &block)?; - let stats_1 = column_stats_state.finalize(HashMap::new(), HashMap::new())?; + let stats_1 = column_stats_state.finalize(HashMap::new())?; assert_eq!(stats_0, stats_1); Ok(()) diff --git a/src/query/storages/fuse/src/statistics/column_statistic.rs b/src/query/storages/fuse/src/statistics/column_statistic.rs index d178dd496dab1..0f980a57a97de 100644 --- a/src/query/storages/fuse/src/statistics/column_statistic.rs +++ b/src/query/storages/fuse/src/statistics/column_statistic.rs @@ -124,12 +124,7 @@ pub fn gen_columns_statistics( .as_ref() .and_then(|v| v.get(&column_id)) { - // value calculated by xor hash function include NULL, need to subtract one. - if unset_bits > 0 && value > 0 { - value as u64 - 1 - } else { - value as u64 - } + value as u64 } else { calc_column_distinct_of_values(&col, rows)? }; From c05ddffe03d71c54d86d67ddffcb25a92a946ef9 Mon Sep 17 00:00:00 2001 From: zhyass Date: Fri, 18 Jul 2025 13:07:38 +0800 Subject: [PATCH 05/11] use metahll --- .../storages/common/table_meta/src/meta/statistics.rs | 1 - .../common/table_meta/src/meta/v4/block_statistics.rs | 6 +++--- src/query/storages/fuse/src/io/write/block_writer.rs | 4 +++- .../storages/fuse/src/io/write/stream/block_builder.rs | 4 +++- .../fuse/src/io/write/stream/column_ndv_estimator.rs | 10 +++++----- 5 files changed, 14 insertions(+), 11 deletions(-) diff --git a/src/query/storages/common/table_meta/src/meta/statistics.rs b/src/query/storages/common/table_meta/src/meta/statistics.rs index b63d53cafa5af..0506f04df1e5e 100644 --- a/src/query/storages/common/table_meta/src/meta/statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/statistics.rs @@ -25,7 +25,6 @@ pub type SnapshotId = Uuid; pub type Location = (String, FormatVersion); pub type ClusterKey = (u32, String); pub type StatisticsOfColumns = HashMap; -pub type ColumnDistinctHLL = simple_hll::HyperLogLog<10>; // Assigned to executors, describes that which blocks of given segment, an executor should take care of #[derive(Clone, Debug, serde::Serialize, serde::Deserialize, PartialEq)] diff --git a/src/query/storages/common/table_meta/src/meta/v4/block_statistics.rs b/src/query/storages/common/table_meta/src/meta/v4/block_statistics.rs index 1ac9703376d5c..e453f47f12630 100644 --- a/src/query/storages/common/table_meta/src/meta/v4/block_statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/v4/block_statistics.rs @@ -26,20 +26,20 @@ use crate::meta::format::compress; use crate::meta::format::encode; use crate::meta::format::read_and_deserialize; use crate::meta::versions::Versioned; -use crate::meta::ColumnDistinctHLL; use crate::meta::FormatVersion; use crate::meta::MetaCompression; use crate::meta::MetaEncoding; +use crate::meta::MetaHLL; #[derive(Serialize, Deserialize, Clone, Debug)] pub struct BlockStatistics { pub format_version: FormatVersion, - pub hll: HashMap, + pub hll: HashMap, } impl BlockStatistics { - pub fn new(hll: HashMap) -> Self { + pub fn new(hll: HashMap) -> Self { Self { format_version: BlockStatistics::VERSION, hll, diff --git a/src/query/storages/fuse/src/io/write/block_writer.rs b/src/query/storages/fuse/src/io/write/block_writer.rs index 007285f691530..342c455204b36 100644 --- a/src/query/storages/fuse/src/io/write/block_writer.rs +++ b/src/query/storages/fuse/src/io/write/block_writer.rs @@ -196,7 +196,9 @@ impl BlockBuilder { &self.ndv_columns_map, )?; if let Some(block_stats_state) = &block_stats_state { - column_distinct_count.extend(block_stats_state.column_distinct_count.clone()); + for (key, val) in &block_stats_state.column_distinct_count { + column_distinct_count.entry(*key).or_insert(*val); + } } let mut inverted_index_states = Vec::with_capacity(self.inverted_index_builders.len()); diff --git a/src/query/storages/fuse/src/io/write/stream/block_builder.rs b/src/query/storages/fuse/src/io/write/stream/block_builder.rs index 46833c3956e39..66114228a0082 100644 --- a/src/query/storages/fuse/src/io/write/stream/block_builder.rs +++ b/src/query/storages/fuse/src/io/write/stream/block_builder.rs @@ -318,7 +318,9 @@ impl StreamBlockBuilder { .block_stats_location(&block_id); let block_stats_state = self.block_stats_builder.finalize(block_stats_location)?; if let Some(state) = &block_stats_state { - column_distinct_count.extend(state.column_distinct_count.clone()); + for (key, val) in &state.column_distinct_count { + column_distinct_count.entry(*key).or_insert(*val); + } } let col_stats = self.column_stats_state.finalize(column_distinct_count)?; diff --git a/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs b/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs index e2fa21f07786c..23d2f2475546a 100644 --- a/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs +++ b/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs @@ -39,7 +39,7 @@ use databend_common_expression::with_number_type; use databend_common_expression::Column; use databend_common_expression::ScalarRef; use databend_common_expression::SELECTIVITY_THRESHOLD; -use databend_storages_common_table_meta::meta::ColumnDistinctHLL; +use databend_storages_common_table_meta::meta::MetaHLL; use enum_dispatch::enum_dispatch; #[enum_dispatch] @@ -47,7 +47,7 @@ pub trait ColumnNDVEstimatorOps: Send + Sync { fn update_column(&mut self, column: &Column); fn update_scalar(&mut self, scalar: &ScalarRef); fn finalize(&self) -> usize; - fn hll(self) -> ColumnDistinctHLL; + fn hll(self) -> MetaHLL; } #[enum_dispatch(ColumnNDVEstimatorOps)] @@ -111,7 +111,7 @@ where T: ValueType + Send + Sync, for<'a> T::ScalarRef<'a>: Hash, { - hll: ColumnDistinctHLL, + hll: MetaHLL, _phantom: PhantomData, } @@ -122,7 +122,7 @@ where { pub fn new() -> Self { Self { - hll: ColumnDistinctHLL::new(), + hll: MetaHLL::new(), _phantom: Default::default(), } } @@ -181,7 +181,7 @@ where self.hll.count() } - fn hll(self) -> ColumnDistinctHLL { + fn hll(self) -> MetaHLL { self.hll } } From 87ee9411c5c4527690d77ec2aa648b4183e6dc96 Mon Sep 17 00:00:00 2001 From: zhyass Date: Fri, 18 Jul 2025 20:25:27 +0800 Subject: [PATCH 06/11] avoid large string --- src/query/expression/src/values.rs | 17 +++++++++++++++++ .../storages/common/index/src/bloom_index.rs | 18 +++--------------- .../src/io/write/block_statistics_writer.rs | 13 ++++++++++++- 3 files changed, 32 insertions(+), 16 deletions(-) diff --git a/src/query/expression/src/values.rs b/src/query/expression/src/values.rs index bff7c60ff1703..cd12b0ee8508b 100755 --- a/src/query/expression/src/values.rs +++ b/src/query/expression/src/values.rs @@ -1694,6 +1694,23 @@ impl Column { _ => (false, None), } } + + /// Checks if the average length of a string column exceeds 256 bytes. + /// If it does, the bloom index for the column will not be established. + pub fn check_large_string(&self) -> bool { + let (inner, len) = if let Column::Nullable(c) = self { + (&c.column, c.validity.true_count()) + } else { + (self, self.len()) + }; + if let Column::String(v) = inner { + let bytes_per_row = v.total_bytes_len() / len.max(1); + if bytes_per_row > 256 { + return true; + } + } + false + } } /// Serialize a column to a base64 string. diff --git a/src/query/storages/common/index/src/bloom_index.rs b/src/query/storages/common/index/src/bloom_index.rs index a23ead502c54b..9af4d2a3734fa 100644 --- a/src/query/storages/common/index/src/bloom_index.rs +++ b/src/query/storages/common/index/src/bloom_index.rs @@ -625,18 +625,6 @@ impl BloomIndex { let data_type = DataType::from(data_type); Xor8Filter::supported_type(&data_type) } - - /// Checks if the average length of a string column exceeds 256 bytes. - /// If it does, the bloom index for the column will not be established. - pub fn check_large_string(column: &Column) -> bool { - if let Column::String(v) = &column { - let bytes_per_row = v.total_bytes_len() / v.len().max(1); - if bytes_per_row > 256 { - return true; - } - } - false - } } pub struct BloomIndexBuilder { @@ -780,14 +768,14 @@ impl BloomIndexBuilder { builder.push_default(); } let str_column = builder.build(); - if BloomIndex::check_large_string(&str_column) { + if str_column.check_large_string() { bloom_keys_to_remove.push(index); continue; } let str_type = DataType::Nullable(Box::new(DataType::String)); (str_column, str_type) } else { - if BloomIndex::check_large_string(&column) { + if column.check_large_string() { bloom_keys_to_remove.push(index); continue; } @@ -795,7 +783,7 @@ impl BloomIndexBuilder { } } _ => { - if BloomIndex::check_large_string(&column) { + if column.check_large_string() { bloom_keys_to_remove.push(index); continue; } diff --git a/src/query/storages/fuse/src/io/write/block_statistics_writer.rs b/src/query/storages/fuse/src/io/write/block_statistics_writer.rs index d6f2fb777cadc..ad54bbb972efa 100644 --- a/src/query/storages/fuse/src/io/write/block_statistics_writer.rs +++ b/src/query/storages/fuse/src/io/write/block_statistics_writer.rs @@ -76,17 +76,28 @@ impl BlockStatsBuilder { } pub fn add_block(&mut self, block: &DataBlock) -> Result<()> { - for column_builder in self.builders.iter_mut() { + let mut keys_to_remove = vec![]; + for (index, column_builder) in self.builders.iter_mut().enumerate() { let entry = block.get_by_offset(column_builder.index); match entry { BlockEntry::Const(s, ..) => { column_builder.builder.update_scalar(&s.as_ref()); } BlockEntry::Column(col) => { + if col.check_large_string() { + keys_to_remove.push(index); + continue; + } column_builder.builder.update_column(col); } } } + + // reverse sorting. + keys_to_remove.sort_by(|a, b| b.cmp(a)); + for k in keys_to_remove { + self.builders.remove(k); + } Ok(()) } From 6dd1065c5623c84119873747a909f9a3243cdd0b Mon Sep 17 00:00:00 2001 From: zhyass Date: Sat, 19 Jul 2025 13:43:02 +0800 Subject: [PATCH 07/11] fix --- src/query/storages/fuse/src/io/locations.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/storages/fuse/src/io/locations.rs b/src/query/storages/fuse/src/io/locations.rs index 76de95a868b9c..95ce57493f5b0 100644 --- a/src/query/storages/fuse/src/io/locations.rs +++ b/src/query/storages/fuse/src/io/locations.rs @@ -155,7 +155,7 @@ impl TableMetaLocationGenerator { pub fn block_stats_location(&self, block_id: &Uuid) -> Location { ( format!( - "{}{}_v{}.parquet", + "{}{}_v{}.mpk", self.block_statistics_location_prefix(), block_id.as_simple(), BlockStatistics::VERSION, From dafe9366dfb764dbc604216a5fe490bc2bac055a Mon Sep 17 00:00:00 2001 From: zhyass Date: Wed, 30 Jul 2025 03:27:32 +0800 Subject: [PATCH 08/11] use segment level stats --- Cargo.lock | 2 + src/common/storage/Cargo.toml | 2 + src/common/storage/src/lib.rs | 2 + src/common/storage/src/meta_hll.rs | 370 ++++++++++++++++++ src/query/catalog/src/plan/partition.rs | 7 +- src/query/config/src/config.rs | 10 + src/query/config/src/inner.rs | 4 + .../storages/fuse/operations/ngram_index.rs | 5 +- .../storages/fuse/operations/vacuum_table.rs | 28 +- .../fuse/operations/vacuum_table_v2.rs | 13 +- .../fuse/operations/virtual_columns.rs | 5 +- .../interpreter_table_set_options.rs | 2 + .../src/pipelines/builders/builder_commit.rs | 7 +- .../src/pipelines/builders/builder_compact.rs | 2 + .../service/src/test_kits/block_writer.rs | 44 +-- src/query/service/src/test_kits/check.rs | 13 +- src/query/service/src/test_kits/fuse.rs | 31 +- .../it/storages/fuse/bloom_index_meta_size.rs | 5 +- .../tests/it/storages/fuse/conflict.rs | 10 + .../it/storages/fuse/meta/column_oriented.rs | 47 +-- .../mutation/block_compact_mutator.rs | 4 +- .../operations/mutation/recluster_mutator.rs | 6 +- .../mutation/segments_compact_mutator.rs | 4 +- .../it/storages/fuse/operations/read_plan.rs | 2 - .../storages/fuse/operations/table_analyze.rs | 5 +- .../tests/it/storages/fuse/statistics.rs | 4 +- .../storages/testdata/configs_table_basic.txt | 1 + .../sql/src/executor/physical_plans/common.rs | 3 + .../storages/common/cache/src/cache_items.rs | 1 + src/query/storages/common/cache/src/caches.rs | 18 + .../storages/common/cache/src/manager.rs | 15 +- .../src/meta/column_oriented_segment/mod.rs | 2 - .../meta/column_oriented_segment/schema.rs | 6 - .../segment_builder.rs | 21 +- .../common/table_meta/src/meta/current/mod.rs | 4 +- .../common/table_meta/src/meta/mod.rs | 2 +- .../common/table_meta/src/meta/statistics.rs | 29 ++ .../common/table_meta/src/meta/v2/mod.rs | 4 +- .../common/table_meta/src/meta/v2/segment.rs | 14 +- .../segment_statistics.rs} | 35 +- .../table_meta/src/meta/v2/statistics.rs | 4 + .../src/meta/v2/table_snapshot_statistics.rs | 6 +- .../src/meta/v3/frozen/block_meta.rs | 2 - .../src/meta/v3/frozen/statistics.rs | 1 + .../src/meta/v3/table_snapshot_statistics.rs | 6 +- .../common/table_meta/src/meta/v4/mod.rs | 2 - .../common/table_meta/src/meta/versions.rs | 16 +- .../src/readers/versioned_reader.rs | 12 +- src/query/storages/fuse/src/constants.rs | 2 +- src/query/storages/fuse/src/io/locations.rs | 38 +- src/query/storages/fuse/src/io/mod.rs | 2 +- .../fuse/src/io/read/meta/meta_readers.rs | 20 + src/query/storages/fuse/src/io/read/mod.rs | 2 + .../fuse/src/io/read/statistics/mod.rs | 18 + .../src/io/read/statistics/segment_stats.rs | 70 ++++ .../src/io/write/block_statistics_writer.rs | 49 +-- .../fuse/src/io/write/block_writer.rs | 60 +-- .../storages/fuse/src/io/write/meta_writer.rs | 8 + src/query/storages/fuse/src/io/write/mod.rs | 2 +- .../fuse/src/io/write/stream/block_builder.rs | 22 +- .../io/write/stream/column_ndv_estimator.rs | 2 +- .../storages/fuse/src/operations/analyze.rs | 6 +- .../operations/common/meta/mutation_log.rs | 2 +- .../transform_mutation_aggregator.rs | 125 ++++-- .../processors/transform_serialize_segment.rs | 43 +- src/query/storages/fuse/src/operations/gc.rs | 61 ++- .../operations/mutation/meta/compact_part.rs | 7 +- .../mutation/mutator/block_compact_mutator.rs | 77 ++-- .../mutation/mutator/recluster_mutator.rs | 129 +++++- .../mutator/segment_compact_mutator.rs | 34 +- .../fuse/src/statistics/accumulator.rs | 26 +- src/query/storages/fuse/src/statistics/mod.rs | 2 + .../storages/fuse/src/statistics/reducers.rs | 4 +- .../fuse/src/statistics/segment_statistics.rs | 13 + .../fuse/src/table_functions/fuse_block.rs | 7 - .../09_0006_func_fuse_history.test | 11 - 76 files changed, 1194 insertions(+), 486 deletions(-) create mode 100644 src/common/storage/src/meta_hll.rs rename src/query/storages/common/table_meta/src/meta/{v4/block_statistics.rs => v2/segment_statistics.rs} (77%) create mode 100644 src/query/storages/fuse/src/io/read/statistics/mod.rs create mode 100644 src/query/storages/fuse/src/io/read/statistics/segment_stats.rs create mode 100644 src/query/storages/fuse/src/statistics/segment_statistics.rs diff --git a/Cargo.lock b/Cargo.lock index d4a188e61ad82..b350bde33d8bf 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4231,9 +4231,11 @@ dependencies = [ name = "databend-common-storage" version = "0.1.0" dependencies = [ + "ahash 0.8.12", "anyhow", "arrow-schema", "async-backtrace", + "borsh", "chrono", "dashmap 6.1.0", "databend-common-ast", diff --git a/src/common/storage/Cargo.toml b/src/common/storage/Cargo.toml index b3763abba49d7..4d4d0e4f205d5 100644 --- a/src/common/storage/Cargo.toml +++ b/src/common/storage/Cargo.toml @@ -10,9 +10,11 @@ edition = { workspace = true } storage-hdfs = ["databend-common-meta-app/storage-hdfs"] [dependencies] +ahash = { workspace = true } anyhow = { workspace = true } arrow-schema = { workspace = true } async-backtrace = { workspace = true } +borsh = { workspace = true } chrono = { workspace = true } dashmap = { workspace = true, features = ["serde"] } databend-common-ast = { workspace = true } diff --git a/src/common/storage/src/lib.rs b/src/common/storage/src/lib.rs index d5c3cfbc95373..aef8bc4cc86b6 100644 --- a/src/common/storage/src/lib.rs +++ b/src/common/storage/src/lib.rs @@ -71,6 +71,7 @@ pub use stage::STDIN_FD; mod copy; mod histogram; mod merge; +mod meta_hll; mod metrics_layer; mod multi_table_insert; mod statistics; @@ -83,6 +84,7 @@ pub use histogram::Histogram; pub use histogram::HistogramBucket; pub use histogram::DEFAULT_HISTOGRAM_BUCKETS; pub use merge::MutationStatus; +pub use meta_hll::MetaHLL; pub use multi_table_insert::MultiTableInsertStatus; pub use statistics::Datum; pub use statistics::F64; diff --git a/src/common/storage/src/meta_hll.rs b/src/common/storage/src/meta_hll.rs new file mode 100644 index 0000000000000..de427e96b1b39 --- /dev/null +++ b/src/common/storage/src/meta_hll.rs @@ -0,0 +1,370 @@ +// Copyright 2021 Datafuse Labs +// +// 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. + +//! Codes are borrowed from [simple_hll](https://github.com/sundy-li/simple_hll) + +use std::hash::Hash; + +use ahash::RandomState; + +const P: usize = 7_usize; +const Q: usize = 64 - P; +const M: usize = 1 << P; + +/// Fixed seed +const SEED: RandomState = RandomState::with_seeds( + 0x355e438b4b1478c7_u64, + 0xd0e8453cd135b473_u64, + 0xf7b252066a57836a_u64, + 0xb8a829e3713c09bf_u64, +); + +/// Note: We don't make MetaHLL as static struct by keeping `PhantomData` +/// Callers should take care of its hash function to be unchanged. +#[derive(Clone, Debug, Eq, PartialEq)] +pub struct MetaHLL { + pub(crate) registers: Vec, +} + +impl Default for MetaHLL { + fn default() -> Self { + Self::new() + } +} + +impl MetaHLL { + /// note that this method should not be invoked in untrusted environment + pub fn new() -> Self { + Self { + registers: vec![0; M], + } + } + + /// Adds an hash to the MetaHLL. + /// hash value is dertermined by caller + #[inline] + pub fn add_hash(&mut self, hash: u64) { + let index = (hash & (M as u64 - 1)) as usize; + let one_position = (((hash >> P) | (1_u64 << Q)).trailing_zeros() + 1) as u8; + unsafe { + let val = self.registers.get_unchecked_mut(index); + if *val < one_position { + *val = one_position; + } + } + } + + /// Adds an object to the MetaHLL. + /// Though we could pass different types into this method, caller should notice that + pub fn add_object(&mut self, obj: &T) { + let hash = SEED.hash_one(obj); + self.add_hash(hash); + } + + /// Merge the other [`MetaHLL`] into this one + pub fn merge(&mut self, other: &Self) { + for i in 0..self.registers.len() { + self.registers[i] = self.registers[i].max(other.registers[i]); + } + } + + /// Get the register histogram (each value in register index into + /// the histogram + #[inline] + fn get_histogram(&self) -> [u32; 64] { + let mut histogram = [0; 64]; + // hopefully this can be unrolled + for r in &self.registers { + histogram[*r as usize] += 1; + } + histogram + } + + /// Guess the number of unique elements seen by the MetaHLL. + #[inline] + pub fn count(&self) -> usize { + let histogram = self.get_histogram(); + let m = M as f64; + let mut z = m * hll_tau((m - histogram[Q + 1] as f64) / m); + for i in histogram[1..=Q].iter().rev() { + z += *i as f64; + z *= 0.5; + } + z += m * hll_sigma(histogram[0] as f64 / m); + + (0.5 / 2_f64.ln() * m * m / z).round() as usize + } + + #[inline] + pub fn num_empty_registers(&self) -> usize { + self.registers.iter().filter(|x| **x == 0).count() + } +} + +/// Helper function sigma as defined in +/// "New cardinality estimation algorithms for MetaHLL sketches" +/// Otmar Ertl, https://arxiv.org/abs/1702.01284 +#[allow(dead_code)] +#[inline] +fn hll_sigma(x: f64) -> f64 { + if x == 1. { + f64::INFINITY + } else { + let mut y = 1.0; + let mut z = x; + let mut x = x; + loop { + x *= x; + let z_prime = z; + z += x * y; + y += y; + + if z_prime == z { + break; + } + } + z + } +} + +/// Helper function tau as defined in +/// "New cardinality estimation algorithms for MetaHLL sketches" +/// Otmar Ertl, https://arxiv.org/abs/1702.01284 +#[inline] +fn hll_tau(x: f64) -> f64 { + if x == 0.0 || x == 1.0 { + 0.0 + } else { + let mut y = 1.0; + let mut z = 1.0 - x; + let mut x = x; + loop { + x = x.sqrt(); + let z_prime = z; + y *= 0.5; + z -= (1.0 - x).powi(2) * y; + if z_prime == z { + break; + } + } + z / 3.0 + } +} + +#[derive(serde::Serialize, borsh::BorshSerialize)] +enum MetaHLLVariantRef<'a> { + Empty, + Sparse { indices: Vec, values: Vec }, + Full(&'a Vec), +} + +#[derive(serde::Deserialize, borsh::BorshDeserialize)] +enum MetaHLLVariant { + Empty, + Sparse { indices: Vec, values: Vec }, + Full(Vec), +} + +impl From for MetaHLL { + fn from(value: MetaHLLVariant) -> Self { + match value { + MetaHLLVariant::Empty => MetaHLL::new(), + MetaHLLVariant::Sparse { indices, values } => { + let mut registers = vec![0; 1 << P]; + for (i, v) in indices.into_iter().zip(values.into_iter()) { + registers[i as usize] = v; + } + MetaHLL { registers } + } + MetaHLLVariant::Full(registers) => MetaHLL { registers }, + } + } +} + +impl<'a> From<&'a MetaHLL> for MetaHLLVariantRef<'a> { + fn from(hll: &'a MetaHLL) -> Self { + let none_empty_registers = M - hll.num_empty_registers(); + + if none_empty_registers == 0 { + MetaHLLVariantRef::Empty + } else if none_empty_registers * 3 <= M { + // If the number of empty registers is larger enough, we can use sparse serialize to reduce the binary size + // each register in sparse format will occupy 3 bytes, 2 for register index and 1 for register value. + let mut indices = Vec::with_capacity(none_empty_registers); + let mut values = Vec::with_capacity(none_empty_registers); + for (index, &value) in hll.registers.iter().enumerate() { + if value != 0 { + indices.push(index as u8); + values.push(value); + } + } + MetaHLLVariantRef::Sparse { indices, values } + } else { + MetaHLLVariantRef::Full(&hll.registers) + } + } +} + +impl serde::Serialize for MetaHLL { + fn serialize(&self, serializer: S) -> Result + where S: serde::Serializer { + let v: MetaHLLVariantRef<'_> = self.into(); + v.serialize(serializer) + } +} + +impl<'de> serde::Deserialize<'de> for MetaHLL { + fn deserialize(deserializer: D) -> Result + where D: serde::Deserializer<'de> { + let v = MetaHLLVariant::deserialize(deserializer)?; + Ok(v.into()) + } +} + +impl borsh::BorshSerialize for MetaHLL { + fn serialize(&self, writer: &mut W) -> std::io::Result<()> { + let v: MetaHLLVariantRef<'_> = self.into(); + v.serialize(writer) + } +} + +impl borsh::BorshDeserialize for MetaHLL { + fn deserialize_reader(reader: &mut R) -> std::io::Result { + let v = MetaHLLVariant::deserialize_reader(reader)?; + Ok(v.into()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + fn compare_with_delta(got: usize, expected: usize) { + let expected = expected as f64; + let diff = (got as f64) - expected; + let diff = diff.abs() / expected; + // times 6 because we want the tests to be stable + // so we allow a rather large margin of error + // this is adopted from redis's unit test version as well + let margin = 1.04 / ((M as f64).sqrt()) * 6.0; + assert!( + diff <= margin, + "{} is not near {} percent of {} which is ({}, {})", + got, + margin, + expected, + expected * (1.0 - margin), + expected * (1.0 + margin) + ); + } + + macro_rules! sized_number_test { + ($SIZE: expr, $T: tt) => {{ + let mut hll = MetaHLL::new(); + for i in 0..$SIZE { + hll.add_object(&(i as $T)); + } + compare_with_delta(hll.count(), $SIZE); + }}; + } + + macro_rules! typed_large_number_test { + ($SIZE: expr) => {{ + sized_number_test!($SIZE, u64); + sized_number_test!($SIZE, u128); + sized_number_test!($SIZE, i64); + sized_number_test!($SIZE, i128); + }}; + } + + macro_rules! typed_number_test { + ($SIZE: expr) => {{ + sized_number_test!($SIZE, u16); + sized_number_test!($SIZE, u32); + sized_number_test!($SIZE, i16); + sized_number_test!($SIZE, i32); + typed_large_number_test!($SIZE); + }}; + } + + #[test] + fn test_empty() { + let hll = MetaHLL::new(); + assert_eq!(hll.count(), 0); + } + + #[test] + fn test_one() { + let mut hll = MetaHLL::new(); + hll.add_hash(1); + assert_eq!(hll.count(), 1); + } + + #[test] + fn test_number_100() { + typed_number_test!(100); + } + + #[test] + fn test_number_1k() { + typed_number_test!(1_000); + } + + #[test] + fn test_number_10k() { + typed_number_test!(10_000); + } + + #[test] + fn test_number_100k() { + typed_large_number_test!(100_000); + } + + #[test] + fn test_number_1m() { + typed_large_number_test!(1_000_000); + } + + #[test] + fn test_empty_merge() { + let mut hll = MetaHLL::new(); + hll.merge(&MetaHLL::new()); + assert_eq!(hll.count(), 0); + } + + #[test] + fn test_merge_overlapped() { + let mut hll = MetaHLL::new(); + for i in 0..1000 { + hll.add_object(&i); + } + + let other = MetaHLL::new(); + for i in 0..1000 { + hll.add_object(&i); + } + + hll.merge(&other); + compare_with_delta(hll.count(), 1000); + } + + #[test] + fn test_repetition() { + let mut hll = MetaHLL::new(); + for i in 0..1_000_000 { + hll.add_object(&(i % 1000)); + } + compare_with_delta(hll.count(), 1000); + } +} diff --git a/src/query/catalog/src/plan/partition.rs b/src/query/catalog/src/plan/partition.rs index 0df56b8647ec6..b13d2ae0f597d 100644 --- a/src/query/catalog/src/plan/partition.rs +++ b/src/query/catalog/src/plan/partition.rs @@ -26,6 +26,7 @@ use databend_common_config::GlobalConfig; use databend_common_exception::Result; use databend_common_meta_types::NodeInfo; use databend_storages_common_table_meta::meta::BlockMeta; +use databend_storages_common_table_meta::meta::RawColumnHLL; use databend_storages_common_table_meta::meta::Statistics; use parking_lot::RwLock; use rand::prelude::SliceRandom; @@ -346,11 +347,13 @@ pub struct ReclusterTask { pub level: i32, } +pub type BlockMetaWithHLL = (Arc, Option); + #[derive(Clone)] pub enum ReclusterParts { Recluster { tasks: Vec, - remained_blocks: Vec>, + remained_blocks: Vec, removed_segment_indexes: Vec, removed_segment_summary: Statistics, }, @@ -398,7 +401,7 @@ impl ReclusterParts { #[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default)] pub struct ReclusterInfoSideCar { - pub merged_blocks: Vec>, + pub merged_blocks: Vec, pub removed_segment_indexes: Vec, pub removed_statistics: Statistics, } diff --git a/src/query/config/src/config.rs b/src/query/config/src/config.rs index c7c1a5c09613b..067ba63736653 100644 --- a/src/query/config/src/config.rs +++ b/src/query/config/src/config.rs @@ -3219,6 +3219,14 @@ pub struct CacheConfig { )] pub table_meta_statistic_count: u64, + /// Max number of cached segment statistic meta + #[clap( + long = "cache-segment-statistic-count", + value_name = "VALUE", + default_value = "0" + )] + pub segment_stastistics_count: u64, + /// Enable bloom index cache. Default is enabled. Set it to false to disable all the bloom index caches #[clap( long = "cache-enable-table-bloom-index-cache", @@ -3635,6 +3643,7 @@ mod cache_config_converters { block_meta_count: value.block_meta_count, segment_block_metas_count: value.segment_block_metas_count, table_meta_statistic_count: value.table_meta_statistic_count, + segment_statistics_count: value.segment_stastistics_count, enable_table_index_bloom: value.enable_table_bloom_index_cache, table_bloom_index_meta_count: value.table_bloom_index_meta_count, table_bloom_index_filter_count: value.table_bloom_index_filter_count, @@ -3671,6 +3680,7 @@ mod cache_config_converters { table_meta_snapshot_count: value.table_meta_snapshot_count, table_meta_segment_bytes: value.table_meta_segment_bytes, table_meta_statistic_count: value.table_meta_statistic_count, + segment_stastistics_count: value.segment_statistics_count, block_meta_count: value.block_meta_count, enable_table_bloom_index_cache: value.enable_table_index_bloom, table_bloom_index_meta_count: value.table_bloom_index_meta_count, diff --git a/src/query/config/src/inner.rs b/src/query/config/src/inner.rs index fe66001d6d489..374f6d13f42ae 100644 --- a/src/query/config/src/inner.rs +++ b/src/query/config/src/inner.rs @@ -583,6 +583,9 @@ pub struct CacheConfig { /// Max number of cached table segment pub table_meta_statistic_count: u64, + /// Max number of cached segment statistics + pub segment_statistics_count: u64, + /// Enable bloom index cache. Default is enabled. Set it to false to disable all the bloom index caches pub enable_table_index_bloom: bool, @@ -751,6 +754,7 @@ impl Default for CacheConfig { block_meta_count: 0, segment_block_metas_count: 0, table_meta_statistic_count: 256, + segment_statistics_count: 0, enable_table_index_bloom: true, table_bloom_index_meta_count: 3000, table_bloom_index_filter_count: 0, diff --git a/src/query/ee/src/storages/fuse/operations/ngram_index.rs b/src/query/ee/src/storages/fuse/operations/ngram_index.rs index 308dbae6759ca..72907c44247c0 100644 --- a/src/query/ee/src/storages/fuse/operations/ngram_index.rs +++ b/src/query/ee/src/storages/fuse/operations/ngram_index.rs @@ -226,13 +226,13 @@ pub async fn do_refresh_ngram_index( vec![], vec![], Statistics::default(), - MutationKind::Update, + MutationKind::Refresh, table_meta_timestamps, ) }); let prev_snapshot_id = snapshot.snapshot_id; - let snapshot_gen = MutationGenerator::new(Some(snapshot), MutationKind::Update); + let snapshot_gen = MutationGenerator::new(Some(snapshot), MutationKind::Refresh); pipeline.add_sink(|input| { CommitSink::try_create( fuse_table, @@ -435,6 +435,7 @@ impl AsyncTransform for NgramIndexTransform { let extended_block_meta = ExtendedBlockMeta { block_meta: new_block_meta, draft_virtual_block_meta: None, + column_hlls: None, }; let entry = MutationLogEntry::ReplacedBlock { diff --git a/src/query/ee/src/storages/fuse/operations/vacuum_table.rs b/src/query/ee/src/storages/fuse/operations/vacuum_table.rs index 19ee34c627efa..77610022a9164 100644 --- a/src/query/ee/src/storages/fuse/operations/vacuum_table.rs +++ b/src/query/ee/src/storages/fuse/operations/vacuum_table.rs @@ -40,7 +40,7 @@ pub struct SnapshotReferencedFiles { pub segments: HashSet, pub blocks: HashSet, pub blocks_index: HashSet, - pub blocks_stats: HashSet, + pub segments_stats: HashSet, } impl SnapshotReferencedFiles { @@ -55,7 +55,7 @@ impl SnapshotReferencedFiles { for file in &self.blocks_index { files.push(file.clone()); } - for file in &self.blocks_stats { + for file in &self.segments_stats { files.push(file.clone()); } files @@ -136,7 +136,7 @@ pub async fn get_snapshot_referenced_files( segments, blocks: locations_referenced.block_location, blocks_index: locations_referenced.bloom_location, - blocks_stats: locations_referenced.stats_location, + segments_stats: locations_referenced.hll_location, })) } @@ -173,7 +173,7 @@ pub async fn do_gc_orphan_files( referenced_files.segments.len(), referenced_files.blocks.len(), referenced_files.blocks_index.len(), - referenced_files.blocks_stats.len(), + referenced_files.segments_stats.len(), start.elapsed() ); ctx.set_status_info(&status); @@ -274,12 +274,12 @@ pub async fn do_gc_orphan_files( ); ctx.set_status_info(&status); - // 5. Purge orphan block stats files. - // 5.1 Get orphan block stats files to be purged + // 5. Purge orphan segment stats files. + // 5.1 Get orphan segment stats files to be purged let stats_locations_to_be_purged = get_orphan_files_to_be_purged( fuse_table, - location_gen.block_statistics_location_prefix(), - referenced_files.blocks_stats, + location_gen.segment_statistics_location_prefix(), + referenced_files.segments_stats, retention_time, ) .await?; @@ -290,7 +290,7 @@ pub async fn do_gc_orphan_files( ); ctx.set_status_info(&status); - // 5.2 Delete all the orphan block stats files to be purged + // 5.2 Delete all the orphan segment stats files to be purged let purged_file_num = stats_locations_to_be_purged.len(); fuse_table .try_purge_location_files( @@ -299,7 +299,7 @@ pub async fn do_gc_orphan_files( ) .await?; let status = format!( - "gc orphan: purged block stats files:{}, cost:{:?}", + "gc orphan: purged segment stats files:{}, cost:{:?}", purged_file_num, start.elapsed() ); @@ -326,7 +326,7 @@ pub async fn do_dry_run_orphan_files( referenced_files.segments.len(), referenced_files.blocks.len(), referenced_files.blocks_index.len(), - referenced_files.blocks_stats.len(), + referenced_files.segments_stats.len(), start.elapsed() ); ctx.set_status_info(&status); @@ -388,11 +388,11 @@ pub async fn do_dry_run_orphan_files( purge_files.extend(index_locations_to_be_purged); - // 5. Get purge orphan block stats files. + // 5. Get purge orphan segment stats files. let stats_locations_to_be_purged = get_orphan_files_to_be_purged( fuse_table, - location_gen.block_statistics_location_prefix(), - referenced_files.blocks_stats, + location_gen.segment_statistics_location_prefix(), + referenced_files.segments_stats, retention_time, ) .await?; diff --git a/src/query/ee/src/storages/fuse/operations/vacuum_table_v2.rs b/src/query/ee/src/storages/fuse/operations/vacuum_table_v2.rs index da2d43f7f708e..b44e917b41afb 100644 --- a/src/query/ee/src/storages/fuse/operations/vacuum_table_v2.rs +++ b/src/query/ee/src/storages/fuse/operations/vacuum_table_v2.rs @@ -248,11 +248,16 @@ pub async fn do_vacuum2( .into_iter() .filter(|s| !gc_root_segments.contains(s)) .collect(); + let stats_to_gc = segments_to_gc + .iter() + .map(|v| TableMetaLocationGenerator::gen_segment_stats_location_from_segment_location(v)) + .collect::>(); ctx.set_status_info(&format!( - "[FUSE-VACUUM2] Filtered segments_to_gc for table {}, elapsed: {:?}, segments_to_gc: {:?}", + "[FUSE-VACUUM2] Filtered segments_to_gc for table {}, elapsed: {:?}, segments_to_gc: {:?}, stats_to_gc: {:?}", fuse_table.get_table_info().desc, start.elapsed(), - slice_summary(&segments_to_gc) + slice_summary(&segments_to_gc), + slice_summary(&stats_to_gc) )); let start = std::time::Instant::now(); @@ -336,9 +341,6 @@ pub async fn do_vacuum2( } indexes_to_gc .push(TableMetaLocationGenerator::gen_bloom_index_location_from_block_location(loc)); - - indexes_to_gc - .push(TableMetaLocationGenerator::gen_block_stats_location_from_block_location(loc)); } ctx.set_status_info(&format!( @@ -352,6 +354,7 @@ pub async fn do_vacuum2( let subject_files_to_gc: Vec<_> = segments_to_gc .into_iter() .chain(blocks_to_gc.into_iter()) + .chain(stats_to_gc.into_iter()) .collect(); let op = Files::create(ctx.clone(), fuse_table.get_operator()); diff --git a/src/query/ee/src/storages/fuse/operations/virtual_columns.rs b/src/query/ee/src/storages/fuse/operations/virtual_columns.rs index 94ece33fcf993..aab980df52da1 100644 --- a/src/query/ee/src/storages/fuse/operations/virtual_columns.rs +++ b/src/query/ee/src/storages/fuse/operations/virtual_columns.rs @@ -191,13 +191,13 @@ pub async fn do_refresh_virtual_column( vec![], vec![], Statistics::default(), - MutationKind::Update, + MutationKind::Refresh, table_meta_timestamps, ) }); let prev_snapshot_id = snapshot.snapshot_id; - let snapshot_gen = MutationGenerator::new(Some(snapshot), MutationKind::Update); + let snapshot_gen = MutationGenerator::new(Some(snapshot), MutationKind::Refresh); pipeline.add_sink(|input| { CommitSink::try_create( fuse_table, @@ -344,6 +344,7 @@ impl AsyncTransform for VirtualColumnTransform { let extended_block_meta = ExtendedBlockMeta { block_meta: block_meta.clone(), draft_virtual_block_meta: Some(virtual_column_state.draft_virtual_block_meta), + column_hlls: None, }; let entry = MutationLogEntry::ReplacedBlock { diff --git a/src/query/service/src/interpreters/interpreter_table_set_options.rs b/src/query/service/src/interpreters/interpreter_table_set_options.rs index 93185f25feeb8..f98f2915778cf 100644 --- a/src/query/service/src/interpreters/interpreter_table_set_options.rs +++ b/src/query/service/src/interpreters/interpreter_table_set_options.rs @@ -243,10 +243,12 @@ async fn set_segment_format( for block in segment.blocks { segment_builder.add_block(block.as_ref().clone())?; } + let hlls = segment.summary.hlls; let segment = segment_builder .build( fuse_table.get_block_thresholds(), fuse_table.cluster_key_id(), + hlls, )? .serialize()?; let location_gen = fuse_table.meta_location_generator(); diff --git a/src/query/service/src/pipelines/builders/builder_commit.rs b/src/query/service/src/pipelines/builders/builder_commit.rs index 23ff8730e7347..e9241a60ae53c 100644 --- a/src/query/service/src/pipelines/builders/builder_commit.rs +++ b/src/query/service/src/pipelines/builders/builder_commit.rs @@ -99,11 +99,12 @@ impl PipelineBuilder { let extended_merged_blocks = recluster_info .merged_blocks - .iter() - .map(|block_meta| { + .into_iter() + .map(|(block_meta, column_hlls)| { Arc::new(ExtendedBlockMeta { - block_meta: Arc::unwrap_or_clone(block_meta.clone()), + block_meta: Arc::unwrap_or_clone(block_meta), draft_virtual_block_meta: None, + column_hlls, }) }) .collect::>>(); diff --git a/src/query/service/src/pipelines/builders/builder_compact.rs b/src/query/service/src/pipelines/builders/builder_compact.rs index 143e20c245220..2a0ca022d0c73 100644 --- a/src/query/service/src/pipelines/builders/builder_compact.rs +++ b/src/query/service/src/pipelines/builders/builder_compact.rs @@ -57,6 +57,7 @@ impl PipelineBuilder { if is_lazy { let query_ctx = self.ctx.clone(); + let dal = table.get_operator(); let lazy_parts = compact_block .parts @@ -78,6 +79,7 @@ impl PipelineBuilder { .block_on(async move { let partitions = BlockCompactMutator::build_compact_tasks( ctx.clone(), + dal.clone(), column_ids.clone(), cluster_key_id, thresholds, diff --git a/src/query/service/src/test_kits/block_writer.rs b/src/query/service/src/test_kits/block_writer.rs index 75914348ed1f2..91a0f5a1f89cf 100644 --- a/src/query/service/src/test_kits/block_writer.rs +++ b/src/query/service/src/test_kits/block_writer.rs @@ -21,8 +21,8 @@ use databend_common_io::constants::DEFAULT_BLOCK_BUFFER_SIZE; use databend_common_io::constants::DEFAULT_BLOCK_INDEX_BUFFER_SIZE; use databend_common_sql::ApproxDistinctColumns; use databend_common_sql::BloomIndexColumns; +use databend_common_storages_fuse::io::build_column_hlls; use databend_common_storages_fuse::io::serialize_block; -use databend_common_storages_fuse::io::BlockStatisticsState; use databend_common_storages_fuse::io::TableMetaLocationGenerator; use databend_common_storages_fuse::io::WriteSettings; use databend_common_storages_fuse::FuseStorageFormat; @@ -30,10 +30,12 @@ use databend_storages_common_blocks::blocks_to_parquet; use databend_storages_common_index::BloomIndex; use databend_storages_common_index::BloomIndexBuilder; use databend_storages_common_index::RangeIndex; +use databend_storages_common_table_meta::meta::encode_column_hll; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::ClusterStatistics; use databend_storages_common_table_meta::meta::Compression; use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::RawColumnHLL; use databend_storages_common_table_meta::meta::StatisticsOfColumns; use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::table::TableCompression; @@ -71,7 +73,7 @@ impl<'a> BlockWriter<'a> { block: DataBlock, col_stats: StatisticsOfColumns, cluster_stats: Option, - ) -> Result<(BlockMeta, Option)> { + ) -> Result<(BlockMeta, Option, RawColumnHLL)> { let (location, block_id) = if !self.is_greater_than_v5 { let location_generator = old_version_generator::TableMetaLocationGenerator::with_prefix( self.location_generator.prefix().to_string(), @@ -88,9 +90,12 @@ impl<'a> BlockWriter<'a> { let (bloom_filter_index_size, bloom_filter_index_location, meta) = self .build_block_index(data_accessor, schema.clone(), &block, block_id) .await?; - let (block_stats_size, block_stats_location) = self - .build_block_stats(data_accessor, schema.clone(), &block, block_id) - .await?; + + let hll_columns = ApproxDistinctColumns::All; + let ndv_columns_map = + hll_columns.distinct_column_fields(schema.clone(), RangeIndex::supported_table_type)?; + let column_hlls = build_column_hlls(&block, &ndv_columns_map)?.unwrap(); + let column_hlls = encode_column_hll(&column_hlls)?; let write_settings = WriteSettings { storage_format, @@ -118,12 +123,10 @@ impl<'a> BlockWriter<'a> { None, None, None, - block_stats_location, - block_stats_size, Compression::Lz4Raw, Some(Utc::now()), ); - Ok((block_meta, meta)) + Ok((block_meta, meta, column_hlls)) } pub async fn build_block_index( @@ -162,29 +165,4 @@ impl<'a> BlockWriter<'a> { Ok((0u64, None, None)) } } - - pub async fn build_block_stats( - &self, - data_accessor: &Operator, - schema: TableSchemaRef, - block: &DataBlock, - block_id: Uuid, - ) -> Result<(u64, Option)> { - let location = self.location_generator.block_stats_location(&block_id); - - let hll_columns = ApproxDistinctColumns::All; - let ndv_columns_map = - hll_columns.distinct_column_fields(schema.clone(), RangeIndex::supported_table_type)?; - let maybe_block_stats = - BlockStatisticsState::from_data_block(location, block, &ndv_columns_map)?; - if let Some(block_stats) = maybe_block_stats { - let size = block_stats.block_stats_size(); - data_accessor - .write(&block_stats.location.0, block_stats.data) - .await?; - Ok((size, Some(block_stats.location))) - } else { - Ok((0u64, None)) - } - } } diff --git a/src/query/service/src/test_kits/check.rs b/src/query/service/src/test_kits/check.rs index 6943810c7d949..5a0ebf2296757 100644 --- a/src/query/service/src/test_kits/check.rs +++ b/src/query/service/src/test_kits/check.rs @@ -24,7 +24,6 @@ use databend_common_meta_app::storage::StorageParams; use databend_common_storages_fuse::operations::load_last_snapshot_hint; use databend_common_storages_fuse::FuseTable; use databend_common_storages_fuse::FUSE_TBL_BLOCK_PREFIX; -use databend_common_storages_fuse::FUSE_TBL_BLOCK_STATISTICS_PREFIX; use databend_common_storages_fuse::FUSE_TBL_SEGMENT_PREFIX; use databend_common_storages_fuse::FUSE_TBL_SNAPSHOT_PREFIX; use databend_common_storages_fuse::FUSE_TBL_SNAPSHOT_STATISTICS_PREFIX; @@ -80,7 +79,7 @@ pub async fn check_data_dir( segment_count: u32, block_count: u32, index_count: u32, - block_stat_count: u32, + _block_stat_count: u32, check_last_snapshot: Option<()>, check_table_statistic_file: Option<()>, ) -> Result<()> { @@ -94,14 +93,12 @@ pub async fn check_data_dir( let mut sg_count = 0; let mut b_count = 0; let mut i_count = 0; - let mut b_stat_count = 0; let mut table_statistic_files = vec![]; let prefix_snapshot = FUSE_TBL_SNAPSHOT_PREFIX; let prefix_snapshot_statistics = FUSE_TBL_SNAPSHOT_STATISTICS_PREFIX; let prefix_segment = FUSE_TBL_SEGMENT_PREFIX; let prefix_block = FUSE_TBL_BLOCK_PREFIX; let prefix_index = FUSE_TBL_XOR_BLOOM_INDEX_PREFIX; - let prefix_block_stats = FUSE_TBL_BLOCK_STATISTICS_PREFIX; for entry in WalkDir::new(root) { let entry = entry.unwrap(); if entry.file_type().is_file() { @@ -113,8 +110,6 @@ pub async fn check_data_dir( ss_count += 1; } else if path.starts_with(prefix_segment) { sg_count += 1; - } else if path.starts_with(prefix_block_stats) { - b_stat_count += 1; } else if path.starts_with(prefix_block) { b_count += 1; } else if path.starts_with(prefix_index) { @@ -148,12 +143,6 @@ pub async fn check_data_dir( case_name ); - assert_eq!( - b_stat_count, block_stat_count, - "case [{}], check block statistics count", - case_name - ); - assert_eq!( i_count, index_count, "case [{}], check index count", diff --git a/src/query/service/src/test_kits/fuse.rs b/src/query/service/src/test_kits/fuse.rs index a7d7c56fbf1ea..547077432ffe1 100644 --- a/src/query/service/src/test_kits/fuse.rs +++ b/src/query/service/src/test_kits/fuse.rs @@ -30,12 +30,14 @@ use databend_common_sql::optimizer::ir::SExpr; use databend_common_sql::plans::Mutation; use databend_common_storages_factory::Table; use databend_common_storages_fuse::io::MetaWriter; +use databend_common_storages_fuse::io::TableMetaLocationGenerator; use databend_common_storages_fuse::statistics::gen_columns_statistics; use databend_common_storages_fuse::statistics::merge_statistics; use databend_common_storages_fuse::statistics::reducers::reduce_block_metas; use databend_common_storages_fuse::FuseStorageFormat; use databend_common_storages_fuse::FuseTable; use databend_common_storages_fuse::FUSE_TBL_SEGMENT_PREFIX; +use databend_storages_common_cache::SegmentStatistics; use databend_storages_common_table_meta::meta::testing::SegmentInfoV2; use databend_storages_common_table_meta::meta::testing::TableSnapshotV2; use databend_storages_common_table_meta::meta::testing::TableSnapshotV4; @@ -107,15 +109,14 @@ pub async fn generate_segments_v2( let mut segs = vec![]; for _ in 0..number_of_segments { let dal = fuse_table.get_operator_ref(); - let block_metas = generate_blocks( + let (block_metas, stats) = generate_blocks( fuse_table, blocks_per_segment, false, TestFixture::default_table_meta_timestamps(), ) .await?; - let summary = reduce_block_metas(&block_metas, BlockThresholds::default(), None); - let segment_info = SegmentInfoV2::new(block_metas, summary); + let mut summary = reduce_block_metas(&block_metas, BlockThresholds::default(), None); let uuid = Uuid::new_v4(); let location = format!( "{}/{}/{}_v{}.json", @@ -124,6 +125,11 @@ pub async fn generate_segments_v2( uuid, SegmentInfoV2::VERSION, ); + let hll_location = + TableMetaLocationGenerator::gen_segment_stats_location_from_segment_location(&location); + stats.write_meta(dal, &hll_location).await?; + summary.hlls = Some((hll_location, SegmentStatistics::VERSION)); + let segment_info = SegmentInfoV2::new(block_metas, summary); write_v2_to_storage(dal, &location, &segment_info).await?; segs.push(((location, SegmentInfoV2::VERSION), segment_info)) } @@ -141,15 +147,14 @@ pub async fn generate_segments( let location_generator = fuse_table.meta_location_generator(); for _ in 0..number_of_segments { let dal = fuse_table.get_operator_ref(); - let block_metas = generate_blocks( + let (block_metas, stats) = generate_blocks( fuse_table, blocks_per_segment, is_greater_than_v5, table_meta_timestamps, ) .await?; - let summary = reduce_block_metas(&block_metas, BlockThresholds::default(), None); - let segment_info = SegmentInfo::new(block_metas, summary); + let mut summary = reduce_block_metas(&block_metas, BlockThresholds::default(), None); let location = if is_greater_than_v5 { location_generator.gen_segment_info_location(table_meta_timestamps, false) } else { @@ -158,6 +163,11 @@ pub async fn generate_segments( ); location_generator.gen_segment_info_location(table_meta_timestamps) }; + let hll_location = + TableMetaLocationGenerator::gen_segment_stats_location_from_segment_location(&location); + stats.write_meta(dal, &hll_location).await?; + summary.hlls = Some((hll_location, SegmentStatistics::VERSION)); + let segment_info = SegmentInfo::new(block_metas, summary); segment_info.write_meta(dal, location.as_str()).await?; segs.push(((location, SegmentInfo::VERSION), segment_info)) } @@ -169,7 +179,7 @@ async fn generate_blocks( num_blocks: usize, is_greater_than_v5: bool, table_meta_timestamps: TableMetaTimestamps, -) -> Result>> { +) -> Result<(Vec>, SegmentStatistics)> { let dal = fuse_table.get_operator_ref(); let schema = fuse_table.schema(); let location_generator = fuse_table.meta_location_generator(); @@ -180,6 +190,7 @@ async fn generate_blocks( is_greater_than_v5, ); let mut block_metas = vec![]; + let mut hlls = vec![]; // does not matter in this suite let rows_per_block = 2; @@ -191,12 +202,14 @@ async fn generate_blocks( let blocks: std::vec::Vec = stream.try_collect().await?; for block in blocks { let stats = gen_columns_statistics(&block, None, &schema)?; - let (block_meta, _index_meta) = block_writer + let (block_meta, _index_meta, hll) = block_writer .write(FuseStorageFormat::Parquet, &schema, block, stats, None) .await?; block_metas.push(Arc::new(block_meta)); + hlls.push(hll); } - Ok(block_metas) + let stats = SegmentStatistics::new(hlls); + Ok((block_metas, stats)) } pub async fn generate_snapshots(fixture: &TestFixture) -> Result<()> { diff --git a/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs b/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs index bc5ecc4e76978..5f23a93f45167 100644 --- a/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs +++ b/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs @@ -340,8 +340,6 @@ fn build_test_segment_info( vector_index_size: None, vector_index_location: None, virtual_block_meta: None, - block_stats_location: None, - block_stats_size: 0, compression: Compression::Lz4, create_on: Some(Utc::now()), }; @@ -365,6 +363,7 @@ fn build_test_segment_info( col_stats: col_stats.clone(), cluster_stats: None, virtual_block_count: None, + hlls: None, }; Ok(SegmentInfo::new(block_metas, statistics)) @@ -404,7 +403,7 @@ async fn setup() -> databend_common_exception::Result { TestFixture::default_table_meta_timestamps(), true, ); - let (_block_meta, thrift_file_meta) = block_writer + let (_block_meta, thrift_file_meta, _) = block_writer .write(FuseStorageFormat::Parquet, &schema, block, col_stats, None) .await?; diff --git a/src/query/service/tests/it/storages/fuse/conflict.rs b/src/query/service/tests/it/storages/fuse/conflict.rs index 2f8c224ccfedb..d04f2db2c9a6a 100644 --- a/src/query/service/tests/it/storages/fuse/conflict.rs +++ b/src/query/service/tests/it/storages/fuse/conflict.rs @@ -106,6 +106,7 @@ fn test_resolvable_delete_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, + hlls: None, }; let mut latest_snapshot = new_empty_snapshot(TableSchema::default(), None); @@ -130,6 +131,7 @@ fn test_resolvable_delete_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, + hlls: None, }; let removed_statistics = Statistics { @@ -147,6 +149,7 @@ fn test_resolvable_delete_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, + hlls: None, }; let merged_statistics = Statistics { @@ -164,6 +167,7 @@ fn test_resolvable_delete_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, + hlls: None, }; let ctx = ConflictResolveContext::ModifiedSegmentExistsInLatest(SnapshotChanges { @@ -207,6 +211,7 @@ fn test_resolvable_delete_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, + hlls: None, }; assert_eq!(actual, expected); } @@ -244,6 +249,7 @@ fn test_resolvable_replace_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, + hlls: None, }; let mut latest_snapshot = new_empty_snapshot(TableSchema::default(), None); @@ -268,6 +274,7 @@ fn test_resolvable_replace_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, + hlls: None, }; let removed_statistics = Statistics { @@ -285,6 +292,7 @@ fn test_resolvable_replace_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, + hlls: None, }; let merged_statistics = Statistics { @@ -302,6 +310,7 @@ fn test_resolvable_replace_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, + hlls: None, }; let ctx = ConflictResolveContext::ModifiedSegmentExistsInLatest(SnapshotChanges { @@ -350,6 +359,7 @@ fn test_resolvable_replace_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, + hlls: None, }; assert_eq!(actual, expected); } diff --git a/src/query/service/tests/it/storages/fuse/meta/column_oriented.rs b/src/query/service/tests/it/storages/fuse/meta/column_oriented.rs index dc50928f2b4e1..28dd954cafcaf 100644 --- a/src/query/service/tests/it/storages/fuse/meta/column_oriented.rs +++ b/src/query/service/tests/it/storages/fuse/meta/column_oriented.rs @@ -95,7 +95,7 @@ async fn generate_column_oriented_segment( TestFixture::default_table_meta_timestamps(), true, ); - let (block_meta, _index_meta) = block_writer + let (block_meta, _index_meta, _) = block_writer .write( FuseStorageFormat::Parquet, &table_schema, @@ -112,7 +112,9 @@ async fn generate_column_oriented_segment( for block_meta in block_metas.iter() { segment_builder.add_block(block_meta.clone()).unwrap(); } - segment_builder.build(Default::default(), Some(0)).unwrap() + segment_builder + .build(Default::default(), Some(0), None) + .unwrap() }; assert_eq!( @@ -288,39 +290,6 @@ fn check_block_level_meta( assert!(is_null); } - // check block stats location - let block_stats_location = column_oriented_segment - .col_by_name(&[BLOCK_STATS_LOCATION]) - .unwrap(); - for (block_stats_location, block_meta) in block_stats_location.iter().zip(block_metas.iter()) { - let block_stats_location = block_stats_location.as_tuple(); - if let Some(block_stats_location) = block_stats_location { - assert_eq!( - block_stats_location[0].as_string().unwrap(), - &block_meta.block_stats_location.as_ref().unwrap().0 - ); - assert_eq!( - block_stats_location[1] - .as_number() - .unwrap() - .as_u_int64() - .unwrap(), - &block_meta.block_stats_location.as_ref().unwrap().1 - ); - } else { - assert!(block_meta.block_stats_location.is_none()); - } - } - - // check block stats size - let block_stats_size = column_oriented_segment - .col_by_name(&[BLOCK_STATS_SIZE]) - .unwrap(); - for (block_stats_size, block_meta) in block_stats_size.iter().zip(block_metas.iter()) { - let block_stats_size = block_stats_size.as_number().unwrap().as_u_int64().unwrap(); - assert_eq!(block_stats_size, &block_meta.block_stats_size); - } - // check compression let compression = column_oriented_segment.col_by_name(&[COMPRESSION]).unwrap(); for (compression, block_meta) in compression.iter().zip(block_metas.iter()) { @@ -405,7 +374,7 @@ async fn test_segment_cache() -> Result<()> { ) .await?; let cached = cache.get(&location).unwrap(); - assert_eq!(cached.segment_schema.fields.len(), 12); + assert_eq!(cached.segment_schema.fields.len(), 10); assert_eq!(cached.segment_schema, segment_schema(&TableSchema::empty())); check_summary(&block_metas, &cached); check_block_level_meta(&block_metas, &cached); @@ -418,7 +387,7 @@ async fn test_segment_cache() -> Result<()> { let _column_oriented_segment = read_column_oriented_segment(operator.clone(), &location, &projection, true).await?; let cached = cache.get(&location).unwrap(); - assert_eq!(cached.segment_schema.fields.len(), 14); + assert_eq!(cached.segment_schema.fields.len(), 12); let column_1 = table_schema.field_of_column_id(col_id).unwrap(); let stat_1 = column_oriented_segment @@ -442,7 +411,7 @@ async fn test_segment_cache() -> Result<()> { read_column_oriented_segment(operator.clone(), &location, &projection, true).await?; let cached = cache.get(&location).unwrap(); // column 2 does not have stats - assert_eq!(cached.segment_schema.fields.len(), 15); + assert_eq!(cached.segment_schema.fields.len(), 13); check_summary(&block_metas, &cached); check_block_level_meta(&block_metas, &cached); check_column_stats_and_meta(&block_metas, &cached, &[1, 2]); @@ -456,7 +425,7 @@ async fn test_segment_cache() -> Result<()> { read_column_oriented_segment(operator.clone(), &location, &projection, true).await?; let cached = cache.get(&location).unwrap(); // column 2 does not have stats - assert_eq!(cached.segment_schema.fields.len(), 15); + assert_eq!(cached.segment_schema.fields.len(), 13); check_summary(&block_metas, &cached); check_block_level_meta(&block_metas, &cached); check_column_stats_and_meta(&block_metas, &cached, &[1, 2]); diff --git a/src/query/service/tests/it/storages/fuse/operations/mutation/block_compact_mutator.rs b/src/query/service/tests/it/storages/fuse/operations/mutation/block_compact_mutator.rs index f57ac1b95d9da..da5e8cc435d11 100644 --- a/src/query/service/tests/it/storages/fuse/operations/mutation/block_compact_mutator.rs +++ b/src/query/service/tests/it/storages/fuse/operations/mutation/block_compact_mutator.rs @@ -285,8 +285,8 @@ pub async fn verify_compact_tasks( compact_segment_indices.insert(extra.segment_index); compact_segment_indices.extend(extra.removed_segment_indexes.iter()); actual_blocks_number += extra.unchanged_blocks.len(); - for b in &extra.unchanged_blocks { - actual_block_ids.insert(b.1.location.clone()); + for (_, b) in &extra.unchanged_blocks { + actual_block_ids.insert(b.0.location.clone()); } } CompactBlockPartInfo::CompactTaskInfo(task) => { diff --git a/src/query/service/tests/it/storages/fuse/operations/mutation/recluster_mutator.rs b/src/query/service/tests/it/storages/fuse/operations/mutation/recluster_mutator.rs index 8d0ec47d7f19e..c70c37a235cb8 100644 --- a/src/query/service/tests/it/storages/fuse/operations/mutation/recluster_mutator.rs +++ b/src/query/service/tests/it/storages/fuse/operations/mutation/recluster_mutator.rs @@ -81,8 +81,6 @@ async fn test_recluster_mutator_block_select() -> Result<()> { None, None, None, - None, - 0, meta::Compression::Lz4Raw, Some(Utc::now()), )); @@ -154,6 +152,7 @@ async fn test_recluster_mutator_block_select() -> Result<()> { let column_ids = snapshot.schema.to_leaf_column_id_set(); let mutator = ReclusterMutator::new( ctx, + data_accessor, schema, vec![DataType::Number(NumberDataType::Int64)], 1.0, @@ -278,6 +277,7 @@ async fn test_safety_for_recluster() -> Result<()> { let mut parts = ReclusterParts::new_recluster_parts(); let mutator = Arc::new(ReclusterMutator::new( ctx.clone(), + data_accessor.clone(), schema.clone(), vec![DataType::Number(NumberDataType::Int32)], 1.0, @@ -331,7 +331,7 @@ async fn test_safety_for_recluster() -> Result<()> { remained_blocks.len() ); for remain in remained_blocks { - blocks.push(remain.location.0.clone()); + blocks.push(remain.0.location.0.clone()); } let block_ids_after_target = HashSet::from_iter(blocks.into_iter()); diff --git a/src/query/service/tests/it/storages/fuse/operations/mutation/segments_compact_mutator.rs b/src/query/service/tests/it/storages/fuse/operations/mutation/segments_compact_mutator.rs index 8387d3c7a2ace..df406a607f94c 100644 --- a/src/query/service/tests/it/storages/fuse/operations/mutation/segments_compact_mutator.rs +++ b/src/query/service/tests/it/storages/fuse/operations/mutation/segments_compact_mutator.rs @@ -781,8 +781,6 @@ impl CompactSegmentTestFixture { None, None, None, - None, - 0, Compression::Lz4Raw, Some(Utc::now()), ); @@ -790,7 +788,7 @@ impl CompactSegmentTestFixture { collected_blocks.push(block_meta.clone()); stats_acc.add_block(block_meta).unwrap(); } - let segment_info = stats_acc.build(thresholds, cluster_key_id)?; + let segment_info = stats_acc.build(thresholds, cluster_key_id, None)?; let path = location_gen .gen_segment_info_location(TestFixture::default_table_meta_timestamps(), false); segment_info.write_meta(&data_accessor, &path).await?; diff --git a/src/query/service/tests/it/storages/fuse/operations/read_plan.rs b/src/query/service/tests/it/storages/fuse/operations/read_plan.rs index eb2d64ad67465..09dccdb11816e 100644 --- a/src/query/service/tests/it/storages/fuse/operations/read_plan.rs +++ b/src/query/service/tests/it/storages/fuse/operations/read_plan.rs @@ -107,8 +107,6 @@ fn test_to_partitions() -> Result<()> { None, None, None, - None, - 0, meta::Compression::Lz4Raw, Some(Utc::now()), )); diff --git a/src/query/service/tests/it/storages/fuse/operations/table_analyze.rs b/src/query/service/tests/it/storages/fuse/operations/table_analyze.rs index 2a8c066b637f8..125b3ded06e39 100644 --- a/src/query/service/tests/it/storages/fuse/operations/table_analyze.rs +++ b/src/query/service/tests/it/storages/fuse/operations/table_analyze.rs @@ -33,7 +33,7 @@ use databend_query::sql::plans::Plan; use databend_query::sql::Planner; use databend_query::test_kits::*; use databend_storages_common_cache::LoadParams; -use databend_storages_common_table_meta::meta::MetaHLL; +use databend_storages_common_table_meta::meta::MetaHLL12; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::Statistics; use databend_storages_common_table_meta::meta::TableSnapshot; @@ -216,7 +216,8 @@ async fn test_table_analyze_without_prev_table_seq() -> Result<()> { // generate table statistics. let col: Vec = vec![1, 3, 0, 0, 0, 118, 5, 1, 21, 6, 3, 229, 13, 3]; - let hll: HashMap = HashMap::from([(0, borsh_deserialize_from_slice(&col)?)]); + let hll: HashMap = + HashMap::from([(0, borsh_deserialize_from_slice(&col)?)]); let table_statistics = TableSnapshotStatistics::new(hll, HashMap::new(), snapshot_1.snapshot_id); let table_statistics_location = location_gen.snapshot_statistics_location_from_uuid( diff --git a/src/query/service/tests/it/storages/fuse/statistics.rs b/src/query/service/tests/it/storages/fuse/statistics.rs index 4bcfc5bf9cf6d..f039a43ed65e5 100644 --- a/src/query/service/tests/it/storages/fuse/statistics.rs +++ b/src/query/service/tests/it/storages/fuse/statistics.rs @@ -327,7 +327,7 @@ async fn test_accumulator() -> databend_common_exception::Result<()> { TestFixture::default_table_meta_timestamps(), true, ); - let (block_meta, _index_meta) = block_writer + let (block_meta, _index_meta, _) = block_writer .write(FuseStorageFormat::Parquet, &schema, block, col_stats, None) .await?; stats_acc.add_block(block_meta).unwrap(); @@ -637,8 +637,6 @@ fn test_reduce_block_meta() -> databend_common_exception::Result<()> { None, None, None, - None, - 0, Compression::Lz4Raw, Some(Utc::now()), ); diff --git a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt index a231a051923c8..536eb79d132c6 100644 --- a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt +++ b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt @@ -21,6 +21,7 @@ DB.Table: 'system'.'configs', Table: configs-table_id:1, ver:0, Engine: SystemCo | 'cache' | 'inverted_index_meta_count' | '3000' | '' | | 'cache' | 'meta_service_ownership_cache' | 'false' | '' | | 'cache' | 'segment_block_metas_count' | '0' | '' | +| 'cache' | 'segment_stastistics_count' | '0' | '' | | 'cache' | 'table_bloom_index_filter_count' | '0' | '' | | 'cache' | 'table_bloom_index_filter_size' | '2147483648' | '' | | 'cache' | 'table_bloom_index_meta_count' | '3000' | '' | diff --git a/src/query/sql/src/executor/physical_plans/common.rs b/src/query/sql/src/executor/physical_plans/common.rs index 545179b4af4d6..e887f496f6e38 100644 --- a/src/query/sql/src/executor/physical_plans/common.rs +++ b/src/query/sql/src/executor/physical_plans/common.rs @@ -78,6 +78,8 @@ pub enum MutationKind { Insert, Compact, MergeInto, + /// refresh index, virtual column. + Refresh, } impl Display for MutationKind { @@ -90,6 +92,7 @@ impl Display for MutationKind { MutationKind::Replace => write!(f, "Replace"), MutationKind::Compact => write!(f, "Compact"), MutationKind::MergeInto => write!(f, "MergeInto"), + MutationKind::Refresh => write!(f, "Refresh"), } } } diff --git a/src/query/storages/common/cache/src/cache_items.rs b/src/query/storages/common/cache/src/cache_items.rs index 1f39fd50c10ef..95e36b5880701 100644 --- a/src/query/storages/common/cache/src/cache_items.rs +++ b/src/query/storages/common/cache/src/cache_items.rs @@ -27,6 +27,7 @@ pub use databend_storages_common_table_meta::meta::column_oriented_segment::Colu pub use databend_storages_common_table_meta::meta::BlockMeta; pub use databend_storages_common_table_meta::meta::CompactSegmentInfo; pub use databend_storages_common_table_meta::meta::SegmentInfo; +pub use databend_storages_common_table_meta::meta::SegmentStatistics; pub use databend_storages_common_table_meta::meta::TableSnapshot; pub use databend_storages_common_table_meta::meta::TableSnapshotStatistics; pub use parquet::file::metadata::ParquetMetaData; diff --git a/src/query/storages/common/cache/src/caches.rs b/src/query/storages/common/cache/src/caches.rs index 944462e5bd4bc..0324f602fd997 100644 --- a/src/query/storages/common/cache/src/caches.rs +++ b/src/query/storages/common/cache/src/caches.rs @@ -43,6 +43,8 @@ pub type BlockMetaCache = InMemoryLruCache; pub type TableSnapshotCache = InMemoryLruCache; /// In memory object cache of TableSnapshotStatistics pub type TableSnapshotStatisticCache = InMemoryLruCache; +/// In memory object cache of SegmentStatistics +pub type SegmentStatisticsCache = InMemoryLruCache; /// In memory object cache of bloom filter. /// For each indexed data block, the bloom xor8 filter of column is cached individually pub type BloomIndexFilterCache = HybridCache; @@ -112,6 +114,13 @@ impl CachedObject for TableSnapshotStatistics { } } +impl CachedObject for SegmentStatistics { + type Cache = SegmentStatisticsCache; + fn cache() -> Option { + CacheManager::instance().get_segment_statistics_cache() + } +} + impl CachedObject for BloomIndexMeta { type Cache = BloomIndexMetaCache; fn cache() -> Option { @@ -252,6 +261,15 @@ impl From for CacheValue { } } +impl From for CacheValue { + fn from(value: SegmentStatistics) -> Self { + CacheValue { + inner: Arc::new(value), + mem_bytes: 0, + } + } +} + impl From for CacheValue { fn from(value: FilterImpl) -> Self { CacheValue { diff --git a/src/query/storages/common/cache/src/manager.rs b/src/query/storages/common/cache/src/manager.rs index 1b8432a52ce98..725c87b6dfb73 100644 --- a/src/query/storages/common/cache/src/manager.rs +++ b/src/query/storages/common/cache/src/manager.rs @@ -50,6 +50,7 @@ use crate::CacheAccessor; use crate::DiskCacheAccessor; use crate::DiskCacheBuilder; use crate::InMemoryLruCache; +use crate::SegmentStatisticsCache; use crate::Unit; static DEFAULT_PARQUET_META_DATA_CACHE_ITEMS: usize = 3000; @@ -100,6 +101,7 @@ pub enum CacheClearanceLevel { pub struct CacheManager { table_snapshot_cache: CacheSlot, table_statistic_cache: CacheSlot, + segment_statistics_cache: CacheSlot, compact_segment_info_cache: CacheSlot, column_oriented_segment_info_cache: CacheSlot, bloom_index_filter_cache: CacheSlot, @@ -232,6 +234,7 @@ impl CacheManager { prune_partitions_cache: CacheSlot::new(None), parquet_meta_data_cache: CacheSlot::new(None), table_statistic_cache: CacheSlot::new(None), + segment_statistics_cache: CacheSlot::new(None), in_memory_table_data_cache, segment_block_metas_cache: CacheSlot::new(None), block_meta_cache: CacheSlot::new(None), @@ -248,6 +251,10 @@ impl CacheManager { MEMORY_CACHE_TABLE_STATISTICS, config.table_meta_statistic_count as usize, ); + let segment_statistics_cache = Self::new_items_cache_slot( + MEMORY_CACHE_SEGMENT_STATISTICS, + config.table_meta_statistic_count as usize, + ); let compact_segment_info_cache = Self::new_bytes_cache_slot( MEMORY_CACHE_COMPACT_SEGMENT_INFO, config.table_meta_segment_bytes as usize, @@ -364,6 +371,7 @@ impl CacheManager { vector_index_file_cache, prune_partitions_cache, table_statistic_cache, + segment_statistics_cache, in_memory_table_data_cache, segment_block_metas_cache, parquet_meta_data_cache, @@ -597,6 +605,10 @@ impl CacheManager { self.table_statistic_cache.get() } + pub fn get_segment_statistics_cache(&self) -> Option { + self.segment_statistics_cache.get() + } + pub fn get_table_segment_cache(&self) -> Option { self.compact_segment_info_cache.get() } @@ -792,6 +804,7 @@ const IN_MEMORY_HYBRID_CACHE_BLOOM_INDEX_FILTER: &str = "memory_cache_bloom_inde const MEMORY_CACHE_COMPACT_SEGMENT_INFO: &str = "memory_cache_compact_segment_info"; const MEMORY_CACHE_COLUMN_ORIENTED_SEGMENT_INFO: &str = "memory_cache_column_oriented_segment_info"; const MEMORY_CACHE_TABLE_STATISTICS: &str = "memory_cache_table_statistics"; +const MEMORY_CACHE_SEGMENT_STATISTICS: &str = "memory_cache_segment_statistics"; const MEMORY_CACHE_TABLE_SNAPSHOT: &str = "memory_cache_table_snapshot"; const MEMORY_CACHE_SEGMENT_BLOCK_METAS: &str = "memory_cache_segment_block_metas"; const MEMORY_CACHE_ICEBERG_TABLE: &str = "memory_cache_iceberg_table"; @@ -1033,8 +1046,6 @@ mod tests { vector_index_location: None, vector_index_size: None, virtual_block_meta: None, - block_stats_location: None, - block_stats_size: 0, compression: Compression::Lz4, create_on: None, }); diff --git a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/mod.rs b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/mod.rs index 21787350577ee..86453a71543fc 100644 --- a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/mod.rs +++ b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/mod.rs @@ -26,8 +26,6 @@ pub use schema::meta_name; pub use schema::segment_schema; pub use schema::stat_name; pub use schema::BLOCK_SIZE; -pub use schema::BLOCK_STATS_LOCATION; -pub use schema::BLOCK_STATS_SIZE; pub use schema::BLOOM_FILTER_INDEX_LOCATION; pub use schema::BLOOM_FILTER_INDEX_SIZE; pub use schema::CLUSTER_STATS; diff --git a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/schema.rs b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/schema.rs index daea76a037897..dbba08ce7c622 100644 --- a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/schema.rs +++ b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/schema.rs @@ -34,8 +34,6 @@ pub const COMPRESSION: &str = "compression"; pub const CREATE_ON: &str = "create_on"; pub const LOCATION_PATH: &str = "path"; pub const LOCATION_FORMAT_VERSION: &str = "format_version"; -pub const BLOCK_STATS_LOCATION: &str = "block_stats_location"; -pub const BLOCK_STATS_SIZE: &str = "block_stats_size"; pub fn block_level_field_names() -> HashSet { let mut set = HashSet::new(); @@ -47,8 +45,6 @@ pub fn block_level_field_names() -> HashSet { set.insert(BLOOM_FILTER_INDEX_LOCATION.to_string()); set.insert(BLOOM_FILTER_INDEX_SIZE.to_string()); set.insert(INVERTED_INDEX_SIZE.to_string()); - set.insert(BLOCK_STATS_LOCATION.to_string()); - set.insert(BLOCK_STATS_SIZE.to_string()); set.insert(COMPRESSION.to_string()); set.insert(CREATE_ON.to_string()); set @@ -134,8 +130,6 @@ pub fn segment_schema(table_schema: &TableSchema) -> TableSchema { TableField::new(BLOOM_FILTER_INDEX_LOCATION, nullable_location_type()), TableField::new(BLOOM_FILTER_INDEX_SIZE, u64_t.clone()), TableField::new(INVERTED_INDEX_SIZE, nullable_u64_t.clone()), - TableField::new(BLOCK_STATS_LOCATION, nullable_location_type()), - TableField::new(BLOCK_STATS_SIZE, u64_t.clone()), TableField::new(COMPRESSION, u8_t.clone()), TableField::new(CREATE_ON, i64_t.clone()), ]; diff --git a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/segment_builder.rs b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/segment_builder.rs index 917fc6e935de6..41bb7ac029399 100644 --- a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/segment_builder.rs +++ b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/segment_builder.rs @@ -58,6 +58,7 @@ pub trait SegmentBuilder: Send + Sync + 'static { &mut self, thresholds: BlockThresholds, default_cluster_key_id: Option, + hlls: Option, ) -> Result; fn new(table_schema: TableSchemaRef, block_per_segment: usize) -> Self; } @@ -72,8 +73,6 @@ pub struct ColumnOrientedSegmentBuilder { bloom_filter_index_size: Vec, inverted_index_size: Vec>, virtual_block_meta: Vec>, - block_stats_location: LocationsWithOption, - block_stats_size: Vec, compression: Vec, create_on: Vec>, column_stats: HashMap, @@ -137,9 +136,6 @@ impl SegmentBuilder for ColumnOrientedSegmentBuilder { self.inverted_index_size .push(block_meta.inverted_index_size); self.virtual_block_meta.push(block_meta.virtual_block_meta); - self.block_stats_location - .add_location(block_meta.block_stats_location.as_ref()); - self.block_stats_size.push(block_meta.block_stats_size); self.compression.push(block_meta.compression.to_u8()); self.create_on .push(block_meta.create_on.map(|t| t.timestamp())); @@ -164,12 +160,13 @@ impl SegmentBuilder for ColumnOrientedSegmentBuilder { &mut self, thresholds: BlockThresholds, default_cluster_key_id: Option, + hlls: Option, ) -> Result { let mut this = std::mem::replace( self, ColumnOrientedSegmentBuilder::new(self.table_schema.clone(), self.block_per_segment), ); - let summary = this.build_summary(thresholds, default_cluster_key_id)?; + let summary = this.build_summary(thresholds, default_cluster_key_id, hlls)?; let cluster_stats = this.cluster_stats; let mut cluster_stats_binary = Vec::with_capacity(cluster_stats.len()); for stats in cluster_stats { @@ -197,14 +194,6 @@ impl SegmentBuilder for ColumnOrientedSegmentBuilder { ))), UInt64Type::from_data(this.bloom_filter_index_size), UInt64Type::from_opt_data(this.inverted_index_size), - Column::Nullable(Box::new(NullableColumn::new( - Column::Tuple(vec![ - StringType::from_data(this.block_stats_location.locations), - UInt64Type::from_data(this.block_stats_location.versions), - ]), - this.block_stats_location.validity.into(), - ))), - UInt64Type::from_data(this.block_stats_size), UInt8Type::from_data(this.compression), Int64Type::from_opt_data(this.create_on), ]; @@ -267,8 +256,6 @@ impl SegmentBuilder for ColumnOrientedSegmentBuilder { bloom_filter_index_size: Vec::with_capacity(block_per_segment), inverted_index_size: Vec::with_capacity(block_per_segment), virtual_block_meta: Vec::with_capacity(block_per_segment), - block_stats_location: LocationsWithOption::new_with_capacity(block_per_segment), - block_stats_size: Vec::with_capacity(block_per_segment), compression: Vec::with_capacity(block_per_segment), create_on: Vec::with_capacity(block_per_segment), column_stats, @@ -285,6 +272,7 @@ impl ColumnOrientedSegmentBuilder { &mut self, thresholds: BlockThresholds, default_cluster_key_id: Option, + hlls: Option, ) -> Result { let row_count = self.row_count.iter().sum(); let block_count = self.row_count.len() as u64; @@ -377,6 +365,7 @@ impl ColumnOrientedSegmentBuilder { col_stats, cluster_stats, virtual_block_count: Some(virtual_block_count), + hlls, }) } } diff --git a/src/query/storages/common/table_meta/src/meta/current/mod.rs b/src/query/storages/common/table_meta/src/meta/current/mod.rs index cc254a566e671..54b9aeae9ae30 100644 --- a/src/query/storages/common/table_meta/src/meta/current/mod.rs +++ b/src/query/storages/common/table_meta/src/meta/current/mod.rs @@ -20,12 +20,12 @@ pub use v2::ColumnStatistics; pub use v2::DraftVirtualBlockMeta; pub use v2::DraftVirtualColumnMeta; pub use v2::ExtendedBlockMeta; -pub use v2::MetaHLL; +pub use v2::MetaHLL12; +pub use v2::SegmentStatistics; pub use v2::Statistics; pub use v2::VirtualBlockMeta; pub use v2::VirtualColumnMeta; pub use v3::TableSnapshotStatistics; -pub use v4::BlockStatistics; pub use v4::CompactSegmentInfo; pub use v4::RawBlockMeta; pub use v4::SegmentInfo; diff --git a/src/query/storages/common/table_meta/src/meta/mod.rs b/src/query/storages/common/table_meta/src/meta/mod.rs index 3c9133ec4fdca..5e1e67a68507f 100644 --- a/src/query/storages/common/table_meta/src/meta/mod.rs +++ b/src/query/storages/common/table_meta/src/meta/mod.rs @@ -49,8 +49,8 @@ pub use utils::VACUUM2_OBJECT_KEY_PREFIX; pub(crate) use utils::*; pub use v0::ColumnMeta as ColumnMetaV0; pub use versions::testify_version; -pub use versions::BlockStatisticsVersion; pub use versions::SegmentInfoVersion; +pub use versions::SegmentStatisticsVersion; pub use versions::SnapshotVersion; pub use versions::TableSnapshotStatisticsVersion; pub use versions::Versioned; diff --git a/src/query/storages/common/table_meta/src/meta/statistics.rs b/src/query/storages/common/table_meta/src/meta/statistics.rs index 0506f04df1e5e..5cc51aebc5bc1 100644 --- a/src/query/storages/common/table_meta/src/meta/statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/statistics.rs @@ -13,18 +13,27 @@ // limitations under the License. use std::collections::HashMap; +use std::io::Cursor; use databend_common_base::base::uuid::Uuid; +use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::ColumnId; +use databend_common_storage::MetaHLL; +use crate::meta::format::compress; +use crate::meta::format::encode; +use crate::meta::format::read_and_deserialize; use crate::meta::ColumnStatistics; +use crate::meta::SegmentStatistics; pub type FormatVersion = u64; pub type SnapshotId = Uuid; pub type Location = (String, FormatVersion); pub type ClusterKey = (u32, String); pub type StatisticsOfColumns = HashMap; +pub type ColumnHLL = HashMap; +pub type RawColumnHLL = Vec; // Assigned to executors, describes that which blocks of given segment, an executor should take care of #[derive(Clone, Debug, serde::Serialize, serde::Deserialize, PartialEq)] @@ -49,3 +58,23 @@ pub fn supported_stat_type(data_type: &DataType) -> bool { | DataType::Decimal(_) ) } + +pub fn encode_column_hll(hll: &ColumnHLL) -> Result { + let encoding = SegmentStatistics::encoding(); + let compression = SegmentStatistics::compression(); + + let data = encode(&encoding, hll)?; + let data_compress = compress(&compression, data)?; + Ok(data_compress) +} + +pub fn decode_column_hll(data: &RawColumnHLL) -> Result> { + if data.is_empty() { + return Ok(None); + } + let encoding = SegmentStatistics::encoding(); + let compression = SegmentStatistics::compression(); + let mut reader = Cursor::new(&data); + let res = read_and_deserialize(&mut reader, data.len() as u64, &encoding, &compression)?; + Ok(Some(res)) +} diff --git a/src/query/storages/common/table_meta/src/meta/v2/mod.rs b/src/query/storages/common/table_meta/src/meta/v2/mod.rs index 45235b7eb7589..2b819ed9a9015 100644 --- a/src/query/storages/common/table_meta/src/meta/v2/mod.rs +++ b/src/query/storages/common/table_meta/src/meta/v2/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. mod segment; +mod segment_statistics; mod snapshot; pub mod statistics; mod table_snapshot_statistics; @@ -25,9 +26,10 @@ pub use segment::ExtendedBlockMeta; pub use segment::SegmentInfo; pub use segment::VirtualBlockMeta; pub use segment::VirtualColumnMeta; +pub use segment_statistics::SegmentStatistics; pub use snapshot::TableSnapshot; pub use statistics::ClusterStatistics; pub use statistics::ColumnStatistics; pub use statistics::Statistics; -pub use table_snapshot_statistics::MetaHLL; +pub use table_snapshot_statistics::MetaHLL12; pub use table_snapshot_statistics::TableSnapshotStatistics; diff --git a/src/query/storages/common/table_meta/src/meta/v2/segment.rs b/src/query/storages/common/table_meta/src/meta/v2/segment.rs index f9b0bcbd403bd..603ec014a19ea 100644 --- a/src/query/storages/common/table_meta/src/meta/v2/segment.rs +++ b/src/query/storages/common/table_meta/src/meta/v2/segment.rs @@ -37,6 +37,7 @@ use crate::meta::ColumnStatistics; use crate::meta::Compression; use crate::meta::FormatVersion; use crate::meta::Location; +use crate::meta::RawColumnHLL; use crate::meta::Statistics; use crate::meta::Versioned; @@ -176,10 +177,6 @@ pub struct BlockMeta { pub virtual_block_meta: Option, pub compression: Compression, - pub block_stats_location: Option, - #[serde(default)] - pub block_stats_size: u64, - // block create_on pub create_on: Option>, } @@ -201,8 +198,6 @@ impl BlockMeta { vector_index_size: Option, vector_index_location: Option, virtual_block_meta: Option, - block_stats_location: Option, - block_stats_size: u64, compression: Compression, create_on: Option>, ) -> Self { @@ -221,8 +216,6 @@ impl BlockMeta { vector_index_size, vector_index_location, virtual_block_meta, - block_stats_location, - block_stats_size, compression, create_on, } @@ -251,6 +244,7 @@ impl BlockMeta { pub struct ExtendedBlockMeta { pub block_meta: BlockMeta, pub draft_virtual_block_meta: Option, + pub column_hlls: Option, } #[typetag::serde(name = "extended_block_meta")] @@ -385,8 +379,6 @@ impl BlockMeta { vector_index_size: None, vector_index_location: None, virtual_block_meta: None, - block_stats_location: None, - block_stats_size: 0, create_on: None, ngram_filter_index_size: None, } @@ -415,8 +407,6 @@ impl BlockMeta { vector_index_size: None, vector_index_location: None, virtual_block_meta: None, - block_stats_location: None, - block_stats_size: 0, create_on: None, ngram_filter_index_size: None, } diff --git a/src/query/storages/common/table_meta/src/meta/v4/block_statistics.rs b/src/query/storages/common/table_meta/src/meta/v2/segment_statistics.rs similarity index 77% rename from src/query/storages/common/table_meta/src/meta/v4/block_statistics.rs rename to src/query/storages/common/table_meta/src/meta/v2/segment_statistics.rs index e453f47f12630..d6f7129d7fea5 100644 --- a/src/query/storages/common/table_meta/src/meta/v4/block_statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/v2/segment_statistics.rs @@ -12,12 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; -use std::io::Cursor; use std::io::Read; use databend_common_exception::Result; -use databend_common_expression::ColumnId; use databend_common_io::prelude::BinaryRead; use serde::Deserialize; use serde::Serialize; @@ -25,27 +22,35 @@ use serde::Serialize; use crate::meta::format::compress; use crate::meta::format::encode; use crate::meta::format::read_and_deserialize; -use crate::meta::versions::Versioned; use crate::meta::FormatVersion; use crate::meta::MetaCompression; use crate::meta::MetaEncoding; -use crate::meta::MetaHLL; +use crate::meta::RawColumnHLL; +use crate::meta::Versioned; #[derive(Serialize, Deserialize, Clone, Debug)] -pub struct BlockStatistics { +pub struct SegmentStatistics { pub format_version: FormatVersion, - pub hll: HashMap, + pub blocks: Vec, } -impl BlockStatistics { - pub fn new(hll: HashMap) -> Self { +impl SegmentStatistics { + pub fn new(blocks: Vec) -> Self { Self { - format_version: BlockStatistics::VERSION, - hll, + format_version: SegmentStatistics::VERSION, + blocks, } } + pub fn encoding() -> MetaEncoding { + MetaEncoding::MessagePack + } + + pub fn compression() -> MetaCompression { + MetaCompression::Zstd + } + pub fn to_bytes(&self) -> Result> { let encoding = MetaEncoding::MessagePack; let compression = MetaCompression::default(); @@ -69,13 +74,9 @@ impl BlockStatistics { Ok(buf) } - pub fn from_slice(buffer: &[u8]) -> Result { - Self::from_read(Cursor::new(buffer)) - } - - pub fn from_read(mut r: impl Read) -> Result { + pub fn from_read(mut r: impl Read) -> Result { let version = r.read_scalar::()?; - assert_eq!(version, BlockStatistics::VERSION); + assert_eq!(version, SegmentStatistics::VERSION); let encoding = MetaEncoding::try_from(r.read_scalar::()?)?; let compression = MetaCompression::try_from(r.read_scalar::()?)?; let statistics_size: u64 = r.read_scalar::()?; diff --git a/src/query/storages/common/table_meta/src/meta/v2/statistics.rs b/src/query/storages/common/table_meta/src/meta/v2/statistics.rs index 19e1fc0303fca..9489cd6748b11 100644 --- a/src/query/storages/common/table_meta/src/meta/v2/statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/v2/statistics.rs @@ -28,6 +28,7 @@ use serde::de::Error; use crate::meta::supported_stat_type; use crate::meta::v0; +use crate::meta::Location; #[derive(serde::Serialize, serde::Deserialize, Debug, Clone, PartialEq, Eq)] pub struct ColumnStatistics { @@ -88,6 +89,8 @@ pub struct Statistics { pub col_stats: HashMap, pub cluster_stats: Option, pub virtual_block_count: Option, + + pub hlls: Option, } // conversions from old meta data @@ -253,6 +256,7 @@ impl Statistics { col_stats, cluster_stats: None, virtual_block_count: None, + hlls: None, } } } diff --git a/src/query/storages/common/table_meta/src/meta/v2/table_snapshot_statistics.rs b/src/query/storages/common/table_meta/src/meta/v2/table_snapshot_statistics.rs index 17aae1a8c6083..86e4d06cab518 100644 --- a/src/query/storages/common/table_meta/src/meta/v2/table_snapshot_statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/v2/table_snapshot_statistics.rs @@ -23,7 +23,7 @@ use crate::meta::FormatVersion; use crate::meta::SnapshotId; use crate::meta::Versioned; -pub type MetaHLL = simple_hll::HyperLogLog<12>; +pub type MetaHLL12 = simple_hll::HyperLogLog<12>; #[derive(Serialize, Deserialize, Clone, Debug)] pub struct TableSnapshotStatistics { @@ -32,11 +32,11 @@ pub struct TableSnapshotStatistics { /// id of snapshot pub snapshot_id: SnapshotId, - pub hll: HashMap, + pub hll: HashMap, } impl TableSnapshotStatistics { - pub fn new(hll: HashMap, snapshot_id: SnapshotId) -> Self { + pub fn new(hll: HashMap, snapshot_id: SnapshotId) -> Self { Self { format_version: TableSnapshotStatistics::VERSION, snapshot_id, diff --git a/src/query/storages/common/table_meta/src/meta/v3/frozen/block_meta.rs b/src/query/storages/common/table_meta/src/meta/v3/frozen/block_meta.rs index a02b2761848f6..ccc22fdbdb600 100644 --- a/src/query/storages/common/table_meta/src/meta/v3/frozen/block_meta.rs +++ b/src/query/storages/common/table_meta/src/meta/v3/frozen/block_meta.rs @@ -66,8 +66,6 @@ impl From for crate::meta::BlockMeta { vector_index_size: None, vector_index_location: None, virtual_block_meta: None, - block_stats_location: None, - block_stats_size: 0, compression: value.compression.into(), create_on: None, } diff --git a/src/query/storages/common/table_meta/src/meta/v3/frozen/statistics.rs b/src/query/storages/common/table_meta/src/meta/v3/frozen/statistics.rs index 58d7409e997d3..7a7c13427b738 100644 --- a/src/query/storages/common/table_meta/src/meta/v3/frozen/statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/v3/frozen/statistics.rs @@ -54,6 +54,7 @@ impl From for crate::meta::Statistics { .collect(), cluster_stats: None, virtual_block_count: None, + hlls: None, } } } diff --git a/src/query/storages/common/table_meta/src/meta/v3/table_snapshot_statistics.rs b/src/query/storages/common/table_meta/src/meta/v3/table_snapshot_statistics.rs index 0689e4a687dfd..7cb19c1455421 100644 --- a/src/query/storages/common/table_meta/src/meta/v3/table_snapshot_statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/v3/table_snapshot_statistics.rs @@ -25,7 +25,7 @@ use crate::meta::FormatVersion; use crate::meta::SnapshotId; use crate::meta::Versioned; -pub type MetaHLL = simple_hll::HyperLogLog<12>; +pub type MetaHLL12 = simple_hll::HyperLogLog<12>; #[derive(Serialize, Deserialize, Clone, Debug)] pub struct TableSnapshotStatistics { @@ -34,13 +34,13 @@ pub struct TableSnapshotStatistics { /// id of snapshot pub snapshot_id: SnapshotId, - pub hll: HashMap, + pub hll: HashMap, pub histograms: HashMap, } impl TableSnapshotStatistics { pub fn new( - hll: HashMap, + hll: HashMap, histograms: HashMap, snapshot_id: SnapshotId, ) -> Self { diff --git a/src/query/storages/common/table_meta/src/meta/v4/mod.rs b/src/query/storages/common/table_meta/src/meta/v4/mod.rs index cf73718ea7c29..6a596b9ec8807 100644 --- a/src/query/storages/common/table_meta/src/meta/v4/mod.rs +++ b/src/query/storages/common/table_meta/src/meta/v4/mod.rs @@ -12,11 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod block_statistics; mod segment; mod snapshot; -pub use block_statistics::BlockStatistics; pub use segment::CompactSegmentInfo; pub use segment::RawBlockMeta; pub use segment::SegmentInfo; diff --git a/src/query/storages/common/table_meta/src/meta/versions.rs b/src/query/storages/common/table_meta/src/meta/versions.rs index 670ce5c73805c..98411980f7762 100644 --- a/src/query/storages/common/table_meta/src/meta/versions.rs +++ b/src/query/storages/common/table_meta/src/meta/versions.rs @@ -128,16 +128,16 @@ impl TableSnapshotStatisticsVersion { } } -impl Versioned<0> for v4::BlockStatistics {} +impl Versioned<0> for v2::SegmentStatistics {} -pub enum BlockStatisticsVersion { - V0(PhantomData), +pub enum SegmentStatisticsVersion { + V0(PhantomData), } -impl BlockStatisticsVersion { +impl SegmentStatisticsVersion { pub fn version(&self) -> u64 { match self { - BlockStatisticsVersion::V0(a) => Self::ver(a), + SegmentStatisticsVersion::V0(a) => Self::ver(a), } } @@ -213,15 +213,15 @@ mod converters { } } - impl TryFrom for BlockStatisticsVersion { + impl TryFrom for SegmentStatisticsVersion { type Error = ErrorCode; fn try_from(value: u64) -> Result { match value { - 0 => Ok(BlockStatisticsVersion::V0(testify_version::<_, 0>( + 0 => Ok(SegmentStatisticsVersion::V0(testify_version::<_, 0>( PhantomData, ))), _ => Err(ErrorCode::Internal(format!( - "unknown block statistics version {value}, versions supported: 0" + "unknown segment statistics version {value}, versions supported: 0" ))), } } diff --git a/src/query/storages/common/table_meta/src/readers/versioned_reader.rs b/src/query/storages/common/table_meta/src/readers/versioned_reader.rs index a67c796d7eda1..7ce87d65e098e 100644 --- a/src/query/storages/common/table_meta/src/readers/versioned_reader.rs +++ b/src/query/storages/common/table_meta/src/readers/versioned_reader.rs @@ -17,8 +17,8 @@ use std::io::Read; use databend_common_exception::Result; use crate::meta::load_json; -use crate::meta::BlockStatistics; -use crate::meta::BlockStatisticsVersion; +use crate::meta::SegmentStatistics; +use crate::meta::SegmentStatisticsVersion; use crate::meta::TableSnapshotStatistics; use crate::meta::TableSnapshotStatisticsVersion; @@ -49,13 +49,13 @@ impl VersionedReader for TableSnapshotStatisticsVersion } } -impl VersionedReader for BlockStatisticsVersion { - type TargetType = BlockStatistics; +impl VersionedReader for SegmentStatisticsVersion { + type TargetType = SegmentStatistics; - fn read(&self, reader: R) -> Result + fn read(&self, reader: R) -> Result where R: Read + Unpin + Send { let r = match self { - BlockStatisticsVersion::V0(_) => BlockStatistics::from_read(reader)?, + SegmentStatisticsVersion::V0(_) => SegmentStatistics::from_read(reader)?, }; Ok(r) } diff --git a/src/query/storages/fuse/src/constants.rs b/src/query/storages/fuse/src/constants.rs index bc95d3fe14a48..50641f7daa739 100644 --- a/src/query/storages/fuse/src/constants.rs +++ b/src/query/storages/fuse/src/constants.rs @@ -31,7 +31,7 @@ pub const FUSE_TBL_XOR_BLOOM_INDEX_PREFIX: &str = "_i_b_v2"; pub const FUSE_TBL_SEGMENT_PREFIX: &str = "_sg"; pub const FUSE_TBL_SNAPSHOT_PREFIX: &str = "_ss"; pub const FUSE_TBL_SNAPSHOT_STATISTICS_PREFIX: &str = "_ts"; -pub const FUSE_TBL_BLOCK_STATISTICS_PREFIX: &str = "_bs"; +pub const FUSE_TBL_SEGMENT_STATISTICS_PREFIX: &str = "_hs"; pub const FUSE_TBL_LAST_SNAPSHOT_HINT: &str = "last_snapshot_location_hint"; pub const FUSE_TBL_LAST_SNAPSHOT_HINT_V2: &str = "last_snapshot_location_hint_v2"; pub const FUSE_TBL_VIRTUAL_BLOCK_PREFIX: &str = "_vb"; diff --git a/src/query/storages/fuse/src/io/locations.rs b/src/query/storages/fuse/src/io/locations.rs index 95ce57493f5b0..e389b9023927b 100644 --- a/src/query/storages/fuse/src/io/locations.rs +++ b/src/query/storages/fuse/src/io/locations.rs @@ -18,9 +18,9 @@ use databend_common_exception::Result; use databend_common_expression::DataBlock; use databend_storages_common_table_meta::meta::trim_object_prefix; use databend_storages_common_table_meta::meta::uuid_from_date_time; -use databend_storages_common_table_meta::meta::BlockStatistics; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SegmentInfo; +use databend_storages_common_table_meta::meta::SegmentStatistics; use databend_storages_common_table_meta::meta::SnapshotVersion; use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshotStatisticsVersion; @@ -37,9 +37,9 @@ use crate::constants::FUSE_TBL_VIRTUAL_BLOCK_PREFIX; use crate::index::filters::BlockFilter; use crate::index::InvertedIndexFile; use crate::FUSE_TBL_AGG_INDEX_PREFIX; -use crate::FUSE_TBL_BLOCK_STATISTICS_PREFIX; use crate::FUSE_TBL_INVERTED_INDEX_PREFIX; use crate::FUSE_TBL_LAST_SNAPSHOT_HINT_V2; +use crate::FUSE_TBL_SEGMENT_STATISTICS_PREFIX; use crate::FUSE_TBL_VECTOR_INDEX_PREFIX; use crate::FUSE_TBL_XOR_BLOOM_INDEX_PREFIX; static SNAPSHOT_V0: SnapshotVersion = SnapshotVersion::V0(PhantomData); @@ -67,7 +67,7 @@ pub struct TableMetaLocationGenerator { agg_index_location_prefix: String, inverted_index_location_prefix: String, vector_index_location_prefix: String, - block_statistics_location_prefix: String, + segment_statistics_location_prefix: String, } impl TableMetaLocationGenerator { @@ -81,8 +81,8 @@ impl TableMetaLocationGenerator { let inverted_index_location_prefix = format!("{}/{}/", &prefix, FUSE_TBL_INVERTED_INDEX_PREFIX); let vector_index_location_prefix = format!("{}/{}/", &prefix, FUSE_TBL_VECTOR_INDEX_PREFIX); - let block_statistics_location_prefix = - format!("{}/{}/", &prefix, FUSE_TBL_BLOCK_STATISTICS_PREFIX); + let segment_statistics_location_prefix = + format!("{}/{}/", &prefix, FUSE_TBL_SEGMENT_STATISTICS_PREFIX); Self { prefix, block_location_prefix, @@ -92,7 +92,7 @@ impl TableMetaLocationGenerator { agg_index_location_prefix, inverted_index_location_prefix, vector_index_location_prefix, - block_statistics_location_prefix, + segment_statistics_location_prefix, } } @@ -120,8 +120,8 @@ impl TableMetaLocationGenerator { &self.snapshot_location_prefix } - pub fn block_statistics_location_prefix(&self) -> &str { - &self.block_statistics_location_prefix + pub fn segment_statistics_location_prefix(&self) -> &str { + &self.segment_statistics_location_prefix } pub fn gen_block_location( @@ -152,18 +152,6 @@ impl TableMetaLocationGenerator { ) } - pub fn block_stats_location(&self, block_id: &Uuid) -> Location { - ( - format!( - "{}{}_v{}.mpk", - self.block_statistics_location_prefix(), - block_id.as_simple(), - BlockStatistics::VERSION, - ), - BlockStatistics::VERSION, - ) - } - pub fn block_vector_index_location(&self) -> Location { let uuid = Uuid::now_v7(); ( @@ -319,18 +307,18 @@ impl TableMetaLocationGenerator { ) } - pub fn gen_block_stats_location_from_block_location(loc: &str) -> String { + pub fn gen_segment_stats_location_from_segment_location(loc: &str) -> String { let splits = loc.split('/').collect::>(); let len = splits.len(); let prefix = splits[..len - 2].join("/"); - let block_name = trim_object_prefix(splits[len - 1]); - let id: String = block_name.chars().take(32).collect(); + let segment_name = trim_object_prefix(splits[len - 1]); + let id: String = segment_name.chars().take(32).collect(); format!( "{}/{}/{}_v{}.mpk", prefix, - FUSE_TBL_BLOCK_STATISTICS_PREFIX, + FUSE_TBL_SEGMENT_STATISTICS_PREFIX, id, - BlockStatistics::VERSION, + SegmentStatistics::VERSION, ) } } diff --git a/src/query/storages/fuse/src/io/mod.rs b/src/query/storages/fuse/src/io/mod.rs index 24fc70297a0cd..e13b252e29b15 100644 --- a/src/query/storages/fuse/src/io/mod.rs +++ b/src/query/storages/fuse/src/io/mod.rs @@ -36,6 +36,7 @@ pub use segments::SegmentsIO; pub use segments::SerializedSegment; pub use snapshots::SnapshotLiteExtended; pub use snapshots::SnapshotsIO; +pub use write::build_column_hlls; pub(crate) use write::create_index_schema; pub(crate) use write::create_inverted_index_builders; pub(crate) use write::create_tokenizer_manager; @@ -43,7 +44,6 @@ pub use write::serialize_block; pub use write::write_data; pub use write::BlockBuilder; pub use write::BlockSerialization; -pub use write::BlockStatisticsState; pub use write::BlockWriter; pub use write::BloomIndexRebuilder; pub use write::BloomIndexState; diff --git a/src/query/storages/fuse/src/io/read/meta/meta_readers.rs b/src/query/storages/fuse/src/io/read/meta/meta_readers.rs index a07e089a42cca..33d564862b0a3 100644 --- a/src/query/storages/fuse/src/io/read/meta/meta_readers.rs +++ b/src/query/storages/fuse/src/io/read/meta/meta_readers.rs @@ -30,6 +30,8 @@ use databend_storages_common_index::InvertedIndexMeta; use databend_storages_common_index::VectorIndexMeta; use databend_storages_common_table_meta::meta::CompactSegmentInfo; use databend_storages_common_table_meta::meta::SegmentInfoVersion; +use databend_storages_common_table_meta::meta::SegmentStatistics; +use databend_storages_common_table_meta::meta::SegmentStatisticsVersion; use databend_storages_common_table_meta::meta::SingleColumnMeta; use databend_storages_common_table_meta::meta::SnapshotVersion; use databend_storages_common_table_meta::meta::TableSnapshot; @@ -53,6 +55,7 @@ pub type CompactSegmentInfoReader = InMemoryCacheReader>; pub type InvertedIndexMetaReader = InMemoryCacheReader>; pub type VectorIndexMetaReader = InMemoryCacheReader>; +pub type SegmentStatsReader = InMemoryCacheReader>; pub struct MetaReaders; @@ -89,6 +92,13 @@ impl MetaReaders { ) } + pub fn segment_stats_reader(dal: Operator) -> SegmentStatsReader { + SegmentStatsReader::new( + CacheManager::instance().get_segment_statistics_cache(), + LoaderWrapper(dal), + ) + } + pub fn bloom_index_meta_reader(dal: Operator) -> BloomIndexMetaReader { BloomIndexMetaReader::new( CacheManager::instance().get_bloom_index_meta_cache(), @@ -135,6 +145,16 @@ impl Loader for LoaderWrapper { } } +#[async_trait::async_trait] +impl Loader for LoaderWrapper { + #[async_backtrace::framed] + async fn load(&self, params: &LoadParams) -> Result { + let version = SegmentStatisticsVersion::try_from(params.ver)?; + let reader = bytes_reader(&self.0, params.location.as_str(), params.len_hint).await?; + version.read(reader.reader()) + } +} + #[async_trait::async_trait] impl Loader for LoaderWrapper<(Operator, TableSchemaRef)> { #[async_backtrace::framed] diff --git a/src/query/storages/fuse/src/io/read/mod.rs b/src/query/storages/fuse/src/io/read/mod.rs index 391f23e33c6e9..45195902680bb 100644 --- a/src/query/storages/fuse/src/io/read/mod.rs +++ b/src/query/storages/fuse/src/io/read/mod.rs @@ -19,6 +19,7 @@ mod inverted_index; pub mod meta; mod segment_reader; mod snapshot_history_reader; +mod statistics; mod utils; mod vector_index; mod virtual_column; @@ -38,6 +39,7 @@ pub use segment_reader::ColumnOrientedSegmentReader; pub use segment_reader::RowOrientedSegmentReader; pub use segment_reader::SegmentReader; pub use snapshot_history_reader::SnapshotHistoryReader; +pub use statistics::*; pub use utils::build_columns_meta; pub use vector_index::VectorIndexReader; pub use virtual_column::VirtualBlockReadResult; diff --git a/src/query/storages/fuse/src/io/read/statistics/mod.rs b/src/query/storages/fuse/src/io/read/statistics/mod.rs new file mode 100644 index 0000000000000..9ece879a7d91c --- /dev/null +++ b/src/query/storages/fuse/src/io/read/statistics/mod.rs @@ -0,0 +1,18 @@ +// Copyright 2021 Datafuse Labs +// +// 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 segment_stats; + +pub use segment_stats::read_segment_stats; +pub use segment_stats::read_segment_stats_in_parallel; diff --git a/src/query/storages/fuse/src/io/read/statistics/segment_stats.rs b/src/query/storages/fuse/src/io/read/statistics/segment_stats.rs new file mode 100644 index 0000000000000..340bfe54da4e2 --- /dev/null +++ b/src/query/storages/fuse/src/io/read/statistics/segment_stats.rs @@ -0,0 +1,70 @@ +// Copyright 2021 Datafuse Labs +// +// 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::sync::Arc; + +use databend_common_base::runtime::execute_futures_in_parallel; +use databend_common_exception::Result; +use databend_storages_common_cache::LoadParams; +use databend_storages_common_cache::SegmentStatistics; +use databend_storages_common_table_meta::meta::Location; +use fastrace::func_path; +use fastrace::prelude::*; +use opendal::Operator; + +use crate::io::MetaReaders; + +#[async_backtrace::framed] +#[fastrace::trace] +pub async fn read_segment_stats(dal: Operator, loc: Location) -> Result> { + let reader = MetaReaders::segment_stats_reader(dal); + let (location, ver) = loc; + let load_params = LoadParams { + location, + len_hint: None, + ver, + put_cache: false, + }; + reader.read(&load_params).await +} + +#[async_backtrace::framed] +#[fastrace::trace] +pub async fn read_segment_stats_in_parallel( + dal: Operator, + locations: &[Location], + threads_nums: usize, +) -> Result>> { + // combine all the tasks. + let mut iter = locations.iter(); + let tasks = std::iter::from_fn(|| { + iter.next().map(|location| { + let dal = dal.clone(); + let loc = location.clone(); + async move { read_segment_stats(dal, loc).await } + .in_span(Span::enter_with_local_parent(func_path!())) + }) + }); + + let permit_nums = threads_nums * 2; + execute_futures_in_parallel( + tasks, + threads_nums, + permit_nums, + "fuse-segment-stats-worker".to_owned(), + ) + .await? + .into_iter() + .collect::>>() +} diff --git a/src/query/storages/fuse/src/io/write/block_statistics_writer.rs b/src/query/storages/fuse/src/io/write/block_statistics_writer.rs index ad54bbb972efa..b191d05b3fe54 100644 --- a/src/query/storages/fuse/src/io/write/block_statistics_writer.rs +++ b/src/query/storages/fuse/src/io/write/block_statistics_writer.rs @@ -17,38 +17,22 @@ use std::collections::HashMap; use databend_common_exception::Result; use databend_common_expression::BlockEntry; -use databend_common_expression::ColumnId; use databend_common_expression::DataBlock; use databend_common_expression::FieldIndex; use databend_common_expression::TableField; -use databend_storages_common_table_meta::meta::BlockStatistics; -use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::ColumnHLL; use crate::io::write::stream::create_column_ndv_estimator; use crate::io::write::stream::ColumnNDVEstimator; use crate::io::write::stream::ColumnNDVEstimatorOps; -#[derive(Debug)] -pub struct BlockStatisticsState { - pub data: Vec, - pub location: Location, - pub column_distinct_count: HashMap, -} - -impl BlockStatisticsState { - pub fn from_data_block( - location: Location, - block: &DataBlock, - ndv_columns_map: &BTreeMap, - ) -> Result> { - let mut builder = BlockStatsBuilder::new(ndv_columns_map); - builder.add_block(block)?; - builder.finalize(location) - } - - pub fn block_stats_size(&self) -> u64 { - self.data.len() as u64 - } +pub fn build_column_hlls( + block: &DataBlock, + ndv_columns_map: &BTreeMap, +) -> Result> { + let mut builder = BlockStatsBuilder::new(ndv_columns_map); + builder.add_block(block)?; + builder.finalize() } pub struct BlockStatsBuilder { @@ -101,27 +85,18 @@ impl BlockStatsBuilder { Ok(()) } - pub fn finalize(self, location: Location) -> Result> { + pub fn finalize(self) -> Result> { if self.builders.is_empty() { return Ok(None); } - let mut hlls = HashMap::with_capacity(self.builders.len()); - let mut column_distinct_count = HashMap::with_capacity(self.builders.len()); + let mut column_hlls = HashMap::with_capacity(self.builders.len()); for column_builder in self.builders { let column_id = column_builder.field.column_id(); - let distinct_count = column_builder.builder.finalize(); let hll = column_builder.builder.hll(); - hlls.insert(column_id, hll); - column_distinct_count.insert(column_id, distinct_count); + column_hlls.insert(column_id, hll); } - let block_stats = BlockStatistics::new(hlls); - let data = block_stats.to_bytes()?; - Ok(Some(BlockStatisticsState { - data, - location, - column_distinct_count, - })) + Ok(Some(column_hlls)) } } diff --git a/src/query/storages/fuse/src/io/write/block_writer.rs b/src/query/storages/fuse/src/io/write/block_writer.rs index 342c455204b36..dd83cb01dc53a 100644 --- a/src/query/storages/fuse/src/io/write/block_writer.rs +++ b/src/query/storages/fuse/src/io/write/block_writer.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::hash_map::Entry; use std::collections::BTreeMap; use std::collections::HashMap; use std::sync::Arc; @@ -34,9 +35,6 @@ use databend_common_metrics::storage::metrics_inc_block_index_write_nums; use databend_common_metrics::storage::metrics_inc_block_inverted_index_write_bytes; use databend_common_metrics::storage::metrics_inc_block_inverted_index_write_milliseconds; use databend_common_metrics::storage::metrics_inc_block_inverted_index_write_nums; -use databend_common_metrics::storage::metrics_inc_block_stats_write_bytes; -use databend_common_metrics::storage::metrics_inc_block_stats_write_milliseconds; -use databend_common_metrics::storage::metrics_inc_block_stats_write_nums; use databend_common_metrics::storage::metrics_inc_block_vector_index_write_bytes; use databend_common_metrics::storage::metrics_inc_block_vector_index_write_milliseconds; use databend_common_metrics::storage::metrics_inc_block_vector_index_write_nums; @@ -48,17 +46,19 @@ use databend_common_metrics::storage::metrics_inc_block_write_nums; use databend_common_native::write::NativeWriter; use databend_storages_common_blocks::blocks_to_parquet; use databend_storages_common_index::NgramArgs; +use databend_storages_common_table_meta::meta::encode_column_hll; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::ClusterStatistics; use databend_storages_common_table_meta::meta::ColumnMeta; use databend_storages_common_table_meta::meta::ExtendedBlockMeta; +use databend_storages_common_table_meta::meta::RawColumnHLL; use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::table::TableCompression; use opendal::Operator; +use crate::io::build_column_hlls; use crate::io::write::virtual_column_builder::VirtualColumnBuilder; use crate::io::write::virtual_column_builder::VirtualColumnState; -use crate::io::write::BlockStatisticsState; use crate::io::write::InvertedIndexBuilder; use crate::io::write::InvertedIndexState; use crate::io::write::VectorIndexBuilder; @@ -143,7 +143,7 @@ pub struct BlockSerialization { pub inverted_index_states: Vec, pub virtual_column_state: Option, pub vector_index_state: Option, - pub block_stats_state: Option, + pub column_hlls: Option, } local_block_meta_serde!(BlockSerialization); @@ -189,15 +189,12 @@ impl BlockBuilder { .map(|i| i.column_distinct_count.clone()) .unwrap_or_default(); - let block_stats_location = self.meta_locations.block_stats_location(&block_id); - let block_stats_state = BlockStatisticsState::from_data_block( - block_stats_location, - &data_block, - &self.ndv_columns_map, - )?; - if let Some(block_stats_state) = &block_stats_state { - for (key, val) in &block_stats_state.column_distinct_count { - column_distinct_count.entry(*key).or_insert(*val); + let column_hlls = build_column_hlls(&data_block, &self.ndv_columns_map)?; + if let Some(hlls) = &column_hlls { + for (key, val) in hlls { + if let Entry::Vacant(entry) = column_distinct_count.entry(*key) { + entry.insert(val.count()); + } } } @@ -276,10 +273,6 @@ impl BlockBuilder { compression: self.write_settings.table_compression.into(), inverted_index_size, virtual_block_meta: None, - block_stats_location: block_stats_state.as_ref().map(|v| v.location.clone()), - block_stats_size: block_stats_state - .as_ref() - .map_or(0, |v| v.block_stats_size()), create_on: Some(Utc::now()), }; @@ -290,7 +283,7 @@ impl BlockBuilder { inverted_index_states, virtual_column_state, vector_index_state, - block_stats_state, + column_hlls: column_hlls.map(|v| encode_column_hll(&v)).transpose()?, }; Ok(serialized) } @@ -304,28 +297,31 @@ impl BlockWriter { serialized: BlockSerialization, ) -> Result { let block_meta = serialized.block_meta; + let column_hlls = serialized.column_hlls; + let block_location = block_meta.location.0.clone(); let extended_block_meta = if let Some(virtual_column_state) = &serialized.virtual_column_state { ExtendedBlockMeta { - block_meta: block_meta.clone(), + block_meta, draft_virtual_block_meta: Some( virtual_column_state.draft_virtual_block_meta.clone(), ), + column_hlls, } } else { ExtendedBlockMeta { - block_meta: block_meta.clone(), + block_meta, draft_virtual_block_meta: None, + column_hlls, } }; - Self::write_down_data_block(dal, serialized.block_raw_data, &block_meta.location.0).await?; + Self::write_down_data_block(dal, serialized.block_raw_data, &block_location).await?; Self::write_down_bloom_index_state(dal, serialized.bloom_index_state).await?; Self::write_down_vector_index_state(dal, serialized.vector_index_state).await?; Self::write_down_inverted_index_state(dal, serialized.inverted_index_states).await?; Self::write_down_virtual_column_state(dal, serialized.virtual_column_state).await?; - Self::write_down_block_stats_state(dal, serialized.block_stats_state).await?; Ok(extended_block_meta) } @@ -427,22 +423,4 @@ impl BlockWriter { } Ok(()) } - - pub async fn write_down_block_stats_state( - dal: &Operator, - block_stats_state: Option, - ) -> Result<()> { - if let Some(block_stats_state) = block_stats_state { - let start = Instant::now(); - - let stats_size = block_stats_state.block_stats_size(); - let location = &block_stats_state.location.0; - write_data(block_stats_state.data, dal, location).await?; - - metrics_inc_block_stats_write_nums(1); - metrics_inc_block_stats_write_bytes(stats_size); - metrics_inc_block_stats_write_milliseconds(start.elapsed().as_millis() as u64); - } - Ok(()) - } } diff --git a/src/query/storages/fuse/src/io/write/meta_writer.rs b/src/query/storages/fuse/src/io/write/meta_writer.rs index 956deb6a4edc1..43daaefd5195b 100644 --- a/src/query/storages/fuse/src/io/write/meta_writer.rs +++ b/src/query/storages/fuse/src/io/write/meta_writer.rs @@ -16,6 +16,7 @@ use databend_common_exception::Result; use databend_storages_common_cache::CacheAccessor; use databend_storages_common_cache::CachedObject; use databend_storages_common_table_meta::meta::SegmentInfo; +use databend_storages_common_table_meta::meta::SegmentStatistics; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::TableSnapshotStatistics; use databend_storages_common_table_meta::meta::Versioned; @@ -95,6 +96,13 @@ impl Marshal for TableSnapshotStatistics { } } +impl Marshal for SegmentStatistics { + fn marshal(&self) -> Result> { + assert_eq!(self.format_version, SegmentStatistics::VERSION); + self.to_bytes() + } +} + #[cfg(test)] mod tests { use chrono::Duration; diff --git a/src/query/storages/fuse/src/io/write/mod.rs b/src/query/storages/fuse/src/io/write/mod.rs index 544d32d988988..368cc31ae1497 100644 --- a/src/query/storages/fuse/src/io/write/mod.rs +++ b/src/query/storages/fuse/src/io/write/mod.rs @@ -22,7 +22,7 @@ mod vector_index_writer; mod virtual_column_builder; mod write_settings; -pub use block_statistics_writer::BlockStatisticsState; +pub use block_statistics_writer::build_column_hlls; pub use block_statistics_writer::BlockStatsBuilder; pub use block_writer::serialize_block; pub use block_writer::write_data; diff --git a/src/query/storages/fuse/src/io/write/stream/block_builder.rs b/src/query/storages/fuse/src/io/write/stream/block_builder.rs index 66114228a0082..6877ce806bcf6 100644 --- a/src/query/storages/fuse/src/io/write/stream/block_builder.rs +++ b/src/query/storages/fuse/src/io/write/stream/block_builder.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::hash_map::Entry; use std::collections::BTreeMap; use std::collections::HashMap; use std::collections::HashSet; @@ -42,6 +43,7 @@ use databend_storages_common_index::BloomIndexBuilder; use databend_storages_common_index::Index; use databend_storages_common_index::NgramArgs; use databend_storages_common_index::RangeIndex; +use databend_storages_common_table_meta::meta::encode_column_hll; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::ColumnMeta; use databend_storages_common_table_meta::meta::TableMetaTimestamps; @@ -312,14 +314,12 @@ impl StreamBlockBuilder { .as_ref() .map(|i| i.column_distinct_count.clone()) .unwrap_or_default(); - let block_stats_location = self - .properties - .meta_locations - .block_stats_location(&block_id); - let block_stats_state = self.block_stats_builder.finalize(block_stats_location)?; - if let Some(state) = &block_stats_state { - for (key, val) in &state.column_distinct_count { - column_distinct_count.entry(*key).or_insert(*val); + let column_hlls = self.block_stats_builder.finalize()?; + if let Some(hlls) = &column_hlls { + for (key, val) in hlls { + if let Entry::Vacant(entry) = column_distinct_count.entry(*key) { + entry.insert(val.count()); + } } } let col_stats = self.column_stats_state.finalize(column_distinct_count)?; @@ -393,10 +393,6 @@ impl StreamBlockBuilder { .map(|v| v.ngram_size) .unwrap_or_default(), virtual_block_meta: None, - block_stats_location: block_stats_state.as_ref().map(|v| v.location.clone()), - block_stats_size: block_stats_state - .as_ref() - .map_or(0, |v| v.block_stats_size()), }; let serialized = BlockSerialization { block_raw_data, @@ -405,7 +401,7 @@ impl StreamBlockBuilder { inverted_index_states, virtual_column_state, vector_index_state, - block_stats_state, + column_hlls: column_hlls.map(|v| encode_column_hll(&v)).transpose()?, }; Ok(serialized) } diff --git a/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs b/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs index 23d2f2475546a..2cef92da92ccf 100644 --- a/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs +++ b/src/query/storages/fuse/src/io/write/stream/column_ndv_estimator.rs @@ -39,7 +39,7 @@ use databend_common_expression::with_number_type; use databend_common_expression::Column; use databend_common_expression::ScalarRef; use databend_common_expression::SELECTIVITY_THRESHOLD; -use databend_storages_common_table_meta::meta::MetaHLL; +use databend_common_storage::MetaHLL; use enum_dispatch::enum_dispatch; #[enum_dispatch] diff --git a/src/query/storages/fuse/src/operations/analyze.rs b/src/query/storages/fuse/src/operations/analyze.rs index 84dc9817826c5..324446fa56231 100644 --- a/src/query/storages/fuse/src/operations/analyze.rs +++ b/src/query/storages/fuse/src/operations/analyze.rs @@ -40,7 +40,7 @@ use databend_common_storage::Datum; use databend_common_storage::Histogram; use databend_common_storage::HistogramBucket; use databend_storages_common_table_meta::meta::ClusterStatistics; -use databend_storages_common_table_meta::meta::MetaHLL; +use databend_storages_common_table_meta::meta::MetaHLL12; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::SnapshotId; use databend_storages_common_table_meta::meta::StatisticsOfColumns; @@ -99,7 +99,7 @@ struct SinkAnalyzeState { histogram_info_receivers: HashMap>, input_data: Option, committed: bool, - ndv_states: HashMap, + ndv_states: HashMap, histograms: HashMap, step: AnalyzeStep, } @@ -187,7 +187,7 @@ impl SinkAnalyzeState { let col = col.index(0).unwrap(); let data = col.as_tuple().unwrap()[0].as_binary().unwrap(); - let hll: MetaHLL = borsh_deserialize_from_slice(data)?; + let hll: MetaHLL12 = borsh_deserialize_from_slice(data)?; if !is_full { ndv_states diff --git a/src/query/storages/fuse/src/operations/common/meta/mutation_log.rs b/src/query/storages/fuse/src/operations/common/meta/mutation_log.rs index 8bb180b08c7fb..1a6d9ff765a75 100644 --- a/src/query/storages/fuse/src/operations/common/meta/mutation_log.rs +++ b/src/query/storages/fuse/src/operations/common/meta/mutation_log.rs @@ -62,7 +62,7 @@ pub enum MutationLogEntry { DoNothing, } -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default, PartialEq)] +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default, PartialEq, Eq, Hash)] pub struct BlockMetaIndex { pub segment_idx: SegmentIndex, pub block_idx: BlockIndex, diff --git a/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs b/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs index c17b7eb770606..59144f0328a38 100644 --- a/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs +++ b/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs @@ -19,6 +19,7 @@ use std::sync::Arc; use std::time::Instant; use databend_common_base::runtime::execute_futures_in_parallel; +use databend_common_catalog::plan::BlockMetaWithHLL; use databend_common_catalog::table::Table; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; @@ -29,10 +30,12 @@ use databend_common_expression::TableSchemaRef; use databend_common_expression::VirtualDataSchema; use databend_common_pipeline_transforms::processors::AsyncAccumulatingTransform; use databend_common_sql::executor::physical_plans::MutationKind; +use databend_storages_common_cache::SegmentStatistics; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::ClusterStatistics; use databend_storages_common_table_meta::meta::ExtendedBlockMeta; use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::RawColumnHLL; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::Statistics; use databend_storages_common_table_meta::meta::TableMetaTimestamps; @@ -45,7 +48,9 @@ use log::info; use log::warn; use opendal::Operator; +use crate::io::read::read_segment_stats; use crate::io::CachedMetaWriter; +use crate::io::MetaWriter; use crate::io::SegmentsIO; use crate::io::TableMetaLocationGenerator; use crate::operations::common::CommitMeta; @@ -279,11 +284,17 @@ impl TableMutationAggregator { MutationLogEntry::CompactExtras { extras } => { match self.mutations.entry(extras.segment_index) { Entry::Occupied(mut v) => { - v.get_mut().replaced_blocks.extend(extras.unchanged_blocks); + for (idx, blocks) in extras.unchanged_blocks { + v.get_mut().replaced_blocks.push((idx, blocks)); + } } Entry::Vacant(v) => { + let mut replaced_blocks = Vec::with_capacity(extras.unchanged_blocks.len()); + for (idx, blocks) in extras.unchanged_blocks { + replaced_blocks.push((idx, blocks)); + } v.insert(BlockMutations { - replaced_blocks: extras.unchanged_blocks, + replaced_blocks, deleted_blocks: vec![], }); } @@ -311,7 +322,7 @@ impl TableMutationAggregator { if let Some(id) = self.default_cluster_key_id { // sort ascending. merged_blocks - .sort_by(|a, b| sort_by_cluster_stats(&a.cluster_stats, &b.cluster_stats, id)); + .sort_by(|a, b| sort_by_cluster_stats(&a.0.cluster_stats, &b.0.cluster_stats, id)); } let mut tasks = Vec::new(); @@ -322,7 +333,17 @@ impl TableMutationAggregator { let set_hilbert_level = self.set_hilbert_level; let kind = self.kind; for chunk in &merged_blocks.into_iter().chunks(chunk_size) { - let new_blocks = chunk.collect::>(); + let (new_blocks, new_hlls): (Vec>, Vec>) = + chunk.unzip(); + let new_hlls = if new_hlls.iter().all(|v| v.is_none()) { + None + } else { + let hlls = new_hlls + .into_iter() + .map(|x| x.unwrap_or_default()) + .collect::>(); + Some(SegmentStatistics::new(hlls)) + }; let all_perfect = new_blocks.len() > 1; let location_gen = self.location_gen.clone(); @@ -333,6 +354,7 @@ impl TableMutationAggregator { op, location_gen, new_blocks, + new_hlls, thresholds, default_cluster_key, all_perfect, @@ -465,18 +487,32 @@ impl TableMutationAggregator { tasks.push(async move { let mut all_perfect = false; let mut set_level = false; - let (new_blocks, origin_summary) = if let Some(loc) = location { + let (new_blocks, new_hlls, origin_summary) = if let Some(loc) = location { // read the old segment let compact_segment_info = SegmentsIO::read_compact_segment(op.clone(), loc, schema, false).await?; let mut segment_info = SegmentInfo::try_from(compact_segment_info)?; + let stats = if let Some(loc) = &segment_info.summary.hlls { + let stats = read_segment_stats(op.clone(), loc.clone()).await?; + Some(stats) + } else { + None + }; + // take away the blocks, they are being mutated - let mut block_editor = BTreeMap::<_, _>::from_iter( - std::mem::take(&mut segment_info.blocks) - .into_iter() - .enumerate(), - ); + let mut block_editor = std::mem::take(&mut segment_info.blocks) + .into_iter() + .enumerate() + .map(|(block_idx, block_meta)| { + let hll = stats + .as_ref() + .and_then(|v| v.blocks.get(block_idx)) + .cloned(); + (block_idx, (block_meta, hll)) + }) + .collect::>(); + for (idx, new_meta) in segment_mutation.replaced_blocks { block_editor.insert(idx, new_meta); } @@ -493,33 +529,36 @@ impl TableMutationAggregator { } // assign back the mutated blocks to segment - let new_blocks = block_editor.into_values().collect::>(); + let (new_blocks, new_hlls) = block_editor.into_values().unzip(); set_level = set_hilbert_level && segment_info .summary .cluster_stats .as_ref() .is_some_and(|v| v.cluster_key_id == default_cluster_key_id.unwrap()); - (new_blocks, Some(segment_info.summary)) + let stats = generate_segment_stats(new_hlls); + (new_blocks, stats, Some(segment_info.summary)) } else { // use by compact. assert!(segment_mutation.deleted_blocks.is_empty()); // There are more than 1 blocks, means that the blocks can no longer be compacted. // They can be marked as perfect blocks. all_perfect = segment_mutation.replaced_blocks.len() > 1; - let new_blocks = segment_mutation + let (new_blocks, new_hlls) = segment_mutation .replaced_blocks .into_iter() .sorted_by(|a, b| a.0.cmp(&b.0)) .map(|(_, meta)| meta) - .collect::>(); - (new_blocks, None) + .unzip(); + let stats = generate_segment_stats(new_hlls); + (new_blocks, stats, None) }; let new_segment_info = write_segment( op, location_gen, new_blocks, + new_hlls, thresholds, default_cluster_key_id, all_perfect, @@ -591,10 +630,18 @@ impl TableMutationAggregator { match self.mutations.entry(segment_idx) { Entry::Occupied(mut v) => { - v.get_mut().push_replaced(block_idx, new_block_meta); + v.get_mut().push_replaced( + block_idx, + new_block_meta, + extended_block_meta.column_hlls.clone(), + ); } Entry::Vacant(v) => { - v.insert(BlockMutations::new_replacement(block_idx, new_block_meta)); + v.insert(BlockMutations::new_replacement( + block_idx, + new_block_meta, + extended_block_meta.column_hlls.clone(), + )); } } } @@ -615,7 +662,7 @@ impl TableMutationAggregator { } // Assign columnId to the virtual column in the merged blocks and generate a new virtual schema. - fn accumulate_merged_blocks(&mut self) -> Vec> { + fn accumulate_merged_blocks(&mut self) -> Vec { let mut virtual_column_accumulator = VirtualColumnAccumulator::try_create( self.ctx.clone(), &self.schema, @@ -645,7 +692,7 @@ impl TableMutationAggregator { } else { Arc::new(extended_block_meta.block_meta.clone()) }; - new_merged_blocks.push(new_block_meta); + new_merged_blocks.push((new_block_meta, extended_block_meta.column_hlls.clone())); } self.virtual_schema = if let Some(virtual_column_accumulator) = virtual_column_accumulator { @@ -706,14 +753,18 @@ impl ExtendedBlockMutations { #[derive(Default)] struct BlockMutations { - replaced_blocks: Vec<(BlockIndex, Arc)>, + replaced_blocks: Vec<(BlockIndex, BlockMetaWithHLL)>, deleted_blocks: Vec, } impl BlockMutations { - fn new_replacement(block_idx: BlockIndex, block_meta: Arc) -> Self { + fn new_replacement( + block_idx: BlockIndex, + block_meta: Arc, + column_hlls: Option, + ) -> Self { BlockMutations { - replaced_blocks: vec![(block_idx, block_meta)], + replaced_blocks: vec![(block_idx, (block_meta, column_hlls))], deleted_blocks: vec![], } } @@ -725,8 +776,14 @@ impl BlockMutations { } } - fn push_replaced(&mut self, block_idx: BlockIndex, block_meta: Arc) { - self.replaced_blocks.push((block_idx, block_meta)); + fn push_replaced( + &mut self, + block_idx: BlockIndex, + block_meta: Arc, + column_hlls: Option, + ) { + self.replaced_blocks + .push((block_idx, (block_meta, column_hlls))); } fn push_deleted(&mut self, block_idx: BlockIndex) { @@ -747,6 +804,7 @@ async fn write_segment( dal: Operator, location_gen: TableMetaLocationGenerator, blocks: Vec>, + stats: Option, thresholds: BlockThresholds, default_cluster_key: Option, all_perfect: bool, @@ -786,6 +844,16 @@ async fn write_segment( pages: None, }); } + + if let Some(stats) = stats { + let segment_stats_location = + TableMetaLocationGenerator::gen_segment_stats_location_from_segment_location( + location.as_str(), + ); + stats.write_meta(&dal, &segment_stats_location).await?; + new_summary.hlls = Some((segment_stats_location, SegmentStatistics::VERSION)); + } + // create new segment info let new_segment = SegmentInfo::new(blocks, new_summary.clone()); new_segment @@ -793,3 +861,12 @@ async fn write_segment( .await?; Ok((location, new_summary)) } + +fn generate_segment_stats(hlls: Vec>) -> Option { + if hlls.iter().all(|v| v.is_none()) { + None + } else { + let blocks = hlls.into_iter().map(|x| x.unwrap_or_default()).collect(); + Some(SegmentStatistics::new(blocks)) + } +} diff --git a/src/query/storages/fuse/src/operations/common/processors/transform_serialize_segment.rs b/src/query/storages/fuse/src/operations/common/processors/transform_serialize_segment.rs index d32bec321ea91..471915c715dfc 100644 --- a/src/query/storages/fuse/src/operations/common/processors/transform_serialize_segment.rs +++ b/src/query/storages/fuse/src/operations/common/processors/transform_serialize_segment.rs @@ -31,15 +31,18 @@ use databend_common_pipeline_core::processors::ProcessorPtr; use databend_storages_common_table_meta::meta::column_oriented_segment::*; use databend_storages_common_table_meta::meta::ExtendedBlockMeta; use databend_storages_common_table_meta::meta::SegmentInfo; +use databend_storages_common_table_meta::meta::SegmentStatistics; use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::Versioned; use databend_storages_common_table_meta::meta::VirtualBlockMeta; use log::info; use opendal::Operator; +use crate::io::MetaWriter; use crate::io::TableMetaLocationGenerator; use crate::operations::common::MutationLogEntry; use crate::operations::common::MutationLogs; +use crate::statistics::ColumnHLLAccumulator; use crate::statistics::RowOrientedSegmentBuilder; use crate::statistics::VirtualColumnAccumulator; use crate::FuseSegmentFormat; @@ -52,6 +55,8 @@ enum State { data: Vec, location: String, segment: B::Segment, + + hlls: Option<(String, SegmentStatistics)>, }, PreCommitSegment { location: String, @@ -65,6 +70,7 @@ pub struct TransformSerializeSegment { meta_locations: TableMetaLocationGenerator, segment_builder: B, virtual_column_accumulator: Option, + hll_accumulator: ColumnHLLAccumulator, state: State, input: Arc, @@ -105,6 +111,7 @@ impl TransformSerializeSegment { state: State::None, segment_builder, virtual_column_accumulator, + hll_accumulator: ColumnHLLAccumulator::default(), thresholds, default_cluster_key_id, table_meta_timestamps, @@ -244,6 +251,9 @@ impl Processor for TransformSerializeSegment { .add_block(extended_block_meta.block_meta)?; } + if let Some(hll) = extended_block_meta.column_hlls { + self.hll_accumulator.add_hll(hll); + } if self.segment_builder.block_count() >= self.thresholds.block_per_segment { self.state = State::GenerateSegment; return Ok(Event::Sync); @@ -257,17 +267,32 @@ impl Processor for TransformSerializeSegment { fn process(&mut self) -> Result<()> { match std::mem::replace(&mut self.state, State::None) { State::GenerateSegment => { - let segment_info = self - .segment_builder - .build(self.thresholds, self.default_cluster_key_id)?; + let location = self + .meta_locations + .gen_segment_info_location(self.table_meta_timestamps, self.is_column_oriented); + + let hlls = if !self.hll_accumulator.is_empty() { + let segment_stats_location = TableMetaLocationGenerator::gen_segment_stats_location_from_segment_location(location.as_str()); + let stats = self.hll_accumulator.build(); + Some((segment_stats_location, stats)) + } else { + None + }; + + let hll_location = hlls + .as_ref() + .map(|(hll_location, _)| (hll_location.clone(), SegmentStatistics::VERSION)); + let segment_info = self.segment_builder.build( + self.thresholds, + self.default_cluster_key_id, + hll_location, + )?; self.state = State::SerializedSegment { data: segment_info.serialize()?, - location: self.meta_locations.gen_segment_info_location( - self.table_meta_timestamps, - self.is_column_oriented, - ), + location, segment: segment_info, + hlls, } } State::PreCommitSegment { location, segment } => { @@ -300,8 +325,12 @@ impl Processor for TransformSerializeSegment { data, location, segment, + hlls, } => { self.data_accessor.write(&location, data).await?; + if let Some((location, stats)) = hlls { + stats.write_meta(&self.data_accessor, &location).await?; + } info!("fuse append wrote down segment {} ", location); self.state = State::PreCommitSegment { location, segment }; diff --git a/src/query/storages/fuse/src/operations/gc.rs b/src/query/storages/fuse/src/operations/gc.rs index 7e3bfee8e566e..501485253ecd6 100644 --- a/src/query/storages/fuse/src/operations/gc.rs +++ b/src/query/storages/fuse/src/operations/gc.rs @@ -32,7 +32,6 @@ use databend_storages_common_index::InvertedIndexFile; use databend_storages_common_index::InvertedIndexMeta; use databend_storages_common_io::Files; use databend_storages_common_table_meta::meta::column_oriented_segment::ColumnOrientedSegment; -use databend_storages_common_table_meta::meta::column_oriented_segment::BLOCK_STATS_LOCATION; use databend_storages_common_table_meta::meta::column_oriented_segment::BLOOM_FILTER_INDEX_LOCATION; use databend_storages_common_table_meta::meta::column_oriented_segment::LOCATION; use databend_storages_common_table_meta::meta::CompactSegmentInfo; @@ -391,13 +390,6 @@ impl FuseTable { purge_files.push(loc.to_string()) } - for loc in &locations.stats_location { - if locations_referenced_by_root.stats_location.contains(loc) { - continue; - } - purge_files.push(loc.to_string()) - } - purge_files.extend(chunk.iter().map(|loc| loc.0.clone())); } purge_files.extend(ts_to_be_purged.iter().map(|loc| loc.to_string())); @@ -464,8 +456,8 @@ impl FuseTable { } let mut stats_to_be_purged = HashSet::new(); - for loc in &locations.stats_location { - if locations_referenced_by_root.stats_location.contains(loc) { + for loc in &locations.hll_location { + if locations_referenced_by_root.hll_location.contains(loc) { continue; } stats_to_be_purged.insert(loc.to_string()); @@ -558,7 +550,7 @@ impl FuseTable { agg_indexes_to_be_purged, inverted_indexes_to_be_purged, root_location_tuple.bloom_location, - root_location_tuple.stats_location, + root_location_tuple.hll_location, segment_locations_to_be_purged, ) .await?; @@ -635,10 +627,10 @@ impl FuseTable { .await?; } - // 3. Try to purge block statistic file chunks. + // 3. Try to purge segment statistic file chunks. let stats_count = stats_to_be_purged.len(); if stats_count > 0 { - counter.block_stats += stats_count; + counter.hlls += stats_count; self.try_purge_location_files(ctx.clone(), stats_to_be_purged) .await?; } @@ -688,10 +680,10 @@ impl FuseTable { // 5. Refresh status. { let status = format!( - "gc: block files purged:{}, bloom files purged:{}, block stats files purged:{}, segment files purged:{}, table statistic files purged:{}, snapshots purged:{}, take:{:?}", + "gc: block files purged:{}, bloom files purged:{}, segment stats files purged:{}, segment files purged:{}, table statistic files purged:{}, snapshots purged:{}, take:{:?}", counter.blocks, counter.blooms, - counter.block_stats, + counter.hlls, counter.segments, counter.table_statistics, counter.snapshots, @@ -742,7 +734,7 @@ impl FuseTable { ) -> Result { let mut blocks = HashSet::new(); let mut blooms = HashSet::new(); - let mut stats = HashSet::new(); + let mut hlls = HashSet::new(); let fuse_segments = SegmentsIO::create(ctx.clone(), self.operator.clone(), self.schema()); let chunk_size = ctx.get_settings().get_max_threads()? as usize * 4; @@ -808,14 +800,14 @@ impl FuseTable { }; blocks.extend(location_tuple.block_location.into_iter()); blooms.extend(location_tuple.bloom_location.into_iter()); - stats.extend(location_tuple.stats_location.into_iter()); + hlls.extend(location_tuple.hll_location.into_iter()); } } Ok(LocationTuple { block_location: blocks, bloom_location: blooms, - stats_location: stats, + hll_location: hlls, }) } @@ -839,7 +831,7 @@ struct RootSnapshotInfo { pub struct LocationTuple { pub block_location: HashSet, pub bloom_location: HashSet, - pub stats_location: HashSet, + pub hll_location: HashSet, } impl TryFrom> for LocationTuple { @@ -847,21 +839,21 @@ impl TryFrom> for LocationTuple { fn try_from(value: Arc) -> Result { let mut block_location = HashSet::new(); let mut bloom_location = HashSet::new(); - let mut stats_location = HashSet::new(); + let mut hll_location = HashSet::new(); let block_metas = value.block_metas()?; for block_meta in block_metas.into_iter() { block_location.insert(block_meta.location.0.clone()); if let Some(bloom_loc) = &block_meta.bloom_filter_index_location { bloom_location.insert(bloom_loc.0.clone()); } - if let Some(stats_loc) = &block_meta.block_stats_location { - stats_location.insert(stats_loc.0.clone()); - } + } + if let Some(hlls) = &value.as_ref().summary.hlls { + hll_location.insert(hlls.0.clone()); } Ok(Self { block_location, bloom_location, - stats_location, + hll_location, }) } } @@ -871,7 +863,7 @@ impl TryFrom> for LocationTuple { fn try_from(value: Arc) -> Result { let mut block_location = HashSet::new(); let mut bloom_location = HashSet::new(); - let mut stats_location = HashSet::new(); + let mut hll_location = HashSet::new(); let location_path = value.location_path_col(); for path in location_path.iter() { @@ -890,22 +882,13 @@ impl TryFrom> for LocationTuple { } } - let (index, _) = value - .segment_schema - .column_with_name(BLOCK_STATS_LOCATION) - .unwrap(); - let column = value.block_metas.get_by_offset(index).to_column(); - for value in column.iter() { - if let ScalarRef::Tuple(values) = value { - let path = values[0].as_string().unwrap(); - stats_location.insert(path.to_string()); - } + if let Some(hlls) = &value.as_ref().summary.hlls { + hll_location.insert(hlls.0.clone()); } - Ok(Self { block_location, bloom_location, - stats_location, + hll_location, }) } } @@ -917,7 +900,7 @@ struct PurgeCounter { agg_indexes: usize, inverted_indexes: usize, blooms: usize, - block_stats: usize, + hlls: usize, segments: usize, table_statistics: usize, snapshots: usize, @@ -931,7 +914,7 @@ impl PurgeCounter { agg_indexes: 0, inverted_indexes: 0, blooms: 0, - block_stats: 0, + hlls: 0, segments: 0, table_statistics: 0, snapshots: 0, diff --git a/src/query/storages/fuse/src/operations/mutation/meta/compact_part.rs b/src/query/storages/fuse/src/operations/mutation/meta/compact_part.rs index c69b144252dc7..aa3f768f21d99 100644 --- a/src/query/storages/fuse/src/operations/mutation/meta/compact_part.rs +++ b/src/query/storages/fuse/src/operations/mutation/meta/compact_part.rs @@ -18,12 +18,13 @@ use std::hash::Hash; use std::hash::Hasher; use std::sync::Arc; +use databend_common_catalog::plan::BlockMetaWithHLL; use databend_common_catalog::plan::PartInfo; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::PartInfoType; use databend_common_exception::ErrorCode; use databend_common_exception::Result; -use databend_storages_common_table_meta::meta::BlockMeta; +use databend_storages_common_cache::BlockMeta; use databend_storages_common_table_meta::meta::CompactSegmentInfo; use databend_storages_common_table_meta::meta::Statistics; @@ -113,7 +114,7 @@ impl CompactBlockPartInfo { #[derive(serde::Serialize, serde::Deserialize, Clone, Debug, PartialEq)] pub struct CompactExtraInfo { pub segment_index: SegmentIndex, - pub unchanged_blocks: Vec<(BlockIndex, Arc)>, + pub unchanged_blocks: Vec<(BlockIndex, BlockMetaWithHLL)>, pub removed_segment_indexes: Vec, pub removed_segment_summary: Statistics, } @@ -121,7 +122,7 @@ pub struct CompactExtraInfo { impl CompactExtraInfo { pub fn create( segment_index: SegmentIndex, - unchanged_blocks: Vec<(BlockIndex, Arc)>, + unchanged_blocks: Vec<(BlockIndex, BlockMetaWithHLL)>, removed_segment_indexes: Vec, removed_segment_summary: Statistics, ) -> Self { diff --git a/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs b/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs index dfa628636cec4..ec8ee051f0708 100644 --- a/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs +++ b/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs @@ -21,6 +21,7 @@ use std::vec; use databend_common_base::base::tokio::sync::Semaphore; use databend_common_base::runtime::GlobalIORuntime; use databend_common_base::runtime::TrySpawn; +use databend_common_catalog::plan::BlockMetaWithHLL; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::Partitions; use databend_common_catalog::plan::PartitionsShuffleKind; @@ -32,10 +33,12 @@ use databend_common_expression::ColumnId; use databend_common_metrics::storage::*; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::CompactSegmentInfo; +use databend_storages_common_table_meta::meta::RawColumnHLL; use databend_storages_common_table_meta::meta::Statistics; use log::info; use opendal::Operator; +use crate::io::read::read_segment_stats; use crate::io::SegmentsIO; use crate::operations::acquire_task_permit; use crate::operations::common::BlockMetaIndex; @@ -211,6 +214,7 @@ impl BlockCompactMutator { PartitionsShuffleKind::Mod, BlockCompactMutator::build_compact_tasks( self.ctx.clone(), + self.operator.clone(), column_ids, self.cluster_key_id, self.thresholds, @@ -227,6 +231,7 @@ impl BlockCompactMutator { #[async_backtrace::framed] pub async fn build_compact_tasks( ctx: Arc, + dal: Operator, column_ids: HashSet, cluster_key_id: Option, thresholds: BlockThresholds, @@ -248,13 +253,18 @@ impl BlockCompactMutator { let column_ids = column_ids.clone(); let semaphore = semaphore.clone(); + let dal = dal.clone(); let batch = lazy_parts.drain(0..batch_size).collect::>(); works.push(async move { let mut res = vec![]; for lazy_part in batch { - let mut builder = - CompactTaskBuilder::new(column_ids.clone(), cluster_key_id, thresholds); + let mut builder = CompactTaskBuilder::new( + dal.clone(), + column_ids.clone(), + cluster_key_id, + thresholds, + ); let parts = builder .build_tasks( lazy_part.segment_indices, @@ -418,11 +428,12 @@ impl SegmentCompactChecker { } struct CompactTaskBuilder { + dal: Operator, column_ids: HashSet, cluster_key_id: Option, thresholds: BlockThresholds, - blocks: Vec>, + blocks: Vec, total_rows: usize, total_size: usize, total_compressed: usize, @@ -430,11 +441,13 @@ struct CompactTaskBuilder { impl CompactTaskBuilder { fn new( + dal: Operator, column_ids: HashSet, cluster_key_id: Option, thresholds: BlockThresholds, ) -> Self { Self { + dal, column_ids, cluster_key_id, thresholds, @@ -449,27 +462,27 @@ impl CompactTaskBuilder { self.blocks.is_empty() } - fn take_blocks(&mut self) -> Vec> { + fn take_blocks(&mut self) -> Vec { self.total_rows = 0; self.total_size = 0; self.total_compressed = 0; std::mem::take(&mut self.blocks) } - fn add(&mut self, block: &Arc) -> (bool, bool) { - let total_rows = self.total_rows + block.row_count as usize; - let total_size = self.total_size + block.block_size as usize; - let total_compressed = self.total_compressed + block.file_size as usize; + fn add(&mut self, block_meta: &Arc, hlls: &Option) -> (bool, bool) { + let total_rows = self.total_rows + block_meta.row_count as usize; + let total_size = self.total_size + block_meta.block_size as usize; + let total_compressed = self.total_compressed + block_meta.file_size as usize; if !self.check_large_enough(total_rows, total_size, total_compressed) { // blocks < N - self.blocks.push(block.clone()); + self.blocks.push((block_meta.clone(), hlls.clone())); self.total_rows = total_rows; self.total_size = total_size; self.total_compressed = total_compressed; (false, false) } else if self.check_for_compact(total_rows, total_size, total_compressed) { // N <= blocks < 2N - self.blocks.push(block.clone()); + self.blocks.push((block_meta.clone(), hlls.clone())); (false, true) } else { // blocks >= 2N @@ -500,20 +513,21 @@ impl CompactTaskBuilder { fn build_task( &self, tasks: &mut VecDeque<(usize, Vec>)>, - unchanged_blocks: &mut Vec<(BlockIndex, Arc)>, + unchanged_blocks: &mut Vec<(BlockIndex, BlockMetaWithHLL)>, block_idx: BlockIndex, - blocks: Vec>, + blocks: Vec, ) -> bool { - if blocks.len() == 1 && !self.check_compact(&blocks[0]) { + if blocks.len() == 1 && !self.check_compact(&blocks[0].0) { unchanged_blocks.push((block_idx, blocks[0].clone())); true } else { + let blocks = blocks.into_iter().map(|v| v.0).collect(); tasks.push_back((block_idx, blocks)); false } } - fn check_compact(&self, block: &Arc) -> bool { + fn check_compact(&self, block: &BlockMeta) -> bool { // The snapshot schema does not contain stream columns, // so the stream columns need to be filtered out. let column_ids = block @@ -554,10 +568,17 @@ impl CompactTaskBuilder { let mut handlers = Vec::with_capacity(compact_segments.len()); for segment in compact_segments.into_iter().rev() { let permit = acquire_task_permit(semaphore.clone()).await?; + let op = self.dal.clone(); let handler = runtime.spawn(async move { + let stats = if let Some(loc) = &segment.summary.hlls { + let stats = read_segment_stats(op.clone(), loc.clone()).await?; + Some(stats) + } else { + None + }; let blocks = segment.block_metas()?; drop(permit); - Ok::<_, ErrorCode>((blocks, segment.summary.clone())) + Ok::<_, ErrorCode>((blocks, segment.summary.clone(), stats)) }); handlers.push(handler); } @@ -573,29 +594,33 @@ impl CompactTaskBuilder { .into_iter() .collect::>>()? .into_iter() - .flat_map(|(blocks, summary)| { + .flat_map(|(blocks, summary, hlls)| { merge_statistics_mut(&mut removed_segment_summary, &summary, self.cluster_key_id); - blocks + + blocks.into_iter().enumerate().map(move |(idx, v)| { + let column_hlls = hlls.as_ref().and_then(|v| v.blocks.get(idx)).cloned(); + (v, column_hlls) + }) }) .collect::>(); if let Some(default_cluster_key) = self.cluster_key_id { // sort ascending. blocks.sort_by(|a, b| { - sort_by_cluster_stats(&a.cluster_stats, &b.cluster_stats, default_cluster_key) + sort_by_cluster_stats(&a.0.cluster_stats, &b.0.cluster_stats, default_cluster_key) }); } let mut tasks = VecDeque::new(); - for block in blocks.iter() { - let (unchanged, need_take) = self.add(block); + for (block_meta, hlls) in blocks.iter() { + let (unchanged, need_take) = self.add(block_meta, hlls); if need_take { let blocks = self.take_blocks(); latest_flag = self.build_task(&mut tasks, &mut unchanged_blocks, block_idx, blocks); block_idx += 1; } if unchanged { - let blocks = vec![block.clone()]; + let blocks = vec![(block_meta.clone(), hlls.clone())]; latest_flag = self.build_task(&mut tasks, &mut unchanged_blocks, block_idx, blocks); block_idx += 1; } @@ -610,16 +635,18 @@ impl CompactTaskBuilder { let mut blocks = if latest_flag { unchanged_blocks.pop().map_or(vec![], |(_, v)| vec![v]) } else { - tasks.pop_back().map_or(vec![], |(_, v)| v) + tasks + .pop_back() + .map_or(vec![], |(_, v)| v.into_iter().map(|v| (v, None)).collect()) }; let (total_rows, total_size, total_compressed) = blocks .iter() .chain(tail.iter()) .fold((0, 0, 0), |mut acc, x| { - acc.0 += x.row_count as usize; - acc.1 += x.block_size as usize; - acc.2 += x.file_size as usize; + acc.0 += x.0.row_count as usize; + acc.1 += x.0.block_size as usize; + acc.2 += x.0.file_size as usize; acc }); if self.check_for_compact(total_rows, total_size, total_compressed) { diff --git a/src/query/storages/fuse/src/operations/mutation/mutator/recluster_mutator.rs b/src/query/storages/fuse/src/operations/mutation/mutator/recluster_mutator.rs index 3c50cd9f83153..157da2468ddbd 100644 --- a/src/query/storages/fuse/src/operations/mutation/mutator/recluster_mutator.rs +++ b/src/query/storages/fuse/src/operations/mutation/mutator/recluster_mutator.rs @@ -27,7 +27,6 @@ use databend_common_catalog::plan::ReclusterParts; use databend_common_catalog::plan::ReclusterTask; use databend_common_catalog::table::Table; use databend_common_catalog::table_context::TableContext; -use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::compare_scalars; use databend_common_expression::types::DataType; @@ -35,9 +34,12 @@ use databend_common_expression::BlockThresholds; use databend_common_expression::Scalar; use databend_common_expression::TableSchemaRef; use databend_common_storage::ColumnNodes; +use databend_storages_common_cache::LoadParams; use databend_storages_common_pruner::BlockMetaIndex; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::CompactSegmentInfo; +use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::RawColumnHLL; use databend_storages_common_table_meta::meta::Statistics; use databend_storages_common_table_meta::meta::TableSnapshot; use fastrace::func_path; @@ -46,7 +48,10 @@ use fastrace::Span; use indexmap::IndexSet; use log::debug; use log::warn; +use opendal::Operator; +use crate::io::MetaReaders; +use crate::operations::common::BlockMetaIndex as BlockIndex; use crate::operations::mutation::SegmentCompactChecker; use crate::operations::BlockCompactMutator; use crate::operations::CompactLazyPartInfo; @@ -65,6 +70,7 @@ pub enum ReclusterMode { #[derive(Clone)] pub struct ReclusterMutator { pub(crate) ctx: Arc, + pub(crate) operator: Operator, pub(crate) depth_threshold: f64, pub(crate) block_thresholds: BlockThresholds, pub(crate) cluster_key_id: u32, @@ -104,6 +110,7 @@ impl ReclusterMutator { Ok(Self { ctx, + operator: table.get_operator(), schema, depth_threshold, block_thresholds, @@ -118,6 +125,7 @@ impl ReclusterMutator { #[allow(clippy::too_many_arguments)] pub fn new( ctx: Arc, + operator: Operator, schema: TableSchemaRef, cluster_key_types: Vec, depth_threshold: f64, @@ -128,6 +136,7 @@ impl ReclusterMutator { ) -> Self { Self { ctx, + operator, schema, depth_threshold, block_thresholds, @@ -168,19 +177,21 @@ impl ReclusterMutator { // Prepare for reclustering by collecting segment indices, statistics, and blocks let mut selected_segs_idx = Vec::with_capacity(compact_segments.len()); let mut selected_statistics = Vec::with_capacity(compact_segments.len()); + let mut selected_seg_stats = Vec::with_capacity(compact_segments.len()); let selected_segments = compact_segments .into_iter() .map(|(loc, info)| { selected_statistics.push(info.summary.clone()); selected_segs_idx.push(loc.segment_idx); - info + selected_seg_stats.push((loc.segment_idx, info.summary.hlls.clone())); + (loc.segment_idx, info) }) .collect::>(); // Gather blocks and create a block map categorized by clustering levels let blocks = self.gather_blocks(selected_segments).await?; let mut blocks_map: BTreeMap> = BTreeMap::new(); - for (idx, block) in blocks.iter().enumerate() { + for (idx, (_, block)) in blocks.iter().enumerate() { if let Some(stats) = &block.cluster_stats { if stats.cluster_key_id == self.cluster_key_id { blocks_map.entry(stats.level).or_default().push(idx); @@ -221,23 +232,23 @@ impl ReclusterMutator { // Analyze each block's statistics and track min/max points for &i in indices.iter() { let block = &blocks[i]; - if let Some(stats) = &block.cluster_stats { + if let Some(stats) = &block.1.cluster_stats { points_map.entry(stats.min().clone()).or_default().0.push(i); points_map.entry(stats.max().clone()).or_default().1.push(i); } // Track small blocks for potential compaction if self.block_thresholds.check_too_small( - block.row_count as usize, - block.block_size as usize, - block.file_size as usize, + block.1.row_count as usize, + block.1.block_size as usize, + block.1.file_size as usize, ) { small_blocks.insert(i); } - total_rows += block.row_count; - total_bytes += block.block_size; - total_compressed += block.file_size; + total_rows += block.1.row_count; + total_bytes += block.1.block_size; + total_compressed += block.1.file_size; } // If total rows and bytes are too small, compact the blocks into one @@ -251,7 +262,7 @@ impl ReclusterMutator { selected_blocks_idx.extend(indices.iter()); let block_metas = indices .into_iter() - .map(|i| (None, blocks[i].clone())) + .map(|i| (None, blocks[i].1.clone())) .collect::>(); tasks.push(self.generate_task( &block_metas, @@ -281,7 +292,7 @@ impl ReclusterMutator { let mut task_indices = Vec::new(); let mut selected_blocks = Vec::new(); for idx in selected_idx { - let block = blocks[idx].clone(); + let block = blocks[idx].1.clone(); let block_size = block.block_size as usize; let row_count = block.row_count as usize; @@ -336,8 +347,8 @@ impl ReclusterMutator { let unordered = || { blocks.windows(2).any(|w| { sort_by_cluster_stats( - &w[0].cluster_stats, - &w[1].cluster_stats, + &w[0].1.cluster_stats, + &w[1].1.cluster_stats, self.cluster_key_id, ) == Ordering::Greater }) @@ -361,8 +372,15 @@ impl ReclusterMutator { let remained_blocks = blocks_idx .difference(&selected_blocks_idx) .map(|&v| blocks[v].clone()) + .collect::>(); + let hlls = self.gather_hlls(selected_seg_stats).await?; + let remained_blocks = remained_blocks + .into_iter() + .map(|(block_index, block_meta)| { + let hll = hlls.get(&block_index).and_then(|hll| hll.clone()); + (block_meta, hll) + }) .collect(); - ReclusterParts::Recluster { tasks, remained_blocks, @@ -424,6 +442,7 @@ impl ReclusterMutator { PartitionsShuffleKind::Mod, BlockCompactMutator::build_compact_tasks( self.ctx.clone(), + self.operator.clone(), self.column_ids.clone(), Some(self.cluster_key_id), self.block_thresholds, @@ -564,15 +583,26 @@ impl ReclusterMutator { #[async_backtrace::framed] async fn gather_blocks( &self, - compact_segments: Vec>, - ) -> Result>> { + compact_segments: Vec<(usize, Arc)>, + ) -> Result)>> { // combine all the tasks. let mut iter = compact_segments.into_iter(); let tasks = std::iter::from_fn(|| { - iter.next().map(|v| { + iter.next().map(|(segment_idx, v)| { async move { - v.block_metas() - .map_err(|_| ErrorCode::Internal("Failed to get block metas")) + v.block_metas().map(|block_metas| { + block_metas + .into_iter() + .enumerate() + .map(|(block_idx, block_meta)| { + let block_index = BlockIndex { + segment_idx, + block_idx, + }; + (block_index, block_meta) + }) + .collect::>() + }) } .in_span(Span::enter_with_local_parent(func_path!())) }) @@ -596,6 +626,65 @@ impl ReclusterMutator { Ok(blocks) } + #[async_backtrace::framed] + async fn gather_hlls( + &self, + hlls: Vec<(usize, Option)>, + ) -> Result>> { + // combine all the tasks. + let mut iter = hlls.into_iter(); + let tasks = std::iter::from_fn(|| { + iter.next().map(|(segment_idx, location)| { + let dal = self.operator.clone(); + async move { + if let Some((loc, ver)) = location { + let reader = MetaReaders::segment_stats_reader(dal); + let load_params = LoadParams { + location: loc, + len_hint: None, + ver, + put_cache: false, + }; + let stats = reader.read(&load_params).await?; + let res = stats + .blocks + .iter() + .enumerate() + .map(|(block_idx, hll)| { + let block_index = BlockIndex { + segment_idx, + block_idx, + }; + (block_index, Some(hll.clone())) + }) + .collect::>(); + Ok(res) + } else { + Ok(vec![]) + } + } + .in_span(Span::enter_with_local_parent(func_path!())) + }) + }); + + let thread_nums = self.ctx.get_settings().get_max_threads()? as usize; + + let joint = execute_futures_in_parallel( + tasks, + thread_nums, + thread_nums * 2, + "convert-segments-worker".to_owned(), + ) + .await?; + let blocks = joint + .into_iter() + .collect::>>()? + .into_iter() + .flatten() + .collect::>(); + Ok(blocks) + } + fn fetch_max_depth( &self, points_map: HashMap, (Vec, Vec)>, diff --git a/src/query/storages/fuse/src/operations/mutation/mutator/segment_compact_mutator.rs b/src/query/storages/fuse/src/operations/mutation/mutator/segment_compact_mutator.rs index 2f92f300f28be..edf39d3046d6b 100644 --- a/src/query/storages/fuse/src/operations/mutation/mutator/segment_compact_mutator.rs +++ b/src/query/storages/fuse/src/operations/mutation/mutator/segment_compact_mutator.rs @@ -17,6 +17,7 @@ use std::time::Instant; use databend_common_exception::Result; use databend_common_metrics::storage::metrics_set_compact_segments_select_duration_second; +use databend_storages_common_cache::SegmentStatistics; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::Statistics; @@ -25,7 +26,9 @@ use databend_storages_common_table_meta::meta::Versioned; use log::info; use opendal::Operator; +use crate::io::read::read_segment_stats_in_parallel; use crate::io::CachedMetaWriter; +use crate::io::MetaWriter; use crate::io::SegmentsIO; use crate::io::TableMetaLocationGenerator; use crate::operations::CompactOptions; @@ -349,6 +352,8 @@ impl<'a> SegmentCompactor<'a> { // 2.1 merge fragmented segments into new segment, and update the statistics let mut blocks = Vec::with_capacity(self.threshold as usize); let mut new_statistics = Statistics::default(); + let mut hlls = Vec::with_capacity(fragments.len()); + let mut hlls_has_none = false; self.compacted_state.num_fragments_compacted += fragments.len(); for (segment, _location) in fragments { @@ -358,13 +363,38 @@ impl<'a> SegmentCompactor<'a> { self.default_cluster_key_id, ); blocks.append(&mut segment.blocks.clone()); + if let Some(val) = segment.summary.hlls { + hlls.push(val); + } else { + hlls_has_none = true; + } } - // 2.2 write down new segment - let new_segment = SegmentInfo::new(blocks, new_statistics); let location = self .location_generator .gen_segment_info_location(self.table_meta_timestamps, false); + // 2.2 merge hlls into new. + let mut block_hlls = Vec::with_capacity(blocks.len()); + if !hlls_has_none { + let max_threads = (self.chunk_size / 4).max(10); + let segment_stats = + read_segment_stats_in_parallel(self.operator.clone(), &hlls, max_threads).await?; + for stats in segment_stats { + block_hlls.append(&mut stats.blocks.clone()); + } + let stats = SegmentStatistics::new(block_hlls); + let segment_stats_location = + TableMetaLocationGenerator::gen_segment_stats_location_from_segment_location( + location.as_str(), + ); + stats + .write_meta(self.operator, &segment_stats_location) + .await?; + new_statistics.hlls = Some((segment_stats_location, SegmentStatistics::VERSION)); + } + + // 2.3 write down new segment + let new_segment = SegmentInfo::new(blocks, new_statistics); new_segment .write_meta_through_cache(self.operator, &location) .await?; diff --git a/src/query/storages/fuse/src/statistics/accumulator.rs b/src/query/storages/fuse/src/statistics/accumulator.rs index 80ba8d7d09b28..d4e1e0fc96d31 100644 --- a/src/query/storages/fuse/src/statistics/accumulator.rs +++ b/src/query/storages/fuse/src/statistics/accumulator.rs @@ -30,7 +30,10 @@ use databend_common_license::license_manager::LicenseManagerSwitch; use databend_storages_common_table_meta::meta::column_oriented_segment::*; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::DraftVirtualColumnMeta; +use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::RawColumnHLL; use databend_storages_common_table_meta::meta::SegmentInfo; +use databend_storages_common_table_meta::meta::SegmentStatistics; use databend_storages_common_table_meta::meta::VirtualColumnMeta; #[derive(Default)] @@ -53,10 +56,12 @@ impl SegmentBuilder for RowOrientedSegmentBuilder { &mut self, thresholds: BlockThresholds, default_cluster_key_id: Option, + hlls: Option, ) -> Result { let builder = std::mem::take(self); - let stat = + let mut stat = super::reduce_block_metas(&builder.blocks_metas, thresholds, default_cluster_key_id); + stat.hlls = hlls; Ok(SegmentInfo::new(builder.blocks_metas, stat)) } @@ -179,3 +184,22 @@ impl VirtualColumnAccumulator { } } } + +#[derive(Default)] +pub struct ColumnHLLAccumulator { + pub hlls: Vec, +} + +impl ColumnHLLAccumulator { + pub fn add_hll(&mut self, hll: RawColumnHLL) { + self.hlls.push(hll); + } + + pub fn build(&mut self) -> SegmentStatistics { + SegmentStatistics::new(std::mem::take(&mut self.hlls)) + } + + pub fn is_empty(&self) -> bool { + self.hlls.is_empty() + } +} diff --git a/src/query/storages/fuse/src/statistics/mod.rs b/src/query/storages/fuse/src/statistics/mod.rs index 7a84340ead9b8..72df7775c56c3 100644 --- a/src/query/storages/fuse/src/statistics/mod.rs +++ b/src/query/storages/fuse/src/statistics/mod.rs @@ -16,8 +16,10 @@ pub mod accumulator; mod cluster_statistics; mod column_statistic; pub mod reducers; +mod segment_statistics; mod traverse; +pub use accumulator::ColumnHLLAccumulator; pub use accumulator::RowOrientedSegmentBuilder; pub use accumulator::VirtualColumnAccumulator; pub use cluster_statistics::sort_by_cluster_stats; diff --git a/src/query/storages/fuse/src/statistics/reducers.rs b/src/query/storages/fuse/src/statistics/reducers.rs index e1aacd53383de..d73e0f5de7f1d 100644 --- a/src/query/storages/fuse/src/statistics/reducers.rs +++ b/src/query/storages/fuse/src/statistics/reducers.rs @@ -42,7 +42,7 @@ pub fn reduce_block_statistics>( // Reduce the `Vec<&ColumnStatistics` into ColumnStatistics`, i.e.: // from : `HashMap)>` - // to : `type BlockStatistics = HashMap` + // to : `type StatisticsOfColumns = HashMap` let len = col_to_stats_lit.len(); col_to_stats_lit .iter() @@ -155,6 +155,7 @@ pub fn merge_statistics_mut( r: &Statistics, default_cluster_key_id: Option, ) { + l.hlls = None; if l.row_count == 0 { l.col_stats = r.col_stats.clone(); l.cluster_stats = r.cluster_stats.clone(); @@ -331,5 +332,6 @@ pub fn reduce_block_metas>( col_stats: merged_col_stats, cluster_stats: merged_cluster_stats, virtual_block_count: merged_virtual_block_count, + hlls: None, } } diff --git a/src/query/storages/fuse/src/statistics/segment_statistics.rs b/src/query/storages/fuse/src/statistics/segment_statistics.rs new file mode 100644 index 0000000000000..4ca0419659671 --- /dev/null +++ b/src/query/storages/fuse/src/statistics/segment_statistics.rs @@ -0,0 +1,13 @@ +// Copyright 2021 Datafuse Labs +// +// 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. diff --git a/src/query/storages/fuse/src/table_functions/fuse_block.rs b/src/query/storages/fuse/src/table_functions/fuse_block.rs index 11c31d1227e0b..eaecb5425c809 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_block.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_block.rs @@ -75,10 +75,6 @@ impl TableMetaFunc for FuseBlock { "virtual_column_size", TableDataType::Nullable(Box::new(TableDataType::Number(NumberDataType::UInt64))), ), - TableField::new( - "block_stats_size", - TableDataType::Number(NumberDataType::UInt64), - ), ]) } @@ -103,7 +99,6 @@ impl TableMetaFunc for FuseBlock { let mut ngram_index_size = Vec::with_capacity(len); let mut vector_index_size = Vec::with_capacity(len); let mut virtual_column_size = Vec::with_capacity(len); - let mut block_stats_size = Vec::with_capacity(len); let segments_io = SegmentsIO::create(ctx.clone(), tbl.operator.clone(), tbl.schema()); @@ -139,7 +134,6 @@ impl TableMetaFunc for FuseBlock { .as_ref() .map(|m| m.virtual_column_size), ); - block_stats_size.push(block.block_stats_size); num_rows += 1; if num_rows >= limit { @@ -163,7 +157,6 @@ impl TableMetaFunc for FuseBlock { UInt64Type::from_opt_data(ngram_index_size).into(), UInt64Type::from_opt_data(vector_index_size).into(), UInt64Type::from_opt_data(virtual_column_size).into(), - UInt64Type::from_data(block_stats_size).into(), ], num_rows, )) diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history.test b/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history.test index cad96b3804842..fe6536dbae773 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history.test +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history.test @@ -130,11 +130,6 @@ insert into t2 values(1, 'a'), (2, 'b'); ---- 2 -query I -select block_stats_size > 0 from fuse_block('db_09_0006', 't2'); ----- -1 - statement ok alter table t2 set options(approx_distinct_columns = ''); @@ -143,12 +138,6 @@ insert into t2 values(3, 'c'), (4, 'd'); ---- 2 -query I -select block_stats_size > 0 from fuse_block('db_09_0006', 't2') order by block_location; ----- -1 -0 - statement ok DROP TABLE t From b39280af1e60857ad5a7db7d34d0bb233116fe35 Mon Sep 17 00:00:00 2001 From: zhyass Date: Fri, 1 Aug 2025 01:27:03 +0800 Subject: [PATCH 09/11] add test --- src/query/catalog/src/plan/partition.rs | 4 +- .../interpreter_table_set_options.rs | 4 +- .../service/src/test_kits/block_writer.rs | 4 +- src/query/service/src/test_kits/fuse.rs | 25 +++++--- .../it/storages/fuse/bloom_index_meta_size.rs | 2 +- .../tests/it/storages/fuse/conflict.rs | 20 +++---- .../tests/it/storages/fuse/statistics.rs | 57 ++++++++++++++----- .../segment_builder.rs | 12 ++-- .../common/table_meta/src/meta/current/mod.rs | 1 + .../common/table_meta/src/meta/statistics.rs | 8 +-- .../common/table_meta/src/meta/v2/mod.rs | 1 + .../common/table_meta/src/meta/v2/segment.rs | 4 +- .../src/meta/v2/segment_statistics.rs | 10 ++-- .../table_meta/src/meta/v2/statistics.rs | 12 +++- .../src/meta/v3/frozen/statistics.rs | 2 +- .../src/io/write/block_statistics_writer.rs | 6 +- .../fuse/src/io/write/block_writer.rs | 4 +- .../transform_mutation_aggregator.rs | 39 +++++++------ .../processors/transform_serialize_segment.rs | 33 +++++------ src/query/storages/fuse/src/operations/gc.rs | 8 +-- .../mutation/mutator/block_compact_mutator.rs | 10 ++-- .../mutation/mutator/recluster_mutator.rs | 14 +++-- .../mutator/segment_compact_mutator.rs | 30 ++++++---- .../fuse/src/statistics/accumulator.rs | 12 ++-- .../storages/fuse/src/statistics/reducers.rs | 4 +- .../fuse/src/table_functions/fuse_segment.rs | 13 +++++ .../09_0006_func_fuse_history.test | 19 +++++++ 27 files changed, 231 insertions(+), 127 deletions(-) diff --git a/src/query/catalog/src/plan/partition.rs b/src/query/catalog/src/plan/partition.rs index b13d2ae0f597d..dbecc0980d831 100644 --- a/src/query/catalog/src/plan/partition.rs +++ b/src/query/catalog/src/plan/partition.rs @@ -26,7 +26,7 @@ use databend_common_config::GlobalConfig; use databend_common_exception::Result; use databend_common_meta_types::NodeInfo; use databend_storages_common_table_meta::meta::BlockMeta; -use databend_storages_common_table_meta::meta::RawColumnHLL; +use databend_storages_common_table_meta::meta::RawBlockHLL; use databend_storages_common_table_meta::meta::Statistics; use parking_lot::RwLock; use rand::prelude::SliceRandom; @@ -347,7 +347,7 @@ pub struct ReclusterTask { pub level: i32, } -pub type BlockMetaWithHLL = (Arc, Option); +pub type BlockMetaWithHLL = (Arc, Option); #[derive(Clone)] pub enum ReclusterParts { diff --git a/src/query/service/src/interpreters/interpreter_table_set_options.rs b/src/query/service/src/interpreters/interpreter_table_set_options.rs index f98f2915778cf..bdbb08667178f 100644 --- a/src/query/service/src/interpreters/interpreter_table_set_options.rs +++ b/src/query/service/src/interpreters/interpreter_table_set_options.rs @@ -243,12 +243,12 @@ async fn set_segment_format( for block in segment.blocks { segment_builder.add_block(block.as_ref().clone())?; } - let hlls = segment.summary.hlls; + let additional_stats_meta = segment.summary.additional_stats_meta; let segment = segment_builder .build( fuse_table.get_block_thresholds(), fuse_table.cluster_key_id(), - hlls, + additional_stats_meta, )? .serialize()?; let location_gen = fuse_table.meta_location_generator(); diff --git a/src/query/service/src/test_kits/block_writer.rs b/src/query/service/src/test_kits/block_writer.rs index 91a0f5a1f89cf..6c984ded333ef 100644 --- a/src/query/service/src/test_kits/block_writer.rs +++ b/src/query/service/src/test_kits/block_writer.rs @@ -35,7 +35,7 @@ use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::ClusterStatistics; use databend_storages_common_table_meta::meta::Compression; use databend_storages_common_table_meta::meta::Location; -use databend_storages_common_table_meta::meta::RawColumnHLL; +use databend_storages_common_table_meta::meta::RawBlockHLL; use databend_storages_common_table_meta::meta::StatisticsOfColumns; use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::table::TableCompression; @@ -73,7 +73,7 @@ impl<'a> BlockWriter<'a> { block: DataBlock, col_stats: StatisticsOfColumns, cluster_stats: Option, - ) -> Result<(BlockMeta, Option, RawColumnHLL)> { + ) -> Result<(BlockMeta, Option, RawBlockHLL)> { let (location, block_id) = if !self.is_greater_than_v5 { let location_generator = old_version_generator::TableMetaLocationGenerator::with_prefix( self.location_generator.prefix().to_string(), diff --git a/src/query/service/src/test_kits/fuse.rs b/src/query/service/src/test_kits/fuse.rs index 547077432ffe1..b5d0109f58589 100644 --- a/src/query/service/src/test_kits/fuse.rs +++ b/src/query/service/src/test_kits/fuse.rs @@ -41,6 +41,7 @@ use databend_storages_common_cache::SegmentStatistics; use databend_storages_common_table_meta::meta::testing::SegmentInfoV2; use databend_storages_common_table_meta::meta::testing::TableSnapshotV2; use databend_storages_common_table_meta::meta::testing::TableSnapshotV4; +use databend_storages_common_table_meta::meta::AdditionalStatsMeta; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SegmentInfo; @@ -125,10 +126,14 @@ pub async fn generate_segments_v2( uuid, SegmentInfoV2::VERSION, ); - let hll_location = + let stats_location = TableMetaLocationGenerator::gen_segment_stats_location_from_segment_location(&location); - stats.write_meta(dal, &hll_location).await?; - summary.hlls = Some((hll_location, SegmentStatistics::VERSION)); + let additional_stats_meta = AdditionalStatsMeta { + size: stats.len() as u64, + location: (stats_location.clone(), SegmentStatistics::VERSION), + }; + dal.write(&stats_location, stats).await?; + summary.additional_stats_meta = Some(additional_stats_meta); let segment_info = SegmentInfoV2::new(block_metas, summary); write_v2_to_storage(dal, &location, &segment_info).await?; segs.push(((location, SegmentInfoV2::VERSION), segment_info)) @@ -163,10 +168,14 @@ pub async fn generate_segments( ); location_generator.gen_segment_info_location(table_meta_timestamps) }; - let hll_location = + let stats_location = TableMetaLocationGenerator::gen_segment_stats_location_from_segment_location(&location); - stats.write_meta(dal, &hll_location).await?; - summary.hlls = Some((hll_location, SegmentStatistics::VERSION)); + let additional_stats_meta = AdditionalStatsMeta { + size: stats.len() as u64, + location: (stats_location.clone(), SegmentStatistics::VERSION), + }; + dal.write(&stats_location, stats).await?; + summary.additional_stats_meta = Some(additional_stats_meta); let segment_info = SegmentInfo::new(block_metas, summary); segment_info.write_meta(dal, location.as_str()).await?; segs.push(((location, SegmentInfo::VERSION), segment_info)) @@ -179,7 +188,7 @@ async fn generate_blocks( num_blocks: usize, is_greater_than_v5: bool, table_meta_timestamps: TableMetaTimestamps, -) -> Result<(Vec>, SegmentStatistics)> { +) -> Result<(Vec>, Vec)> { let dal = fuse_table.get_operator_ref(); let schema = fuse_table.schema(); let location_generator = fuse_table.meta_location_generator(); @@ -208,7 +217,7 @@ async fn generate_blocks( block_metas.push(Arc::new(block_meta)); hlls.push(hll); } - let stats = SegmentStatistics::new(hlls); + let stats = SegmentStatistics::new(hlls).to_bytes()?; Ok((block_metas, stats)) } diff --git a/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs b/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs index 5f23a93f45167..556ee91b87ec6 100644 --- a/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs +++ b/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs @@ -363,7 +363,7 @@ fn build_test_segment_info( col_stats: col_stats.clone(), cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, }; Ok(SegmentInfo::new(block_metas, statistics)) diff --git a/src/query/service/tests/it/storages/fuse/conflict.rs b/src/query/service/tests/it/storages/fuse/conflict.rs index d04f2db2c9a6a..cecab510afa54 100644 --- a/src/query/service/tests/it/storages/fuse/conflict.rs +++ b/src/query/service/tests/it/storages/fuse/conflict.rs @@ -106,7 +106,7 @@ fn test_resolvable_delete_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, }; let mut latest_snapshot = new_empty_snapshot(TableSchema::default(), None); @@ -131,7 +131,7 @@ fn test_resolvable_delete_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, }; let removed_statistics = Statistics { @@ -149,7 +149,7 @@ fn test_resolvable_delete_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, }; let merged_statistics = Statistics { @@ -167,7 +167,7 @@ fn test_resolvable_delete_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, }; let ctx = ConflictResolveContext::ModifiedSegmentExistsInLatest(SnapshotChanges { @@ -211,7 +211,7 @@ fn test_resolvable_delete_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, }; assert_eq!(actual, expected); } @@ -249,7 +249,7 @@ fn test_resolvable_replace_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, }; let mut latest_snapshot = new_empty_snapshot(TableSchema::default(), None); @@ -274,7 +274,7 @@ fn test_resolvable_replace_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, }; let removed_statistics = Statistics { @@ -292,7 +292,7 @@ fn test_resolvable_replace_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, }; let merged_statistics = Statistics { @@ -310,7 +310,7 @@ fn test_resolvable_replace_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, }; let ctx = ConflictResolveContext::ModifiedSegmentExistsInLatest(SnapshotChanges { @@ -359,7 +359,7 @@ fn test_resolvable_replace_conflict() { col_stats: HashMap::new(), cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, }; assert_eq!(actual, expected); } diff --git a/src/query/service/tests/it/storages/fuse/statistics.rs b/src/query/service/tests/it/storages/fuse/statistics.rs index f039a43ed65e5..366d6a667dfa8 100644 --- a/src/query/service/tests/it/storages/fuse/statistics.rs +++ b/src/query/service/tests/it/storages/fuse/statistics.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use chrono::Utc; use databend_common_base::base::tokio; +use databend_common_exception::Result; use databend_common_expression::type_check::check; use databend_common_expression::types::number::Int32Type; use databend_common_expression::types::number::NumberScalar; @@ -38,6 +39,8 @@ use databend_common_expression::TableSchema; use databend_common_functions::aggregates::eval_aggr; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_sql::evaluator::BlockOperator; +use databend_common_sql::ApproxDistinctColumns; +use databend_common_storages_fuse::io::build_column_hlls; use databend_common_storages_fuse::statistics::reducers::reduce_block_metas; use databend_common_storages_fuse::statistics::Trim; use databend_common_storages_fuse::statistics::STATS_REPLACEMENT_CHAR; @@ -49,7 +52,10 @@ use databend_query::storages::fuse::statistics::reducers; use databend_query::storages::fuse::statistics::ClusterStatsGenerator; use databend_query::storages::fuse::statistics::RowOrientedSegmentBuilder; use databend_query::test_kits::*; +use databend_storages_common_index::RangeIndex; use databend_storages_common_table_meta::meta::column_oriented_segment::SegmentBuilder; +use databend_storages_common_table_meta::meta::decode_column_hll; +use databend_storages_common_table_meta::meta::encode_column_hll; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::ClusterStatistics; use databend_storages_common_table_meta::meta::ColumnStatistics; @@ -57,8 +63,9 @@ use databend_storages_common_table_meta::meta::Compression; use databend_storages_common_table_meta::meta::Statistics; use opendal::Operator; use rand::Rng; + #[test] -fn test_ft_stats_block_stats() -> databend_common_exception::Result<()> { +fn test_ft_stats_block_stats() -> Result<()> { let schema = Arc::new(TableSchema::new(vec![ TableField::new("a", TableDataType::Number(NumberDataType::Int32)), TableField::new("b", TableDataType::String), @@ -82,7 +89,7 @@ fn test_ft_stats_block_stats() -> databend_common_exception::Result<()> { } #[test] -fn test_ft_stats_block_stats_with_column_distinct_count() -> databend_common_exception::Result<()> { +fn test_ft_stats_block_stats_with_column_distinct_count() -> Result<()> { let schema = Arc::new(TableSchema::new(vec![ TableField::new_from_column_id("a", TableDataType::Number(NumberDataType::Int32), 0), TableField::new_from_column_id("b", TableDataType::String, 1), @@ -109,7 +116,7 @@ fn test_ft_stats_block_stats_with_column_distinct_count() -> databend_common_exc } #[test] -fn test_ft_tuple_stats_block_stats() -> databend_common_exception::Result<()> { +fn test_ft_tuple_stats_block_stats() -> Result<()> { let schema = Arc::new(TableSchema::new(vec![TableField::new( "a", TableDataType::Tuple { @@ -142,7 +149,7 @@ fn test_ft_tuple_stats_block_stats() -> databend_common_exception::Result<()> { } #[test] -fn test_ft_stats_col_stats_reduce() -> databend_common_exception::Result<()> { +fn test_ft_stats_col_stats_reduce() -> Result<()> { let num_of_blocks = 10; let rows_per_block = 3; let val_start_with = 1; @@ -188,7 +195,7 @@ fn test_ft_stats_col_stats_reduce() -> databend_common_exception::Result<()> { } #[test] -fn test_reduce_block_statistics_in_memory_size() -> databend_common_exception::Result<()> { +fn test_reduce_block_statistics_in_memory_size() -> Result<()> { let iter = |mut idx| { std::iter::from_fn(move || { idx += 1; @@ -219,7 +226,7 @@ fn test_reduce_block_statistics_in_memory_size() -> databend_common_exception::R } #[test] -fn test_reduce_cluster_statistics() -> databend_common_exception::Result<()> { +fn test_reduce_cluster_statistics() -> Result<()> { let default_cluster_key_id = Some(0); let cluster_stats_0 = Some(ClusterStatistics::new( 0, @@ -312,7 +319,7 @@ fn test_reduce_cluster_statistics() -> databend_common_exception::Result<()> { } #[tokio::test(flavor = "multi_thread")] -async fn test_accumulator() -> databend_common_exception::Result<()> { +async fn test_accumulator() -> Result<()> { let (schema, blocks) = TestFixture::gen_sample_blocks(10, 1); let mut stats_acc = RowOrientedSegmentBuilder::default(); @@ -339,7 +346,7 @@ async fn test_accumulator() -> databend_common_exception::Result<()> { } #[tokio::test(flavor = "multi_thread")] -async fn test_ft_cluster_stats_with_stats() -> databend_common_exception::Result<()> { +async fn test_ft_cluster_stats_with_stats() -> Result<()> { let schema = DataSchemaRefExt::create(vec![DataField::new( "a", DataType::Number(NumberDataType::Int32), @@ -436,8 +443,8 @@ async fn test_ft_cluster_stats_with_stats() -> databend_common_exception::Result } #[test] -fn test_ft_stats_block_stats_string_columns_trimming() -> databend_common_exception::Result<()> { - let suite = || -> databend_common_exception::Result<()> { +fn test_ft_stats_block_stats_string_columns_trimming() -> Result<()> { + let suite = || -> Result<()> { // prepare random strings // 100 string, length ranges from 0 to 100 (chars) let mut rand_strings: Vec = vec![]; @@ -489,8 +496,7 @@ fn test_ft_stats_block_stats_string_columns_trimming() -> databend_common_except } #[test] -fn test_ft_stats_block_stats_string_columns_trimming_using_eval( -) -> databend_common_exception::Result<()> { +fn test_ft_stats_block_stats_string_columns_trimming_using_eval() -> Result<()> { // verifies (randomly) the following assumptions: // // https://github.com/datafuselabs/databend/issues/7829 @@ -499,7 +505,7 @@ fn test_ft_stats_block_stats_string_columns_trimming_using_eval( // > the trimmed max should be larger than the non-trimmed one, and the trimmed min // > should be lesser than the non-trimmed one. - let suite = || -> databend_common_exception::Result<()> { + let suite = || -> Result<()> { // prepare random strings // 100 string, length ranges from 0 to 100 (chars) let mut rand_strings: Vec = vec![]; @@ -597,7 +603,7 @@ fn char_len(value: &str) -> usize { } #[test] -fn test_reduce_block_meta() -> databend_common_exception::Result<()> { +fn test_reduce_block_meta() -> Result<()> { // case 1: empty input should return the default statistics let block_metas: Vec = vec![]; let reduced = reduce_block_metas(&block_metas, BlockThresholds::default(), None); @@ -652,3 +658,26 @@ fn test_reduce_block_meta() -> databend_common_exception::Result<()> { Ok(()) } + +#[test] +fn test_encode_column_hll() -> Result<()> { + let schema = Arc::new(TableSchema::new(vec![ + TableField::new("a", TableDataType::Number(NumberDataType::Int32)), + TableField::new("b", TableDataType::String), + ])); + let block = DataBlock::new_from_columns(vec![ + Int32Type::from_data(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 9]), + StringType::from_data(vec![ + "aa", "aa", "bb", "bb", "cc", "dd", "dd", "dd", "ee", "ff", + ]), + ]); + + let hll_columns = ApproxDistinctColumns::All; + let ndv_columns_map = + hll_columns.distinct_column_fields(schema.clone(), RangeIndex::supported_table_type)?; + let column_hlls = build_column_hlls(&block, &ndv_columns_map)?.unwrap(); + let encoded = encode_column_hll(&column_hlls)?; + let decoded = decode_column_hll(&encoded)?.unwrap(); + assert_eq!(column_hlls, decoded); + Ok(()) +} diff --git a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/segment_builder.rs b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/segment_builder.rs index 41bb7ac029399..f4817388c7420 100644 --- a/src/query/storages/common/table_meta/src/meta/column_oriented_segment/segment_builder.rs +++ b/src/query/storages/common/table_meta/src/meta/column_oriented_segment/segment_builder.rs @@ -42,6 +42,7 @@ use super::segment::ColumnOrientedSegment; use super::AbstractSegment; use crate::meta::format::encode; use crate::meta::supported_stat_type; +use crate::meta::AdditionalStatsMeta; use crate::meta::BlockMeta; use crate::meta::ClusterStatistics; use crate::meta::ColumnStatistics; @@ -58,7 +59,7 @@ pub trait SegmentBuilder: Send + Sync + 'static { &mut self, thresholds: BlockThresholds, default_cluster_key_id: Option, - hlls: Option, + additional_stats_meta: Option, ) -> Result; fn new(table_schema: TableSchemaRef, block_per_segment: usize) -> Self; } @@ -160,13 +161,14 @@ impl SegmentBuilder for ColumnOrientedSegmentBuilder { &mut self, thresholds: BlockThresholds, default_cluster_key_id: Option, - hlls: Option, + additional_stats_meta: Option, ) -> Result { let mut this = std::mem::replace( self, ColumnOrientedSegmentBuilder::new(self.table_schema.clone(), self.block_per_segment), ); - let summary = this.build_summary(thresholds, default_cluster_key_id, hlls)?; + let summary = + this.build_summary(thresholds, default_cluster_key_id, additional_stats_meta)?; let cluster_stats = this.cluster_stats; let mut cluster_stats_binary = Vec::with_capacity(cluster_stats.len()); for stats in cluster_stats { @@ -272,7 +274,7 @@ impl ColumnOrientedSegmentBuilder { &mut self, thresholds: BlockThresholds, default_cluster_key_id: Option, - hlls: Option, + additional_stats_meta: Option, ) -> Result { let row_count = self.row_count.iter().sum(); let block_count = self.row_count.len() as u64; @@ -365,7 +367,7 @@ impl ColumnOrientedSegmentBuilder { col_stats, cluster_stats, virtual_block_count: Some(virtual_block_count), - hlls, + additional_stats_meta, }) } } diff --git a/src/query/storages/common/table_meta/src/meta/current/mod.rs b/src/query/storages/common/table_meta/src/meta/current/mod.rs index 54b9aeae9ae30..cdb11e4b13e02 100644 --- a/src/query/storages/common/table_meta/src/meta/current/mod.rs +++ b/src/query/storages/common/table_meta/src/meta/current/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. pub use v0::ColumnMeta as SingleColumnMeta; +pub use v2::AdditionalStatsMeta; pub use v2::BlockMeta; pub use v2::ClusterStatistics; pub use v2::ColumnMeta; diff --git a/src/query/storages/common/table_meta/src/meta/statistics.rs b/src/query/storages/common/table_meta/src/meta/statistics.rs index 5cc51aebc5bc1..b9def865cb408 100644 --- a/src/query/storages/common/table_meta/src/meta/statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/statistics.rs @@ -32,8 +32,8 @@ pub type SnapshotId = Uuid; pub type Location = (String, FormatVersion); pub type ClusterKey = (u32, String); pub type StatisticsOfColumns = HashMap; -pub type ColumnHLL = HashMap; -pub type RawColumnHLL = Vec; +pub type BlockHLL = HashMap; +pub type RawBlockHLL = Vec; // Assigned to executors, describes that which blocks of given segment, an executor should take care of #[derive(Clone, Debug, serde::Serialize, serde::Deserialize, PartialEq)] @@ -59,7 +59,7 @@ pub fn supported_stat_type(data_type: &DataType) -> bool { ) } -pub fn encode_column_hll(hll: &ColumnHLL) -> Result { +pub fn encode_column_hll(hll: &BlockHLL) -> Result { let encoding = SegmentStatistics::encoding(); let compression = SegmentStatistics::compression(); @@ -68,7 +68,7 @@ pub fn encode_column_hll(hll: &ColumnHLL) -> Result { Ok(data_compress) } -pub fn decode_column_hll(data: &RawColumnHLL) -> Result> { +pub fn decode_column_hll(data: &RawBlockHLL) -> Result> { if data.is_empty() { return Ok(None); } diff --git a/src/query/storages/common/table_meta/src/meta/v2/mod.rs b/src/query/storages/common/table_meta/src/meta/v2/mod.rs index 2b819ed9a9015..26e79eab9bc32 100644 --- a/src/query/storages/common/table_meta/src/meta/v2/mod.rs +++ b/src/query/storages/common/table_meta/src/meta/v2/mod.rs @@ -28,6 +28,7 @@ pub use segment::VirtualBlockMeta; pub use segment::VirtualColumnMeta; pub use segment_statistics::SegmentStatistics; pub use snapshot::TableSnapshot; +pub use statistics::AdditionalStatsMeta; pub use statistics::ClusterStatistics; pub use statistics::ColumnStatistics; pub use statistics::Statistics; diff --git a/src/query/storages/common/table_meta/src/meta/v2/segment.rs b/src/query/storages/common/table_meta/src/meta/v2/segment.rs index 603ec014a19ea..d7ba70f50cc10 100644 --- a/src/query/storages/common/table_meta/src/meta/v2/segment.rs +++ b/src/query/storages/common/table_meta/src/meta/v2/segment.rs @@ -37,7 +37,7 @@ use crate::meta::ColumnStatistics; use crate::meta::Compression; use crate::meta::FormatVersion; use crate::meta::Location; -use crate::meta::RawColumnHLL; +use crate::meta::RawBlockHLL; use crate::meta::Statistics; use crate::meta::Versioned; @@ -244,7 +244,7 @@ impl BlockMeta { pub struct ExtendedBlockMeta { pub block_meta: BlockMeta, pub draft_virtual_block_meta: Option, - pub column_hlls: Option, + pub column_hlls: Option, } #[typetag::serde(name = "extended_block_meta")] diff --git a/src/query/storages/common/table_meta/src/meta/v2/segment_statistics.rs b/src/query/storages/common/table_meta/src/meta/v2/segment_statistics.rs index d6f7129d7fea5..885359efa7168 100644 --- a/src/query/storages/common/table_meta/src/meta/v2/segment_statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/v2/segment_statistics.rs @@ -25,21 +25,23 @@ use crate::meta::format::read_and_deserialize; use crate::meta::FormatVersion; use crate::meta::MetaCompression; use crate::meta::MetaEncoding; -use crate::meta::RawColumnHLL; +use crate::meta::RawBlockHLL; use crate::meta::Versioned; #[derive(Serialize, Deserialize, Clone, Debug)] pub struct SegmentStatistics { pub format_version: FormatVersion, - pub blocks: Vec, + /// HLL data for blocks within the segment. + /// This stores the HyperLogLog statistics for each block in the segment. + pub block_hlls: Vec, } impl SegmentStatistics { - pub fn new(blocks: Vec) -> Self { + pub fn new(block_hlls: Vec) -> Self { Self { format_version: SegmentStatistics::VERSION, - blocks, + block_hlls, } } diff --git a/src/query/storages/common/table_meta/src/meta/v2/statistics.rs b/src/query/storages/common/table_meta/src/meta/v2/statistics.rs index 9489cd6748b11..29d8fdeafaaca 100644 --- a/src/query/storages/common/table_meta/src/meta/v2/statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/v2/statistics.rs @@ -70,6 +70,14 @@ pub struct ClusterStatistics { pub pages: Option>, } +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, PartialEq, Eq)] +pub struct AdditionalStatsMeta { + /// The size of the stats data in bytes. + pub size: u64, + /// The file location of the stats data. + pub location: Location, +} + #[derive(serde::Serialize, serde::Deserialize, Debug, Clone, PartialEq, Eq, Default)] pub struct Statistics { pub row_count: u64, @@ -90,7 +98,7 @@ pub struct Statistics { pub cluster_stats: Option, pub virtual_block_count: Option, - pub hlls: Option, + pub additional_stats_meta: Option, } // conversions from old meta data @@ -256,7 +264,7 @@ impl Statistics { col_stats, cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, } } } diff --git a/src/query/storages/common/table_meta/src/meta/v3/frozen/statistics.rs b/src/query/storages/common/table_meta/src/meta/v3/frozen/statistics.rs index 7a7c13427b738..13a2a71acddd5 100644 --- a/src/query/storages/common/table_meta/src/meta/v3/frozen/statistics.rs +++ b/src/query/storages/common/table_meta/src/meta/v3/frozen/statistics.rs @@ -54,7 +54,7 @@ impl From for crate::meta::Statistics { .collect(), cluster_stats: None, virtual_block_count: None, - hlls: None, + additional_stats_meta: None, } } } diff --git a/src/query/storages/fuse/src/io/write/block_statistics_writer.rs b/src/query/storages/fuse/src/io/write/block_statistics_writer.rs index b191d05b3fe54..3257863962f40 100644 --- a/src/query/storages/fuse/src/io/write/block_statistics_writer.rs +++ b/src/query/storages/fuse/src/io/write/block_statistics_writer.rs @@ -20,7 +20,7 @@ use databend_common_expression::BlockEntry; use databend_common_expression::DataBlock; use databend_common_expression::FieldIndex; use databend_common_expression::TableField; -use databend_storages_common_table_meta::meta::ColumnHLL; +use databend_storages_common_table_meta::meta::BlockHLL; use crate::io::write::stream::create_column_ndv_estimator; use crate::io::write::stream::ColumnNDVEstimator; @@ -29,7 +29,7 @@ use crate::io::write::stream::ColumnNDVEstimatorOps; pub fn build_column_hlls( block: &DataBlock, ndv_columns_map: &BTreeMap, -) -> Result> { +) -> Result> { let mut builder = BlockStatsBuilder::new(ndv_columns_map); builder.add_block(block)?; builder.finalize() @@ -85,7 +85,7 @@ impl BlockStatsBuilder { Ok(()) } - pub fn finalize(self) -> Result> { + pub fn finalize(self) -> Result> { if self.builders.is_empty() { return Ok(None); } diff --git a/src/query/storages/fuse/src/io/write/block_writer.rs b/src/query/storages/fuse/src/io/write/block_writer.rs index dd83cb01dc53a..af345da01a4b7 100644 --- a/src/query/storages/fuse/src/io/write/block_writer.rs +++ b/src/query/storages/fuse/src/io/write/block_writer.rs @@ -51,7 +51,7 @@ use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::ClusterStatistics; use databend_storages_common_table_meta::meta::ColumnMeta; use databend_storages_common_table_meta::meta::ExtendedBlockMeta; -use databend_storages_common_table_meta::meta::RawColumnHLL; +use databend_storages_common_table_meta::meta::RawBlockHLL; use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::table::TableCompression; use opendal::Operator; @@ -143,7 +143,7 @@ pub struct BlockSerialization { pub inverted_index_states: Vec, pub virtual_column_state: Option, pub vector_index_state: Option, - pub column_hlls: Option, + pub column_hlls: Option, } local_block_meta_serde!(BlockSerialization); diff --git a/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs b/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs index 59144f0328a38..9861eaebe6119 100644 --- a/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs +++ b/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs @@ -31,11 +31,12 @@ use databend_common_expression::VirtualDataSchema; use databend_common_pipeline_transforms::processors::AsyncAccumulatingTransform; use databend_common_sql::executor::physical_plans::MutationKind; use databend_storages_common_cache::SegmentStatistics; +use databend_storages_common_table_meta::meta::AdditionalStatsMeta; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::ClusterStatistics; use databend_storages_common_table_meta::meta::ExtendedBlockMeta; use databend_storages_common_table_meta::meta::Location; -use databend_storages_common_table_meta::meta::RawColumnHLL; +use databend_storages_common_table_meta::meta::RawBlockHLL; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::Statistics; use databend_storages_common_table_meta::meta::TableMetaTimestamps; @@ -50,7 +51,6 @@ use opendal::Operator; use crate::io::read::read_segment_stats; use crate::io::CachedMetaWriter; -use crate::io::MetaWriter; use crate::io::SegmentsIO; use crate::io::TableMetaLocationGenerator; use crate::operations::common::CommitMeta; @@ -333,7 +333,7 @@ impl TableMutationAggregator { let set_hilbert_level = self.set_hilbert_level; let kind = self.kind; for chunk in &merged_blocks.into_iter().chunks(chunk_size) { - let (new_blocks, new_hlls): (Vec>, Vec>) = + let (new_blocks, new_hlls): (Vec>, Vec>) = chunk.unzip(); let new_hlls = if new_hlls.iter().all(|v| v.is_none()) { None @@ -342,7 +342,7 @@ impl TableMutationAggregator { .into_iter() .map(|x| x.unwrap_or_default()) .collect::>(); - Some(SegmentStatistics::new(hlls)) + Some(SegmentStatistics::new(hlls).to_bytes()?) }; let all_perfect = new_blocks.len() > 1; @@ -493,8 +493,8 @@ impl TableMutationAggregator { SegmentsIO::read_compact_segment(op.clone(), loc, schema, false).await?; let mut segment_info = SegmentInfo::try_from(compact_segment_info)?; - let stats = if let Some(loc) = &segment_info.summary.hlls { - let stats = read_segment_stats(op.clone(), loc.clone()).await?; + let stats = if let Some(meta) = &segment_info.summary.additional_stats_meta { + let stats = read_segment_stats(op.clone(), meta.location.clone()).await?; Some(stats) } else { None @@ -507,7 +507,7 @@ impl TableMutationAggregator { .map(|(block_idx, block_meta)| { let hll = stats .as_ref() - .and_then(|v| v.blocks.get(block_idx)) + .and_then(|v| v.block_hlls.get(block_idx)) .cloned(); (block_idx, (block_meta, hll)) }) @@ -536,7 +536,7 @@ impl TableMutationAggregator { .cluster_stats .as_ref() .is_some_and(|v| v.cluster_key_id == default_cluster_key_id.unwrap()); - let stats = generate_segment_stats(new_hlls); + let stats = generate_segment_stats(new_hlls)?; (new_blocks, stats, Some(segment_info.summary)) } else { // use by compact. @@ -550,7 +550,7 @@ impl TableMutationAggregator { .sorted_by(|a, b| a.0.cmp(&b.0)) .map(|(_, meta)| meta) .unzip(); - let stats = generate_segment_stats(new_hlls); + let stats = generate_segment_stats(new_hlls)?; (new_blocks, stats, None) }; @@ -761,7 +761,7 @@ impl BlockMutations { fn new_replacement( block_idx: BlockIndex, block_meta: Arc, - column_hlls: Option, + column_hlls: Option, ) -> Self { BlockMutations { replaced_blocks: vec![(block_idx, (block_meta, column_hlls))], @@ -780,7 +780,7 @@ impl BlockMutations { &mut self, block_idx: BlockIndex, block_meta: Arc, - column_hlls: Option, + column_hlls: Option, ) { self.replaced_blocks .push((block_idx, (block_meta, column_hlls))); @@ -804,7 +804,7 @@ async fn write_segment( dal: Operator, location_gen: TableMetaLocationGenerator, blocks: Vec>, - stats: Option, + stats: Option>, thresholds: BlockThresholds, default_cluster_key: Option, all_perfect: bool, @@ -850,8 +850,12 @@ async fn write_segment( TableMetaLocationGenerator::gen_segment_stats_location_from_segment_location( location.as_str(), ); - stats.write_meta(&dal, &segment_stats_location).await?; - new_summary.hlls = Some((segment_stats_location, SegmentStatistics::VERSION)); + let additional_stats_meta = AdditionalStatsMeta { + size: stats.len() as u64, + location: (segment_stats_location.clone(), SegmentStatistics::VERSION), + }; + dal.write(&segment_stats_location, stats).await?; + new_summary.additional_stats_meta = Some(additional_stats_meta); } // create new segment info @@ -862,11 +866,12 @@ async fn write_segment( Ok((location, new_summary)) } -fn generate_segment_stats(hlls: Vec>) -> Option { +fn generate_segment_stats(hlls: Vec>) -> Result>> { if hlls.iter().all(|v| v.is_none()) { - None + Ok(None) } else { let blocks = hlls.into_iter().map(|x| x.unwrap_or_default()).collect(); - Some(SegmentStatistics::new(blocks)) + let data = SegmentStatistics::new(blocks).to_bytes()?; + Ok(Some(data)) } } diff --git a/src/query/storages/fuse/src/operations/common/processors/transform_serialize_segment.rs b/src/query/storages/fuse/src/operations/common/processors/transform_serialize_segment.rs index 471915c715dfc..f0f7fcfb1992f 100644 --- a/src/query/storages/fuse/src/operations/common/processors/transform_serialize_segment.rs +++ b/src/query/storages/fuse/src/operations/common/processors/transform_serialize_segment.rs @@ -29,6 +29,7 @@ use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_storages_common_table_meta::meta::column_oriented_segment::*; +use databend_storages_common_table_meta::meta::AdditionalStatsMeta; use databend_storages_common_table_meta::meta::ExtendedBlockMeta; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::SegmentStatistics; @@ -38,7 +39,6 @@ use databend_storages_common_table_meta::meta::VirtualBlockMeta; use log::info; use opendal::Operator; -use crate::io::MetaWriter; use crate::io::TableMetaLocationGenerator; use crate::operations::common::MutationLogEntry; use crate::operations::common::MutationLogs; @@ -56,7 +56,7 @@ enum State { location: String, segment: B::Segment, - hlls: Option<(String, SegmentStatistics)>, + stats: Option<(String, Vec)>, }, PreCommitSegment { location: String, @@ -271,28 +271,29 @@ impl Processor for TransformSerializeSegment { .meta_locations .gen_segment_info_location(self.table_meta_timestamps, self.is_column_oriented); - let hlls = if !self.hll_accumulator.is_empty() { + let mut additional_stats_meta = None; + let mut stats = None; + if !self.hll_accumulator.is_empty() { let segment_stats_location = TableMetaLocationGenerator::gen_segment_stats_location_from_segment_location(location.as_str()); - let stats = self.hll_accumulator.build(); - Some((segment_stats_location, stats)) - } else { - None - }; + let stats_data = self.hll_accumulator.build().to_bytes()?; + additional_stats_meta = Some(AdditionalStatsMeta { + size: stats_data.len() as u64, + location: (segment_stats_location.clone(), SegmentStatistics::VERSION), + }); + stats = Some((segment_stats_location, stats_data)); + } - let hll_location = hlls - .as_ref() - .map(|(hll_location, _)| (hll_location.clone(), SegmentStatistics::VERSION)); let segment_info = self.segment_builder.build( self.thresholds, self.default_cluster_key_id, - hll_location, + additional_stats_meta, )?; self.state = State::SerializedSegment { data: segment_info.serialize()?, location, segment: segment_info, - hlls, + stats, } } State::PreCommitSegment { location, segment } => { @@ -325,11 +326,11 @@ impl Processor for TransformSerializeSegment { data, location, segment, - hlls, + stats, } => { self.data_accessor.write(&location, data).await?; - if let Some((location, stats)) = hlls { - stats.write_meta(&self.data_accessor, &location).await?; + if let Some((location, stats)) = stats { + self.data_accessor.write(&location, stats).await?; } info!("fuse append wrote down segment {} ", location); diff --git a/src/query/storages/fuse/src/operations/gc.rs b/src/query/storages/fuse/src/operations/gc.rs index 501485253ecd6..a42998b4a9fb5 100644 --- a/src/query/storages/fuse/src/operations/gc.rs +++ b/src/query/storages/fuse/src/operations/gc.rs @@ -847,8 +847,8 @@ impl TryFrom> for LocationTuple { bloom_location.insert(bloom_loc.0.clone()); } } - if let Some(hlls) = &value.as_ref().summary.hlls { - hll_location.insert(hlls.0.clone()); + if let Some(meta) = &value.as_ref().summary.additional_stats_meta { + hll_location.insert(meta.location.0.clone()); } Ok(Self { block_location, @@ -882,8 +882,8 @@ impl TryFrom> for LocationTuple { } } - if let Some(hlls) = &value.as_ref().summary.hlls { - hll_location.insert(hlls.0.clone()); + if let Some(meta) = &value.as_ref().summary.additional_stats_meta { + hll_location.insert(meta.location.0.clone()); } Ok(Self { block_location, diff --git a/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs b/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs index ec8ee051f0708..a56c22666c32c 100644 --- a/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs +++ b/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs @@ -33,7 +33,7 @@ use databend_common_expression::ColumnId; use databend_common_metrics::storage::*; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::CompactSegmentInfo; -use databend_storages_common_table_meta::meta::RawColumnHLL; +use databend_storages_common_table_meta::meta::RawBlockHLL; use databend_storages_common_table_meta::meta::Statistics; use log::info; use opendal::Operator; @@ -469,7 +469,7 @@ impl CompactTaskBuilder { std::mem::take(&mut self.blocks) } - fn add(&mut self, block_meta: &Arc, hlls: &Option) -> (bool, bool) { + fn add(&mut self, block_meta: &Arc, hlls: &Option) -> (bool, bool) { let total_rows = self.total_rows + block_meta.row_count as usize; let total_size = self.total_size + block_meta.block_size as usize; let total_compressed = self.total_compressed + block_meta.file_size as usize; @@ -570,8 +570,8 @@ impl CompactTaskBuilder { let permit = acquire_task_permit(semaphore.clone()).await?; let op = self.dal.clone(); let handler = runtime.spawn(async move { - let stats = if let Some(loc) = &segment.summary.hlls { - let stats = read_segment_stats(op.clone(), loc.clone()).await?; + let stats = if let Some(meta) = &segment.summary.additional_stats_meta { + let stats = read_segment_stats(op.clone(), meta.location.clone()).await?; Some(stats) } else { None @@ -598,7 +598,7 @@ impl CompactTaskBuilder { merge_statistics_mut(&mut removed_segment_summary, &summary, self.cluster_key_id); blocks.into_iter().enumerate().map(move |(idx, v)| { - let column_hlls = hlls.as_ref().and_then(|v| v.blocks.get(idx)).cloned(); + let column_hlls = hlls.as_ref().and_then(|v| v.block_hlls.get(idx)).cloned(); (v, column_hlls) }) }) diff --git a/src/query/storages/fuse/src/operations/mutation/mutator/recluster_mutator.rs b/src/query/storages/fuse/src/operations/mutation/mutator/recluster_mutator.rs index 157da2468ddbd..31683958c8b44 100644 --- a/src/query/storages/fuse/src/operations/mutation/mutator/recluster_mutator.rs +++ b/src/query/storages/fuse/src/operations/mutation/mutator/recluster_mutator.rs @@ -39,7 +39,7 @@ use databend_storages_common_pruner::BlockMetaIndex; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::CompactSegmentInfo; use databend_storages_common_table_meta::meta::Location; -use databend_storages_common_table_meta::meta::RawColumnHLL; +use databend_storages_common_table_meta::meta::RawBlockHLL; use databend_storages_common_table_meta::meta::Statistics; use databend_storages_common_table_meta::meta::TableSnapshot; use fastrace::func_path; @@ -183,7 +183,13 @@ impl ReclusterMutator { .map(|(loc, info)| { selected_statistics.push(info.summary.clone()); selected_segs_idx.push(loc.segment_idx); - selected_seg_stats.push((loc.segment_idx, info.summary.hlls.clone())); + selected_seg_stats.push(( + loc.segment_idx, + info.summary + .additional_stats_meta + .as_ref() + .map(|v| v.location.clone()), + )); (loc.segment_idx, info) }) .collect::>(); @@ -630,7 +636,7 @@ impl ReclusterMutator { async fn gather_hlls( &self, hlls: Vec<(usize, Option)>, - ) -> Result>> { + ) -> Result>> { // combine all the tasks. let mut iter = hlls.into_iter(); let tasks = std::iter::from_fn(|| { @@ -647,7 +653,7 @@ impl ReclusterMutator { }; let stats = reader.read(&load_params).await?; let res = stats - .blocks + .block_hlls .iter() .enumerate() .map(|(block_idx, hll)| { diff --git a/src/query/storages/fuse/src/operations/mutation/mutator/segment_compact_mutator.rs b/src/query/storages/fuse/src/operations/mutation/mutator/segment_compact_mutator.rs index edf39d3046d6b..af4402c4f11b3 100644 --- a/src/query/storages/fuse/src/operations/mutation/mutator/segment_compact_mutator.rs +++ b/src/query/storages/fuse/src/operations/mutation/mutator/segment_compact_mutator.rs @@ -18,6 +18,7 @@ use std::time::Instant; use databend_common_exception::Result; use databend_common_metrics::storage::metrics_set_compact_segments_select_duration_second; use databend_storages_common_cache::SegmentStatistics; +use databend_storages_common_table_meta::meta::AdditionalStatsMeta; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::Statistics; @@ -28,7 +29,6 @@ use opendal::Operator; use crate::io::read::read_segment_stats_in_parallel; use crate::io::CachedMetaWriter; -use crate::io::MetaWriter; use crate::io::SegmentsIO; use crate::io::TableMetaLocationGenerator; use crate::operations::CompactOptions; @@ -352,7 +352,7 @@ impl<'a> SegmentCompactor<'a> { // 2.1 merge fragmented segments into new segment, and update the statistics let mut blocks = Vec::with_capacity(self.threshold as usize); let mut new_statistics = Statistics::default(); - let mut hlls = Vec::with_capacity(fragments.len()); + let mut stats_locations = Vec::with_capacity(fragments.len()); let mut hlls_has_none = false; self.compacted_state.num_fragments_compacted += fragments.len(); @@ -363,8 +363,8 @@ impl<'a> SegmentCompactor<'a> { self.default_cluster_key_id, ); blocks.append(&mut segment.blocks.clone()); - if let Some(val) = segment.summary.hlls { - hlls.push(val); + if let Some(meta) = segment.summary.additional_stats_meta { + stats_locations.push(meta.location); } else { hlls_has_none = true; } @@ -377,20 +377,28 @@ impl<'a> SegmentCompactor<'a> { let mut block_hlls = Vec::with_capacity(blocks.len()); if !hlls_has_none { let max_threads = (self.chunk_size / 4).max(10); - let segment_stats = - read_segment_stats_in_parallel(self.operator.clone(), &hlls, max_threads).await?; + let segment_stats = read_segment_stats_in_parallel( + self.operator.clone(), + &stats_locations, + max_threads, + ) + .await?; for stats in segment_stats { - block_hlls.append(&mut stats.blocks.clone()); + block_hlls.append(&mut stats.block_hlls.clone()); } - let stats = SegmentStatistics::new(block_hlls); + let stats_data = SegmentStatistics::new(block_hlls).to_bytes()?; let segment_stats_location = TableMetaLocationGenerator::gen_segment_stats_location_from_segment_location( location.as_str(), ); - stats - .write_meta(self.operator, &segment_stats_location) + let additional_stats_meta = AdditionalStatsMeta { + size: stats_data.len() as u64, + location: (segment_stats_location.clone(), SegmentStatistics::VERSION), + }; + self.operator + .write(&segment_stats_location, stats_data) .await?; - new_statistics.hlls = Some((segment_stats_location, SegmentStatistics::VERSION)); + new_statistics.additional_stats_meta = Some(additional_stats_meta); } // 2.3 write down new segment diff --git a/src/query/storages/fuse/src/statistics/accumulator.rs b/src/query/storages/fuse/src/statistics/accumulator.rs index d4e1e0fc96d31..2805cf7c8698c 100644 --- a/src/query/storages/fuse/src/statistics/accumulator.rs +++ b/src/query/storages/fuse/src/statistics/accumulator.rs @@ -28,10 +28,10 @@ use databend_common_expression::VirtualDataSchema; use databend_common_license::license::Feature; use databend_common_license::license_manager::LicenseManagerSwitch; use databend_storages_common_table_meta::meta::column_oriented_segment::*; +use databend_storages_common_table_meta::meta::AdditionalStatsMeta; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::DraftVirtualColumnMeta; -use databend_storages_common_table_meta::meta::Location; -use databend_storages_common_table_meta::meta::RawColumnHLL; +use databend_storages_common_table_meta::meta::RawBlockHLL; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::SegmentStatistics; use databend_storages_common_table_meta::meta::VirtualColumnMeta; @@ -56,12 +56,12 @@ impl SegmentBuilder for RowOrientedSegmentBuilder { &mut self, thresholds: BlockThresholds, default_cluster_key_id: Option, - hlls: Option, + additional_stats_meta: Option, ) -> Result { let builder = std::mem::take(self); let mut stat = super::reduce_block_metas(&builder.blocks_metas, thresholds, default_cluster_key_id); - stat.hlls = hlls; + stat.additional_stats_meta = additional_stats_meta; Ok(SegmentInfo::new(builder.blocks_metas, stat)) } @@ -187,11 +187,11 @@ impl VirtualColumnAccumulator { #[derive(Default)] pub struct ColumnHLLAccumulator { - pub hlls: Vec, + pub hlls: Vec, } impl ColumnHLLAccumulator { - pub fn add_hll(&mut self, hll: RawColumnHLL) { + pub fn add_hll(&mut self, hll: RawBlockHLL) { self.hlls.push(hll); } diff --git a/src/query/storages/fuse/src/statistics/reducers.rs b/src/query/storages/fuse/src/statistics/reducers.rs index d73e0f5de7f1d..1148146cee60c 100644 --- a/src/query/storages/fuse/src/statistics/reducers.rs +++ b/src/query/storages/fuse/src/statistics/reducers.rs @@ -155,7 +155,7 @@ pub fn merge_statistics_mut( r: &Statistics, default_cluster_key_id: Option, ) { - l.hlls = None; + l.additional_stats_meta = None; if l.row_count == 0 { l.col_stats = r.col_stats.clone(); l.cluster_stats = r.cluster_stats.clone(); @@ -332,6 +332,6 @@ pub fn reduce_block_metas>( col_stats: merged_col_stats, cluster_stats: merged_cluster_stats, virtual_block_count: merged_virtual_block_count, - hlls: None, + additional_stats_meta: None, } } diff --git a/src/query/storages/fuse/src/table_functions/fuse_segment.rs b/src/query/storages/fuse/src/table_functions/fuse_segment.rs index 7b0d142de8346..338f37ec0922e 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_segment.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_segment.rs @@ -82,6 +82,10 @@ impl TableMetaFunc for FuseSegment { "virtual_block_count", TableDataType::Nullable(Box::new(TableDataType::Number(NumberDataType::UInt64))), ), + TableField::new( + "segment_stats_size", + TableDataType::Nullable(Box::new(TableDataType::Number(NumberDataType::UInt64))), + ), ]) } @@ -108,6 +112,7 @@ impl TableMetaFunc for FuseSegment { let mut virtual_column_size: Vec> = Vec::with_capacity(len); let mut virtual_block_count: Vec> = Vec::with_capacity(len); let mut file_location: Vec = Vec::with_capacity(len); + let mut segment_stats_size: Vec> = Vec::with_capacity(len); let segments_io = SegmentsIO::create(ctx.clone(), tbl.operator.clone(), tbl.schema()); @@ -135,6 +140,13 @@ impl TableMetaFunc for FuseSegment { virtual_column_size.push(segment.summary.virtual_column_size); virtual_block_count.push(segment.summary.virtual_block_count); file_location.push(segment_locations[idx].0.clone()); + segment_stats_size.push( + segment + .summary + .additional_stats_meta + .as_ref() + .map(|v| v.size), + ); row_num += 1; if row_num >= limit { @@ -162,6 +174,7 @@ impl TableMetaFunc for FuseSegment { UInt64Type::from_opt_data(vector_index_size), UInt64Type::from_opt_data(virtual_column_size), UInt64Type::from_opt_data(virtual_block_count), + UInt64Type::from_opt_data(segment_stats_size), ])) } } diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history.test b/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history.test index fe6536dbae773..e184963bab0cf 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history.test +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history.test @@ -130,6 +130,11 @@ insert into t2 values(1, 'a'), (2, 'b'); ---- 2 +query B +select segment_stats_size is null from fuse_segment('db_09_0006','t2'); +---- +0 + statement ok alter table t2 set options(approx_distinct_columns = ''); @@ -138,6 +143,20 @@ insert into t2 values(3, 'c'), (4, 'd'); ---- 2 +query B +select segment_stats_size is null from fuse_segment('db_09_0006','t2'); +---- +1 +0 + +statement ok +optimize table t2 compact segment; + +query B +select segment_stats_size is null from fuse_segment('db_09_0006','t2'); +---- +1 + statement ok DROP TABLE t From 0f579dcf5edb92ee53de763add02d8f51a4d2a66 Mon Sep 17 00:00:00 2001 From: zhyass <34016424+zhyass@users.noreply.github.com> Date: Fri, 1 Aug 2025 18:17:06 +0800 Subject: [PATCH 10/11] remove unused code --- src/common/metrics/src/metrics/storage.rs | 21 --------------------- 1 file changed, 21 deletions(-) diff --git a/src/common/metrics/src/metrics/storage.rs b/src/common/metrics/src/metrics/storage.rs index d7ae34d57f9ec..8059be0b39645 100644 --- a/src/common/metrics/src/metrics/storage.rs +++ b/src/common/metrics/src/metrics/storage.rs @@ -337,14 +337,6 @@ static BLOCK_VIRTUAL_COLUMN_WRITE_MILLISECONDS: LazyLock = LazyLock:: register_histogram_in_milliseconds("fuse_block_virtual_column_write_milliseconds") }); -// Block statistics metrics. -static BLOCK_STATS_WRITE_NUMS: LazyLock = - LazyLock::new(|| register_counter("fuse_block_stats_write_nums")); -static BLOCK_STATS_WRITE_BYTES: LazyLock = - LazyLock::new(|| register_counter("fuse_block_stats_write_bytes")); -static BLOCK_STATS_WRITE_MILLISECONDS: LazyLock = - LazyLock::new(|| register_histogram_in_milliseconds("fuse_block_stats_write_milliseconds")); - /// Common metrics. pub fn metrics_inc_omit_filter_rowgroups(c: u64) { OMIT_FILTER_ROWGROUPS.inc_by(c); @@ -915,16 +907,3 @@ pub fn metrics_inc_block_virtual_column_write_bytes(c: u64) { pub fn metrics_inc_block_virtual_column_write_milliseconds(c: u64) { BLOCK_VIRTUAL_COLUMN_WRITE_MILLISECONDS.observe(c as f64); } - -/// Block stats metrics. -pub fn metrics_inc_block_stats_write_nums(c: u64) { - BLOCK_STATS_WRITE_NUMS.inc_by(c); -} - -pub fn metrics_inc_block_stats_write_bytes(c: u64) { - BLOCK_STATS_WRITE_BYTES.inc_by(c); -} - -pub fn metrics_inc_block_stats_write_milliseconds(c: u64) { - BLOCK_STATS_WRITE_MILLISECONDS.observe(c as f64); -} From 1cb3b876a1ce8c876213cd177f2edd63639c62e6 Mon Sep 17 00:00:00 2001 From: zhyass <34016424+zhyass@users.noreply.github.com> Date: Sat, 2 Aug 2025 23:53:09 +0800 Subject: [PATCH 11/11] fix review comment --- src/common/storage/src/meta_hll.rs | 1 - src/query/config/src/config.rs | 6 +++--- .../it/storages/testdata/configs_table_basic.txt | 2 +- src/query/storages/fuse/src/statistics/mod.rs | 1 - .../fuse/src/statistics/segment_statistics.rs | 13 ------------- 5 files changed, 4 insertions(+), 19 deletions(-) delete mode 100644 src/query/storages/fuse/src/statistics/segment_statistics.rs diff --git a/src/common/storage/src/meta_hll.rs b/src/common/storage/src/meta_hll.rs index de427e96b1b39..dff041f9b155e 100644 --- a/src/common/storage/src/meta_hll.rs +++ b/src/common/storage/src/meta_hll.rs @@ -115,7 +115,6 @@ impl MetaHLL { /// Helper function sigma as defined in /// "New cardinality estimation algorithms for MetaHLL sketches" /// Otmar Ertl, https://arxiv.org/abs/1702.01284 -#[allow(dead_code)] #[inline] fn hll_sigma(x: f64) -> f64 { if x == 1. { diff --git a/src/query/config/src/config.rs b/src/query/config/src/config.rs index 067ba63736653..9025b8195d64f 100644 --- a/src/query/config/src/config.rs +++ b/src/query/config/src/config.rs @@ -3225,7 +3225,7 @@ pub struct CacheConfig { value_name = "VALUE", default_value = "0" )] - pub segment_stastistics_count: u64, + pub segment_statistics_count: u64, /// Enable bloom index cache. Default is enabled. Set it to false to disable all the bloom index caches #[clap( @@ -3643,7 +3643,7 @@ mod cache_config_converters { block_meta_count: value.block_meta_count, segment_block_metas_count: value.segment_block_metas_count, table_meta_statistic_count: value.table_meta_statistic_count, - segment_statistics_count: value.segment_stastistics_count, + segment_statistics_count: value.segment_statistics_count, enable_table_index_bloom: value.enable_table_bloom_index_cache, table_bloom_index_meta_count: value.table_bloom_index_meta_count, table_bloom_index_filter_count: value.table_bloom_index_filter_count, @@ -3680,7 +3680,7 @@ mod cache_config_converters { table_meta_snapshot_count: value.table_meta_snapshot_count, table_meta_segment_bytes: value.table_meta_segment_bytes, table_meta_statistic_count: value.table_meta_statistic_count, - segment_stastistics_count: value.segment_statistics_count, + segment_statistics_count: value.segment_statistics_count, block_meta_count: value.block_meta_count, enable_table_bloom_index_cache: value.enable_table_index_bloom, table_bloom_index_meta_count: value.table_bloom_index_meta_count, diff --git a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt index 536eb79d132c6..f04301a022561 100644 --- a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt +++ b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt @@ -21,7 +21,7 @@ DB.Table: 'system'.'configs', Table: configs-table_id:1, ver:0, Engine: SystemCo | 'cache' | 'inverted_index_meta_count' | '3000' | '' | | 'cache' | 'meta_service_ownership_cache' | 'false' | '' | | 'cache' | 'segment_block_metas_count' | '0' | '' | -| 'cache' | 'segment_stastistics_count' | '0' | '' | +| 'cache' | 'segment_statistics_count' | '0' | '' | | 'cache' | 'table_bloom_index_filter_count' | '0' | '' | | 'cache' | 'table_bloom_index_filter_size' | '2147483648' | '' | | 'cache' | 'table_bloom_index_meta_count' | '3000' | '' | diff --git a/src/query/storages/fuse/src/statistics/mod.rs b/src/query/storages/fuse/src/statistics/mod.rs index 72df7775c56c3..8ea71673f4b2b 100644 --- a/src/query/storages/fuse/src/statistics/mod.rs +++ b/src/query/storages/fuse/src/statistics/mod.rs @@ -16,7 +16,6 @@ pub mod accumulator; mod cluster_statistics; mod column_statistic; pub mod reducers; -mod segment_statistics; mod traverse; pub use accumulator::ColumnHLLAccumulator; diff --git a/src/query/storages/fuse/src/statistics/segment_statistics.rs b/src/query/storages/fuse/src/statistics/segment_statistics.rs deleted file mode 100644 index 4ca0419659671..0000000000000 --- a/src/query/storages/fuse/src/statistics/segment_statistics.rs +++ /dev/null @@ -1,13 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// 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.