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/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 31c4eaba5a784..b561da8c846e9 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 49ff235c6ed3b..eddc249008d2f 100644 --- a/src/query/service/src/interpreters/interpreter_table_recluster.rs +++ b/src/query/service/src/interpreters/interpreter_table_recluster.rs @@ -28,6 +28,7 @@ use databend_common_catalog::table::TableExt; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_function; +use databend_common_expression::types::NumberScalar; use databend_common_expression::DataBlock; use databend_common_expression::Scalar; use databend_common_functions::BUILTIN_FUNCTIONS; @@ -52,6 +53,8 @@ use databend_common_sql::plans::plan_hilbert_sql; use databend_common_sql::plans::replace_with_constant; use databend_common_sql::plans::set_update_stream_columns; use databend_common_sql::plans::BoundColumnRef; +use databend_common_sql::plans::ConstantExpr; +use databend_common_sql::plans::FunctionCall; use databend_common_sql::plans::Plan; use databend_common_sql::plans::ReclusterPlan; use databend_common_sql::IdentifierNormalizer; @@ -320,24 +323,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: {}", @@ -439,22 +433,44 @@ impl ReclusterTableInterpreter { // For distributed execution, add an exchange operator to distribute work if is_distributed { + let nodes_num = cluster.nodes.len() as u64; + let scalar_expr = ScalarExpr::FunctionCall(FunctionCall { + span: None, + func_name: "div".to_string(), + params: vec![], + arguments: vec![ + ScalarExpr::FunctionCall(FunctionCall { + span: None, + func_name: "multiply".to_string(), + params: vec![], + arguments: vec![ + ScalarExpr::BoundColumnRef(BoundColumnRef { + span: None, + column: bind_context.columns.last().unwrap().clone(), + }), + ScalarExpr::ConstantExpr(ConstantExpr { + span: None, + value: Scalar::Number(NumberScalar::UInt64(nodes_num)), + }), + ], + }), + ScalarExpr::ConstantExpr(ConstantExpr { + span: None, + value: Scalar::Number(NumberScalar::UInt64(total_partitions as u64)), + }), + ], + }); + // Create an expression for the partition column, // i.e.`range_partition_id(hilbert_range_index({hilbert_keys_str}), [...]) AS _predicate` - let expr = scalar_expr_to_remote_expr( - &ScalarExpr::BoundColumnRef(BoundColumnRef { - span: None, - column: bind_context.columns.last().unwrap().clone(), - }), - plan.output_schema()?.as_ref(), - )?; + let expr = scalar_expr_to_remote_expr(&scalar_expr, plan.output_schema()?.as_ref())?; // Add exchange operator for data distribution, // shuffling data based on the hash of range partition IDs derived from the Hilbert index. plan = Box::new(PhysicalPlan::Exchange(Exchange { plan_id: 0, input: plan, - kind: FragmentKind::Normal, + kind: FragmentKind::Modulo, keys: vec![expr], allow_adjust_parallelism: true, ignore_exchange: false, @@ -471,8 +487,10 @@ impl ReclusterTableInterpreter { plan_id: 0, input: plan, table_info: table_info.clone(), - num_partitions: total_partitions, + range_start: 0, + range_width: total_partitions, table_meta_timestamps, + bytes_per_block, rows_per_block, })); @@ -613,7 +631,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 +652,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..aebafaa53566d 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.range_start, partition.range_width), ); 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.range_width, + 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.range_width, + 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..853ccbc2b2fef 100644 --- a/src/query/service/src/pipelines/builders/builder_recluster.rs +++ b/src/query/service/src/pipelines/builders/builder_recluster.rs @@ -12,29 +12,60 @@ // 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::row::RowConverter as CommonConverter; +use databend_common_expression::types::AccessType; +use databend_common_expression::types::ArgType; +use databend_common_expression::types::DataType; +use databend_common_expression::types::DateType; +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::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::CommonRows; +use databend_common_pipeline_transforms::sort::RowConverter; +use databend_common_pipeline_transforms::sort::Rows; +use databend_common_pipeline_transforms::sort::SimpleRowConverter; +use databend_common_pipeline_transforms::sort::SimpleRowsAsc; 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 match_template::match_template; use crate::pipelines::builders::SortPipelineBuilder; +use crate::pipelines::processors::transforms::ReclusterPartitionExchange; +use crate::pipelines::processors::transforms::SampleState; +use crate::pipelines::processors::transforms::TransformAddOrderColumn; use crate::pipelines::processors::transforms::TransformAddStreamColumns; +use crate::pipelines::processors::transforms::TransformRangePartitionIndexer; +use crate::pipelines::processors::transforms::TransformReclusterCollect; +use crate::pipelines::processors::transforms::TransformReclusterPartition; use crate::pipelines::PipelineBuilder; impl PipelineBuilder { @@ -71,7 +102,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 +148,154 @@ 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 fields_with_cluster_key = properties.fields_with_cluster_key(); + let schema = DataSchemaRefExt::create(fields_with_cluster_key); + let sort_descs: Vec<_> = properties + .cluster_key_index() + .iter() + .map(|&offset| SortColumnDescription { + offset, + asc: true, + nulls_first: false, + }) + .collect(); + + 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_descs.into(), sample_size) + .with_state(state); + recluster_pipeline_builder + .build_recluster_sample_pipeline(&mut self.main_pipeline)?; + + self.main_pipeline.exchange( + num_processors, + ReclusterPartitionExchange::create(0, partitions), + ); + let processor_id = AtomicUsize::new(0); + self.main_pipeline.add_transform(|input, output| { + TransformReclusterPartition::try_create( + input, + output, + properties.clone(), + processor_id.fetch_add(1, atomic::Ordering::AcqRel), + num_processors, + partitions, ) - }); - } + })?; - // 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_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.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 +303,89 @@ impl PipelineBuilder { } } } + +struct ReclusterPipelineBuilder { + schema: DataSchemaRef, + sort_desc: Arc<[SortColumnDescription]>, + state: Option>, + sample_size: usize, + seed: u64, +} + +impl ReclusterPipelineBuilder { + fn create( + schema: DataSchemaRef, + sort_desc: Arc<[SortColumnDescription]>, + 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<()> { + match self.sort_desc.as_ref() { + [desc] => { + let schema = self.schema.clone(); + let sort_type = schema.field(desc.offset).data_type(); + assert!(desc.asc); + + match_template! { + T = [ Date => DateType, Timestamp => TimestampType, String => StringType ], + match sort_type { + DataType::T => { + self.visit_type::, SimpleRowConverter>(pipeline) + }, + DataType::Number(num_ty) => with_number_mapped_type!(|NUM_TYPE| match num_ty { + NumberDataType::NUM_TYPE => { + self.visit_type::>, SimpleRowConverter>>(pipeline) + } + }), + _ => self.visit_type::(pipeline) + } + } + } + _ => self.visit_type::(pipeline), + } + } + + fn visit_type(&self, pipeline: &mut Pipeline) -> Result<()> + where + R: Rows + 'static, + C: RowConverter + Send + 'static, + R::Type: ArgType + Send + Sync, + ::Scalar: Ord + Send + Sync, + { + pipeline.try_add_transformer(|| { + TransformAddOrderColumn::::try_new(self.sort_desc.clone(), self.schema.clone()) + })?; + let offset = self.schema.num_fields(); + pipeline.add_accumulating_transformer(|| { + TransformReclusterCollect::::new(offset, self.sample_size, self.seed) + }); + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(TransformRangePartitionIndexer::< + R::Type, + >::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/aggregator/aggregate_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs index 55688a4347259..40904ea2c8e16 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs @@ -233,6 +233,7 @@ impl ExchangeInjector for AggregateInjector { match exchange { DataExchange::Merge(_) => unreachable!(), DataExchange::Broadcast(_) => unreachable!(), + DataExchange::Modulo(_) => unreachable!(), DataExchange::ShuffleDataExchange(exchange) => { Ok(Arc::new(Box::new(HashTableHashScatter { buckets: exchange.destination_ids.len(), 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..a76417256416f --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/range_bound_sampler.rs @@ -0,0 +1,81 @@ +// 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::marker::PhantomData; + +use databend_common_expression::types::ArgType; +use databend_common_expression::types::ValueType; +use databend_common_expression::DataBlock; +use databend_common_expression::Scalar; +use rand::prelude::SliceRandom; +use rand::prelude::SmallRng; +use rand::SeedableRng; + +pub struct RangeBoundSampler +where T: ValueType +{ + offset: usize, + sample_size: usize, + rng: SmallRng, + + values: Vec<(u64, Vec)>, + _t: PhantomData, +} + +impl RangeBoundSampler +where T: ValueType +{ + 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![], + _t: PhantomData, + } + } +} + +impl RangeBoundSampler +where + T: ArgType, + T::Scalar: Ord + Send, +{ + 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(rows); + + 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 = T::try_downcast_column(&column).unwrap(); + let sample_values = sampled_indices + .iter() + .map(|i| { + T::upcast_scalar(T::to_owned_scalar(unsafe { + T::index_column_unchecked(&column, *i) + })) + }) + .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 55% 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..dd5257850ac9f 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,10 +12,6 @@ // 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 databend_common_exception::Result; @@ -24,20 +20,20 @@ use databend_common_pipeline_core::processors::Exchange; use crate::pipelines::processors::transforms::WindowPartitionMeta; -pub struct HilbertPartitionExchange { - num_partitions: usize, +pub struct ReclusterPartitionExchange { + start: u64, + width: usize, } -impl HilbertPartitionExchange { - pub fn create(num_partitions: usize) -> Arc { - Arc::new(HilbertPartitionExchange { num_partitions }) +impl ReclusterPartitionExchange { + pub fn create(start: u64, width: usize) -> Arc { + Arc::new(ReclusterPartitionExchange { start, width }) } } -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() @@ -48,20 +44,19 @@ impl Exchange for HilbertPartitionExchange { // Scatter the data block to different partitions. let indices = range_ids .iter() - .map(|&id| (id % self.num_partitions as u64) as u16) + .map(|&id| (id - self.start) as u16) .collect::>(); data_block.pop_columns(1); - let scatter_indices = - DataBlock::divide_indices_by_scatter_size(&indices, self.num_partitions); + + let scatter_indices = DataBlock::divide_indices_by_scatter_size(&indices, self.width); // 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() - { - if indices.is_empty() { - continue; + for (partition_id, indices) in scatter_indices.into_iter().take(self.width).enumerate() { + if !indices.is_empty() { + let target = (partition_id * n) / self.width; + let block = data_block.take_with_optimize_size(&indices)?; + output_data_blocks[target].push((partition_id, block)); } - let block = data_block.take_with_optimize_size(indices)?; - output_data_blocks[partition_id % n].push((partition_id, block)); } // Union data blocks for each processor. 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..269aae2e3abea --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/recluster_partition_strategy.rs @@ -0,0 +1,177 @@ +// 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 } + } + + fn concat_blocks(blocks: Vec) -> Result { + DataBlock::concat(&blocks) + } +} + +impl PartitionProcessStrategy for ReclusterPartitionStrategy { + const NAME: &'static str = "Recluster"; + + fn calc_partitions( + &self, + processor_id: usize, + num_processors: usize, + num_partitions: usize, + ) -> Vec { + (0..num_partitions) + .filter(|&partition| (partition * num_processors) / num_partitions == processor_id) + .collect() + } + + /// 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(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() { + compacted.push(Self::concat_blocks(std::mem::take(&mut staged_blocks))?); + } + + let mut result = Vec::new(); + let mut builder = StreamBlockBuilder::try_new_with_config(self.properties.clone())?; + for block in compacted { + 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"; + + fn calc_partitions( + &self, + processor_id: usize, + num_processors: usize, + num_partitions: usize, + ) -> Vec { + (0..num_partitions) + .filter(|&partition| (partition * num_processors) / num_partitions == processor_id) + .collect() + } + + /// 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..12d50653b8b68 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/recluster_sample_state.rs @@ -0,0 +1,156 @@ +// 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 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::ArgType; +use databend_common_expression::Scalar; + +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<()> + where + T: ArgType, + T::Scalar: Ord, + { + 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) + where + T: ArgType, + T::Scalar: Ord, + { + let inner = self.inner.read().unwrap(); + let bounds = inner + .bounds + .iter() + .map(|v| T::to_owned_scalar(T::try_downcast_scalar(&v.as_ref()).unwrap())) + .collect(); + let max_value = inner + .max_value + .as_ref() + .map(|v| T::to_owned_scalar(T::try_downcast_scalar(&v.as_ref()).unwrap())); + (bounds, max_value) + } +} + +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<()> + where + T: ArgType, + T::Scalar: Ord, + { + 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| { + let val = T::to_owned_scalar(T::try_downcast_scalar(&v.as_ref()).unwrap()); + data.push(val); + weights.push(weight); + }); + } + let col = T::upcast_column(T::column_from_vec(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].clone(); + + 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 weight = weights[idx]; + cum_weight += weight; + if cum_weight >= target { + let data = &data[idx]; + if previous_bound.as_ref().is_none_or(|prev| data > prev) { + if data == &max_val { + self.max_value = Some(T::upcast_scalar(max_val)); + break; + } + + let bound = T::upcast_scalar(data.clone()); + bounds.push(bound); + target += step; + j += 1; + previous_bound = Some(data.clone()); + } + } + 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..7b40593e887c3 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/transform_add_order_column.rs @@ -0,0 +1,72 @@ +// 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::marker::PhantomData; +use std::sync::Arc; + +use databend_common_exception::Result; +use databend_common_expression::BlockEntry; +use databend_common_expression::DataBlock; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::SortColumnDescription; +use databend_common_expression::Value; +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: C, + sort_desc: Arc<[SortColumnDescription]>, + _r: PhantomData, +} + +impl TransformAddOrderColumn +where + R: Rows, + C: RowConverter, +{ + pub fn try_new(sort_desc: Arc<[SortColumnDescription]>, schema: DataSchemaRef) -> Result { + let row_converter = C::create(&sort_desc, schema.clone())?; + Ok(Self { + row_converter, + sort_desc, + _r: PhantomData, + }) + } +} + +impl Transform for TransformAddOrderColumn +where + R: Rows + 'static, + C: RowConverter + Send + 'static, +{ + 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(BlockEntry { + data_type: order_col.data_type(), + value: Value::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..215e0c977eb8f --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/transform_range_partition_indexer.rs @@ -0,0 +1,173 @@ +// 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 databend_common_exception::Result; +use databend_common_expression::types::ArgType; +use databend_common_expression::types::DataType; +use databend_common_expression::types::NumberDataType; +use databend_common_expression::types::UInt64Type; +use databend_common_expression::BlockEntry; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_expression::FromData; +use databend_common_expression::Value; +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 +where T: ArgType +{ + input: Arc, + output: Arc, + + state: Arc, + input_data: Vec, + output_data: VecDeque, + bounds: Vec, + max_value: Option, +} + +impl TransformRangePartitionIndexer +where + T: ArgType + Send + Sync, + T::Scalar: Ord + Send + Sync, +{ + 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, + }) + } +} + +#[async_trait::async_trait] +impl Processor for TransformRangePartitionIndexer +where + T: ArgType + Send + Sync, + T::Scalar: Ord + Send + Sync, +{ + 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)?; + Ok(Event::Async) + } + + fn process(&mut self) -> Result<()> { + if let Some(mut block) = self.input_data.pop() { + let bound_len = self.bounds.len(); + let num_rows = block.num_rows(); + let last = block.get_last_column().clone(); + block.pop_columns(1); + let mut builder = Vec::with_capacity(num_rows); + let last_col = T::try_downcast_column(&last.remove_nullable()).unwrap(); + for index in 0..num_rows { + let val = + T::to_owned_scalar(unsafe { T::index_column_unchecked(&last_col, index) }); + if self.max_value.as_ref().is_some_and(|v| val >= *v) { + let range_id = bound_len + 1; + builder.push(range_id as u64); + continue; + } + + let mut low = 0; + let mut high = bound_len; + while low < high { + let mid = low + ((high - low) / 2); + let bound = unsafe { self.bounds.get_unchecked(mid) }.clone(); + if val > bound { + low = mid + 1; + } else { + high = mid; + } + } + builder.push(low as u64); + } + + block.add_column(BlockEntry::new( + DataType::Number(NumberDataType::UInt64), + Value::Column(UInt64Type::from_data(builder)), + )); + self.output_data.push_back(block); + } + 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..46684b42b31e3 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/transform_recluster_collect.rs @@ -0,0 +1,81 @@ +// 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::types::ArgType; +use databend_common_expression::BlockMetaInfo; +use databend_common_expression::DataBlock; +use databend_common_expression::Scalar; +use databend_common_pipeline_transforms::AccumulatingTransform; + +use crate::pipelines::processors::transforms::RangeBoundSampler; + +pub struct TransformReclusterCollect +where + T: ArgType + Send + Sync, + T::Scalar: Ord + Send, +{ + input_data: Vec, + sampler: RangeBoundSampler, +} + +impl TransformReclusterCollect +where + T: ArgType + Send + Sync, + T::Scalar: Ord + Send, +{ + 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 +where + T: ArgType + Send + Sync, + T::Scalar: Ord + Send, +{ + 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..9bec274cb90f6 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/recluster/transform_recluster_partition.rs @@ -0,0 +1,230 @@ +// 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 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, + + 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, + }, + ))) + } +} + +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() { + 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 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; + } + } + } + } + } + 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..bec3f8a84e91f 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,21 @@ 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; - } + /// Partition assignment: map partition index to processor via proportional mapping. + fn calc_partitions( + &self, + processor_id: usize, + num_processors: usize, + num_partitions: usize, + ) -> Vec; - 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 +42,7 @@ pub struct SortStrategy { have_order_col: bool, } -impl SortStrategy { +impl WindowPartitionStrategy { pub fn try_create( settings: &Settings, sort_desc: Vec, @@ -114,8 +63,19 @@ impl SortStrategy { } } -impl DataProcessorStrategy for SortStrategy { - const NAME: &'static str = "Sort"; +impl PartitionProcessStrategy for WindowPartitionStrategy { + const NAME: &'static str = "Window"; + + fn calc_partitions( + &self, + processor_id: usize, + num_processors: usize, + num_partitions: usize, + ) -> Vec { + (0..num_partitions) + .filter(|&partition| partition % num_processors == processor_id) + .collect() + } 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..d1f011404223b 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, @@ -92,9 +92,7 @@ impl TransformWindowPartitionCollect { strategy: S, ) -> Result { // Calculate the partition ids collected by the processor. - let partitions: Vec = (0..num_partitions) - .filter(|&partition| partition % num_processors == processor_id) - .collect(); + let partitions = strategy.calc_partitions(processor_id, num_processors, num_partitions); // Map each partition id to new partition id. let mut partition_id = vec![0; num_partitions]; @@ -162,11 +160,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 +203,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 +265,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..dc267c896dc17 100644 --- a/src/query/service/src/schedulers/fragments/fragmenter.rs +++ b/src/query/service/src/schedulers/fragments/fragmenter.rs @@ -15,6 +15,7 @@ use std::sync::Arc; use databend_common_catalog::table_context::TableContext; +use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_meta_types::NodeInfo; use databend_common_sql::executor::physical_plans::CompactSource; @@ -26,6 +27,7 @@ use databend_common_sql::executor::physical_plans::ExchangeSink; use databend_common_sql::executor::physical_plans::ExchangeSource; use databend_common_sql::executor::physical_plans::FragmentKind; use databend_common_sql::executor::physical_plans::HashJoin; +use databend_common_sql::executor::physical_plans::HilbertPartition; use databend_common_sql::executor::physical_plans::MutationSource; use databend_common_sql::executor::physical_plans::Recluster; use databend_common_sql::executor::physical_plans::ReplaceInto; @@ -39,9 +41,9 @@ use crate::schedulers::PlanFragment; use crate::servers::flight::v1::exchange::BroadcastExchange; use crate::servers::flight::v1::exchange::DataExchange; use crate::servers::flight::v1::exchange::MergeExchange; +use crate::servers::flight::v1::exchange::ModuloExchange; 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. @@ -67,6 +69,7 @@ enum State { Compact, Recluster, Other, + HilbertRecluster, } impl Fragmenter { @@ -115,6 +118,15 @@ impl Fragmenter { FragmentKind::Expansive => { Ok(Some(BroadcastExchange::create(Self::get_executors(ctx)))) } + FragmentKind::Modulo => { + if plan.keys.len() != 1 { + return Err(ErrorCode::Internal("Modulo exchange require one key")); + } + Ok(Some(ModuloExchange::create( + Self::get_executors(ctx), + plan.keys[0].clone(), + ))) + } _ => Ok(None), }, _ => Ok(None), @@ -170,14 +182,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; @@ -209,6 +213,15 @@ impl PhysicalPlanReplacer for Fragmenter { Ok(PhysicalPlan::Recluster(Box::new(plan.clone()))) } + fn replace_hilbert_serialize(&mut self, plan: &HilbertPartition) -> Result { + let input = self.replace(&plan.input)?; + self.state = State::HilbertRecluster; + Ok(PhysicalPlan::HilbertPartition(Box::new(HilbertPartition { + input: Box::new(input), + ..plan.clone() + }))) + } + fn replace_compact_source(&mut self, plan: &CompactSource) -> Result { self.state = State::Compact; Ok(PhysicalPlan::CompactSource(Box::new(plan.clone()))) @@ -310,6 +323,7 @@ impl PhysicalPlanReplacer for Fragmenter { State::ReplaceInto => FragmentType::ReplaceInto, State::Compact => FragmentType::Compact, State::Recluster => FragmentType::Recluster, + State::HilbertRecluster => FragmentType::HilbertRecluster, }; self.state = State::Other; let exchange = Self::get_exchange(self.ctx.clone(), &plan)?; diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index 9cec8f558c989..e5f3502108d91 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -29,6 +29,7 @@ use databend_common_sql::executor::physical_plans::CompactSource; use databend_common_sql::executor::physical_plans::ConstantTableScan; use databend_common_sql::executor::physical_plans::CopyIntoTable; use databend_common_sql::executor::physical_plans::CopyIntoTableSource; +use databend_common_sql::executor::physical_plans::HilbertPartition; use databend_common_sql::executor::physical_plans::MutationSource; use databend_common_sql::executor::physical_plans::Recluster; use databend_common_sql::executor::physical_plans::ReplaceDeduplicate; @@ -65,6 +66,7 @@ pub enum FragmentType { Compact, Recluster, MutationSource, + HilbertRecluster, } #[derive(Clone)] @@ -137,6 +139,9 @@ impl PlanFragment { FragmentType::Recluster => { self.redistribute_recluster(ctx, &mut fragment_actions)?; } + FragmentType::HilbertRecluster => { + self.redistribute_hilbert(ctx, &mut fragment_actions)?; + } } if let Some(ref exchange) = self.exchange { @@ -381,6 +386,40 @@ impl PlanFragment { Ok(()) } + fn redistribute_hilbert( + &self, + ctx: Arc, + fragment_actions: &mut QueryFragmentActions, + ) -> Result<()> { + let exchange_sink = match &self.plan { + PhysicalPlan::ExchangeSink(plan) => plan, + _ => unreachable!("logic error"), + }; + let hilbert = match exchange_sink.input.as_ref() { + PhysicalPlan::HilbertPartition(plan) => plan, + _ => unreachable!("logic error"), + }; + + let total_ranges = hilbert.range_width; + let executors = Fragmenter::get_executors(ctx); + let num_executors = executors.len(); + let base_width = total_ranges / num_executors; + let remainder = total_ranges % num_executors; + for (executor_idx, executor) in executors.into_iter().enumerate() { + let width = base_width + if executor_idx < remainder { 1 } else { 0 }; + let min = executor_idx * base_width + std::cmp::min(executor_idx, remainder); + let mut plan = self.plan.clone(); + let mut replace_hilbert = ReplaceHilbert { + range_width: width, + range_start: min as u64, + }; + plan = replace_hilbert.replace(&plan)?; + fragment_actions.add_action(QueryFragmentAction::create(executor, plan)); + } + + Ok(()) + } + fn reshuffle( executors: Vec, partitions: Vec, @@ -556,8 +595,25 @@ impl PhysicalPlanReplacer for ReplaceReadSource { } } +struct ReplaceHilbert { + range_width: usize, + range_start: u64, +} + +impl PhysicalPlanReplacer for ReplaceHilbert { + fn replace_hilbert_serialize(&mut self, plan: &HilbertPartition) -> Result { + let input = self.replace(&plan.input)?; + Ok(PhysicalPlan::HilbertPartition(Box::new(HilbertPartition { + input: Box::new(input), + range_width: self.range_width, + range_start: self.range_start, + ..plan.clone() + }))) + } +} + struct ReplaceRecluster { - pub tasks: Vec, + tasks: Vec, } impl PhysicalPlanReplacer for ReplaceRecluster { @@ -570,7 +626,7 @@ impl PhysicalPlanReplacer for ReplaceRecluster { } struct ReplaceMutationSource { - pub partitions: Partitions, + partitions: Partitions, } impl PhysicalPlanReplacer for ReplaceMutationSource { @@ -583,7 +639,7 @@ impl PhysicalPlanReplacer for ReplaceMutationSource { } struct ReplaceCompactBlock { - pub partitions: Partitions, + partitions: Partitions, } impl PhysicalPlanReplacer for ReplaceCompactBlock { @@ -596,10 +652,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/schedulers/fragments/query_fragment_actions_display.rs b/src/query/service/src/schedulers/fragments/query_fragment_actions_display.rs index adb0b6c3bcd18..36d8f0c257eb1 100644 --- a/src/query/service/src/schedulers/fragments/query_fragment_actions_display.rs +++ b/src/query/service/src/schedulers/fragments/query_fragment_actions_display.rs @@ -72,6 +72,7 @@ impl Display for QueryFragmentActionsWrap<'_> { DataExchange::Merge(_) => writeln!(f, " DataExchange: Merge")?, DataExchange::Broadcast(_) => writeln!(f, " DataExchange: Broadcast")?, DataExchange::ShuffleDataExchange(_) => writeln!(f, " DataExchange: Shuffle")?, + DataExchange::Modulo(_) => writeln!(f, " DataExchange: Modulo")?, } } diff --git a/src/query/service/src/servers/flight/v1/exchange/data_exchange.rs b/src/query/service/src/servers/flight/v1/exchange/data_exchange.rs index f23c7582559a7..0fba30c72ec7b 100644 --- a/src/query/service/src/servers/flight/v1/exchange/data_exchange.rs +++ b/src/query/service/src/servers/flight/v1/exchange/data_exchange.rs @@ -19,6 +19,7 @@ pub enum DataExchange { Merge(MergeExchange), Broadcast(BroadcastExchange), ShuffleDataExchange(ShuffleDataExchange), + Modulo(ModuloExchange), } impl DataExchange { @@ -27,6 +28,7 @@ impl DataExchange { DataExchange::Merge(exchange) => vec![exchange.destination_id.clone()], DataExchange::Broadcast(exchange) => exchange.destination_ids.clone(), DataExchange::ShuffleDataExchange(exchange) => exchange.destination_ids.clone(), + DataExchange::Modulo(exchange) => exchange.destination_ids.clone(), } } } @@ -77,3 +79,18 @@ impl BroadcastExchange { DataExchange::Broadcast(BroadcastExchange { destination_ids }) } } + +#[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] +pub struct ModuloExchange { + pub destination_ids: Vec, + pub shuffle_key: RemoteExpr, +} + +impl ModuloExchange { + pub fn create(destination_ids: Vec, shuffle_key: RemoteExpr) -> DataExchange { + DataExchange::Modulo(ModuloExchange { + destination_ids, + shuffle_key, + }) + } +} diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs index 4aa65ba175a83..5b10b4f346960 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs @@ -29,6 +29,7 @@ use crate::servers::flight::v1::exchange::ShuffleExchangeParams; use crate::servers::flight::v1::scatter::BroadcastFlightScatter; use crate::servers::flight::v1::scatter::FlightScatter; use crate::servers::flight::v1::scatter::HashFlightScatter; +use crate::servers::flight::v1::scatter::ModFlightScatter; use crate::sessions::QueryContext; pub trait ExchangeInjector: Send + Sync + 'static { @@ -100,6 +101,11 @@ impl ExchangeInjector for DefaultExchangeInjector { local_pos, )? } + DataExchange::Modulo(exchange) => ModFlightScatter::try_create( + ctx.get_function_context()?, + &exchange.shuffle_key, + exchange.destination_ids.len(), + )?, })) } diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs index 64b0e23c23617..9790b7865c929 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs @@ -1011,6 +1011,18 @@ impl FragmentCoordinator { .flight_scatter(&info.query_ctx, data_exchange)?, }), )), + DataExchange::Modulo(exchange) => Ok(Some(ExchangeParams::ShuffleExchange( + ShuffleExchangeParams { + exchange_injector: exchange_injector.clone(), + schema: self.physical_plan.output_schema()?, + fragment_id: self.fragment_id, + query_id: info.query_id.to_string(), + executor_id: info.current_executor.to_string(), + destination_ids: exchange.destination_ids.to_owned(), + shuffle_scatter: exchange_injector + .flight_scatter(&info.query_ctx, data_exchange)?, + }, + ))), } } diff --git a/src/query/service/src/servers/flight/v1/exchange/mod.rs b/src/query/service/src/servers/flight/v1/exchange/mod.rs index 194f2cbe1e3e5..ada27909df959 100644 --- a/src/query/service/src/servers/flight/v1/exchange/mod.rs +++ b/src/query/service/src/servers/flight/v1/exchange/mod.rs @@ -32,6 +32,7 @@ pub mod serde; pub use data_exchange::BroadcastExchange; pub use data_exchange::DataExchange; pub use data_exchange::MergeExchange; +pub use data_exchange::ModuloExchange; pub use data_exchange::ShuffleDataExchange; pub use exchange_injector::DefaultExchangeInjector; pub use exchange_injector::ExchangeInjector; diff --git a/src/query/service/src/servers/flight/v1/scatter/flight_scatter_mod.rs b/src/query/service/src/servers/flight/v1/scatter/flight_scatter_mod.rs new file mode 100644 index 0000000000000..f83fea3f574c2 --- /dev/null +++ b/src/query/service/src/servers/flight/v1/scatter/flight_scatter_mod.rs @@ -0,0 +1,92 @@ +// 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::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::type_check::check_function; +use databend_common_expression::types::DataType; +use databend_common_expression::types::NumberDataType; +use databend_common_expression::types::NumberScalar; +use databend_common_expression::DataBlock; +use databend_common_expression::Evaluator; +use databend_common_expression::Expr; +use databend_common_expression::FunctionContext; +use databend_common_expression::RemoteExpr; +use databend_common_expression::Scalar; +use databend_common_functions::BUILTIN_FUNCTIONS; + +use crate::servers::flight::v1::scatter::FlightScatter; + +#[derive(Clone)] +pub struct ModFlightScatter { + scatter_size: usize, + func_ctx: FunctionContext, + expr: Expr, +} + +impl ModFlightScatter { + pub fn try_create( + func_ctx: FunctionContext, + expr: &RemoteExpr, + scatter_size: usize, + ) -> Result> { + let expr = check_function( + None, + "modulo", + &[], + &[ + expr.as_expr(&BUILTIN_FUNCTIONS), + Expr::constant( + Scalar::Number(NumberScalar::UInt64(scatter_size as u64)), + Some(DataType::Number(NumberDataType::UInt64)), + ), + ], + &BUILTIN_FUNCTIONS, + )?; + let return_type = expr.data_type(); + if !matches!(return_type, DataType::Number(NumberDataType::UInt64)) { + return Err(ErrorCode::Internal(format!( + "ModFlightScatter expects modulo expression to return UInt64, but got {:?}", + return_type + ))); + } + + Ok(Box::new(ModFlightScatter { + scatter_size, + func_ctx, + expr, + })) + } +} + +impl FlightScatter for ModFlightScatter { + fn execute(&self, data_block: DataBlock) -> Result> { + let evaluator = Evaluator::new(&data_block, &self.func_ctx, &BUILTIN_FUNCTIONS); + let num = data_block.num_rows(); + + let column = evaluator + .run(&self.expr)? + .into_full_column(&DataType::Number(NumberDataType::UInt64), num); + let indices = column.as_number().unwrap().as_u_int64().unwrap(); + let data_blocks = DataBlock::scatter(&data_block, indices, self.scatter_size)?; + + let block_meta = data_block.get_meta(); + let mut res = Vec::with_capacity(data_blocks.len()); + for data_block in data_blocks { + res.push(data_block.add_meta(block_meta.cloned())?); + } + + Ok(res) + } +} diff --git a/src/query/service/src/servers/flight/v1/scatter/mod.rs b/src/query/service/src/servers/flight/v1/scatter/mod.rs index b5f5f900dab71..2904ed87684ca 100644 --- a/src/query/service/src/servers/flight/v1/scatter/mod.rs +++ b/src/query/service/src/servers/flight/v1/scatter/mod.rs @@ -15,7 +15,9 @@ mod flight_scatter; mod flight_scatter_broadcast; mod flight_scatter_hash; +mod flight_scatter_mod; pub use flight_scatter::FlightScatter; pub use flight_scatter_broadcast::BroadcastFlightScatter; pub use flight_scatter_hash::HashFlightScatter; +pub use flight_scatter_mod::ModFlightScatter; 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 fced5a435ac8f..d49fde7779931 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 97468f1d0102c..eb908bb564a8a 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/format.rs b/src/query/sql/src/executor/format.rs index 1e3f8879339f3..29f5bc2529dad 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -1639,6 +1639,14 @@ fn exchange_to_format_tree( ), FragmentKind::Expansive => "Broadcast".to_string(), FragmentKind::Merge => "Merge".to_string(), + FragmentKind::Modulo => format!( + "Modulo({})", + plan.keys + .iter() + .map(|key| { key.as_expr(&BUILTIN_FUNCTIONS).sql_display() }) + .collect::>() + .join(", ") + ), })), to_format_tree(&plan.input, metadata, profs, context)?, ])) diff --git a/src/query/sql/src/executor/physical_plans/common.rs b/src/query/sql/src/executor/physical_plans/common.rs index 545179b4af4d6..10859f8391da1 100644 --- a/src/query/sql/src/executor/physical_plans/common.rs +++ b/src/query/sql/src/executor/physical_plans/common.rs @@ -67,6 +67,8 @@ pub enum FragmentKind { // Broadcast Expansive, Merge, + // Partitioned by a specified expression % node_nums + Modulo, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize, Copy)] diff --git a/src/query/sql/src/executor/physical_plans/physical_exchange.rs b/src/query/sql/src/executor/physical_plans/physical_exchange.rs index 1e831519c415b..b4507942dc8ba 100644 --- a/src/query/sql/src/executor/physical_plans/physical_exchange.rs +++ b/src/query/sql/src/executor/physical_plans/physical_exchange.rs @@ -81,6 +81,14 @@ impl PhysicalPlanBuilder { allow_adjust_parallelism = false; FragmentKind::Merge } + crate::plans::Exchange::Modulo(scalar) => { + let expr = scalar + .type_check(input_schema.as_ref())? + .project_column_ref(|index| input_schema.index_of(&index.to_string()).unwrap()); + let (expr, _) = ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS); + keys.push(expr.as_remote_expr()); + FragmentKind::Modulo + } }; Ok(PhysicalPlan::Exchange(Exchange { plan_id: 0, 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..0a5520ccb5ba2 100644 --- a/src/query/sql/src/executor/physical_plans/physical_recluster.rs +++ b/src/query/sql/src/executor/physical_plans/physical_recluster.rs @@ -31,7 +31,9 @@ pub struct HilbertPartition { pub plan_id: u32, pub input: Box, pub table_info: TableInfo, - pub num_partitions: usize, pub table_meta_timestamps: TableMetaTimestamps, pub rows_per_block: usize, + pub bytes_per_block: usize, + pub range_start: u64, + pub range_width: 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/sql/src/planner/format/display_rel_operator.rs b/src/query/sql/src/planner/format/display_rel_operator.rs index 9835bbedf0cdd..a82e84ddc59ac 100644 --- a/src/query/sql/src/planner/format/display_rel_operator.rs +++ b/src/query/sql/src/planner/format/display_rel_operator.rs @@ -397,6 +397,7 @@ fn exchange_to_format_tree(id_humanizer: &I, op: &Exchange) -> F Exchange::Broadcast => "Exchange(Broadcast)", Exchange::Merge => "Exchange(Merge)", Exchange::MergeSort => "Exchange(MergeSort)", + Exchange::Modulo(_) => "Exchange(Modulo)", }; match op { diff --git a/src/query/sql/src/planner/optimizer/ir/format.rs b/src/query/sql/src/planner/optimizer/ir/format.rs index f9613af6b35ef..017c9bec97203 100644 --- a/src/query/sql/src/planner/optimizer/ir/format.rs +++ b/src/query/sql/src/planner/optimizer/ir/format.rs @@ -66,6 +66,7 @@ fn display_rel_op(rel_op: &RelOperator) -> String { Exchange::Broadcast => "Broadcast".to_string(), Exchange::Merge => "Merge".to_string(), Exchange::MergeSort => "MergeSort".to_string(), + Exchange::Modulo(scalar) => format!("Modulo({})", scalar.as_raw_expr()), }) } RelOperator::DummyTableScan(_) => "DummyTableScan".to_string(), diff --git a/src/query/sql/src/planner/optimizer/ir/property/enforcer.rs b/src/query/sql/src/planner/optimizer/ir/property/enforcer.rs index 1229898f5bef4..69abb144e11a4 100644 --- a/src/query/sql/src/planner/optimizer/ir/property/enforcer.rs +++ b/src/query/sql/src/planner/optimizer/ir/property/enforcer.rs @@ -73,6 +73,7 @@ impl Enforcer for DistributionEnforcer { Distribution::Random | Distribution::Any => Err(ErrorCode::Internal( "Cannot enforce random or any distribution", )), + Distribution::Modulo(key) => Ok(Exchange::Modulo(key.clone()).into()), } } } diff --git a/src/query/sql/src/planner/optimizer/ir/property/property.rs b/src/query/sql/src/planner/optimizer/ir/property/property.rs index 9ae23730ca7e7..3eff4f594a2d2 100644 --- a/src/query/sql/src/planner/optimizer/ir/property/property.rs +++ b/src/query/sql/src/planner/optimizer/ir/property/property.rs @@ -92,6 +92,7 @@ pub enum Distribution { Serial, Broadcast, Hash(Vec), + Modulo(Box), } impl Default for Distribution { @@ -110,11 +111,15 @@ impl Distribution { | (Distribution::Random, _) | (Distribution::Serial, Distribution::Serial) | (Distribution::Broadcast, Distribution::Broadcast) - | (Distribution::Hash(_), Distribution::Broadcast) => true, + | (Distribution::Hash(_), Distribution::Broadcast) + | (Distribution::Modulo(_), Distribution::Broadcast) => true, (Distribution::Hash(ref keys), Distribution::Hash(ref other_keys)) => { keys == other_keys } + (Distribution::Modulo(ref key), Distribution::Modulo(ref other_key)) => { + key == other_key + } _ => false, } } @@ -135,6 +140,7 @@ impl Display for Distribution { .collect::>() .join(", ") ), + Distribution::Modulo(ref key) => write!(f, "Modulo({})", key.as_raw_expr()), } } } diff --git a/src/query/sql/src/planner/optimizer/optimizers/cascades/cost/model.rs b/src/query/sql/src/planner/optimizer/optimizers/cascades/cost/model.rs index 6d39e793a7231..5bd737365a76b 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/cascades/cost/model.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/cascades/cost/model.rs @@ -158,7 +158,7 @@ impl DefaultCostModel { let exchange: Exchange = (*m_expr.plan.clone()).clone().try_into()?; let group = memo.group(m_expr.group_index)?; let cost = match exchange { - Exchange::Hash(_) => { + Exchange::Hash(_) | Exchange::Modulo(_) => { group.stat_info.cardinality * self.network_per_row + group.stat_info.cardinality * self.compute_per_row } diff --git a/src/query/sql/src/planner/plans/exchange.rs b/src/query/sql/src/planner/plans/exchange.rs index a7aca885b2ed1..db8dffd95d8cf 100644 --- a/src/query/sql/src/planner/plans/exchange.rs +++ b/src/query/sql/src/planner/plans/exchange.rs @@ -30,7 +30,8 @@ pub enum Exchange { Hash(Vec), Broadcast, Merge, - MergeSort, // For distributed sort + MergeSort, // For distributed sort + Modulo(Box), // For recluster } impl Operator for Exchange { @@ -49,6 +50,7 @@ impl Operator for Exchange { Exchange::Broadcast => Distribution::Broadcast, Exchange::Merge => Distribution::Serial, Exchange::MergeSort => Distribution::Serial, + Exchange::Modulo(key) => Distribution::Modulo(key.clone()), }, }) } diff --git a/src/query/storages/common/index/src/bloom_index.rs b/src/query/storages/common/index/src/bloom_index.rs index d35c01ae7e9b5..59698ea79e6b7 100644 --- a/src/query/storages/common/index/src/bloom_index.rs +++ b/src/query/storages/common/index/src/bloom_index.rs @@ -18,6 +18,7 @@ use std::hash::Hasher; use std::ops::ControlFlow; use std::ops::Deref; use std::sync::Arc; +use std::time::Instant; use bytes::Bytes; use databend_common_ast::Span; @@ -771,6 +772,7 @@ impl BloomIndexBuilder { } pub fn finalize(&mut self) -> Result> { + let start = Instant::now(); let mut column_distinct_count = HashMap::with_capacity(self.columns_len()); let mut filters = Vec::with_capacity(self.columns_len()); let mut filter_fields = Vec::with_capacity(self.columns_len()); @@ -806,6 +808,7 @@ impl BloomIndexBuilder { return Ok(None); } let filter_schema = Arc::new(TableSchema::new(filter_fields)); + log::info!("build bloom index cost: {:?}", start.elapsed()); Ok(Some(BloomIndex { func_ctx: self.func_ctx.clone(), version: BlockFilter::VERSION, 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 32900fa6f8e86..16a7e750dcc5e 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 1b14fa0df99ca..009e444c04c1d 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 623cd688c18b8..e27e0c423260d 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 b388419c66bf6..d83dbfc6df19c 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) } @@ -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() @@ -397,7 +407,7 @@ 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![]; @@ -429,4 +439,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 6e0641069600b..4fa885883549f 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 { + out_fields: Vec, + level: i32, cluster_key_id: u32, 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 } @@ -90,14 +95,26 @@ impl ClusterStatisticsBuilder { cluster_key_id: table.cluster_key_meta.as_ref().unwrap().0, 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(num_rows)) + .map(|&i| input.get_by_offset(i).to_column(num_rows)) .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,7 +178,7 @@ impl ClusterStatisticsState { let level = if min == max && perfect { -1 } else { - self.level + self.builder.level }; Ok(Some(ClusterStatistics { 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..8df60aa61f03f 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 @@ -160,7 +160,7 @@ fn column_update_hll_cardinality(col: &Column, ty: &DataType, hll: &mut ColumnDi let col = col.as_nullable().unwrap(); for (i, v) in col.validity.iter().enumerate() { if v { - let scalar = col.column.index(i).unwrap(); + let scalar = unsafe { col.column.index_unchecked(i) }; scalar_update_hll_cardinality(&scalar, inner, hll); } } 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..fc14e9589071e 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,42 @@ 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_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 +119,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 +178,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 +215,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 469a7088d4cd9..effa8edb18037 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/window.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/window.test @@ -56,7 +56,7 @@ CompoundBlockOperator(Project) × 1 SortPartialTransform × 4 Merge to Resize × 4 Transform Window × 1 - TransformWindowPartitionCollect(Sort) × 1 + TransformPartitionCollect(Window) × 1 ShuffleMergePartition(Window) × 1 ShufflePartition(Window) × 1 DeserializeDataTransform × 1 @@ -77,7 +77,7 @@ CompoundBlockOperator(Project) × 1 SortPartialTransform × 4 Merge to Resize × 4 Transform Window × 1 - TransformWindowPartitionCollect(Sort) × 1 + TransformPartitionCollect(Window) × 1 ShuffleMergePartition(Window) × 1 ShufflePartition(Window) × 1 DeserializeDataTransform × 1 @@ -367,7 +367,7 @@ explain pipeline select a, sum(a) over (partition by a order by a desc) from t l CompoundBlockOperator(Project) × 1 LimitTransform × 1 Transform Window × 1 - TransformWindowPartitionCollect(Sort) × 1 + TransformPartitionCollect(Window) × 1 ShuffleMergePartition(Window) × 1 ShufflePartition(Window) × 1 DeserializeDataTransform × 1 @@ -385,7 +385,7 @@ explain pipeline select a, sum(a) over (partition by a order by a desc) from t l CompoundBlockOperator(Project) × 1 LimitTransform × 1 Transform Window × 1 - TransformWindowPartitionCollect(Sort) × 1 + TransformPartitionCollect(Window) × 1 ShuffleMergePartition(Window) × 1 ShufflePartition(Window) × 1 DeserializeDataTransform × 1 @@ -404,7 +404,7 @@ explain pipeline select a, dense_rank() over (partition by a order by a desc) fr CompoundBlockOperator(Project) × 1 LimitTransform × 1 Transform Window × 1 - TransformWindowPartitionCollect(Sort) × 1 + TransformPartitionCollect(Window) × 1 ShuffleMergePartition(Window) × 1 ShufflePartition(Window) × 1 DeserializeDataTransform × 1 @@ -418,7 +418,7 @@ explain pipeline select a, sum(a) over (partition by a order by a desc rows betw CompoundBlockOperator(Project) × 1 LimitTransform × 1 Transform Window × 1 - TransformWindowPartitionCollect(Sort) × 1 + TransformPartitionCollect(Window) × 1 ShuffleMergePartition(Window) × 1 ShufflePartition(Window) × 1 DeserializeDataTransform × 1 @@ -432,7 +432,7 @@ explain pipeline select a, sum(a) over (partition by a order by a desc rows betw CompoundBlockOperator(Project) × 1 LimitTransform × 1 Transform Window × 1 - TransformWindowPartitionCollect(Sort) × 1 + TransformPartitionCollect(Window) × 1 ShuffleMergePartition(Window) × 1 ShufflePartition(Window) × 1 DeserializeDataTransform × 1 @@ -452,7 +452,7 @@ CompoundBlockOperator(Project) × 1 SortPartialTransform × 4 Merge to Resize × 4 Transform Window × 1 - TransformWindowPartitionCollect(Sort) × 1 + TransformPartitionCollect(Window) × 1 ShuffleMergePartition(Window) × 1 ShufflePartition(Window) × 1 DeserializeDataTransform × 1 @@ -471,7 +471,7 @@ CompoundBlockOperator(Project) × 1 SortPartialTransform × 4 Merge to Resize × 4 Transform Window × 1 - TransformWindowPartitionCollect(Sort) × 1 + TransformPartitionCollect(Window) × 1 ShuffleMergePartition(Window) × 1 ShufflePartition(Window) × 1 TransformFilter × 1 @@ -535,7 +535,7 @@ CompoundBlockOperator(Project) × 1 LimitTransform × 1 TransformFilter × 1 Transform Window × 1 - TransformWindowPartitionCollect(Sort) × 1 + TransformPartitionCollect(Window) × 1 ShuffleMergePartition(WindowTopN) × 1 ShufflePartition(WindowTopN) × 1 DeserializeDataTransform × 1 @@ -573,7 +573,7 @@ explain pipeline select number, lead(number,1, 0) over (partition by number % 3 CompoundBlockOperator(Project) × 1 Transform Window × 1 Transform Window × 1 - TransformWindowPartitionCollect(Sort) × 1 + TransformPartitionCollect(Window) × 1 ShuffleMergePartition(Window) × 1 ShufflePartition(Window) × 1 CompoundBlockOperator(Map) × 1