diff --git a/src/common/base/src/base/watch_notify.rs b/src/common/base/src/base/watch_notify.rs index be05dfc9028c0..c7a677474b63f 100644 --- a/src/common/base/src/base/watch_notify.rs +++ b/src/common/base/src/base/watch_notify.rs @@ -44,6 +44,10 @@ impl WatchNotify { pub fn notify_waiters(&self) { let _ = self.tx.send_replace(true); } + + pub fn is_notified(&self) -> bool { + *self.rx.borrow() + } } #[cfg(test)] diff --git a/src/query/ee/src/hilbert_clustering/handler.rs b/src/query/ee/src/hilbert_clustering/handler.rs index cebfbadc5947e..dc116c57ab13b 100644 --- a/src/query/ee/src/hilbert_clustering/handler.rs +++ b/src/query/ee/src/hilbert_clustering/handler.rs @@ -76,6 +76,7 @@ impl HilbertClusteringHandler for RealHilbertClusteringHandler { let mut checker = ReclusterChecker::new( cluster_key_id, hilbert_min_bytes, + block_per_seg, push_downs.as_ref().is_none_or(|v| v.filters.is_none()), ); 'FOR: for chunk in segment_locations.chunks(chunk_size) { @@ -139,19 +140,29 @@ struct ReclusterChecker { hilbert_min_bytes: usize, total_bytes: usize, + hilbert_min_blocks: usize, + total_blocks: usize, + finished: bool, // Whether the target segments is at the head of snapshot. head_of_snapshot: bool, } impl ReclusterChecker { - fn new(default_cluster_id: u32, hilbert_min_bytes: usize, head_of_snapshot: bool) -> Self { + fn new( + default_cluster_id: u32, + hilbert_min_bytes: usize, + hilbert_min_blocks: usize, + head_of_snapshot: bool, + ) -> Self { Self { segments: vec![], last_segment: None, default_cluster_id, + hilbert_min_blocks, hilbert_min_bytes, total_bytes: 0, + total_blocks: 0, finished: false, head_of_snapshot, } @@ -164,10 +175,14 @@ impl ReclusterChecker { if segment_should_recluster || !self.head_of_snapshot { self.total_bytes += segment.summary.uncompressed_byte_size as usize; + self.total_blocks += segment.summary.block_count as usize; self.segments.push((location.clone(), segment.clone())); } - if !segment_should_recluster || self.total_bytes >= self.hilbert_min_bytes { + if !segment_should_recluster + || (self.total_bytes >= self.hilbert_min_bytes + && self.total_blocks >= self.hilbert_min_blocks) + { if self.check_for_recluster() { self.finished = true; return true; @@ -208,6 +223,7 @@ impl ReclusterChecker { fn reset(&mut self) { self.total_bytes = 0; + self.total_blocks = 0; self.head_of_snapshot = false; self.segments.clear(); } diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index 03c12fc8f6962..0ee73b994beae 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -504,8 +504,8 @@ impl DataBlock { } #[inline] - pub fn remove_column(&mut self, index: usize) { - self.entries.remove(index); + pub fn remove_column(&mut self, index: usize) -> BlockEntry { + self.entries.remove(index) } #[inline] diff --git a/src/query/expression/src/sampler/fixed_size_sampler.rs b/src/query/expression/src/sampler/fixed_size_sampler.rs index dd7500d40759b..b1317c38ba693 100644 --- a/src/query/expression/src/sampler/fixed_size_sampler.rs +++ b/src/query/expression/src/sampler/fixed_size_sampler.rs @@ -162,7 +162,7 @@ fn compact_indices(indices: &mut Vec, blocks: &mut Vec .collect(); } -mod reservoir_sampling { +pub mod reservoir_sampling { use std::num::NonZeroUsize; use rand::Rng; diff --git a/src/query/expression/src/sampler/mod.rs b/src/query/expression/src/sampler/mod.rs index c34b36905bd0f..558770c854f7b 100644 --- a/src/query/expression/src/sampler/mod.rs +++ b/src/query/expression/src/sampler/mod.rs @@ -16,4 +16,5 @@ mod fixed_rate_sampler; mod fixed_size_sampler; pub use fixed_rate_sampler::FixedRateSampler; +pub use fixed_size_sampler::reservoir_sampling::AlgoL; pub use fixed_size_sampler::FixedSizeSampler; diff --git a/src/query/expression/src/utils/block_thresholds.rs b/src/query/expression/src/utils/block_thresholds.rs index f19a26f6dedee..66cb1fbabc9e1 100644 --- a/src/query/expression/src/utils/block_thresholds.rs +++ b/src/query/expression/src/utils/block_thresholds.rs @@ -39,7 +39,7 @@ impl Default for BlockThresholds { max_bytes_per_block: DEFAULT_BLOCK_BUFFER_SIZE * 2, min_bytes_per_block: (DEFAULT_BLOCK_BUFFER_SIZE * 4).div_ceil(5), max_compressed_per_block: DEFAULT_BLOCK_COMPRESSED_SIZE, - min_compressed_per_block: (DEFAULT_BLOCK_COMPRESSED_SIZE * 4).div_ceil(5), + min_compressed_per_block: (DEFAULT_BLOCK_COMPRESSED_SIZE * 3).div_ceil(5), block_per_segment: DEFAULT_BLOCK_PER_SEGMENT, } } @@ -58,7 +58,7 @@ impl BlockThresholds { max_bytes_per_block: bytes_per_block * 2, min_bytes_per_block: (bytes_per_block * 4).div_ceil(5), max_compressed_per_block, - min_compressed_per_block: (max_compressed_per_block * 4).div_ceil(5), + min_compressed_per_block: (max_compressed_per_block * 3).div_ceil(5), block_per_segment, } } @@ -152,8 +152,8 @@ impl BlockThresholds { let bytes_per_block = total_bytes.div_ceil(block_num_by_compressed); // Adjust the number of blocks based on block size thresholds. - let max_bytes_per_block = (4 * self.min_bytes_per_block).min(400 * 1024 * 1024); - let min_bytes_per_block = (self.min_bytes_per_block / 2).min(50 * 1024 * 1024); + let max_bytes_per_block = self.max_bytes_per_block.min(400 * 1024 * 1024); + let min_bytes_per_block = self.min_bytes_per_block.min(100 * 1024 * 1024); let block_nums = if bytes_per_block > max_bytes_per_block { // Case 1: If the block size is too bigger. total_bytes.div_ceil(max_bytes_per_block) @@ -166,4 +166,53 @@ impl BlockThresholds { }; total_rows.div_ceil(block_nums.max(1)).max(1) } + + /// Calculates the optimal number of partitions (blocks) based on total data size and row count. + /// + /// # Parameters + /// - `total_rows`: The total number of rows in the data. + /// - `total_bytes`: The total uncompressed size of the data in bytes. + /// - `total_compressed`: The total compressed size of the data in bytes. + /// + /// # Returns + /// - The calculated number of partitions (blocks) needed. + #[inline] + pub fn calc_partitions_for_recluster( + &self, + total_rows: usize, + total_bytes: usize, + total_compressed: usize, + ) -> usize { + // If the data is already compact enough, return a single partition. + if self.check_for_compact(total_rows, total_bytes) + && total_compressed < 2 * self.min_compressed_per_block + { + return 1; + } + + // Estimate the number of blocks based on row count and compressed size. + let by_rows = std::cmp::max(total_rows / self.max_rows_per_block, 1); + let by_compressed = total_compressed / self.max_compressed_per_block; + // If row-based block count is greater, use max rows per block as limit. + if by_rows >= by_compressed { + return by_rows; + } + + // Adjust block count based on byte size thresholds. + let bytes_per_block = total_bytes.div_ceil(by_compressed); + let max_bytes = self.max_bytes_per_block.min(400 * 1024 * 1024); + let min_bytes = self.min_bytes_per_block.min(100 * 1024 * 1024); + let total_partitions = if bytes_per_block > max_bytes { + // Block size is too large. + total_bytes / max_bytes + } else if bytes_per_block < min_bytes { + // Block size is too small. + total_bytes / min_bytes + } else { + // Block size is acceptable. + by_compressed + }; + + std::cmp::max(total_partitions, 1) + } } diff --git a/src/query/expression/tests/it/block_thresholds.rs b/src/query/expression/tests/it/block_thresholds.rs index 08793eb2a78e4..b7409208386aa 100644 --- a/src/query/expression/tests/it/block_thresholds.rs +++ b/src/query/expression/tests/it/block_thresholds.rs @@ -15,7 +15,7 @@ use databend_common_expression::BlockThresholds; fn default_thresholds() -> BlockThresholds { - BlockThresholds::new(1000, 1_000_000, 100_000, 4) + BlockThresholds::new(1_000, 1_000_000, 100_000, 4) } #[test] @@ -101,14 +101,40 @@ fn test_calc_rows_for_recluster() { ); // Case 1: If the block size is too bigger. - let result = t.calc_rows_for_recluster(4_000, 30_000_000, 600_000); - assert_eq!(result, 400); + let result = t.calc_rows_for_recluster(4_500, 30_000_000, 600_000); + assert_eq!(result, 300); // Case 2: If the block size is too smaller. - let result = t.calc_rows_for_recluster(4_000, 2_000_000, 600_000); - assert_eq!(result, 800); + let result = t.calc_rows_for_recluster(4_000, 1_600_000, 600_000); + assert_eq!(result, 2000); // Case 3: use the compressed-based block count. let result = t.calc_rows_for_recluster(4_000, 10_000_000, 600_000); assert_eq!(result, 667); } + +#[test] +fn test_calc_partitions_for_recluster() { + let t = default_thresholds(); + + // compact enough to skip further calculations + assert_eq!(t.calc_partitions_for_recluster(1000, 500_000, 100_000), 1); + + // row-based block count exceeds compressed-based block count, use max rows per block. + assert_eq!( + t.calc_partitions_for_recluster(10_000, 2_000_000, 100_000), + 10 + ); + + // Case 1: If the block size is too bigger. + let result = t.calc_partitions_for_recluster(4_500, 30_000_000, 600_000); + assert_eq!(result, 15); + + // Case 2: If the block size is too smaller. + let result = t.calc_partitions_for_recluster(4_000, 1_600_000, 800_000); + assert_eq!(result, 2); + + // Case 3: use the compressed-based block count. + let result = t.calc_partitions_for_recluster(4_000, 10_000_000, 600_000); + assert_eq!(result, 6); +} diff --git a/src/query/functions/src/aggregates/aggregate_range_bound.rs b/src/query/functions/src/aggregates/aggregate_range_bound.rs index 9776caac786c8..2572429300182 100644 --- a/src/query/functions/src/aggregates/aggregate_range_bound.rs +++ b/src/query/functions/src/aggregates/aggregate_range_bound.rs @@ -326,9 +326,7 @@ pub fn try_create_aggregate_range_bound_function( /// For a column with values `(0, 1, 3, 6, 8)` and `partition_num = 3`, the function calculates the /// partition boundaries based on the distribution of the data. The boundaries might be `[1, 6]`. pub fn aggregate_range_bound_function_desc() -> AggregateFunctionDescription { - AggregateFunctionDescription::creator(Box::new( - crate::aggregates::try_create_aggregate_range_bound_function, - )) + AggregateFunctionDescription::creator(Box::new(try_create_aggregate_range_bound_function)) } fn get_partitions( diff --git a/src/query/functions/src/scalars/hilbert.rs b/src/query/functions/src/scalars/hilbert.rs index 060fe5ab97abe..b57c6aa77a17a 100644 --- a/src/query/functions/src/scalars/hilbert.rs +++ b/src/query/functions/src/scalars/hilbert.rs @@ -21,24 +21,33 @@ use databend_common_expression::types::BinaryType; use databend_common_expression::types::DataType; use databend_common_expression::types::GenericType; use databend_common_expression::types::NullableType; +use databend_common_expression::types::NumberDataType; use databend_common_expression::types::NumberType; use databend_common_expression::types::ReturnType; +use databend_common_expression::types::StringType; use databend_common_expression::types::ValueType; +use databend_common_expression::types::ALL_NUMERICS_TYPES; +use databend_common_expression::vectorize_with_builder_1_arg; use databend_common_expression::vectorize_with_builder_2_arg; +use databend_common_expression::with_number_mapped_type; use databend_common_expression::Column; use databend_common_expression::FixedLengthEncoding; use databend_common_expression::Function; use databend_common_expression::FunctionDomain; use databend_common_expression::FunctionEval; use databend_common_expression::FunctionFactory; +use databend_common_expression::FunctionProperty; use databend_common_expression::FunctionRegistry; use databend_common_expression::FunctionSignature; use databend_common_expression::ScalarRef; use databend_common_expression::Value; +use rand::rngs::SmallRng; +use rand::Rng; +use rand::SeedableRng; /// Registers Hilbert curve related functions with the function registry. pub fn register(registry: &mut FunctionRegistry) { - // Register the hilbert_range_index function that calculates Hilbert indices for multi-dimensional data + // Register the hilbert_range_index function that calculates Hilbert indices for multidimensional data let factory = FunctionFactory::Closure(Box::new(|_, args_type: &[DataType]| { let args_num = args_type.len(); // The function supports 2, 3, 4, or 5 dimensions (each dimension requires 2 arguments) @@ -97,7 +106,7 @@ pub fn register(registry: &mut FunctionRegistry) { points.push(key); } - // Convert the multi-dimensional point to a Hilbert index + // Convert the multidimensional point to a Hilbert index // This maps the n-dimensional point to a 1-dimensional value let points = points .iter() @@ -153,6 +162,88 @@ pub fn register(registry: &mut FunctionRegistry) { builder.push(id); }), ); + + // We use true randomness by appending a random u8 value at the end of the binary key. + // This introduces noise to break tie cases in clustering keys that are not uniformly distributed. + // Although this may slightly affect the accuracy of range_bound estimation, + // it ensures that Hilbert index + scatter will no longer suffer from data skew. + // Moreover, since the noise is added at the tail, the original order of the keys is preserved. + registry.properties.insert( + "add_noise".to_string(), + FunctionProperty::default().non_deterministic(), + ); + + registry.register_passthrough_nullable_1_arg::( + "add_noise", + |_, _| FunctionDomain::Full, + vectorize_with_builder_1_arg::(|val, builder, _| { + let mut bytes = val.as_bytes().to_vec(); + let mut rng = SmallRng::from_entropy(); + bytes.push(rng.gen::()); + builder.put_slice(&bytes); + builder.commit_row(); + }), + ); + + for ty in ALL_NUMERICS_TYPES { + with_number_mapped_type!(|NUM_TYPE| match ty { + NumberDataType::NUM_TYPE => { + registry + .register_passthrough_nullable_1_arg::, BinaryType, _, _>( + "add_noise", + |_, _| FunctionDomain::Full, + vectorize_with_builder_1_arg::, BinaryType>( + |val, builder, _| { + let mut encoded = val.encode().to_vec(); + let mut rng = SmallRng::from_entropy(); + encoded.push(rng.gen::()); + builder.put_slice(&encoded); + builder.commit_row(); + }, + ), + ); + } + }) + } + + registry.register_passthrough_nullable_2_arg::, BinaryType, _, _>( + "add_noise", + |_, _, _| FunctionDomain::Full, + vectorize_with_builder_2_arg::, BinaryType>( + |val, level, builder, _| { + let mut bytes = val.as_bytes().to_vec(); + let mut rng = SmallRng::from_entropy(); + for _ in 0..level { + bytes.push(rng.gen::()); + } + builder.put_slice(&bytes); + builder.commit_row(); + }, + ), + ); + + for ty in ALL_NUMERICS_TYPES { + with_number_mapped_type!(|NUM_TYPE| match ty { + NumberDataType::NUM_TYPE => { + registry + .register_passthrough_nullable_2_arg::, NumberType, BinaryType, _, _>( + "add_noise", + |_, _, _| FunctionDomain::Full, + vectorize_with_builder_2_arg::, NumberType, BinaryType>( + |val, level, builder, _| { + let mut encoded = val.encode().to_vec(); + let mut rng = SmallRng::from_entropy(); + for _ in 0..level { + encoded.push(rng.gen::()); + } + builder.put_slice(&encoded); + builder.commit_row(); + }, + ), + ); + } + }) + } } /// Calculates the partition ID for a value based on range boundaries. @@ -166,10 +257,10 @@ pub fn register(registry: &mut FunctionRegistry) { /// /// # Example /// For boundaries [10, 20, 30]: -/// - Values < 10 get partition ID 0 -/// - Values >= 10 and < 20 get partition ID 1 -/// - Values >= 20 and < 30 get partition ID 2 -/// - Values >= 30 get partition ID 3 +/// - Values <= 10 get partition ID 0 +/// - Values > 10 and <= 20 get partition ID 1 +/// - Values > 20 and <= 30 get partition ID 2 +/// - Values > 30 get partition ID 3 fn calc_range_partition_id(val: ScalarRef, arr: &Column) -> u64 { let mut low = 0; let mut high = arr.len(); diff --git a/src/query/functions/tests/it/scalars/testdata/function_list.txt b/src/query/functions/tests/it/scalars/testdata/function_list.txt index 06d4012daf92b..1119a04aa7373 100644 --- a/src/query/functions/tests/it/scalars/testdata/function_list.txt +++ b/src/query/functions/tests/it/scalars/testdata/function_list.txt @@ -115,6 +115,50 @@ Functions overloads: 1 add_months(Date NULL, Int64 NULL) :: Date NULL 2 add_months(Timestamp, Int64) :: Timestamp 3 add_months(Timestamp NULL, Int64 NULL) :: Timestamp NULL +0 add_noise(String) :: Binary +1 add_noise(String NULL) :: Binary NULL +2 add_noise(UInt8) :: Binary +3 add_noise(UInt8 NULL) :: Binary NULL +4 add_noise(UInt16) :: Binary +5 add_noise(UInt16 NULL) :: Binary NULL +6 add_noise(UInt32) :: Binary +7 add_noise(UInt32 NULL) :: Binary NULL +8 add_noise(UInt64) :: Binary +9 add_noise(UInt64 NULL) :: Binary NULL +10 add_noise(Int8) :: Binary +11 add_noise(Int8 NULL) :: Binary NULL +12 add_noise(Int16) :: Binary +13 add_noise(Int16 NULL) :: Binary NULL +14 add_noise(Int32) :: Binary +15 add_noise(Int32 NULL) :: Binary NULL +16 add_noise(Int64) :: Binary +17 add_noise(Int64 NULL) :: Binary NULL +18 add_noise(Float32) :: Binary +19 add_noise(Float32 NULL) :: Binary NULL +20 add_noise(Float64) :: Binary +21 add_noise(Float64 NULL) :: Binary NULL +22 add_noise(String, UInt64) :: Binary +23 add_noise(String NULL, UInt64 NULL) :: Binary NULL +24 add_noise(UInt8, UInt64) :: Binary +25 add_noise(UInt8 NULL, UInt64 NULL) :: Binary NULL +26 add_noise(UInt16, UInt64) :: Binary +27 add_noise(UInt16 NULL, UInt64 NULL) :: Binary NULL +28 add_noise(UInt32, UInt64) :: Binary +29 add_noise(UInt32 NULL, UInt64 NULL) :: Binary NULL +30 add_noise(UInt64, UInt64) :: Binary +31 add_noise(UInt64 NULL, UInt64 NULL) :: Binary NULL +32 add_noise(Int8, UInt64) :: Binary +33 add_noise(Int8 NULL, UInt64 NULL) :: Binary NULL +34 add_noise(Int16, UInt64) :: Binary +35 add_noise(Int16 NULL, UInt64 NULL) :: Binary NULL +36 add_noise(Int32, UInt64) :: Binary +37 add_noise(Int32 NULL, UInt64 NULL) :: Binary NULL +38 add_noise(Int64, UInt64) :: Binary +39 add_noise(Int64 NULL, UInt64 NULL) :: Binary NULL +40 add_noise(Float32, UInt64) :: Binary +41 add_noise(Float32 NULL, UInt64 NULL) :: Binary NULL +42 add_noise(Float64, UInt64) :: Binary +43 add_noise(Float64 NULL, UInt64 NULL) :: Binary NULL 0 add_quarters(Date, Int64) :: Date 1 add_quarters(Date NULL, Int64 NULL) :: Date NULL 2 add_quarters(Timestamp, Int64) :: Timestamp diff --git a/src/query/service/src/interpreters/interpreter_table_recluster.rs b/src/query/service/src/interpreters/interpreter_table_recluster.rs index f3c53597b06d7..2468c6b2cd159 100644 --- a/src/query/service/src/interpreters/interpreter_table_recluster.rs +++ b/src/query/service/src/interpreters/interpreter_table_recluster.rs @@ -320,24 +320,15 @@ impl ReclusterTableInterpreter { let total_rows = recluster_info.removed_statistics.row_count as usize; let total_compressed = recluster_info.removed_statistics.compressed_byte_size as usize; - // Determine rows per block based on data size and compression ratio - let rows_per_block = - block_thresholds.calc_rows_for_recluster(total_rows, total_bytes, total_compressed); - + // Determine rows per block based on data size and compression ratio, // Calculate initial partition count based on data volume and block size - let mut total_partitions = std::cmp::max(total_rows / rows_per_block, 1); - - // Adjust number of partitions according to the block size thresholds - if total_partitions < block_thresholds.block_per_segment - && block_thresholds.check_perfect_segment( - block_thresholds.block_per_segment, // this effectively by-pass the total_blocks criteria - total_rows, - total_bytes, - total_compressed, - ) - { - total_partitions = block_thresholds.block_per_segment; - } + let total_partitions = block_thresholds.calc_partitions_for_recluster( + total_rows, + total_bytes, + total_compressed, + ); + let bytes_per_block = (total_bytes / total_partitions).max(1); + let rows_per_block = (total_rows / total_partitions).max(1); warn!( "Do hilbert recluster, total_bytes: {}, total_rows: {}, total_partitions: {}", @@ -473,6 +464,7 @@ impl ReclusterTableInterpreter { table_info: table_info.clone(), num_partitions: total_partitions, table_meta_timestamps, + bytes_per_block, rows_per_block, })); @@ -613,7 +605,7 @@ impl ReclusterTableInterpreter { let database = &self.plan.database; let table = &self.plan.table; let settings = self.ctx.get_settings(); - let sample_size = settings.get_hilbert_sample_size_per_block()?; + let sample_size = settings.get_recluster_sample_size_per_block()?; let name_resolution_ctx = NameResolutionContext::try_from(settings.as_ref())?; let ast_exprs = tbl.resolve_cluster_keys(self.ctx.clone()).unwrap(); @@ -634,7 +626,7 @@ impl ReclusterTableInterpreter { "range_bound(1000, {sample_size})({cluster_key_str})" )); - hilbert_keys.push(format!("{table}.{cluster_key_str}, []")); + hilbert_keys.push(format!("{cluster_key_str}, []")); } let hilbert_keys_str = hilbert_keys.join(", "); diff --git a/src/query/service/src/local/display.rs b/src/query/service/src/local/display.rs index 32546082de770..b782accabdd00 100644 --- a/src/query/service/src/local/display.rs +++ b/src/query/service/src/local/display.rs @@ -259,7 +259,7 @@ impl FormatDisplay<'_> { rows_str, self.start.elapsed().as_secs_f64(), humanize_count(stats.total_rows as f64), - HumanBytes(stats.total_rows as u64), + HumanBytes(stats.total_bytes as u64), humanize_count(stats.total_rows as f64 / self.start.elapsed().as_secs_f64()), HumanBytes((stats.total_bytes as f64 / self.start.elapsed().as_secs_f64()) as u64), ); diff --git a/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs b/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs index 86b23bfca9238..870db16444a42 100644 --- a/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs +++ b/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs @@ -15,23 +15,26 @@ use std::sync::atomic; use std::sync::atomic::AtomicUsize; +use databend_common_catalog::table::Table; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; -use databend_common_io::constants::DEFAULT_BLOCK_BUFFER_SIZE; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_transforms::MemorySettings; +use databend_common_pipeline_transforms::TransformPipelineHelper; use databend_common_sql::executor::physical_plans::HilbertPartition; use databend_common_sql::executor::physical_plans::MutationKind; +use databend_common_storages_fuse::io::StreamBlockProperties; +use databend_common_storages_fuse::operations::TransformBlockWriter; use databend_common_storages_fuse::operations::TransformSerializeBlock; use databend_common_storages_fuse::statistics::ClusterStatsGenerator; use databend_common_storages_fuse::FuseTable; -use databend_common_storages_fuse::FUSE_OPT_KEY_BLOCK_IN_MEM_SIZE_THRESHOLD; use databend_storages_common_cache::TempDirManager; use crate::pipelines::memory_settings::MemorySettingsExt; -use crate::pipelines::processors::transforms::CompactStrategy; -use crate::pipelines::processors::transforms::HilbertPartitionExchange; -use crate::pipelines::processors::transforms::TransformWindowPartitionCollect; +use crate::pipelines::processors::transforms::CompactPartitionStrategy; +use crate::pipelines::processors::transforms::ReclusterPartitionExchange; +use crate::pipelines::processors::transforms::ReclusterPartitionStrategy; +use crate::pipelines::processors::transforms::TransformPartitionCollect; use crate::pipelines::PipelineBuilder; use crate::spillers::SpillerDiskConfig; @@ -43,10 +46,12 @@ impl PipelineBuilder { .ctx .build_table_by_table_info(&partition.table_info, None)?; let table = FuseTable::try_from_table(table.as_ref())?; + let enable_stream_writer = self.ctx.get_settings().get_enable_block_stream_write()? + && table.storage_format_as_parquet(); self.main_pipeline.exchange( num_processors, - HilbertPartitionExchange::create(partition.num_partitions), + ReclusterPartitionExchange::create(partition.num_partitions), ); let settings = self.ctx.get_settings(); @@ -61,36 +66,68 @@ impl PipelineBuilder { let window_spill_settings = MemorySettings::from_window_settings(&self.ctx)?; let processor_id = AtomicUsize::new(0); - let max_bytes_per_block = std::cmp::min( - 4 * table.get_option( - FUSE_OPT_KEY_BLOCK_IN_MEM_SIZE_THRESHOLD, - DEFAULT_BLOCK_BUFFER_SIZE, - ), - 400 * 1024 * 1024, - ); - self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(Box::new( - TransformWindowPartitionCollect::new( + + if enable_stream_writer { + let properties = StreamBlockProperties::try_create( + self.ctx.clone(), + table, + MutationKind::Recluster, + None, + partition.table_meta_timestamps, + )?; + + self.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(Box::new( + TransformPartitionCollect::new( + self.ctx.clone(), + input, + output, + &settings, + processor_id.fetch_add(1, atomic::Ordering::AcqRel), + num_processors, + partition.num_partitions, + window_spill_settings.clone(), + disk_spill.clone(), + ReclusterPartitionStrategy::new(properties.clone()), + )?, + ))) + })?; + + self.main_pipeline.add_async_accumulating_transformer(|| { + TransformBlockWriter::create( self.ctx.clone(), - input, - output, - &settings, - processor_id.fetch_add(1, atomic::Ordering::AcqRel), - num_processors, - partition.num_partitions, - window_spill_settings.clone(), - disk_spill.clone(), - CompactStrategy::new(partition.rows_per_block, max_bytes_per_block), - )?, - ))) - })?; + MutationKind::Recluster, + table, + false, + ) + }); + Ok(()) + } else { + self.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(Box::new( + TransformPartitionCollect::new( + self.ctx.clone(), + input, + output, + &settings, + processor_id.fetch_add(1, atomic::Ordering::AcqRel), + num_processors, + partition.num_partitions, + window_spill_settings.clone(), + disk_spill.clone(), + CompactPartitionStrategy::new( + partition.rows_per_block, + partition.bytes_per_block, + ), + )?, + ))) + })?; - self.main_pipeline - .add_transform(|transform_input_port, transform_output_port| { + self.main_pipeline.add_transform(|input, output| { let proc = TransformSerializeBlock::try_create( self.ctx.clone(), - transform_input_port, - transform_output_port, + input, + output, table, ClusterStatsGenerator::default(), MutationKind::Recluster, @@ -98,5 +135,6 @@ impl PipelineBuilder { )?; proc.into_processor() }) + } } } diff --git a/src/query/service/src/pipelines/builders/builder_recluster.rs b/src/query/service/src/pipelines/builders/builder_recluster.rs index 05d2d63dd55d7..8364e45d726fa 100644 --- a/src/query/service/src/pipelines/builders/builder_recluster.rs +++ b/src/query/service/src/pipelines/builders/builder_recluster.rs @@ -12,29 +12,47 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::atomic; +use std::sync::atomic::AtomicUsize; +use std::sync::Arc; + use databend_common_catalog::plan::DataSourceInfo; use databend_common_catalog::plan::DataSourcePlan; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::SortColumnDescription; use databend_common_metrics::storage::metrics_inc_recluster_block_bytes_to_read; use databend_common_metrics::storage::metrics_inc_recluster_block_nums_to_read; use databend_common_metrics::storage::metrics_inc_recluster_row_nums_to_read; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_pipeline_core::Pipeline; use databend_common_pipeline_sources::EmptySource; use databend_common_pipeline_transforms::processors::build_compact_block_no_split_pipeline; use databend_common_pipeline_transforms::processors::TransformPipelineHelper; +use databend_common_pipeline_transforms::sort::utils::add_order_field; +use databend_common_pipeline_transforms::MemorySettings; use databend_common_sql::evaluator::CompoundBlockOperator; use databend_common_sql::executor::physical_plans::MutationKind; use databend_common_sql::executor::physical_plans::Recluster; use databend_common_sql::StreamContext; use databend_common_storages_factory::Table; +use databend_common_storages_fuse::io::StreamBlockProperties; +use databend_common_storages_fuse::operations::TransformBlockWriter; use databend_common_storages_fuse::operations::TransformSerializeBlock; use databend_common_storages_fuse::FuseTable; use databend_common_storages_fuse::TableContext; use crate::pipelines::builders::SortPipelineBuilder; +use crate::pipelines::processors::transforms::ReclusterPartitionExchange; +use crate::pipelines::processors::transforms::ReclusterPartitionStrategy; +use crate::pipelines::processors::transforms::SampleState; +use crate::pipelines::processors::transforms::TransformAddOrderColumn; use crate::pipelines::processors::transforms::TransformAddStreamColumns; +use crate::pipelines::processors::transforms::TransformPartitionCollect; +use crate::pipelines::processors::transforms::TransformRangePartitionIndexer; +use crate::pipelines::processors::transforms::TransformReclusterCollect; use crate::pipelines::PipelineBuilder; impl PipelineBuilder { @@ -71,7 +89,7 @@ impl PipelineBuilder { let recluster_block_nums = task.parts.len(); let block_thresholds = table.get_block_thresholds(); let table_info = table.get_table_info(); - let schema = table.schema_with_stream(); + let schema = Arc::new(table.schema_with_stream().remove_virtual_computed_fields()); let description = task.stats.get_description(&table_info.desc); let plan = DataSourcePlan { source_info: DataSourceInfo::TableSource(table_info.clone()), @@ -117,72 +135,164 @@ impl PipelineBuilder { .add_transformer(|| TransformAddStreamColumns::new(stream_ctx.clone())); } - let cluster_stats_gen = table.get_cluster_stats_gen( - self.ctx.clone(), - task.level + 1, - block_thresholds, - None, - )?; - let operators = cluster_stats_gen.operators.clone(); - if !operators.is_empty() { - let func_ctx2 = cluster_stats_gen.func_ctx.clone(); - self.main_pipeline.add_transformer(move || { - CompoundBlockOperator::new( - operators.clone(), - func_ctx2.clone(), - num_input_columns, - ) - }); - } + let level = task.level + 1; + let enable_stream_writer = + self.ctx.get_settings().get_enable_block_stream_write()? + && table.storage_format_as_parquet(); + if enable_stream_writer { + let properties = StreamBlockProperties::try_create( + self.ctx.clone(), + table, + MutationKind::Recluster, + Some(level), + recluster.table_meta_timestamps, + )?; + let operators = properties.cluster_operators(); + if !operators.is_empty() { + let func_ctx = self.ctx.get_function_context()?; + self.main_pipeline.add_transformer(move || { + CompoundBlockOperator::new( + operators.clone(), + func_ctx.clone(), + num_input_columns, + ) + }); + } + + let sort_desc: Vec<_> = properties + .cluster_key_index() + .iter() + .map(|&offset| SortColumnDescription { + offset, + asc: true, + nulls_first: false, + }) + .collect(); + let fields_with_cluster_key = properties.fields_with_cluster_key(); + let schema = DataSchemaRefExt::create(fields_with_cluster_key); + let schema = add_order_field(schema, &sort_desc); + + let num_processors = self.main_pipeline.output_len(); + let sample_size = self + .ctx + .get_settings() + .get_recluster_sample_size_per_block()? + as usize; + let partitions = block_thresholds.calc_partitions_for_recluster( + task.total_rows, + task.total_bytes, + task.total_compressed, + ); + let state = SampleState::new(num_processors, partitions); + let recluster_pipeline_builder = + ReclusterPipelineBuilder::create(schema, sort_desc.clone(), sample_size) + .with_state(state); + recluster_pipeline_builder + .build_recluster_sample_pipeline(&mut self.main_pipeline)?; + + self.main_pipeline.exchange( + num_processors, + ReclusterPartitionExchange::create(partitions), + ); + let processor_id = AtomicUsize::new(0); + + let settings = self.ctx.get_settings(); + let memory_settings = MemorySettings::disable_spill(); + self.main_pipeline.add_transform(|input, output| { + let strategy = ReclusterPartitionStrategy::new(properties.clone()); + Ok(ProcessorPtr::create(Box::new( + TransformPartitionCollect::new( + self.ctx.clone(), + input, + output, + &settings, + processor_id.fetch_add(1, atomic::Ordering::AcqRel), + num_processors, + partitions, + memory_settings.clone(), + None, + strategy, + )?, + ))) + })?; - // construct output fields - let output_fields = cluster_stats_gen.out_fields.clone(); - let schema = DataSchemaRefExt::create(output_fields); - let sort_descs: Vec<_> = cluster_stats_gen - .cluster_key_index - .iter() - .map(|offset| SortColumnDescription { - offset: *offset, - asc: true, - nulls_first: false, - }) - .collect(); - - // merge sort - let sort_block_size = block_thresholds.calc_rows_for_recluster( - task.total_rows, - task.total_bytes, - task.total_compressed, - ); - - let sort_pipeline_builder = - SortPipelineBuilder::create(self.ctx.clone(), schema, sort_descs.into())? - .with_block_size_hit(sort_block_size) - .remove_order_col_at_last(); - // Todo(zhyass): Recluster will no longer perform sort in the near future. - sort_pipeline_builder.build_full_sort_pipeline(&mut self.main_pipeline)?; - - // Compact after merge sort. - let max_threads = self.ctx.get_settings().get_max_threads()? as usize; - build_compact_block_no_split_pipeline( - &mut self.main_pipeline, - block_thresholds, - max_threads, - )?; - - self.main_pipeline - .add_transform(|transform_input_port, transform_output_port| { - let proc = TransformSerializeBlock::try_create( + self.main_pipeline.add_async_accumulating_transformer(|| { + TransformBlockWriter::create( self.ctx.clone(), - transform_input_port, - transform_output_port, - table, - cluster_stats_gen.clone(), MutationKind::Recluster, - recluster.table_meta_timestamps, - )?; - proc.into_processor() - }) + table, + false, + ) + }); + Ok(()) + } else { + let cluster_stats_gen = table.get_cluster_stats_gen( + self.ctx.clone(), + level, + block_thresholds, + None, + )?; + let operators = cluster_stats_gen.operators.clone(); + if !operators.is_empty() { + let func_ctx2 = cluster_stats_gen.func_ctx.clone(); + self.main_pipeline.add_transformer(move || { + CompoundBlockOperator::new( + operators.clone(), + func_ctx2.clone(), + num_input_columns, + ) + }); + } + + // construct output fields + let output_fields = cluster_stats_gen.out_fields.clone(); + let schema = DataSchemaRefExt::create(output_fields); + let sort_desc: Vec<_> = cluster_stats_gen + .cluster_key_index + .iter() + .map(|offset| SortColumnDescription { + offset: *offset, + asc: true, + nulls_first: false, + }) + .collect(); + + // merge sort + let sort_block_size = block_thresholds.calc_rows_for_recluster( + task.total_rows, + task.total_bytes, + task.total_compressed, + ); + + let sort_pipeline_builder = + SortPipelineBuilder::create(self.ctx.clone(), schema, sort_desc.into())? + .with_block_size_hit(sort_block_size) + .remove_order_col_at_last(); + sort_pipeline_builder.build_full_sort_pipeline(&mut self.main_pipeline)?; + + // Compact after merge sort. + let max_threads = self.ctx.get_settings().get_max_threads()? as usize; + build_compact_block_no_split_pipeline( + &mut self.main_pipeline, + block_thresholds, + max_threads, + )?; + + self.main_pipeline.add_transform( + |transform_input_port, transform_output_port| { + let proc = TransformSerializeBlock::try_create( + self.ctx.clone(), + transform_input_port, + transform_output_port, + table, + cluster_stats_gen.clone(), + MutationKind::Recluster, + recluster.table_meta_timestamps, + )?; + proc.into_processor() + }, + ) + } } _ => Err(ErrorCode::Internal( "A node can only execute one recluster task".to_string(), @@ -190,3 +300,53 @@ impl PipelineBuilder { } } } + +struct ReclusterPipelineBuilder { + schema: DataSchemaRef, + sort_desc: Vec, + state: Option>, + sample_size: usize, + seed: u64, +} + +impl ReclusterPipelineBuilder { + fn create( + schema: DataSchemaRef, + sort_desc: Vec, + sample_size: usize, + ) -> Self { + Self { + schema, + sort_desc, + state: None, + sample_size, + seed: rand::random(), + } + } + + #[allow(unused)] + fn with_seed(mut self, seed: u64) -> Self { + self.seed = seed; + self + } + + fn with_state(mut self, state: Arc) -> Self { + self.state = Some(state); + self + } + + fn build_recluster_sample_pipeline(&self, pipeline: &mut Pipeline) -> Result<()> { + pipeline.try_add_transformer(|| { + TransformAddOrderColumn::try_new(self.sort_desc.clone(), self.schema.clone()) + })?; + let offset = self.schema.num_fields() - 1; + pipeline.add_accumulating_transformer(|| { + TransformReclusterCollect::new(offset, self.sample_size, self.seed) + }); + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create( + TransformRangePartitionIndexer::create(input, output, self.state.clone().unwrap()), + )) + }) + } +} diff --git a/src/query/service/src/pipelines/builders/builder_window.rs b/src/query/service/src/pipelines/builders/builder_window.rs index 187bb25d7dd77..64dbbe0e41e18 100644 --- a/src/query/service/src/pipelines/builders/builder_window.rs +++ b/src/query/service/src/pipelines/builders/builder_window.rs @@ -30,11 +30,11 @@ use databend_storages_common_cache::TempDirManager; use crate::pipelines::memory_settings::MemorySettingsExt; use crate::pipelines::processors::transforms::FrameBound; -use crate::pipelines::processors::transforms::SortStrategy; +use crate::pipelines::processors::transforms::TransformPartitionCollect; use crate::pipelines::processors::transforms::TransformWindow; -use crate::pipelines::processors::transforms::TransformWindowPartitionCollect; use crate::pipelines::processors::transforms::WindowFunctionInfo; use crate::pipelines::processors::transforms::WindowPartitionExchange; +use crate::pipelines::processors::transforms::WindowPartitionStrategy; use crate::pipelines::processors::transforms::WindowPartitionTopNExchange; use crate::pipelines::processors::transforms::WindowSortDesc; use crate::pipelines::PipelineBuilder; @@ -203,14 +203,14 @@ impl PipelineBuilder { let processor_id = AtomicUsize::new(0); self.main_pipeline.add_transform(|input, output| { - let strategy = SortStrategy::try_create( + let strategy = WindowPartitionStrategy::try_create( &settings, sort_desc.clone(), plan_schema.clone(), have_order_col, )?; Ok(ProcessorPtr::create(Box::new( - TransformWindowPartitionCollect::new( + TransformPartitionCollect::new( self.ctx.clone(), input, output, diff --git a/src/query/service/src/pipelines/processors/transforms/mod.rs b/src/query/service/src/pipelines/processors/transforms/mod.rs index 80966daa5fa8d..5c7c2264f7a4f 100644 --- a/src/query/service/src/pipelines/processors/transforms/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/mod.rs @@ -17,6 +17,7 @@ pub mod aggregator; mod broadcast; mod hash_join; pub(crate) mod range_join; +mod recluster; mod runtime_pool; mod transform_add_computed_columns; mod transform_add_const_columns; @@ -46,6 +47,7 @@ mod window; pub use broadcast::BroadcastSinkProcessor; pub use broadcast::BroadcastSourceProcessor; pub use hash_join::*; +pub use recluster::*; pub use transform_add_computed_columns::TransformAddComputedColumns; pub use transform_add_const_columns::TransformAddConstColumns; pub use transform_add_internal_columns::TransformAddInternalColumns; diff --git a/src/query/service/src/pipelines/processors/transforms/recluster/mod.rs b/src/query/service/src/pipelines/processors/transforms/recluster/mod.rs new file mode 100644 index 0000000000000..a024e330be25b --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/mod.rs @@ -0,0 +1,33 @@ +// 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 range_bound_sampler; +mod recluster_partition_exchange; +mod recluster_partition_strategy; +mod recluster_sample_state; +mod transform_add_order_column; +mod transform_range_partition_indexer; +mod transform_recluster_collect; +mod transform_recluster_partition; + +pub use range_bound_sampler::RangeBoundSampler; +pub use recluster_partition_exchange::ReclusterPartitionExchange; +pub use recluster_partition_strategy::CompactPartitionStrategy; +pub use recluster_partition_strategy::ReclusterPartitionStrategy; +pub use recluster_sample_state::SampleState; +pub use transform_add_order_column::TransformAddOrderColumn; +pub use transform_range_partition_indexer::TransformRangePartitionIndexer; +pub use transform_recluster_collect::ReclusterSampleMeta; +pub use transform_recluster_collect::TransformReclusterCollect; +pub use transform_recluster_partition::TransformReclusterPartition; diff --git a/src/query/service/src/pipelines/processors/transforms/recluster/range_bound_sampler.rs b/src/query/service/src/pipelines/processors/transforms/recluster/range_bound_sampler.rs new file mode 100644 index 0000000000000..b3fe9a77a4660 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/range_bound_sampler.rs @@ -0,0 +1,62 @@ +// 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 databend_common_expression::DataBlock; +use rand::prelude::SliceRandom; +use rand::prelude::SmallRng; +use rand::SeedableRng; + +pub struct RangeBoundSampler { + offset: usize, + sample_size: usize, + rng: SmallRng, + + values: Vec<(u64, Vec>)>, +} + +impl RangeBoundSampler { + pub fn new(offset: usize, sample_size: usize, seed: u64) -> Self { + let rng = SmallRng::seed_from_u64(seed); + Self { + offset, + sample_size, + rng, + values: vec![], + } + } +} + +impl RangeBoundSampler { + pub fn add_block(&mut self, data: &DataBlock) { + let rows = data.num_rows(); + assert!(rows > 0); + let column = data.get_by_offset(self.offset).to_column(); + + let sample_size = std::cmp::min(self.sample_size, rows); + let mut indices = (0..rows).collect::>(); + indices.shuffle(&mut self.rng); + let sampled_indices = &indices[..sample_size]; + + let column = column.as_binary().unwrap(); + let sample_values = sampled_indices + .iter() + .map(|i| unsafe { column.index_unchecked(*i) }.to_vec()) + .collect::>(); + self.values.push((rows as u64, sample_values)); + } + + pub fn sample_values(&mut self) -> Vec<(u64, Vec>)> { + std::mem::take(&mut self.values) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/hilbert_partition_exchange.rs b/src/query/service/src/pipelines/processors/transforms/recluster/recluster_partition_exchange.rs similarity index 63% rename from src/query/service/src/pipelines/processors/transforms/window/partition/hilbert_partition_exchange.rs rename to src/query/service/src/pipelines/processors/transforms/recluster/recluster_partition_exchange.rs index 93a6ce2aa4b6e..7fc006d3afad2 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/hilbert_partition_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/recluster/recluster_partition_exchange.rs @@ -12,11 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Some variables and functions are named and designed with reference to ClickHouse. -// - https://github.com/ClickHouse/ClickHouse/blob/master/src/Processors/Transforms/WindowTransform.h -// - https://github.com/ClickHouse/ClickHouse/blob/master/src/Processors/Transforms/WindowTransform.cpp - use std::sync::Arc; +use std::time::Instant; use databend_common_exception::Result; use databend_common_expression::DataBlock; @@ -24,20 +21,23 @@ use databend_common_pipeline_core::processors::Exchange; use crate::pipelines::processors::transforms::WindowPartitionMeta; -pub struct HilbertPartitionExchange { +pub struct ReclusterPartitionExchange { num_partitions: usize, + start_time: Instant, } -impl HilbertPartitionExchange { - pub fn create(num_partitions: usize) -> Arc { - Arc::new(HilbertPartitionExchange { num_partitions }) +impl ReclusterPartitionExchange { + pub fn create(num_partitions: usize) -> Arc { + Arc::new(ReclusterPartitionExchange { + num_partitions, + start_time: Instant::now(), + }) } } -impl Exchange for HilbertPartitionExchange { - const NAME: &'static str = "Hilbert"; - fn partition(&self, data_block: DataBlock, n: usize) -> Result> { - let mut data_block = data_block; +impl Exchange for ReclusterPartitionExchange { + const NAME: &'static str = "Recluster"; + fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { let range_ids = data_block .get_last_column() .as_number() @@ -55,14 +55,17 @@ impl Exchange for HilbertPartitionExchange { DataBlock::divide_indices_by_scatter_size(&indices, self.num_partitions); // Partition the data blocks to different processors. let mut output_data_blocks = vec![vec![]; n]; - for (partition_id, indices) in scatter_indices.iter().take(self.num_partitions).enumerate() + for (partition_id, indices) in scatter_indices + .into_iter() + .take(self.num_partitions) + .enumerate() { - if indices.is_empty() { - continue; + if !indices.is_empty() { + let block = data_block.take_with_optimize_size(&indices)?; + output_data_blocks[partition_id % n].push((partition_id, block)); } - let block = data_block.take_with_optimize_size(indices)?; - output_data_blocks[partition_id % n].push((partition_id, block)); } + log::info!("Recluster range exchange: {:?}", self.start_time.elapsed()); // Union data blocks for each processor. Ok(output_data_blocks diff --git a/src/query/service/src/pipelines/processors/transforms/recluster/recluster_partition_strategy.rs b/src/query/service/src/pipelines/processors/transforms/recluster/recluster_partition_strategy.rs new file mode 100644 index 0000000000000..d8f3443c4c6e0 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/recluster_partition_strategy.rs @@ -0,0 +1,153 @@ +// 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_exception::Result; +use databend_common_expression::DataBlock; +use databend_common_storages_fuse::io::StreamBlockBuilder; +use databend_common_storages_fuse::io::StreamBlockProperties; + +use crate::pipelines::processors::transforms::PartitionProcessStrategy; + +/// `ReclusterPartitionStrategy` is used when block stream writing is enabled. +/// It incrementally writes blocks using `StreamBlockBuilder`, which allows +/// partial serialization and flush during reclustering (e.g., Hilbert clustering). +pub struct ReclusterPartitionStrategy { + properties: Arc, +} + +impl ReclusterPartitionStrategy { + pub fn new(properties: Arc) -> Self { + Self { properties } + } +} + +impl PartitionProcessStrategy for ReclusterPartitionStrategy { + const NAME: &'static str = "Recluster"; + + /// Stream write each block, and flush it conditionally based on builder status + /// and input size estimation. + fn process_data_blocks(&self, data_blocks: Vec) -> Result> { + let blocks_num = data_blocks.len(); + let mut accumulated_rows = 0; + let mut accumulated_bytes = 0; + let mut pending_blocks = Vec::with_capacity(blocks_num); + let mut staged_blocks = Vec::with_capacity(blocks_num); + let mut compacted = Vec::with_capacity(blocks_num); + for block in data_blocks { + accumulated_rows += block.num_rows(); + accumulated_bytes += block.estimate_block_size(); + pending_blocks.push(block); + if !self + .properties + .check_large_enough(accumulated_rows, accumulated_bytes) + { + continue; + } + if !staged_blocks.is_empty() { + compacted.push(std::mem::take(&mut staged_blocks)); + } + std::mem::swap(&mut staged_blocks, &mut pending_blocks); + accumulated_rows = 0; + accumulated_bytes = 0; + } + staged_blocks.append(&mut pending_blocks); + if !staged_blocks.is_empty() { + compacted.push(std::mem::take(&mut staged_blocks)); + } + + let mut result = Vec::new(); + let mut builder = StreamBlockBuilder::try_new_with_config(self.properties.clone())?; + for blocks in compacted { + for block in blocks { + builder.write(block)?; + } + if builder.need_flush() { + let serialized = builder.finish()?; + result.push(DataBlock::empty_with_meta(Box::new(serialized))); + builder = StreamBlockBuilder::try_new_with_config(self.properties.clone())?; + } + } + if !builder.is_empty() { + let serialized = builder.finish()?; + result.push(DataBlock::empty_with_meta(Box::new(serialized))); + } + Ok(result) + } +} + +/// `CompactPartitionStrategy` is used when stream write is NOT enabled. +/// It uses a traditional "accumulate and concat" strategy to build large blocks +/// once input thresholds (row count or size) are exceeded. +pub struct CompactPartitionStrategy { + max_bytes_per_block: usize, + max_rows_per_block: usize, +} + +impl CompactPartitionStrategy { + pub fn new(max_rows_per_block: usize, max_bytes_per_block: usize) -> Self { + Self { + max_bytes_per_block, + max_rows_per_block, + } + } + + fn concat_blocks(blocks: Vec) -> Result { + DataBlock::concat(&blocks) + } + + fn check_large_enough(&self, rows: usize, bytes: usize) -> bool { + rows >= self.max_rows_per_block || bytes >= self.max_bytes_per_block + } +} + +impl PartitionProcessStrategy for CompactPartitionStrategy { + const NAME: &'static str = "Compact"; + + /// Collects blocks into batches and merges them via `concat` when size threshold is reached. + fn process_data_blocks(&self, data_blocks: Vec) -> Result> { + let blocks_num = data_blocks.len(); + if blocks_num < 2 { + return Ok(data_blocks); + } + + let mut accumulated_rows = 0; + let mut accumulated_bytes = 0; + let mut pending_blocks = Vec::with_capacity(blocks_num); + let mut staged_blocks = Vec::with_capacity(blocks_num); + let mut result = Vec::with_capacity(blocks_num); + for block in data_blocks { + accumulated_rows += block.num_rows(); + accumulated_bytes += block.estimate_block_size(); + pending_blocks.push(block); + if !self.check_large_enough(accumulated_rows, accumulated_bytes) { + continue; + } + if !staged_blocks.is_empty() { + result.push(Self::concat_blocks(std::mem::take(&mut staged_blocks))?); + } + std::mem::swap(&mut staged_blocks, &mut pending_blocks); + accumulated_rows = 0; + accumulated_bytes = 0; + } + + staged_blocks.append(&mut pending_blocks); + if !staged_blocks.is_empty() { + result.push(Self::concat_blocks(std::mem::take(&mut staged_blocks))?); + } + + Ok(result) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/recluster/recluster_sample_state.rs b/src/query/service/src/pipelines/processors/transforms/recluster/recluster_sample_state.rs new file mode 100644 index 0000000000000..07960939ee538 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/recluster_sample_state.rs @@ -0,0 +1,133 @@ +// 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::intrinsics::unlikely; +use std::sync::Arc; +use std::sync::RwLock; + +use databend_common_base::base::WatchNotify; +use databend_common_exception::Result; +use databend_common_expression::compare_columns; +use databend_common_expression::types::BinaryType; +use databend_common_expression::FromData; + +pub struct SampleState { + pub inner: RwLock, + pub done: Arc, +} + +impl SampleState { + pub fn new(total_inputs: usize, partitions: usize) -> Arc { + Arc::new(SampleState { + inner: RwLock::new(SampleStateInner { + partitions, + total_inputs, + completed_inputs: 0, + values: vec![], + bounds: vec![], + max_value: None, + }), + done: Arc::new(WatchNotify::new()), + }) + } + + pub fn merge_sample(&self, values: Vec<(u64, Vec>)>) -> Result<()> { + let mut inner = self.inner.write().unwrap(); + inner.completed_inputs += 1; + inner.values.extend_from_slice(&values); + + if inner.completed_inputs >= inner.total_inputs { + inner.determine_bounds()?; + self.done.notify_waiters(); + } + Ok(()) + } + + pub fn get_bounds(&self) -> (Vec>, Option>) { + let inner = self.inner.read().unwrap(); + (inner.bounds.clone(), inner.max_value.clone()) + } +} + +pub struct SampleStateInner { + partitions: usize, + total_inputs: usize, + + completed_inputs: usize, + bounds: Vec>, + max_value: Option>, + + values: Vec<(u64, Vec>)>, +} + +impl SampleStateInner { + fn determine_bounds(&mut self) -> Result<()> { + if self.partitions < 2 { + return Ok(()); + } + + let (total_samples, total_rows) = self + .values + .iter() + .fold((0, 0), |(acc_samples, acc_rows), (rows, vals)| { + (acc_samples + vals.len(), acc_rows + *rows) + }); + let step = total_rows as f64 / self.partitions as f64; + let values = std::mem::take(&mut self.values); + let mut data = Vec::with_capacity(total_samples); + let mut weights = Vec::with_capacity(total_samples); + + for (num, values) in values.into_iter() { + let weight = num as f64 / values.len() as f64; + values.into_iter().for_each(|v| { + data.push(v); + weights.push(weight); + }); + } + let col = BinaryType::from_data(data.clone()); + let indices = compare_columns(vec![col], total_samples)?; + + let max_index = indices[total_samples - 1] as usize; + let max_val = &data[max_index]; + + let mut cum_weight = 0.0; + let mut target = step; + let mut bounds = Vec::with_capacity(self.partitions - 1); + let mut previous_bound = None; + + let mut i = 0; + let mut j = 0; + while i < total_samples && j < self.partitions - 1 { + let idx = indices[i] as usize; + let value = &data[idx]; + let weight = weights[idx]; + cum_weight += weight; + + if cum_weight >= target && previous_bound.is_none_or(|prev| value > prev) { + if unlikely(value == max_val) { + self.max_value = Some(max_val.clone()); + break; + } + + bounds.push(value.clone()); + previous_bound = Some(value); + target += step; + j += 1; + } + i += 1; + } + self.bounds = bounds; + Ok(()) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/recluster/transform_add_order_column.rs b/src/query/service/src/pipelines/processors/transforms/recluster/transform_add_order_column.rs new file mode 100644 index 0000000000000..b3fa11ba7ddda --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/transform_add_order_column.rs @@ -0,0 +1,55 @@ +// 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 databend_common_exception::Result; +use databend_common_expression::row::RowConverter as CommonConverter; +use databend_common_expression::DataBlock; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::SortColumnDescription; +use databend_common_pipeline_transforms::sort::RowConverter; +use databend_common_pipeline_transforms::sort::Rows; +use databend_common_pipeline_transforms::Transform; + +pub struct TransformAddOrderColumn { + row_converter: CommonConverter, + sort_desc: Vec, +} + +impl TransformAddOrderColumn { + pub fn try_new(sort_desc: Vec, schema: DataSchemaRef) -> Result { + let row_converter = CommonConverter::create(&sort_desc, schema.clone())?; + Ok(Self { + row_converter, + sort_desc, + }) + } +} + +impl Transform for TransformAddOrderColumn { + const NAME: &'static str = "TransformAddOrderColumn"; + + fn transform(&mut self, mut data: DataBlock) -> Result { + let order_by_cols = self + .sort_desc + .iter() + .map(|desc| data.get_by_offset(desc.offset).clone()) + .collect::>(); + let rows = self + .row_converter + .convert(&order_by_cols, data.num_rows())?; + let order_col = rows.to_column(); + data.add_column(order_col); + Ok(data) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/recluster/transform_range_partition_indexer.rs b/src/query/service/src/pipelines/processors/transforms/recluster/transform_range_partition_indexer.rs new file mode 100644 index 0000000000000..39efbec0b20a7 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/transform_range_partition_indexer.rs @@ -0,0 +1,165 @@ +// 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::any::Any; +use std::collections::VecDeque; +use std::intrinsics::unlikely; +use std::sync::Arc; +use std::time::Instant; + +use databend_common_exception::Result; +use databend_common_expression::types::UInt64Type; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_expression::FromData; +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 crate::pipelines::processors::transforms::ReclusterSampleMeta; +use crate::pipelines::processors::transforms::SampleState; + +pub struct TransformRangePartitionIndexer { + input: Arc, + output: Arc, + + state: Arc, + input_data: Vec, + output_data: VecDeque, + bounds: Vec>, + max_value: Option>, + + start: Instant, +} + +impl TransformRangePartitionIndexer { + pub fn create( + input: Arc, + output: Arc, + state: Arc, + ) -> Box { + Box::new(Self { + input, + output, + state, + input_data: vec![], + output_data: VecDeque::new(), + bounds: vec![], + max_value: None, + start: Instant::now(), + }) + } +} + +#[async_trait::async_trait] +impl Processor for TransformRangePartitionIndexer { + fn name(&self) -> String { + "TransformRangePartitionIndexer".to_owned() + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if self.output.is_finished() { + self.input.finish(); + return Ok(Event::Finished); + } + + if !self.output.can_push() { + self.input.set_not_need_data(); + return Ok(Event::NeedConsume); + } + + if let Some(data_block) = self.output_data.pop_front() { + self.output.push_data(Ok(data_block)); + return Ok(Event::NeedConsume); + } + + if !self.input_data.is_empty() { + return Ok(Event::Sync); + } + + if self.input.is_finished() { + assert!(self.state.done.is_notified()); + self.output.finish(); + return Ok(Event::Finished); + } + + if !self.input.has_data() { + self.input.set_need_data(); + return Ok(Event::NeedData); + } + + let mut input_data = self.input.pull_data().unwrap()?; + let meta = input_data + .take_meta() + .and_then(ReclusterSampleMeta::downcast_from) + .expect("require a ReclusterSampleMeta"); + self.input_data = meta.blocks; + self.state.merge_sample(meta.sample_values)?; + log::info!("Recluster range partition: {:?}", self.start.elapsed()); + Ok(Event::Async) + } + + fn process(&mut self) -> Result<()> { + let start = Instant::now(); + if let Some(mut block) = self.input_data.pop() { + let num_rows = block.num_rows(); + let mut builder = Vec::with_capacity(num_rows); + let last_col = block.get_last_column().as_binary().unwrap(); + if let Some(max_value) = self.max_value.as_ref() { + let bound_len = self.bounds.len(); + for index in 0..num_rows { + let val = unsafe { last_col.index_unchecked(index) }; + if unlikely(val >= max_value.as_slice()) { + let range_id = bound_len + 1; + builder.push(range_id as u64); + continue; + } + + let idx = self + .bounds + .binary_search_by(|b| b.as_slice().cmp(val)) + .unwrap_or_else(|i| i); + builder.push(idx as u64); + } + } else { + for index in 0..num_rows { + let val = unsafe { last_col.index_unchecked(index) }; + let idx = self + .bounds + .binary_search_by(|b| b.as_slice().cmp(val)) + .unwrap_or_else(|i| i); + builder.push(idx as u64); + } + } + block.pop_columns(1); + block.add_column(UInt64Type::from_data(builder)); + self.output_data.push_back(block); + } + + log::info!("Recluster range output: {:?}", start.elapsed()); + Ok(()) + } + + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + self.state.done.notified().await; + (self.bounds, self.max_value) = self.state.get_bounds(); + Ok(()) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/recluster/transform_recluster_collect.rs b/src/query/service/src/pipelines/processors/transforms/recluster/transform_recluster_collect.rs new file mode 100644 index 0000000000000..28d1c0aed8b54 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/transform_recluster_collect.rs @@ -0,0 +1,67 @@ +// 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 databend_common_exception::Result; +use databend_common_expression::local_block_meta_serde; +use databend_common_expression::BlockMetaInfo; +use databend_common_expression::DataBlock; +use databend_common_pipeline_transforms::AccumulatingTransform; + +use crate::pipelines::processors::transforms::RangeBoundSampler; + +pub struct TransformReclusterCollect { + input_data: Vec, + sampler: RangeBoundSampler, +} + +impl TransformReclusterCollect { + pub fn new(offset: usize, sample_size: usize, seed: u64) -> Self { + Self { + input_data: vec![], + sampler: RangeBoundSampler::new(offset, sample_size, seed), + } + } +} + +impl AccumulatingTransform for TransformReclusterCollect { + const NAME: &'static str = "TransformReclusterCollect"; + + fn transform(&mut self, data: DataBlock) -> Result> { + self.sampler.add_block(&data); + self.input_data.push(data); + Ok(vec![]) + } + + fn on_finish(&mut self, _output: bool) -> Result> { + let sample_values = self.sampler.sample_values(); + let blocks = std::mem::take(&mut self.input_data); + let meta = ReclusterSampleMeta { + blocks, + sample_values, + }; + + Ok(vec![DataBlock::empty_with_meta(Box::new(meta))]) + } +} + +#[derive(Debug)] +pub struct ReclusterSampleMeta { + pub blocks: Vec, + pub sample_values: Vec<(u64, Vec>)>, +} + +local_block_meta_serde!(ReclusterSampleMeta); + +#[typetag::serde(name = "recluster_sample")] +impl BlockMetaInfo for ReclusterSampleMeta {} diff --git a/src/query/service/src/pipelines/processors/transforms/recluster/transform_recluster_partition.rs b/src/query/service/src/pipelines/processors/transforms/recluster/transform_recluster_partition.rs new file mode 100644 index 0000000000000..5a3e19d2b0e3e --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/transform_recluster_partition.rs @@ -0,0 +1,246 @@ +// 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::any::Any; +use std::collections::VecDeque; +use std::sync::Arc; +use std::time::Instant; + +use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +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_storages_fuse::io::StreamBlockBuilder; +use databend_common_storages_fuse::io::StreamBlockProperties; + +use crate::pipelines::processors::transforms::WindowPartitionMeta; + +enum Step { + Consume, + Collect, + Flush, +} + +struct PartitionData { + builder: Option, + data_blocks: Vec, + block_size: usize, + block_rows: usize, +} + +impl PartitionData { + fn new() -> Self { + Self { + builder: None, + data_blocks: vec![], + block_size: 0, + block_rows: 0, + } + } + + fn is_empty(&self) -> bool { + self.builder.as_ref().is_none_or(|v| v.is_empty()) && self.data_blocks.is_empty() + } +} + +pub struct TransformReclusterPartition { + input: Arc, + output: Arc, + + properties: Arc, + + // The partition id is used to map the partition id to the new partition id. + partition_id: Vec, + partition_data: Vec, + output_data: VecDeque, + + start: Instant, + cnt: usize, + + step: Step, +} + +impl TransformReclusterPartition { + pub fn try_create( + input: Arc, + output: Arc, + properties: Arc, + processor_id: usize, + num_processors: usize, + num_partitions: usize, + ) -> Result { + let partitions = (0..num_partitions) + .filter(|&partition| (partition * num_processors) / num_partitions == processor_id) + .collect::>(); + let mut partition_id = vec![0; num_partitions]; + let mut partition_data = Vec::with_capacity(num_partitions); + for (new_partition_id, partition) in partitions.iter().enumerate() { + partition_id[*partition] = new_partition_id; + partition_data.push(PartitionData::new()); + } + Ok(ProcessorPtr::create(Box::new( + TransformReclusterPartition { + input, + output, + properties, + partition_id, + partition_data, + output_data: VecDeque::new(), + step: Step::Consume, + start: Instant::now(), + cnt: 0, + }, + ))) + } +} + +impl Processor for TransformReclusterPartition { + fn name(&self) -> String { + "TransformReclusterPartition".to_string() + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if matches!(self.step, Step::Collect | Step::Flush) { + return Ok(Event::Sync); + } + + if self.output.is_finished() { + self.input.finish(); + return Ok(Event::Finished); + } + + if !self.output.can_push() { + return Ok(Event::NeedConsume); + } + + if let Some(data_block) = self.output_data.pop_front() { + self.output.push_data(Ok(data_block)); + return Ok(Event::NeedConsume); + } + + if self.input.is_finished() { + if !self.partition_data.is_empty() { + if self.cnt == 0 { + log::info!("Recluster: start flush: {:?}", self.start.elapsed()); + } + self.cnt += 1; + self.step = Step::Flush; + return Ok(Event::Sync); + } + self.output.finish(); + return Ok(Event::Finished); + } + + if self.input.has_data() { + self.step = Step::Collect; + return Ok(Event::Sync); + } + + self.input.set_need_data(); + Ok(Event::NeedData) + } + + fn process(&mut self) -> Result<()> { + match std::mem::replace(&mut self.step, Step::Consume) { + Step::Collect => { + let start_cost = self.start.elapsed(); + let data_block = self.input.pull_data().unwrap()?; + if let Some(meta) = data_block + .get_owned_meta() + .and_then(WindowPartitionMeta::downcast_from) + { + for (partition_id, data_block) in meta.partitioned_data.into_iter() { + if data_block.is_empty() { + continue; + } + + let new_id = self.partition_id[partition_id]; + let partition_data = + unsafe { self.partition_data.get_unchecked_mut(new_id) }; + if partition_data.builder.is_none() { + partition_data.builder = Some(StreamBlockBuilder::try_new_with_config( + self.properties.clone(), + )?); + } + let builder = partition_data.builder.as_mut().unwrap(); + if !builder.need_flush() { + builder.write(data_block)?; + } else { + partition_data.block_size += data_block.estimate_block_size(); + partition_data.block_rows += data_block.num_rows(); + partition_data.data_blocks.push(data_block); + + if self.properties.check_large_enough( + partition_data.block_rows, + partition_data.block_size, + ) { + let builder = partition_data.builder.take().unwrap(); + let serialized = builder.finish()?; + self.output_data + .push_back(DataBlock::empty_with_meta(Box::new(serialized))); + + let mut builder = StreamBlockBuilder::try_new_with_config( + self.properties.clone(), + )?; + for block in + std::mem::take(&mut partition_data.data_blocks).into_iter() + { + builder.write(block)?; + } + partition_data.builder = Some(builder); + partition_data.block_rows = 0; + partition_data.block_size = 0; + } + } + } + } + log::info!( + "Recluster: start collect: {:?}, end: {:?}", + start_cost, + self.start.elapsed() + ); + } + Step::Flush => { + while let Some(mut partition_data) = self.partition_data.pop() { + if partition_data.is_empty() { + continue; + } + + let mut builder = if partition_data.builder.is_none() { + StreamBlockBuilder::try_new_with_config(self.properties.clone())? + } else { + partition_data.builder.take().unwrap() + }; + for block in partition_data.data_blocks { + builder.write(block)?; + } + let serialized = builder.finish()?; + self.output_data + .push_back(DataBlock::empty_with_meta(Box::new(serialized))); + break; + } + } + _ => unreachable!(), + } + Ok(()) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/mod.rs b/src/query/service/src/pipelines/processors/transforms/window/partition/mod.rs index 5aa4562c98865..1418388cf2553 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/partition/mod.rs @@ -12,16 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod data_processor_strategy; -mod hilbert_partition_exchange; +mod partition_process_strategy; mod transform_window_partition_collect; mod window_partition_buffer; mod window_partition_exchange; mod window_partition_meta; mod window_partition_partial_top_n_exchange; -pub use data_processor_strategy::*; -pub use hilbert_partition_exchange::*; +pub use partition_process_strategy::*; pub use transform_window_partition_collect::*; pub use window_partition_buffer::*; pub use window_partition_exchange::*; diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/data_processor_strategy.rs b/src/query/service/src/pipelines/processors/transforms/window/partition/partition_process_strategy.rs similarity index 52% rename from src/query/service/src/pipelines/processors/transforms/window/partition/data_processor_strategy.rs rename to src/query/service/src/pipelines/processors/transforms/window/partition/partition_process_strategy.rs index 75793aa415e08..cffa542136623 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/data_processor_strategy.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/partition/partition_process_strategy.rs @@ -19,72 +19,13 @@ use databend_common_expression::SortColumnDescription; use databend_common_pipeline_transforms::sort_merge; use databend_common_settings::Settings; -pub trait DataProcessorStrategy: Send + Sync + 'static { +pub trait PartitionProcessStrategy: Send + Sync + 'static { const NAME: &'static str; - fn process_data_blocks(&self, data_blocks: Vec) -> Result>; -} - -pub struct CompactStrategy { - max_bytes_per_block: usize, - max_rows_per_block: usize, -} - -impl CompactStrategy { - pub fn new(max_rows_per_block: usize, max_bytes_per_block: usize) -> Self { - Self { - max_bytes_per_block, - max_rows_per_block, - } - } - - fn concat_blocks(blocks: Vec) -> Result { - DataBlock::concat(&blocks) - } - - fn check_large_enough(&self, rows: usize, bytes: usize) -> bool { - rows >= self.max_rows_per_block || bytes >= self.max_bytes_per_block - } -} - -impl DataProcessorStrategy for CompactStrategy { - const NAME: &'static str = "Compact"; - - fn process_data_blocks(&self, data_blocks: Vec) -> Result> { - let blocks_num = data_blocks.len(); - if blocks_num < 2 { - return Ok(data_blocks); - } - let mut accumulated_rows = 0; - let mut accumulated_bytes = 0; - let mut pending_blocks = Vec::with_capacity(blocks_num); - let mut staged_blocks = Vec::with_capacity(blocks_num); - let mut result = Vec::with_capacity(blocks_num); - for block in data_blocks { - accumulated_rows += block.num_rows(); - accumulated_bytes += block.estimate_block_size(); - pending_blocks.push(block); - if !self.check_large_enough(accumulated_rows, accumulated_bytes) { - continue; - } - if !staged_blocks.is_empty() { - result.push(Self::concat_blocks(std::mem::take(&mut staged_blocks))?); - } - std::mem::swap(&mut staged_blocks, &mut pending_blocks); - accumulated_rows = 0; - accumulated_bytes = 0; - } - - staged_blocks.append(&mut pending_blocks); - if !staged_blocks.is_empty() { - result.push(Self::concat_blocks(std::mem::take(&mut staged_blocks))?); - } - - Ok(result) - } + fn process_data_blocks(&self, data_blocks: Vec) -> Result>; } -pub struct SortStrategy { +pub struct WindowPartitionStrategy { sort_desc: Vec, schema: DataSchemaRef, max_block_size: usize, @@ -93,7 +34,7 @@ pub struct SortStrategy { have_order_col: bool, } -impl SortStrategy { +impl WindowPartitionStrategy { pub fn try_create( settings: &Settings, sort_desc: Vec, @@ -114,8 +55,8 @@ impl SortStrategy { } } -impl DataProcessorStrategy for SortStrategy { - const NAME: &'static str = "Sort"; +impl PartitionProcessStrategy for WindowPartitionStrategy { + const NAME: &'static str = "Window"; fn process_data_blocks(&self, data_blocks: Vec) -> Result> { let data_blocks = data_blocks diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/transform_window_partition_collect.rs b/src/query/service/src/pipelines/processors/transforms/window/partition/transform_window_partition_collect.rs index 3051a2f0f018c..0171af6053d7c 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/transform_window_partition_collect.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/partition/transform_window_partition_collect.rs @@ -33,7 +33,7 @@ use databend_common_storage::DataOperator; use super::WindowPartitionBuffer; use super::WindowPartitionMeta; -use crate::pipelines::processors::transforms::DataProcessorStrategy; +use crate::pipelines::processors::transforms::PartitionProcessStrategy; use crate::sessions::QueryContext; use crate::spillers::Spiller; use crate::spillers::SpillerConfig; @@ -59,7 +59,7 @@ pub enum AsyncStep { Restore, } -pub struct TransformWindowPartitionCollect { +pub struct TransformPartitionCollect { input: Arc, output: Arc, @@ -78,7 +78,7 @@ pub struct TransformWindowPartitionCollect { is_collect_finished: bool, } -impl TransformWindowPartitionCollect { +impl TransformPartitionCollect { pub fn new( ctx: Arc, input: Arc, @@ -162,11 +162,7 @@ impl TransformWindowPartitionCollect { } if self.input.has_data() { - Self::collect_data_block( - self.input.pull_data().unwrap()?, - &self.partition_id, - &mut self.buffer, - ); + self.collect_data_block()?; } // Check again. flush memory data to external storage if need @@ -209,9 +205,9 @@ impl TransformWindowPartitionCollect { } #[async_trait::async_trait] -impl Processor for TransformWindowPartitionCollect { +impl Processor for TransformPartitionCollect { fn name(&self) -> String { - format!("TransformWindowPartitionCollect({})", S::NAME) + format!("TransformPartitionCollect({})", S::NAME) } fn as_any(&mut self) -> &mut dyn Any { @@ -271,21 +267,19 @@ impl Processor for TransformWindowPartitionCollect } } -impl TransformWindowPartitionCollect { - fn collect_data_block( - data_block: DataBlock, - partition_ids: &[usize], - buffer: &mut WindowPartitionBuffer, - ) { +impl TransformPartitionCollect { + fn collect_data_block(&mut self) -> Result<()> { + let data_block = self.input.pull_data().unwrap()?; if let Some(meta) = data_block .get_owned_meta() .and_then(WindowPartitionMeta::downcast_from) { for (partition_id, data_block) in meta.partitioned_data.into_iter() { - let partition_id = partition_ids[partition_id]; - buffer.add_data_block(partition_id, data_block); + let new_id = self.partition_id[partition_id]; + self.buffer.add_data_block(new_id, data_block); } } + Ok(()) } fn need_spill(&mut self) -> bool { diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer.rs b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer.rs index bf01acedc586c..b58bafca0ee9a 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer.rs @@ -145,75 +145,84 @@ impl WindowPartitionBuffer { while self.next_to_restore_partition_id + 1 < self.num_partitions as isize { self.next_to_restore_partition_id += 1; let partition_id = self.next_to_restore_partition_id as usize; - // Restore large partitions from spilled files. - let mut result = self.spiller.read_spilled_partition(&partition_id).await?; - - // Restore small merged partitions from spilled files. - let spilled_small_partitions = - std::mem::take(&mut self.spilled_small_partitions[partition_id]); - for index in spilled_small_partitions { - let out_of_memory_limit = self.out_of_memory_limit(); - let (merged_partitions, restored, partial_restored) = - &mut self.spilled_merged_partitions[index]; - if *restored { - continue; - } - let MergedPartition { - location, - partitions, - } = merged_partitions; - if out_of_memory_limit || *partial_restored { - if let Some(pos) = partitions.iter().position(|(id, _)| *id == partition_id) { - let data_block = self - .spiller - .read_chunk(location, &partitions[pos].1) - .await?; - self.restored_partition_buffer - .add_data_block(partition_id, data_block); - partitions.remove(pos); - *partial_restored = true; - } - } else { - let partitioned_data = self + let result = self.restore_by_id(partition_id, false).await?; + if !result.is_empty() { + return Ok(result); + } + } + Ok(vec![]) + } + + pub async fn restore_by_id( + &mut self, + partition_id: usize, + partial_restore: bool, + ) -> Result> { + // Restore large partitions from spilled files. + let mut result = self.spiller.take_spilled_partition(&partition_id).await?; + + // Restore small merged partitions from spilled files. + let spilled_small_partitions = + std::mem::take(&mut self.spilled_small_partitions[partition_id]); + for index in spilled_small_partitions { + let out_of_memory_limit = self.out_of_memory_limit(); + let (merged_partitions, restored, partial_restored) = + &mut self.spilled_merged_partitions[index]; + if *restored { + continue; + } + let MergedPartition { + location, + partitions, + } = merged_partitions; + if out_of_memory_limit || *partial_restored || partial_restore { + if let Some(pos) = partitions.iter().position(|(id, _)| *id == partition_id) { + let data_block = self .spiller - .read_merged_partitions(merged_partitions) + .read_chunk(location, &partitions[pos].1) .await?; - for (partition_id, data_block) in partitioned_data.into_iter() { - self.restored_partition_buffer - .add_data_block(partition_id, data_block); - } - *restored = true; + self.restored_partition_buffer + .add_data_block(partition_id, data_block); + partitions.remove(pos); + *partial_restored = true; } - } - - if !self.partition_buffer.is_partition_empty(partition_id) { - let option = PartitionBufferFetchOption::PickPartitionWithThreshold(0); - if let Some(data_blocks) = self - .partition_buffer - .fetch_data_blocks(partition_id, &option)? - { - result.extend(self.concat_data_blocks(data_blocks)?); + } else { + let partitioned_data = self + .spiller + .read_merged_partitions(merged_partitions) + .await?; + for (partition_id, data_block) in partitioned_data.into_iter() { + self.restored_partition_buffer + .add_data_block(partition_id, data_block); } + *restored = true; } + } - if !self - .restored_partition_buffer - .is_partition_empty(partition_id) + if !self.partition_buffer.is_partition_empty(partition_id) { + let option = PartitionBufferFetchOption::PickPartitionWithThreshold(0); + if let Some(data_blocks) = self + .partition_buffer + .fetch_data_blocks(partition_id, &option)? { - let option = PartitionBufferFetchOption::PickPartitionWithThreshold(0); - if let Some(data_blocks) = self - .restored_partition_buffer - .fetch_data_blocks(partition_id, &option)? - { - result.extend(self.concat_data_blocks(data_blocks)?); - } + result.extend(self.concat_data_blocks(data_blocks)?); } + } - if !result.is_empty() { - return Ok(result); + if !self + .restored_partition_buffer + .is_partition_empty(partition_id) + { + let option = PartitionBufferFetchOption::PickPartitionWithThreshold(0); + if let Some(data_blocks) = self + .restored_partition_buffer + .fetch_data_blocks(partition_id, &option)? + { + result.extend(self.concat_data_blocks(data_blocks)?); } } - Ok(vec![]) + + Ok(result) } fn concat_data_blocks(&self, data_blocks: Vec) -> Result> { diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs index 2e7a6e878b819..5b83c20a670d8 100644 --- a/src/query/service/src/schedulers/fragments/fragmenter.rs +++ b/src/query/service/src/schedulers/fragments/fragmenter.rs @@ -41,7 +41,6 @@ use crate::servers::flight::v1::exchange::DataExchange; use crate::servers::flight::v1::exchange::MergeExchange; use crate::servers::flight::v1::exchange::ShuffleDataExchange; use crate::sessions::QueryContext; -use crate::sql::executor::physical_plans::Mutation; use crate::sql::executor::PhysicalPlan; /// Visitor to split a `PhysicalPlan` into fragments. @@ -170,14 +169,6 @@ impl PhysicalPlanReplacer for Fragmenter { Ok(PhysicalPlan::MutationSource(plan.clone())) } - fn replace_mutation(&mut self, plan: &Mutation) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::Mutation(Box::new(Mutation { - input: Box::new(input), - ..plan.clone() - }))) - } - fn replace_replace_into(&mut self, plan: &ReplaceInto) -> Result { let input = self.replace(&plan.input)?; self.state = State::ReplaceInto; diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index 18f2b35267eb4..2f52da8b04ce3 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -552,7 +552,7 @@ impl PhysicalPlanReplacer for ReplaceReadSource { } struct ReplaceRecluster { - pub tasks: Vec, + tasks: Vec, } impl PhysicalPlanReplacer for ReplaceRecluster { @@ -565,7 +565,7 @@ impl PhysicalPlanReplacer for ReplaceRecluster { } struct ReplaceMutationSource { - pub partitions: Partitions, + partitions: Partitions, } impl PhysicalPlanReplacer for ReplaceMutationSource { @@ -578,7 +578,7 @@ impl PhysicalPlanReplacer for ReplaceMutationSource { } struct ReplaceCompactBlock { - pub partitions: Partitions, + partitions: Partitions, } impl PhysicalPlanReplacer for ReplaceCompactBlock { @@ -591,10 +591,10 @@ impl PhysicalPlanReplacer for ReplaceCompactBlock { } struct ReplaceReplaceInto { - pub partitions: Vec<(usize, Location)>, + partitions: Vec<(usize, Location)>, // for standalone mode, slot is None - pub slot: Option, - pub need_insert: bool, + slot: Option, + need_insert: bool, } impl PhysicalPlanReplacer for ReplaceReplaceInto { diff --git a/src/query/service/src/spillers/spiller.rs b/src/query/service/src/spillers/spiller.rs index 6c454ab89638b..3627f1b83a117 100644 --- a/src/query/service/src/spillers/spiller.rs +++ b/src/query/service/src/spillers/spiller.rs @@ -386,6 +386,24 @@ impl Spiller { } } + #[async_backtrace::framed] + /// Read and remove spilled data with partition id + pub async fn take_spilled_partition(&mut self, p_id: &usize) -> Result> { + if let Some(locs) = self.partition_location.remove(p_id) { + let mut spilled_data = Vec::with_capacity(locs.len()); + for loc in locs { + let block = self.read_spilled_file(&loc).await?; + + if block.num_rows() != 0 { + spilled_data.push(block); + } + } + Ok(spilled_data) + } else { + Ok(vec![]) + } + } + pub async fn read_merged_partitions( &self, MergedPartition { 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 b3e8cf59c5a65..bd091a35ef5f2 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 @@ -156,6 +156,7 @@ async fn test_recluster_mutator_block_select() -> Result<()> { cluster_key_id, 1, column_ids, + 1, ); let (_, parts) = mutator .target_select(compact_segments, ReclusterMode::Recluster) @@ -280,6 +281,7 @@ async fn test_safety_for_recluster() -> Result<()> { cluster_key_id, max_tasks, column_ids, + 500, )); let (mode, selected_segs) = mutator.select_segments(&compact_segments, 8)?; // select the blocks with the highest depth. diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index f5c42c4e05053..a686891024a90 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -755,7 +755,7 @@ impl DefaultSettings { range: Some(SettingRange::Numeric(0..=1)), }), ("enable_distributed_compact", DefaultSettingValue { - value: UserSettingValue::UInt64(0), + value: UserSettingValue::UInt64(1), desc: "Enables distributed execution of table compaction.", mode: SettingMode::Both, scope: SettingScope::Both, @@ -860,7 +860,7 @@ impl DefaultSettings { desc: "Sets the maximum byte size of blocks for recluster", mode: SettingMode::Both, scope: SettingScope::Both, - range: Some(SettingRange::Numeric(0..=u64::MAX)), + range: Some(SettingRange::Numeric(0..=80 * 1024 * 1024 * 1024)), }), ("compact_max_block_selection", DefaultSettingValue { value: UserSettingValue::UInt64(10000), @@ -870,7 +870,7 @@ impl DefaultSettings { range: Some(SettingRange::Numeric(2..=u64::MAX)), }), ("enable_distributed_recluster", DefaultSettingValue { - value: UserSettingValue::UInt64(0), + value: UserSettingValue::UInt64(1), desc: "Enable distributed execution of table recluster.", mode: SettingMode::Both, scope: SettingScope::Both, @@ -1220,9 +1220,9 @@ impl DefaultSettings { scope: SettingScope::Both, range: Some(SettingRange::Numeric(1..=65535)), }), - ("hilbert_sample_size_per_block", DefaultSettingValue { + ("recluster_sample_size_per_block", DefaultSettingValue { value: UserSettingValue::UInt64(1000), - desc: "Specifies the number of sample points per block used in Hilbert clustering.", + desc: "Specifies the number of sample points per block used in clustering.", mode: SettingMode::Both, scope: SettingScope::Both, range: Some(SettingRange::Numeric(1..=u64::MAX)), diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index a3ef08cc9264b..89061370021da 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -889,8 +889,8 @@ impl Settings { self.try_get_u64("hilbert_num_range_ids") } - pub fn get_hilbert_sample_size_per_block(&self) -> Result { - self.try_get_u64("hilbert_sample_size_per_block") + pub fn get_recluster_sample_size_per_block(&self) -> Result { + self.try_get_u64("recluster_sample_size_per_block") } pub fn get_hilbert_clustering_min_bytes(&self) -> Result { diff --git a/src/query/sql/src/executor/physical_plans/physical_recluster.rs b/src/query/sql/src/executor/physical_plans/physical_recluster.rs index 9227c86b64199..a90df3a9fdef5 100644 --- a/src/query/sql/src/executor/physical_plans/physical_recluster.rs +++ b/src/query/sql/src/executor/physical_plans/physical_recluster.rs @@ -34,4 +34,5 @@ pub struct HilbertPartition { pub num_partitions: usize, pub table_meta_timestamps: TableMetaTimestamps, pub rows_per_block: usize, + pub bytes_per_block: usize, } diff --git a/src/query/sql/src/planner/binder/ddl/table.rs b/src/query/sql/src/planner/binder/ddl/table.rs index 8f95e1d17958f..7d3263a8cdceb 100644 --- a/src/query/sql/src/planner/binder/ddl/table.rs +++ b/src/query/sql/src/planner/binder/ddl/table.rs @@ -149,6 +149,7 @@ use crate::plans::VacuumTemporaryFilesPlan; use crate::BindContext; use crate::DefaultExprBinder; use crate::Planner; +use crate::ScalarExpr; use crate::SelectBuilder; pub(in crate::planner::binder) struct AnalyzeCreateTableResult { @@ -1767,7 +1768,7 @@ impl Binder { let mut cluster_keys = Vec::with_capacity(expr_len); for cluster_expr in cluster_exprs.iter() { - let (cluster_key, _) = scalar_binder.bind(cluster_expr)?; + let (mut cluster_key, _) = scalar_binder.bind(cluster_expr)?; if cluster_key.used_columns().len() != 1 || !cluster_key.evaluable() { return Err(ErrorCode::InvalidClusterKeys(format!( "Cluster by expression `{:#}` is invalid", @@ -1775,6 +1776,14 @@ impl Binder { ))); } + if let ScalarExpr::FunctionCall(func) = &cluster_key { + if func.func_name == "add_noise" && matches!(cluster_type, AstClusterType::Hilbert) + { + debug_assert!(func.arguments.len() == 1); + cluster_key = func.arguments[0].clone(); + } + } + let expr = cluster_key.as_expr()?; if !expr.is_deterministic(&BUILTIN_FUNCTIONS) { return Err(ErrorCode::InvalidClusterKeys(format!( 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/constants.rs b/src/query/storages/fuse/src/constants.rs index 890153cb5f15b..9f56c913e5ba1 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_STATS_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/io/mod.rs b/src/query/storages/fuse/src/io/mod.rs index 63b43a9ff785f..93c695bd06f5c 100644 --- a/src/query/storages/fuse/src/io/mod.rs +++ b/src/query/storages/fuse/src/io/mod.rs @@ -50,8 +50,8 @@ pub use write::CachedMetaWriter; pub use write::InvertedIndexBuilder; pub use write::InvertedIndexWriter; pub use write::MetaWriter; -pub(crate) use write::StreamBlockBuilder; -pub(crate) use write::StreamBlockProperties; +pub use write::StreamBlockBuilder; +pub use write::StreamBlockProperties; pub use write::VirtualColumnBuilder; pub use write::WriteSettings; pub use write::MAX_BLOCK_UNCOMPRESSED_SIZE; 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 8b9b269327b5f..1264a757e94f8 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; @@ -124,6 +126,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, @@ -132,6 +135,11 @@ pub struct BlockSerialization { pub virtual_column_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/mod.rs b/src/query/storages/fuse/src/io/write/mod.rs index b0af3633055dc..24bf6fd52c042 100644 --- a/src/query/storages/fuse/src/io/write/mod.rs +++ b/src/query/storages/fuse/src/io/write/mod.rs @@ -35,8 +35,8 @@ pub(crate) use inverted_index_writer::InvertedIndexState; pub use inverted_index_writer::InvertedIndexWriter; pub use meta_writer::CachedMetaWriter; pub use meta_writer::MetaWriter; -pub(crate) use stream::StreamBlockBuilder; -pub(crate) use stream::StreamBlockProperties; +pub use stream::StreamBlockBuilder; +pub use stream::StreamBlockProperties; pub use virtual_column_builder::VirtualColumnBuilder; pub use write_settings::WriteSettings; pub use write_settings::MAX_BLOCK_UNCOMPRESSED_SIZE; 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 27eaf89c6616d..c125ae026d49f 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 @@ -28,6 +28,7 @@ use databend_common_expression::Column; use databend_common_expression::ColumnId; use databend_common_expression::ComputedExpr; use databend_common_expression::DataBlock; +use databend_common_expression::DataField; use databend_common_expression::FieldIndex; use databend_common_expression::TableField; use databend_common_expression::TableSchema; @@ -35,6 +36,8 @@ 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_native::write::NativeWriter; +use databend_common_sql::evaluator::BlockOperator; +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; @@ -235,7 +238,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) } @@ -355,8 +358,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, @@ -367,17 +370,24 @@ impl StreamBlockProperties { pub fn try_create( ctx: Arc, table: &FuseTable, + kind: MutationKind, + level: Option, 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, ..table.schema().as_ref().clone() @@ -400,19 +410,19 @@ impl StreamBlockProperties { let inverted_index_builders = create_inverted_index_builders(&table.table_info.meta); let cluster_stats_builder = - ClusterStatisticsBuilder::try_create(table, ctx.clone(), &source_schema)?; + ClusterStatisticsBuilder::try_create(table, ctx.clone(), &source_schema, level)?; let mut stats_columns = vec![]; let mut distinct_columns = vec![]; 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)); } } } @@ -432,4 +442,21 @@ impl StreamBlockProperties { table_meta_timestamps, })) } + + pub fn check_large_enough(&self, num_rows: usize, data_size: usize) -> bool { + self.block_thresholds + .check_large_enough(num_rows, data_size) + } + + pub fn cluster_operators(&self) -> Vec { + self.cluster_stats_builder.operators() + } + + pub fn fields_with_cluster_key(&self) -> Vec { + self.cluster_stats_builder.out_fields() + } + + pub fn cluster_key_index(&self) -> &Vec { + self.cluster_stats_builder.cluster_key_index() + } } diff --git a/src/query/storages/fuse/src/io/write/stream/cluster_statistics.rs b/src/query/storages/fuse/src/io/write/stream/cluster_statistics.rs index 84bba6b663db1..4c33ff80566db 100644 --- a/src/query/storages/fuse/src/io/write/stream/cluster_statistics.rs +++ b/src/query/storages/fuse/src/io/write/stream/cluster_statistics.rs @@ -20,9 +20,9 @@ use databend_common_exception::Result; use databend_common_expression::Column; use databend_common_expression::ColumnRef; use databend_common_expression::DataBlock; +use databend_common_expression::DataField; use databend_common_expression::DataSchema; use databend_common_expression::Expr; -use databend_common_expression::FunctionContext; use databend_common_expression::Scalar; use databend_common_expression::TableSchemaRef; use databend_common_functions::aggregates::eval_aggr; @@ -35,12 +35,13 @@ use crate::FuseTable; #[derive(Default, Clone)] pub struct ClusterStatisticsBuilder { - cluster_key_id: u32, + out_fields: Vec, + level: i32, + cluster_key_id: Option, cluster_key_index: Vec, extra_key_num: usize, operators: Vec, - func_ctx: FunctionContext, } impl ClusterStatisticsBuilder { @@ -48,6 +49,7 @@ impl ClusterStatisticsBuilder { table: &FuseTable, ctx: Arc, source_schema: &TableSchemaRef, + level: Option, ) -> Result> { let cluster_type = table.cluster_type(); if cluster_type.is_none_or(|v| v == ClusterType::Hilbert) { @@ -55,9 +57,9 @@ impl ClusterStatisticsBuilder { } let input_schema: Arc = DataSchema::from(source_schema).into(); - let input_filed_len = input_schema.fields.len(); + let mut out_fields = input_schema.fields().clone(); - let cluster_keys = table.linear_cluster_keys(ctx.clone()); + let cluster_keys = table.linear_cluster_keys(ctx); let mut cluster_key_index = Vec::with_capacity(cluster_keys.len()); let mut extra_key_num = 0; @@ -69,8 +71,11 @@ impl ClusterStatisticsBuilder { let index = match &expr { Expr::ColumnRef(ColumnRef { id, .. }) => *id, _ => { + let cname = format!("{}", expr); + out_fields.push(DataField::new(cname.as_str(), expr.data_type().clone())); exprs.push(expr); - let offset = input_filed_len + extra_key_num; + + let offset = out_fields.len() - 1; extra_key_num += 1; offset } @@ -87,17 +92,29 @@ impl ClusterStatisticsBuilder { }] }; Ok(Arc::new(Self { - cluster_key_id: table.cluster_key_meta.as_ref().unwrap().0, + cluster_key_id: table.cluster_key_id(), cluster_key_index, extra_key_num, - func_ctx: ctx.get_function_context()?, operators, + out_fields, + level: level.unwrap_or(0), })) } + + pub fn operators(&self) -> Vec { + self.operators.clone() + } + + pub fn out_fields(&self) -> Vec { + self.out_fields.clone() + } + + pub fn cluster_key_index(&self) -> &Vec { + &self.cluster_key_index + } } pub struct ClusterStatisticsState { - level: i32, mins: Vec, maxs: Vec, @@ -107,29 +124,23 @@ pub struct ClusterStatisticsState { impl ClusterStatisticsState { pub fn new(builder: Arc) -> Self { Self { - level: 0, mins: vec![], maxs: vec![], builder, } } - pub fn add_block(&mut self, input: DataBlock) -> Result { + pub fn add_block(&mut self, mut input: DataBlock) -> Result { if self.builder.cluster_key_index.is_empty() { return Ok(input); } let num_rows = input.num_rows(); - let mut block = self - .builder - .operators - .iter() - .try_fold(input, |input, op| op.execute(&self.builder.func_ctx, input))?; let cols = self .builder .cluster_key_index .iter() - .map(|&i| block.get_by_offset(i).to_column()) + .map(|&i| input.get_by_offset(i).to_column()) .collect(); let tuple = Column::Tuple(cols); let (min, _) = eval_aggr("min", vec![], &[tuple.clone()], num_rows, vec![])?; @@ -138,8 +149,8 @@ impl ClusterStatisticsState { assert_eq!(max.len(), 1); self.mins.push(min.index(0).unwrap().to_owned()); self.maxs.push(max.index(0).unwrap().to_owned()); - block.pop_columns(self.builder.extra_key_num); - Ok(block) + input.pop_columns(self.builder.extra_key_num); + Ok(input) } pub fn finalize(self, perfect: bool) -> Result> { @@ -167,14 +178,14 @@ impl ClusterStatisticsState { let level = if min == max && perfect { -1 } else { - self.level + self.builder.level }; Ok(Some(ClusterStatistics { max, min, level, - cluster_key_id: self.builder.cluster_key_id, + cluster_key_id: self.builder.cluster_key_id.unwrap(), pages: None, })) } 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 index 4c0e3cd715227..b0da462368914 100644 --- a/src/query/storages/fuse/src/io/write/stream/column_statistics.rs +++ b/src/query/storages/fuse/src/io/write/stream/column_statistics.rs @@ -12,19 +12,31 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::cmp::Ordering; +use std::collections::{HashMap, HashSet}; +use std::hash::Hash; +use std::marker::PhantomData; +use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::sampler::AlgoL; +use databend_common_expression::types::boolean::TrueIdxIter; use databend_common_expression::types::AccessType; +use databend_common_expression::types::ArrayType; +use databend_common_expression::types::BinaryColumn; +use databend_common_expression::types::BinaryType; 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::Decimal; +use databend_common_expression::types::Decimal128Type; +use databend_common_expression::types::Decimal256Type; 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::types::ValueType; +use databend_common_expression::{with_number_mapped_type, BlockRowIndex}; +use databend_common_expression::BlockEntry; use databend_common_expression::Column; use databend_common_expression::ColumnId; use databend_common_expression::DataBlock; @@ -32,30 +44,34 @@ 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_common_expression::SELECTIVITY_THRESHOLD; +use databend_storages_common_table_meta::meta::{ColumnDistinctHLL, Location}; use databend_storages_common_table_meta::meta::ColumnStatistics; use databend_storages_common_table_meta::meta::StatisticsOfColumns; - -use crate::statistics::reducers::reduce_column_statistics; +use rand::rngs::SmallRng; +use databend_common_expression::types::binary::BinaryColumnBuilder; use crate::statistics::traverse_values_dfs; use crate::statistics::Trim; pub struct ColumnStatisticsState { - col_stats: HashMap>, - distinct_columns: HashMap, + col_stats: HashMap>, + distinct_columns: HashMap>, + // cluster_key: Option, } impl ColumnStatisticsState { - pub fn new(stats_columns: &[ColumnId], distinct_columns: &[ColumnId]) -> Self { + pub fn new( + stats_columns: &[(ColumnId, DataType)], + distinct_columns: &[(ColumnId, DataType)], + ) -> Self { let col_stats = stats_columns .iter() - .map(|&col_id| (col_id, Vec::new())) + .map(|(col_id, data_type)| (*col_id, create_column_minmax_state(data_type))) .collect(); let distinct_columns = distinct_columns .iter() - .map(|&col_id| (col_id, ColumnDistinctHLL::default())) + .map(|(col_id, data_type)| (*col_id, create_estimator(data_type))) .collect(); Self { @@ -70,66 +86,23 @@ impl ColumnStatisticsState { 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, + self.col_stats.get_mut(&column_id).unwrap().update_scalar( + &s.as_ref(), + rows, + &data_type, ); - if let Some(hll) = self.distinct_columns.get_mut(&column_id) { - scalar_update_hll_cardinality(&s.as_ref(), &data_type, hll); + if let Some(estimator) = self.distinct_columns.get_mut(&column_id) { + estimator.update_scalar(&s.as_ref()); } - 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()], 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()], 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); - + 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(hll) = self.distinct_columns.get_mut(&column_id) { - column_update_hll_cardinality(&col, &data_type, hll); + if let Some(estimator) = self.distinct_columns.get_mut(&column_id) { + estimator.update_column(&col); } } } @@ -142,106 +115,453 @@ impl ColumnStatisticsState { 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) { + for (id, stats) in self.col_stats { + let mut col_stats = stats.finalize()?; + 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); + } else if let Some(estimator) = self.distinct_columns.get(&id) { + col_stats.distinct_of_values = Some(estimator.finalize()); } - statistics.insert(*id, col_stats); + 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); +#[derive(Debug)] +pub struct BlockStatisticsState { + pub(crate) data: Vec, + pub(crate) size: u64, + pub(crate) location: Location, +} + +pub struct ClusterStateSampler { + k: usize, + origins: Vec, + indices: Vec, + core: AlgoL, + + s: usize, +} + +impl ClusterStateSampler { + pub fn new(k: usize, rng: SmallRng) -> Self { + let core = AlgoL::new(k.try_into().unwrap(), rng); + Self { + origins: Vec::new(), + indices: Vec::with_capacity(k), + k, + core, + s: usize::MAX, + } + } + + pub fn add_column(&mut self, data: BinaryColumn) { + let rows = data.len(); + assert!(rows > 0); + let block_idx = self.origins.len() as u32; + let change = self.add_indices(rows, block_idx); + if change { + self.origins.push(data); + } + } + + fn add_indices(&mut self, rows: usize, block_idx: u32) -> bool { + let mut change = false; + let mut cur = 0; + + // Fill initial reservoir + if self.indices.len() < self.k { + let remain = self.k - self.indices.len(); + + if rows <= remain { + self.indices.extend((0..rows).map(|i| (block_idx, i as u32, 1))); + if self.indices.len() == self.k { + self.s = self.core.search(); + } + return true; } + + self.indices.extend((0..remain).map(|i| (block_idx, i as u32, 1))); + cur += remain; + self.s = self.core.search(); + change = true; } - return; + + // Apply AlgoL + while rows - cur > self.s { + cur += self.s; + let pos = self.core.pos(); + self.indices[pos] = (block_idx, cur as u32, 1); + self.core.update_w(); + self.s = self.core.search(); + change = true; + } + + self.s -= rows - cur; + change } - 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); + pub fn finalize(self) -> BlockEntry { + let columns = self.origins; + let mut builder = BinaryColumnBuilder::with_capacity(self.k, 0); + for (block_index, row, times) in self.indices { + let val = + unsafe { BinaryType::index_column_unchecked(&columns[block_index as usize], row as usize) }; + for _ in 0..times { + BinaryType::push_item(&mut builder, val.clone()) } } + BlockEntry::new_const_column_arg::>(builder.build(), 1) + } +} + +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_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 => { - let col = StringType::try_downcast_column(col).unwrap(); - for v in col.iter() { - hll.add_object(&v); - } + ColumnNDVEstimatorImpl::::create() } DataType::Date => { - let col = DateType::try_downcast_column(col).unwrap(); - for v in col.iter() { - hll.add_object(v); - } + ColumnNDVEstimatorImpl::::create() } DataType::Timestamp => { - let col = TimestampType::try_downcast_column(col).unwrap(); - for v in col.iter() { - hll.add_object(v); - } + ColumnNDVEstimatorImpl::::create() + } + DataType::Decimal(s) if s.can_carried_by_128() => { + ColumnNDVEstimatorImpl::::create() } DataType::Decimal(_) => { - match col { - 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); + 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) = 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 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); } } - _ => unreachable!(), - }; + } 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 + } +} + +pub trait ColumnMinMaxState: 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 trait MinMaxAdapter: 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); +} + +pub struct CommonAdapter; + +impl MinMaxAdapter 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); } - _ => unreachable!("Unsupported data type: {:?}", ty), - }); + } } -fn scalar_update_hll_cardinality(scalar: &ScalarRef, ty: &DataType, hll: &mut ColumnDistinctHLL) { - if matches!(scalar, ScalarRef::Null) { - return; +pub struct DecimalAdapter; + +impl MinMaxAdapter 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)) } - let ty = ty.remove_nullable(); + fn value_to_scalar(val: Self::Value) -> T::Scalar { + T::Scalar::from_u64_array(val) + } - with_number_mapped_type!(|NUM_TYPE| match ty { + 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)); + } + } +} + +pub fn create_column_minmax_state(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) => { - let val = NumberType::::try_downcast_scalar(scalar).unwrap(); - hll.add_object(&val); + GenericColumnMinMaxState::, CommonAdapter>::create(inner_type) } DataType::String => { - let val = StringType::try_downcast_scalar(scalar).unwrap(); - hll.add_object(&val); + GenericColumnMinMaxState::::create(inner_type) } DataType::Date => { - let val = DateType::try_downcast_scalar(scalar).unwrap(); - hll.add_object(&val); + GenericColumnMinMaxState::::create(inner_type) } DataType::Timestamp => { - let val = TimestampType::try_downcast_scalar(scalar).unwrap(); - hll.add_object(&val); + GenericColumnMinMaxState::::create(inner_type) + } + DataType::Decimal(s) if s.can_carried_by_128() => { + GenericColumnMinMaxState::::create(inner_type) } DataType::Decimal(_) => { - match scalar { - ScalarRef::Decimal(DecimalScalar::Decimal128(v, _)) => hll.add_object(&v), - ScalarRef::Decimal(DecimalScalar::Decimal256(v, _)) => hll.add_object(&v), - _ => unreachable!(), + GenericColumnMinMaxState::::create(inner_type) + } + _ => unreachable!("Unsupported data type: {:?}", data_type), + }) +} + +pub struct GenericColumnMinMaxState +where + T: ValueType, + A: MinMaxAdapter, +{ + min: Option, + max: Option, + null_count: usize, + in_memory_size: usize, + data_type: DataType, + + _phantom: PhantomData<(T, A)>, +} + +impl GenericColumnMinMaxState +where + T: ValueType + Send + Sync, + T::Scalar: Send + Sync, + A: MinMaxAdapter + 'static, + for<'a, 'b> T::ScalarRef<'a>: PartialOrd>, +{ + pub 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 ColumnMinMaxState for GenericColumnMinMaxState +where + T: ValueType + Send + Sync, + T::Scalar: Send + Sync, + A: MinMaxAdapter + 'static, + for<'a, 'b> T::ScalarRef<'a>: PartialOrd>, +{ + fn update_column(&mut self, column: &Column) { + self.in_memory_size += column.memory_size(); + 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) + }; + 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); } - _ => unreachable!("Unsupported data type: {:?}", ty), - }); + } + + 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/mod.rs b/src/query/storages/fuse/src/io/write/stream/mod.rs index 26d32ee679582..0c99368220ed4 100644 --- a/src/query/storages/fuse/src/io/write/stream/mod.rs +++ b/src/query/storages/fuse/src/io/write/stream/mod.rs @@ -16,5 +16,5 @@ mod block_builder; mod cluster_statistics; mod column_statistics; -pub(crate) use block_builder::StreamBlockBuilder; -pub(crate) use block_builder::StreamBlockProperties; +pub use block_builder::StreamBlockBuilder; +pub use block_builder::StreamBlockProperties; diff --git a/src/query/storages/fuse/src/operations/append.rs b/src/query/storages/fuse/src/operations/append.rs index 9316374128528..1d02e58b52f0b 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,24 +51,45 @@ 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 block_thresholds = self.get_block_thresholds(); + build_compact_block_pipeline(pipeline, block_thresholds)?; + + let enable_stream_block_write = + ctx.get_settings().get_enable_block_stream_write()? && self.storage_format_as_parquet(); if enable_stream_block_write { + let properties = StreamBlockProperties::try_create( + ctx.clone(), + self, + MutationKind::Insert, + None, + table_meta_timestamps, + )?; + + let cluster_key_index = properties.cluster_key_index(); + if !cluster_key_index.is_empty() { + let cluster_operators = properties.cluster_operators(); + if !cluster_operators.is_empty() { + let num_input_columns = self.table_info.schema().num_fields(); + let func_ctx = ctx.get_function_context()?; + pipeline.add_transformer(move || { + CompoundBlockOperator::new( + cluster_operators.clone(), + func_ctx.clone(), + num_input_columns, + ) + }); + } + } + pipeline.add_transform(|input, output| { - TransformBlockWriter::try_create( - ctx.clone(), - input, - output, - self, - table_meta_timestamps, - false, - ) + TransformBlockBuilder::try_create(input, output, properties.clone()) })?; - } else { - let block_thresholds = self.get_block_thresholds(); - build_compact_block_pipeline(pipeline, block_thresholds)?; - let schema = DataSchema::from(self.schema()).into(); + pipeline.add_async_accumulating_transformer(|| { + TransformBlockWriter::create(ctx.clone(), MutationKind::Insert, self, false) + }); + } else { + let schema = DataSchema::from(&self.schema().remove_virtual_computed_fields()).into(); let cluster_stats_gen = self.cluster_gen_for_append(ctx.clone(), pipeline, block_thresholds, Some(schema))?; pipeline.add_transform(|input, output| { @@ -100,7 +122,7 @@ impl FuseTable { let operators = cluster_stats_gen.operators.clone(); if !operators.is_empty() { - let num_input_columns = self.table_info.schema().fields().len(); + let num_input_columns = self.table_info.schema().num_fields(); let func_ctx2 = cluster_stats_gen.func_ctx.clone(); let mut builder = pipeline.try_create_transform_pipeline_builder_with_len( move || { @@ -159,7 +181,7 @@ impl FuseTable { let operators = cluster_stats_gen.operators.clone(); if !operators.is_empty() { - let num_input_columns = self.table_info.schema().fields().len(); + let num_input_columns = self.table_info.schema().num_fields(); let func_ctx2 = cluster_stats_gen.func_ctx.clone(); pipeline.add_transformer(move || { @@ -196,8 +218,9 @@ impl FuseTable { return Ok(ClusterStatsGenerator::default()); } - let input_schema = - modified_schema.unwrap_or(DataSchema::from(self.schema_with_stream()).into()); + let input_schema = modified_schema.unwrap_or( + DataSchema::from(&self.schema_with_stream().remove_virtual_computed_fields()).into(), + ); let mut merged = input_schema.fields().clone(); let cluster_keys = self.linear_cluster_keys(ctx.clone()); 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 7a3615233b8ce..ea5b4b9dc5c0c 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_size: usize, input_data: VecDeque, 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_size = 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_size, }))) } @@ -110,24 +95,12 @@ impl TransformBlockWriter { } Ok(self.builder.as_mut().unwrap()) } - - fn calc_max_block_size(&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_size; - } - let num_rows = block.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_size.min(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 +108,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); } @@ -160,7 +133,6 @@ impl Processor for TransformBlockWriter { if self.need_flush && self .properties - .block_thresholds .check_large_enough(self.input_num_rows, self.input_data_size) { self.state = State::Flush; @@ -198,9 +170,7 @@ impl Processor for TransformBlockWriter { 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_size(&block); - let blocks = block.split_by_rows_no_tail(max_rows_per_block); - self.input_data.extend(blocks); + self.input_data.push_back(block); } State::Serialize => { while let Some(b) = self.input_data.pop_front() { @@ -227,7 +197,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 +205,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 +251,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/operations/mutation/mutator/recluster_mutator.rs b/src/query/storages/fuse/src/operations/mutation/mutator/recluster_mutator.rs index 3c50cd9f83153..be706f9d123b9 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 @@ -72,6 +72,8 @@ pub struct ReclusterMutator { pub(crate) max_tasks: usize, pub(crate) cluster_key_types: Vec, pub(crate) column_ids: HashSet, + + average_size: usize, } impl ReclusterMutator { @@ -102,6 +104,13 @@ impl ReclusterMutator { // NOTE: The snapshot schema does not contain the stream column. let column_ids = snapshot.schema.to_leaf_column_id_set(); + let average_size = cmp::max( + snapshot + .summary + .uncompressed_byte_size + .div_ceil(snapshot.summary.block_count) as usize, + block_thresholds.min_bytes_per_block, + ); Ok(Self { ctx, schema, @@ -111,6 +120,7 @@ impl ReclusterMutator { max_tasks, cluster_key_types, column_ids, + average_size, }) } @@ -125,6 +135,7 @@ impl ReclusterMutator { cluster_key_id: u32, max_tasks: usize, column_ids: HashSet, + average_size: usize, ) -> Self { Self { ctx, @@ -135,6 +146,7 @@ impl ReclusterMutator { max_tasks, cluster_key_types, column_ids, + average_size, } } @@ -196,8 +208,7 @@ impl ReclusterMutator { .get_recluster_block_size()? .min(avail_memory_usage * 30 / 100) as usize; // specify a rather small value, so that `recluster_block_size` might be tuned to lower value. - let max_blocks_num = - (memory_threshold / self.block_thresholds.max_bytes_per_block).max(2) * self.max_tasks; + let mut max_blocks_per_task = (memory_threshold / self.average_size).max(2); let block_per_seg = self.block_thresholds.block_per_segment; // Prepare task generation parameters @@ -265,8 +276,11 @@ impl ReclusterMutator { } // Select blocks for reclustering based on depth threshold and max block size - let mut selected_idx = - self.fetch_max_depth(points_map, self.depth_threshold, max_blocks_num)?; + let mut selected_idx = self.fetch_max_depth( + points_map, + self.depth_threshold, + max_blocks_per_task * self.max_tasks, + )?; if selected_idx.is_empty() { if level != 0 || small_blocks.len() < 2 { continue; @@ -280,13 +294,19 @@ impl ReclusterMutator { let mut task_compressed = 0; let mut task_indices = Vec::new(); let mut selected_blocks = Vec::new(); + if selected_idx.len() > max_blocks_per_task { + max_blocks_per_task = selected_idx.len().div_ceil(self.max_tasks).max(10); + } for idx in selected_idx { let block = blocks[idx].clone(); let block_size = block.block_size as usize; let row_count = block.row_count as usize; + let selected_len = selected_blocks.len(); // If memory threshold exceeded, generate a new task and reset accumulators - if task_bytes + block_size > memory_threshold && selected_blocks.len() > 1 { + if selected_len > max_blocks_per_task + || (task_bytes + block_size > memory_threshold && selected_len > 1) + { selected_blocks_idx.extend(std::mem::take(&mut task_indices)); tasks.push(self.generate_task( diff --git a/src/query/storages/fuse/src/operations/recluster.rs b/src/query/storages/fuse/src/operations/recluster.rs index 4ce55ee9b9052..3cbf4cd1f922b 100644 --- a/src/query/storages/fuse/src/operations/recluster.rs +++ b/src/query/storages/fuse/src/operations/recluster.rs @@ -71,6 +71,10 @@ impl FuseTable { return Ok(None); }; + if snapshot.summary.block_count == 0 { + return Ok(None); + } + let mutator = Arc::new(ReclusterMutator::try_create( self, ctx.clone(), 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..176d17124c5d5 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 @@ -510,12 +510,12 @@ select segment_count, block_count from fuse_snapshot('db_09_0008', 't9') limit 2 2 2 query I -select a from t9 +select a from t9 order by a ---- +-5 1 2 4 --5 statement ok insert into t9 values(-3) diff --git a/tests/sqllogictests/suites/ee/07_hilbert_clustering/07_0000_recluster_final.test b/tests/sqllogictests/suites/ee/07_hilbert_clustering/07_0000_recluster_final.test index b3354e66740f6..4e0822f4589db 100644 --- a/tests/sqllogictests/suites/ee/07_hilbert_clustering/07_0000_recluster_final.test +++ b/tests/sqllogictests/suites/ee/07_hilbert_clustering/07_0000_recluster_final.test @@ -21,6 +21,9 @@ USE test_hilbert statement ok create or replace table t(a int, b int) cluster by hilbert(a, b) row_per_block=2 block_per_segment=2 block_size_threshold = 18; +statement ok +set enable_block_stream_write = 0 + statement ok set hilbert_clustering_min_bytes = 35; @@ -48,12 +51,12 @@ select * EXCLUDE(timestamp) from clustering_information('test_hilbert','t'); statement ok -alter table t recluster final; +alter table t recluster; query I select count() from fuse_snapshot('test_hilbert','t'); ---- -6 +5 query II select count(a), sum(a) from t; @@ -97,19 +100,19 @@ select * EXCLUDE(timestamp) from clustering_information('test_hilbert','t'); statement ok alter table t recluster final; -query T -select * EXCLUDE(timestamp) from clustering_information('test_hilbert','t'); +query II +select info:partial_segment_count, info:unclustered_segment_count from clustering_information('test_hilbert','t'); ---- -(b, a) hilbert {"partial_block_count":0,"partial_segment_count":0,"stable_block_count":5,"stable_segment_count":2,"total_block_count":5,"total_segment_count":2,"unclustered_block_count":0,"unclustered_segment_count":0} +0 0 ######################################################## # force eval as linear clustering by specify columns # ######################################################## query T -select * EXCLUDE(timestamp) from clustering_information('test_hilbert','t', 'a,b'); +select cluster_key, type, info:constant_block_count from clustering_information('test_hilbert','t', 'a,b'); ---- -(a, b) linear {"average_depth":1.4,"average_overlaps":0.4,"block_depth_histogram":{"00001":3,"00002":2},"constant_block_count":0,"total_block_count":5} +(a, b) linear 0 # column specified not exist statement error 1065 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/window.test b/tests/sqllogictests/suites/mode/standalone/explain/window.test index 9846febbd9819..11943d90e42eb 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/window.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/window.test @@ -59,7 +59,7 @@ digraph { 2 [ label = "DeserializeDataTransform" ] 3 [ label = "ShufflePartition(Window)" ] 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] + 5 [ label = "TransformPartitionCollect(Window)" ] 6 [ label = "Transform Window" ] 7 [ label = "Resize" ] 8 [ label = "SortPartialTransform" ] @@ -108,7 +108,7 @@ digraph { 2 [ label = "DeserializeDataTransform" ] 3 [ label = "ShufflePartition(Window)" ] 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] + 5 [ label = "TransformPartitionCollect(Window)" ] 6 [ label = "Transform Window" ] 7 [ label = "Resize" ] 8 [ label = "SortPartialTransform" ] @@ -429,7 +429,7 @@ digraph { 2 [ label = "DeserializeDataTransform" ] 3 [ label = "ShufflePartition(Window)" ] 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] + 5 [ label = "TransformPartitionCollect(Window)" ] 6 [ label = "Transform Window" ] 7 [ label = "LimitTransform" ] 8 [ label = "CompoundBlockOperator(Project)" ] @@ -457,7 +457,7 @@ digraph { 2 [ label = "DeserializeDataTransform" ] 3 [ label = "ShufflePartition(Window)" ] 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] + 5 [ label = "TransformPartitionCollect(Window)" ] 6 [ label = "Transform Window" ] 7 [ label = "LimitTransform" ] 8 [ label = "CompoundBlockOperator(Project)" ] @@ -486,7 +486,7 @@ digraph { 2 [ label = "DeserializeDataTransform" ] 3 [ label = "ShufflePartition(Window)" ] 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] + 5 [ label = "TransformPartitionCollect(Window)" ] 6 [ label = "Transform Window" ] 7 [ label = "LimitTransform" ] 8 [ label = "CompoundBlockOperator(Project)" ] @@ -510,7 +510,7 @@ digraph { 2 [ label = "DeserializeDataTransform" ] 3 [ label = "ShufflePartition(Window)" ] 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] + 5 [ label = "TransformPartitionCollect(Window)" ] 6 [ label = "Transform Window" ] 7 [ label = "LimitTransform" ] 8 [ label = "CompoundBlockOperator(Project)" ] @@ -534,7 +534,7 @@ digraph { 2 [ label = "DeserializeDataTransform" ] 3 [ label = "ShufflePartition(Window)" ] 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] + 5 [ label = "TransformPartitionCollect(Window)" ] 6 [ label = "Transform Window" ] 7 [ label = "LimitTransform" ] 8 [ label = "CompoundBlockOperator(Project)" ] @@ -559,7 +559,7 @@ digraph { 2 [ label = "DeserializeDataTransform" ] 3 [ label = "ShufflePartition(Window)" ] 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] + 5 [ label = "TransformPartitionCollect(Window)" ] 6 [ label = "Transform Window" ] 7 [ label = "Resize" ] 8 [ label = "SortPartialTransform" ] @@ -610,7 +610,7 @@ digraph { 4 [ label = "TransformFilter" ] 5 [ label = "ShufflePartition(Window)" ] 6 [ label = "ShuffleMergePartition(Window)" ] - 7 [ label = "TransformWindowPartitionCollect(Sort)" ] + 7 [ label = "TransformPartitionCollect(Window)" ] 8 [ label = "Transform Window" ] 9 [ label = "Resize" ] 10 [ label = "SortPartialTransform" ] @@ -708,7 +708,7 @@ digraph { 2 [ label = "DeserializeDataTransform" ] 3 [ label = "ShufflePartition(WindowTopN)" ] 4 [ label = "ShuffleMergePartition(WindowTopN)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] + 5 [ label = "TransformPartitionCollect(Window)" ] 6 [ label = "Transform Window" ] 7 [ label = "TransformFilter" ] 8 [ label = "LimitTransform" ] @@ -786,12 +786,12 @@ digraph { 1 [ label = "CompoundBlockOperator(Map)" ] 2 [ label = "ShufflePartition(Window)" ] 3 [ label = "ShuffleMergePartition(Window)" ] - 4 [ label = "TransformWindowPartitionCollect(Sort)" ] + 4 [ label = "TransformPartitionCollect(Window)" ] 5 [ label = "Transform Window" ] 6 [ label = "CompoundBlockOperator(Map)" ] 7 [ label = "ShufflePartition(Window)" ] 8 [ label = "ShuffleMergePartition(Window)" ] - 9 [ label = "TransformWindowPartitionCollect(Sort)" ] + 9 [ label = "TransformPartitionCollect(Window)" ] 10 [ label = "Transform Window" ] 11 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ]