From 67056a542e0f85aa318e9a34e091183540c8fe4e Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Mon, 30 Oct 2023 16:10:06 +0800 Subject: [PATCH 01/14] refactor(optimizer): remove some ColIndexMapping::without_tar_size usage (#13134) Co-authored-by: Dylan Chen --- src/common/src/util/column_index_mapping.rs | 32 +++++++------------ .../src/handler/alter_table_column.rs | 2 +- src/frontend/src/optimizer/mod.rs | 2 +- .../optimizer/plan_node/eq_join_predicate.rs | 4 +-- .../src/optimizer/plan_node/generic/agg.rs | 4 +-- .../src/optimizer/plan_node/generic/expand.rs | 2 +- .../src/optimizer/plan_node/generic/join.rs | 2 +- .../optimizer/plan_node/generic/project.rs | 4 +-- .../plan_node/generic/project_set.rs | 4 +-- .../src/optimizer/plan_node/logical_agg.rs | 2 +- .../src/optimizer/plan_node/logical_expand.rs | 5 +-- .../src/optimizer/plan_node/logical_join.rs | 3 +- .../optimizer/plan_node/logical_multi_join.rs | 4 +-- .../src/optimizer/plan_node/logical_now.rs | 5 +-- .../optimizer/plan_node/logical_project.rs | 2 +- .../plan_node/logical_project_set.rs | 2 +- .../src/optimizer/plan_node/logical_scan.rs | 6 ++-- .../src/optimizer/plan_node/logical_source.rs | 2 +- .../rule/apply_join_transpose_rule.rs | 27 ++++++++++------ .../optimizer/rule/apply_offset_rewriter.rs | 9 ++++-- .../rule/push_calculation_of_join_rule.rs | 4 +-- .../optimizer/rule/translate_apply_rule.rs | 3 +- 22 files changed, 63 insertions(+), 67 deletions(-) diff --git a/src/common/src/util/column_index_mapping.rs b/src/common/src/util/column_index_mapping.rs index 49030a06c5001..2c12dc47efb11 100644 --- a/src/common/src/util/column_index_mapping.rs +++ b/src/common/src/util/column_index_mapping.rs @@ -32,19 +32,9 @@ pub struct ColIndexMapping { } impl ColIndexMapping { - /// Create a partial mapping which maps the subscripts range `(0..map.len())` to the - /// corresponding element. **This method is not recommended**, please use `with_target_size` instead, see for more information** - pub fn without_target_size(map: Vec>) -> Self { - let target_size = match map.iter().filter_map(|x| *x).max_by_key(|x| *x) { - Some(target_max) => target_max + 1, - None => 0, - }; - Self { target_size, map } - } - /// Create a partial mapping which maps from the subscripts range `(0..map.len())` to /// `(0..target_size)`. Each subscript is mapped to the corresponding element. - pub fn with_target_size(map: Vec>, target_size: usize) -> Self { + pub fn new(map: Vec>, target_size: usize) -> Self { if let Some(target_max) = map.iter().filter_map(|x| *x).max_by_key(|x| *x) { assert!(target_max < target_size) }; @@ -69,7 +59,7 @@ impl ColIndexMapping { pub fn identity(size: usize) -> Self { let map = (0..size).map(Some).collect(); - Self::without_target_size(map) + Self::new(map, size) } pub fn is_identity(&self) -> bool { @@ -90,12 +80,12 @@ impl ColIndexMapping { let map = (0..source_size) .map(|i| if i < target_size { Some(i) } else { None }) .collect(); - Self::with_target_size(map, target_size) + Self::new(map, target_size) } pub fn empty(source_size: usize, target_size: usize) -> Self { let map = vec![None; source_size]; - Self::with_target_size(map, target_size) + Self::new(map, target_size) } /// Create a partial mapping which maps range `(0..source_num)` to range @@ -134,7 +124,7 @@ impl ColIndexMapping { }) .collect_vec(); let target_size = usize::try_from(source_num as isize + offset).unwrap(); - Self::with_target_size(map, target_size) + Self::new(map, target_size) } /// Maps the smallest index to 0, the next smallest to 1, and so on. @@ -159,7 +149,7 @@ impl ColIndexMapping { for (tar, &src) in cols.iter().enumerate() { map[src] = Some(tar); } - Self::without_target_size(map) + Self::new(map, cols.len()) } // TODO(yuchao): isn't this the same as `with_remaining_columns`? @@ -170,7 +160,7 @@ impl ColIndexMapping { map[src] = Some(tar); } } - Self::without_target_size(map) + Self::new(map, cols.len()) } /// Remove the given columns, and maps the remaining columns to a consecutive range starting @@ -205,7 +195,7 @@ impl ColIndexMapping { for target in &mut map { *target = target.and_then(|index| following.try_map(index)); } - Self::with_target_size(map, following.target_size()) + Self::new(map, following.target_size()) } pub fn clone_with_offset(&self, offset: usize) -> Self { @@ -213,7 +203,7 @@ impl ColIndexMapping { for target in &mut map { *target = target.and_then(|index| index.checked_add(offset)); } - Self::with_target_size(map, self.target_size() + offset) + Self::new(map, self.target_size() + offset) } /// Union two mapping, the result mapping `target_size` and source size will be the max size @@ -236,7 +226,7 @@ impl ColIndexMapping { assert_eq!(map[src], None); map[src] = Some(dst); } - Self::with_target_size(map, target_size) + Self::new(map, target_size) } /// Inverse the mapping. If a target corresponds to more than one source, return `None`. @@ -249,7 +239,7 @@ impl ColIndexMapping { } map[dst] = Some(src); } - Some(Self::with_target_size(map, self.source_size())) + Some(Self::new(map, self.source_size())) } /// return iter of (src, dst) order by src diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index 18313e0458a04..1b3babc41ceaf 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -240,7 +240,7 @@ pub async fn handle_alter_table_column( }; // Calculate the mapping from the original columns to the new columns. - let col_index_mapping = ColIndexMapping::with_target_size( + let col_index_mapping = ColIndexMapping::new( original_catalog .columns() .iter() diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 4004748a2f4f9..20da006dcc992 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -378,7 +378,7 @@ impl PlanRoot { } let plan = LogicalProject::with_out_col_idx(plan, output_indices.into_iter()); - let out_col_change = ColIndexMapping::with_target_size(map, target_size); + let out_col_change = ColIndexMapping::new(map, target_size); (plan.into(), out_col_change) } }; diff --git a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs index 0811bc0531a25..2c714bea0f7de 100644 --- a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs +++ b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs @@ -238,7 +238,7 @@ impl EqJoinPredicate { for (left, right, _) in self.eq_keys() { map[right.index - left_cols_num] = Some(left.index); } - ColIndexMapping::with_target_size(map, left_cols_num) + ColIndexMapping::new(map, left_cols_num) } /// return the eq columns index mapping from left inputs to right inputs @@ -251,7 +251,7 @@ impl EqJoinPredicate { for (left, right, _) in self.eq_keys() { map[left.index] = Some(right.index - left_cols_num); } - ColIndexMapping::with_target_size(map, right_cols_num) + ColIndexMapping::new(map, right_cols_num) } /// Reorder the `eq_keys` according to the `reorder_idx`. diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index c7a0ea4f39389..18ac5525182f3 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -74,7 +74,7 @@ impl Agg { for (i, key) in self.group_key.indices().enumerate() { map[i] = Some(key); } - ColIndexMapping::with_target_size(map, self.input.schema().len()) + ColIndexMapping::new(map, self.input.schema().len()) } /// get the Mapping of columnIndex from input column index to out column index @@ -83,7 +83,7 @@ impl Agg { for (i, key) in self.group_key.indices().enumerate() { map[key] = Some(i); } - ColIndexMapping::with_target_size(map, self.output_len()) + ColIndexMapping::new(map, self.output_len()) } fn two_phase_agg_forced(&self) -> bool { diff --git a/src/frontend/src/optimizer/plan_node/generic/expand.rs b/src/frontend/src/optimizer/plan_node/generic/expand.rs index d346677ac2d62..426589d3983e6 100644 --- a/src/frontend/src/optimizer/plan_node/generic/expand.rs +++ b/src/frontend/src/optimizer/plan_node/generic/expand.rs @@ -134,7 +134,7 @@ impl Expand { let map = (0..input_len) .map(|source| Some(source + input_len)) .collect_vec(); - ColIndexMapping::with_target_size(map, self.output_len()) + ColIndexMapping::new(map, self.output_len()) } pub fn o2i_col_mapping(&self) -> ColIndexMapping { diff --git a/src/frontend/src/optimizer/plan_node/generic/join.rs b/src/frontend/src/optimizer/plan_node/generic/join.rs index 2536cee984558..ee8c43b388684 100644 --- a/src/frontend/src/optimizer/plan_node/generic/join.rs +++ b/src/frontend/src/optimizer/plan_node/generic/join.rs @@ -431,7 +431,7 @@ impl Join { pub fn o2i_col_mapping(&self) -> ColIndexMapping { // If output_indices = [0, 0, 1], we should use it as `o2i_col_mapping` directly. // If we use `self.i2o_col_mapping().inverse()`, we will lose the first 0. - ColIndexMapping::with_target_size( + ColIndexMapping::new( self.output_indices.iter().map(|x| Some(*x)).collect(), self.internal_column_num(), ) diff --git a/src/frontend/src/optimizer/plan_node/generic/project.rs b/src/frontend/src/optimizer/plan_node/generic/project.rs index d8b6988af4391..e58ac40e918a8 100644 --- a/src/frontend/src/optimizer/plan_node/generic/project.rs +++ b/src/frontend/src/optimizer/plan_node/generic/project.rs @@ -210,7 +210,7 @@ impl Project { map[i] = Some(input.index()) } } - ColIndexMapping::with_target_size(map, input_len) + ColIndexMapping::new(map, input_len) } /// get the Mapping of columnIndex from input column index to output column index,if a input @@ -224,7 +224,7 @@ impl Project { map[input.index()] = Some(i) } } - ColIndexMapping::with_target_size(map, exprs.len()) + ColIndexMapping::new(map, exprs.len()) } pub fn is_all_inputref(&self) -> bool { diff --git a/src/frontend/src/optimizer/plan_node/generic/project_set.rs b/src/frontend/src/optimizer/plan_node/generic/project_set.rs index fef26d1b32993..195b420802076 100644 --- a/src/frontend/src/optimizer/plan_node/generic/project_set.rs +++ b/src/frontend/src/optimizer/plan_node/generic/project_set.rs @@ -122,7 +122,7 @@ impl ProjectSet { map[1 + i] = Some(input.index()) } } - ColIndexMapping::with_target_size(map, input_len) + ColIndexMapping::new(map, input_len) } /// Gets the Mapping of columnIndex from input column index to output column index,if a input @@ -135,7 +135,7 @@ impl ProjectSet { map[input.index()] = Some(1 + i) } } - ColIndexMapping::with_target_size(map, 1 + self.select_list.len()) + ColIndexMapping::new(map, 1 + self.select_list.len()) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 0aed32abec40e..b0e04f0598778 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -1214,7 +1214,7 @@ impl ToStream for LogicalAgg { let (input, input_col_change) = self.input().logical_rewrite_for_stream(ctx)?; let (agg, out_col_change) = self.rewrite_with_input(input, input_col_change); let (map, _) = out_col_change.into_parts(); - let out_col_change = ColIndexMapping::with_target_size(map, agg.schema().len()); + let out_col_change = ColIndexMapping::new(map, agg.schema().len()); Ok((agg.into(), out_col_change)) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index 054621ef967bf..77388ee33cf4a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -121,10 +121,7 @@ impl PlanTreeNodeUnary for LogicalExpand { let expand = Self::new(input, column_subsets); let output_col_num = expand.schema().len(); - ( - expand, - ColIndexMapping::with_target_size(mapping, output_col_num), - ) + (expand, ColIndexMapping::new(mapping, output_col_num)) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index a928481230d3c..c594ededa40cf 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -518,8 +518,7 @@ impl PlanTreeNodeBinary for LogicalJoin { *i += left.schema().len(); } map.append(&mut right_map); - let mut mapping = - ColIndexMapping::with_target_size(map, left.schema().len() + right.schema().len()); + let mut mapping = ColIndexMapping::new(map, left.schema().len() + right.schema().len()); let new_output_indices = self .output_indices() diff --git a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs index 819f84e963cfd..0bbe59dd70754 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -87,7 +87,7 @@ impl LogicalMultiJoinBuilder { /// add a predicate above the plan, so they will be rewritten from the `output_indices` to the /// input indices pub fn add_predicate_above(&mut self, exprs: impl Iterator) { - let mut mapping = ColIndexMapping::with_target_size( + let mut mapping = ColIndexMapping::new( self.output_indices.iter().map(|i| Some(*i)).collect(), self.tot_input_col_num, ); @@ -240,7 +240,7 @@ impl LogicalMultiJoin { i2o_maps .into_iter() - .map(|map| ColIndexMapping::with_target_size(map, tot_col_num)) + .map(|map| ColIndexMapping::new(map, tot_col_num)) .collect_vec() }; diff --git a/src/frontend/src/optimizer/plan_node/logical_now.rs b/src/frontend/src/optimizer/plan_node/logical_now.rs index c13a0c93a6e70..9e2d586c0fd2a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_now.rs +++ b/src/frontend/src/optimizer/plan_node/logical_now.rs @@ -81,10 +81,7 @@ impl ToStream for LogicalNow { &self, _ctx: &mut RewriteStreamContext, ) -> Result<(PlanRef, ColIndexMapping)> { - Ok(( - self.clone().into(), - ColIndexMapping::with_target_size(vec![Some(0)], 1), - )) + Ok((self.clone().into(), ColIndexMapping::new(vec![Some(0)], 1))) } /// `to_stream` is equivalent to `to_stream_with_dist_required(RequiredDist::Any)` diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index 59a5509ebcd70..aee136e57ca9c 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -284,7 +284,7 @@ impl ToStream for LogicalProject { // But the target size of `out_col_change` should be the same as the length of the new // schema. let (map, _) = out_col_change.into_parts(); - let out_col_change = ColIndexMapping::with_target_size(map, proj.base.schema().len()); + let out_col_change = ColIndexMapping::new(map, proj.base.schema().len()); Ok((proj.into(), out_col_change)) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_project_set.rs b/src/frontend/src/optimizer/plan_node/logical_project_set.rs index fc894713c2f43..727c641d52974 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -385,7 +385,7 @@ impl ToStream for LogicalProjectSet { // But the target size of `out_col_change` should be the same as the length of the new // schema. let (map, _) = out_col_change.into_parts(); - let out_col_change = ColIndexMapping::with_target_size(map, project_set.schema().len()); + let out_col_change = ColIndexMapping::new(map, project_set.schema().len()); Ok((project_set.into(), out_col_change)) } diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 200302ae70f97..c9f5494b86be6 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -233,7 +233,7 @@ impl LogicalScan { } let mut inverse_mapping = { - let mapping = ColIndexMapping::with_target_size( + let mapping = ColIndexMapping::new( self.required_col_idx().iter().map(|i| Some(*i)).collect(), self.table_desc().columns.len(), ); @@ -242,7 +242,7 @@ impl LogicalScan { for (src, dst) in mapping.mapping_pairs() { inverse_map[dst] = Some(src); } - ColIndexMapping::with_target_size(inverse_map, mapping.source_size()) + ColIndexMapping::new(inverse_map, mapping.source_size()) }; predicate = predicate.rewrite_expr(&mut inverse_mapping); @@ -412,7 +412,7 @@ impl PredicatePushdown for LogicalScan { .conjunctions .extract_if(|expr| expr.count_nows() > 0 || HasCorrelated {}.visit_expr(expr)) .collect(); - let predicate = predicate.rewrite_expr(&mut ColIndexMapping::with_target_size( + let predicate = predicate.rewrite_expr(&mut ColIndexMapping::new( self.output_col_idx().iter().map(|i| Some(*i)).collect(), self.table_desc().columns.len(), )); diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 01166e74f1359..cac051957b0a5 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -127,7 +127,7 @@ impl LogicalSource { mapping[idx] = None; } } - ColIndexMapping::with_target_size(mapping, columns.len()) + ColIndexMapping::new(mapping, columns.len()) }; let mut rewriter = IndexRewriter::new(col_mapping); diff --git a/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs index 7ac121692c81d..98363f2aba213 100644 --- a/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs @@ -356,7 +356,7 @@ impl ApplyJoinTransposeRule { correlated_indices, false, ); - let output_indices: Vec<_> = { + let (output_indices, target_size) = { let (apply_left_len, join_right_len) = match apply_join_type { JoinType::LeftSemi | JoinType::LeftAnti => (apply_left_len, 0), JoinType::RightSemi | JoinType::RightAnti => (0, join.right().schema().len()), @@ -368,14 +368,19 @@ impl ApplyJoinTransposeRule { join_left_len + apply_left_len..join_left_len + apply_left_len + join_right_len, ); - match join.join_type() { + let output_indices: Vec<_> = match join.join_type() { JoinType::LeftSemi | JoinType::LeftAnti => left_iter.collect(), JoinType::RightSemi | JoinType::RightAnti => right_iter.collect(), _ => left_iter.chain(right_iter).collect(), - } + }; + + let target_size = join_left_len + apply_left_len + join_right_len; + (output_indices, target_size) }; - let mut output_indices_mapping = - ColIndexMapping::without_target_size(output_indices.iter().map(|x| Some(*x)).collect()); + let mut output_indices_mapping = ColIndexMapping::new( + output_indices.iter().map(|x| Some(*x)).collect(), + target_size, + ); let new_join = LogicalJoin::new( join.left(), new_join_right, @@ -518,7 +523,7 @@ impl ApplyJoinTransposeRule { false, ); - let output_indices: Vec<_> = { + let (output_indices, target_size) = { let (apply_left_len, join_right_len) = match apply_join_type { JoinType::LeftSemi | JoinType::LeftAnti => (apply_left_len, 0), JoinType::RightSemi | JoinType::RightAnti => (0, join.right().schema().len()), @@ -529,11 +534,14 @@ impl ApplyJoinTransposeRule { let right_iter = join_left_len + apply_left_len * 2 ..join_left_len + apply_left_len * 2 + join_right_len; - match join.join_type() { + let output_indices: Vec<_> = match join.join_type() { JoinType::LeftSemi | JoinType::LeftAnti => left_iter.collect(), JoinType::RightSemi | JoinType::RightAnti => right_iter.collect(), _ => left_iter.chain(right_iter).collect(), - } + }; + + let target_size = join_left_len + apply_left_len * 2 + join_right_len; + (output_indices, target_size) }; let new_join = LogicalJoin::new( new_join_left, @@ -548,8 +556,9 @@ impl ApplyJoinTransposeRule { new_join.into() } JoinType::Inner | JoinType::LeftOuter | JoinType::RightOuter | JoinType::FullOuter => { - let mut output_indices_mapping = ColIndexMapping::without_target_size( + let mut output_indices_mapping = ColIndexMapping::new( output_indices.iter().map(|x| Some(*x)).collect(), + target_size, ); // Leave other condition for predicate push down to deal with LogicalFilter::create( diff --git a/src/frontend/src/optimizer/rule/apply_offset_rewriter.rs b/src/frontend/src/optimizer/rule/apply_offset_rewriter.rs index 4a18f1ba37e44..559ace9d5dbe0 100644 --- a/src/frontend/src/optimizer/rule/apply_offset_rewriter.rs +++ b/src/frontend/src/optimizer/rule/apply_offset_rewriter.rs @@ -74,13 +74,18 @@ pub struct ApplyCorrelatedIndicesConverter {} impl ApplyCorrelatedIndicesConverter { pub fn convert_to_index_mapping(correlated_indices: &[usize]) -> ColIndexMapping { // Inverse anyway. - let col_mapping = ColIndexMapping::without_target_size( + let target_size = match correlated_indices.iter().max_by_key(|&&x| x) { + Some(target_max) => target_max + 1, + None => 0, + }; + let col_mapping = ColIndexMapping::new( correlated_indices.iter().copied().map(Some).collect_vec(), + target_size, ); let mut map = vec![None; col_mapping.target_size()]; for (src, dst) in col_mapping.mapping_pairs() { map[dst] = Some(src); } - ColIndexMapping::with_target_size(map, col_mapping.source_size()) + ColIndexMapping::new(map, col_mapping.source_size()) } } diff --git a/src/frontend/src/optimizer/rule/push_calculation_of_join_rule.rs b/src/frontend/src/optimizer/rule/push_calculation_of_join_rule.rs index ce5004b069a31..a4cdb78990a77 100644 --- a/src/frontend/src/optimizer/rule/push_calculation_of_join_rule.rs +++ b/src/frontend/src/optimizer/rule/push_calculation_of_join_rule.rs @@ -63,7 +63,7 @@ impl Rule for PushCalculationOfJoinRule { ) .map(Some) .collect_vec(); - ColIndexMapping::with_target_size(map, new_internal_col_num) + ColIndexMapping::new(map, new_internal_col_num) }; let (mut exprs, new_output_indices) = Self::remap_exprs_and_output_indices(exprs, output_indices, &mut col_index_mapping); @@ -82,7 +82,7 @@ impl Rule for PushCalculationOfJoinRule { .map(|i| i + left_col_num + left_exprs_non_input_ref.len()) .map(Some) .collect_vec(); - ColIndexMapping::with_target_size(map, new_internal_col_num) + ColIndexMapping::new(map, new_internal_col_num) }; // replace chosen function calls. for (((index_of_func_call, ty), left_expr), right_expr) in indices_and_ty_of_func_calls diff --git a/src/frontend/src/optimizer/rule/translate_apply_rule.rs b/src/frontend/src/optimizer/rule/translate_apply_rule.rs index aee6a2a06e1a9..ec6f58a2422d9 100644 --- a/src/frontend/src/optimizer/rule/translate_apply_rule.rs +++ b/src/frontend/src/optimizer/rule/translate_apply_rule.rs @@ -59,8 +59,7 @@ impl Rule for TranslateApplyRule { let apply_left_len = left.schema().len(); let correlated_indices = apply.correlated_indices(); - let mut index_mapping = - ColIndexMapping::with_target_size(vec![None; apply_left_len], apply_left_len); + let mut index_mapping = ColIndexMapping::new(vec![None; apply_left_len], apply_left_len); let mut data_types = HashMap::new(); let mut index = 0; From 85274f1dc323d1980c18d869f27a17a4d49bb116 Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Mon, 30 Oct 2023 16:14:49 +0800 Subject: [PATCH 02/14] refactor(source): remove `upsert_avro_primary_key ` and separate extracting columns and bind pk (#13023) --- e2e_test/source/basic/ddl.slt | 2 +- e2e_test/source/basic/nosim_kafka.slt | 78 +-- .../basic/old_row_format_syntax/ddl.slt | 2 +- .../old_row_format_syntax/nosim_kafka.slt | 81 +-- proto/catalog.proto | 3 +- src/connector/src/parser/avro/parser.rs | 10 +- src/connector/src/parser/mod.rs | 2 - src/connector/src/parser/upsert_parser.rs | 24 +- .../tests/testdata/output/create_source.yaml | 2 +- src/frontend/src/handler/create_sink.rs | 2 +- src/frontend/src/handler/create_source.rs | 578 +++++++++--------- src/frontend/src/handler/create_table.rs | 32 +- src/sqlparser/src/ast/ddl.rs | 6 + 13 files changed, 391 insertions(+), 431 deletions(-) diff --git a/e2e_test/source/basic/ddl.slt b/e2e_test/source/basic/ddl.slt index c1941d4697ffa..c6c1c0590d558 100644 --- a/e2e_test/source/basic/ddl.slt +++ b/e2e_test/source/basic/ddl.slt @@ -28,7 +28,7 @@ create source invalid_startup_timestamp ( properties.bootstrap.server = 'message_queue:29092' ) FORMAT PLAIN ENCODE JSON; -statement error db error: ERROR: QueryError: Invalid input syntax: schema definition is required for ENCODE JSON +statement error db error: ERROR: QueryError: Protocol error: Schema definition is required, either from SQL or schema registry. create source invalid_schema_definition with ( connector = 'kafka', diff --git a/e2e_test/source/basic/nosim_kafka.slt b/e2e_test/source/basic/nosim_kafka.slt index 945f60e732fb8..beafb02cb96ba 100644 --- a/e2e_test/source/basic/nosim_kafka.slt +++ b/e2e_test/source/basic/nosim_kafka.slt @@ -30,26 +30,17 @@ WITH ( FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); -statement ok -CREATE TABLE upsert_avro_json ( - PRIMARY KEY("ID") -) -WITH ( -connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', -topic = 'upsert_avro_json') -FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); +# TODO: Uncomment this when we add test data kafka key with format `"ID":id` +# statement ok +# CREATE TABLE upsert_avro_json ( +# PRIMARY KEY("ID") +# ) +# WITH ( +# connector = 'kafka', +# properties.bootstrap.server = 'message_queue:29092', +# topic = 'upsert_avro_json') +# FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); -# Just ignore the kafka key, it works -statement ok -CREATE TABLE upsert_avro_json2 ( - PRIMARY KEY("ID") -) -WITH ( - connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', - topic = 'upsert_avro_json') -FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); statement ok CREATE TABLE debezium_non_compact (PRIMARY KEY(order_id)) with ( @@ -89,7 +80,7 @@ statement ok flush; # Wait enough time to ensure SourceExecutor consumes all Kafka data. -sleep 5s +sleep 8s query II SELECT @@ -104,33 +95,18 @@ delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -query II -SELECT - * -FROM - upsert_avro_json -ORDER BY - "ID"; ----- -update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z -delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z - - -query II -SELECT - * -FROM - upsert_avro_json2 -ORDER BY - "ID"; ----- -update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z -delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z - +# query II +# SELECT +# * +# FROM +# upsert_avro_json +# ORDER BY +# "ID"; +# ---- +# update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z +# delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z +# delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z +# delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z query II SELECT @@ -172,12 +148,8 @@ select * from kafka_json_schema_upsert order by id statement ok DROP TABLE upsert_avro_json_default_key; -statement ok -DROP TABLE upsert_avro_json; - - -statement ok -DROP TABLE upsert_avro_json2; +# statement ok +# DROP TABLE upsert_avro_json; statement ok DROP TABLE upsert_student_avro_json; diff --git a/e2e_test/source/basic/old_row_format_syntax/ddl.slt b/e2e_test/source/basic/old_row_format_syntax/ddl.slt index d0a8cd9ba08ea..6d1290463beeb 100644 --- a/e2e_test/source/basic/old_row_format_syntax/ddl.slt +++ b/e2e_test/source/basic/old_row_format_syntax/ddl.slt @@ -28,7 +28,7 @@ create source invalid_startup_timestamp ( properties.bootstrap.server = 'message_queue:29092' ) ROW FORMAT JSON; -statement error db error: ERROR: QueryError: Invalid input syntax: schema definition is required for ENCODE JSON +statement error db error: ERROR: QueryError: Protocol error: Schema definition is required, either from SQL or schema registry. create source invalid_schema_definition with ( connector = 'kafka', diff --git a/e2e_test/source/basic/old_row_format_syntax/nosim_kafka.slt b/e2e_test/source/basic/old_row_format_syntax/nosim_kafka.slt index 582aff7d958fb..37e2ef2266ff0 100644 --- a/e2e_test/source/basic/old_row_format_syntax/nosim_kafka.slt +++ b/e2e_test/source/basic/old_row_format_syntax/nosim_kafka.slt @@ -33,28 +33,17 @@ ROW FORMAT UPSERT_AVRO row schema location confluent schema registry 'http://message_queue:8081' -statement ok -CREATE TABLE upsert_avro_json ( - PRIMARY KEY("ID") -) -WITH ( -connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', -topic = 'upsert_avro_json') -ROW FORMAT UPSERT_AVRO -row schema location confluent schema registry 'http://message_queue:8081' - -# Just ignore the kafka key, it works -statement ok -CREATE TABLE upsert_avro_json2 ( - PRIMARY KEY("ID") -) -WITH ( - connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', - topic = 'upsert_avro_json') -ROW FORMAT UPSERT_AVRO -row schema location confluent schema registry 'http://message_queue:8081' +# TODO: Uncomment this when we add test data kafka key with format `"ID":id` +# statement ok +# CREATE TABLE upsert_avro_json ( +# PRIMARY KEY("ID") +# ) +# WITH ( +# connector = 'kafka', +# properties.bootstrap.server = 'message_queue:29092', +# topic = 'upsert_avro_json') +# ROW FORMAT UPSERT_AVRO +# row schema location confluent schema registry 'http://message_queue:8081' statement ok CREATE TABLE debezium_non_compact (PRIMARY KEY(order_id)) with ( @@ -65,7 +54,6 @@ CREATE TABLE debezium_non_compact (PRIMARY KEY(order_id)) with ( ) ROW FORMAT DEBEZIUM_AVRO ROW SCHEMA LOCATION CONFLUENT SCHEMA REGISTRY 'http://message_queue:8081'; - statement ok CREATE TABLE debezium_compact (PRIMARY KEY(order_id)) with ( connector = 'kafka', @@ -78,7 +66,7 @@ statement ok flush; # Wait enough time to ensure SourceExecutor consumes all Kafka data. -sleep 5s +sleep 10s query II SELECT @@ -93,32 +81,18 @@ delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -query II -SELECT - * -FROM - upsert_avro_json -ORDER BY - "ID"; ----- -update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z -delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z - - -query II -SELECT - * -FROM - upsert_avro_json2 -ORDER BY - "ID"; ----- -update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z -delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z +# query II +# SELECT +# * +# FROM +# upsert_avro_json +# ORDER BY +# "ID"; +# ---- +# update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z +# delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z +# delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z +# delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z query II @@ -150,12 +124,9 @@ select count(*) from debezium_compact; statement ok DROP TABLE upsert_avro_json_default_key; -statement ok -DROP TABLE upsert_avro_json; - +# statement ok +# DROP TABLE upsert_avro_json; -statement ok -DROP TABLE upsert_avro_json2; statement ok DROP TABLE upsert_student_avro_json; diff --git a/proto/catalog.proto b/proto/catalog.proto index c966b7bbe5eb0..4f421305c4eea 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -55,7 +55,8 @@ message StreamSourceInfo { string proto_message_name = 4; int32 csv_delimiter = 5; bool csv_has_header = 6; - string upsert_avro_primary_key = 7; + reserved 7; + reserved "upsert_avro_primary_key"; // deprecated plan_common.FormatType format = 8; plan_common.EncodeType row_encode = 9; SchemaRegistryNameStrategy name_strategy = 10; diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index e02bcca7fe2ae..61c55decfc040 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -108,7 +108,6 @@ pub struct AvroParserConfig { pub schema: Arc, pub key_schema: Option>, pub schema_resolver: Option>, - pub upsert_primary_key_column_name: Option, } impl AvroParserConfig { @@ -120,12 +119,7 @@ impl AvroParserConfig { if avro_config.use_schema_registry { let client = Client::new(url, &avro_config.client_config)?; let resolver = ConfluentSchemaResolver::new(client); - let upsert_primary_key_column_name = - if enable_upsert && !avro_config.upsert_primary_key.is_empty() { - Some(avro_config.upsert_primary_key.clone()) - } else { - None - }; + let subject_key = if enable_upsert { Some(get_subject_by_strategy( &avro_config.name_strategy, @@ -157,7 +151,6 @@ impl AvroParserConfig { None }, schema_resolver: Some(Arc::new(resolver)), - upsert_primary_key_column_name, }) } else { if enable_upsert { @@ -184,7 +177,6 @@ impl AvroParserConfig { schema: Arc::new(schema), key_schema: None, schema_resolver: None, - upsert_primary_key_column_name: None, }) } } diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 584e0e497d1f3..c7b8bf702e1cc 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -794,7 +794,6 @@ impl SpecificParserConfig { pub struct AvroProperties { pub use_schema_registry: bool, pub row_schema_location: String, - pub upsert_primary_key: String, pub client_config: SchemaRegistryAuth, pub aws_auth_props: Option, pub topic: String, @@ -895,7 +894,6 @@ impl SpecificParserConfig { .unwrap(), use_schema_registry: info.use_schema_registry, row_schema_location: info.row_schema_location.clone(), - upsert_primary_key: info.upsert_avro_primary_key.clone(), ..Default::default() }; if format == SourceFormat::Upsert { diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index 214775851103b..f9ce0caa7e254 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -34,7 +34,6 @@ pub struct UpsertParser { payload_builder: AccessBuilderImpl, pub(crate) rw_columns: Vec, source_ctx: SourceContextRef, - avro_primary_key_column_name: Option, } async fn build_accessor_builder( @@ -68,23 +67,18 @@ impl UpsertParser { rw_columns: Vec, source_ctx: SourceContextRef, ) -> Result { - let mut avro_primary_key_column_name = None; - let key_builder: AccessBuilderImpl; // check whether columns has `DEFAULT_KEY_COLUMN_NAME`, if so, the key accessor should be // bytes - if check_rw_default_key(&rw_columns) { - key_builder = AccessBuilderImpl::Bytes(BytesAccessBuilder::new( - EncodingProperties::Bytes(BytesProperties { + let key_builder = if check_rw_default_key(&rw_columns) { + AccessBuilderImpl::Bytes(BytesAccessBuilder::new(EncodingProperties::Bytes( + BytesProperties { column_name: Some(DEFAULT_KEY_COLUMN_NAME.into()), - }), - )?); + }, + ))?) } else { - if let EncodingProperties::Avro(config) = &props.encoding_config { - avro_primary_key_column_name = Some(config.upsert_primary_key.clone()) - } let (key_config, key_type) = extract_key_config!(props); - key_builder = build_accessor_builder(key_config, key_type).await?; - } + build_accessor_builder(key_config, key_type).await? + }; let payload_builder = build_accessor_builder(props.encoding_config, EncodingType::Value).await?; Ok(Self { @@ -92,7 +86,6 @@ impl UpsertParser { payload_builder, rw_columns, source_ctx, - avro_primary_key_column_name, }) } @@ -113,9 +106,6 @@ impl UpsertParser { row_op = row_op.with_value(self.payload_builder.generate_accessor(data).await?); change_event_op = ChangeEventOperation::Upsert; } - if let Some(primary_key_name) = &self.avro_primary_key_column_name { - row_op = row_op.with_key_as_column_name(primary_key_name); - } apply_row_operation_on_stream_chunk_writer_with_op(row_op, &mut writer, change_event_op) .map_err(Into::into) diff --git a/src/frontend/planner_test/tests/testdata/output/create_source.yaml b/src/frontend/planner_test/tests/testdata/output/create_source.yaml index d1d6c314595b8..50178ccb1c6e5 100644 --- a/src/frontend/planner_test/tests/testdata/output/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/create_source.yaml @@ -10,7 +10,7 @@ - id: create_source_without_schema_in_json sql: | create source s with(connector='kafka') FORMAT PLAIN ENCODE JSON; - planner_error: 'Invalid input syntax: schema definition is required for ENCODE JSON' + planner_error: 'Protocol error: Schema definition is required, either from SQL or schema registry.' - id: csv_delimiter_tab sql: | explain create table s0 (v1 int, v2 varchar) with ( diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 85d867d538e7f..e4081fbee4fcf 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -226,7 +226,7 @@ pub async fn handle_create_sink( } /// Transforms the (format, encode, options) from sqlparser AST into an internal struct `SinkFormatDesc`. -/// This is an analogy to (part of) [`crate::handler::create_source::try_bind_columns_from_source`] +/// This is an analogy to (part of) [`crate::handler::create_source::bind_columns_from_source`] /// which transforms sqlparser AST `SourceSchemaV2` into `StreamSourceInfo`. fn bind_sink_format_desc(value: ConnectorSchema) -> Result { use risingwave_connector::sink::catalog::{SinkEncode, SinkFormat}; diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 0ce3e32ed584e..8bca367351641 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -45,8 +45,8 @@ use risingwave_pb::catalog::{ }; use risingwave_pb::plan_common::{EncodeType, FormatType}; use risingwave_sqlparser::ast::{ - self, get_delimiter, AstString, AvroSchema, ColumnDef, ColumnOption, ConnectorSchema, - CreateSourceStatement, DebeziumAvroSchema, Encode, Format, ProtobufSchema, SourceWatermark, + get_delimiter, AstString, AvroSchema, ColumnDef, ConnectorSchema, CreateSourceStatement, + DebeziumAvroSchema, Encode, Format, ProtobufSchema, SourceWatermark, }; use super::RwPgResponse; @@ -54,7 +54,7 @@ use crate::binder::Binder; use crate::catalog::ColumnId; use crate::expr::Expr; use crate::handler::create_table::{ - bind_pk_names, bind_pk_on_relation, bind_sql_column_constraints, bind_sql_columns, + bind_pk_on_relation, bind_sql_column_constraints, bind_sql_columns, bind_sql_pk_names, ensure_table_constraints_supported, ColumnIdGenerator, }; use crate::handler::util::{get_connector, is_kafka_connector}; @@ -110,45 +110,35 @@ async fn extract_avro_table_schema( .collect_vec()) } -/// Map an Avro schema to a relational schema. And extract primary key columns. -async fn extract_upsert_avro_table_schema( +/// Extract Avro primary key columns. +async fn extract_upsert_avro_table_pk_columns( info: &StreamSourceInfo, with_properties: &HashMap, -) -> Result<(Vec, Vec)> { +) -> Result>> { let parser_config = SpecificParserConfig::new(info, with_properties)?; let conf = AvroParserConfig::new(parser_config.encoding_config).await?; let vec_column_desc = conf.map_to_columns()?; - let mut vec_column_catalog = vec_column_desc - .clone() - .into_iter() - .map(|col| ColumnCatalog { - column_desc: col.into(), - is_hidden: false, - }) - .collect_vec(); - // For upsert avro, if we can't extract pk from schema, use message key as primary key - let pks = if let Ok(pk_desc) = conf.extract_pks() { - pk_desc - .into_iter() - .map(|desc| { - vec_column_desc - .iter() - .find(|x| x.name == desc.name) - .ok_or_else(|| { - RwError::from(ErrorCode::InternalError(format!( - "Can not found primary key column {} in value schema", - desc.name - ))) - }) - }) - .map_ok(|desc| desc.name.clone()) - .collect::>>()? - } else { - add_upsert_default_key_column(&mut vec_column_catalog); - vec![DEFAULT_KEY_COLUMN_NAME.into()] - }; - Ok((vec_column_catalog, pks)) + conf.extract_pks() + .ok() + .map(|pk_desc| { + pk_desc + .into_iter() + .map(|desc| { + vec_column_desc + .iter() + .find(|x| x.name == desc.name) + .ok_or_else(|| { + RwError::from(ErrorCode::InternalError(format!( + "Can not found primary key column {} in value schema", + desc.name + ))) + }) + }) + .map_ok(|desc| desc.name.clone()) + .collect::>>() + }) + .transpose() } async fn extract_debezium_avro_table_pk_columns( @@ -208,11 +198,7 @@ async fn extract_protobuf_table_schema( fn non_generated_sql_columns(columns: &[ColumnDef]) -> Vec { columns .iter() - .filter(|c| { - c.options - .iter() - .all(|option| !matches!(option.option, ColumnOption::GeneratedColumns(_))) - }) + .filter(|c| !c.is_generated()) .cloned() .collect() } @@ -275,19 +261,15 @@ fn get_name_strategy_or_default(name_strategy: Option) -> Result for more information. -/// return `(columns, pk_names, source info)` -pub(crate) async fn try_bind_columns_from_source( +/// return `(columns, source info)` +pub(crate) async fn bind_columns_from_source( source_schema: &ConnectorSchema, - sql_defined_pk_names: Vec, - sql_defined_columns: &[ColumnDef], with_properties: &HashMap, -) -> Result<(Option>, Vec, StreamSourceInfo)> { +) -> Result<(Option>, StreamSourceInfo)> { const MESSAGE_NAME_KEY: &str = "message"; const KEY_MESSAGE_NAME_KEY: &str = "key.message"; const NAME_STRATEGY_KEY: &str = "schema.registry.name.strategy"; - let sql_defined_pk = !sql_defined_pk_names.is_empty(); - let sql_defined_schema = !sql_defined_columns.is_empty(); let is_kafka: bool = is_kafka_connector(with_properties); let mut options = WithOptions::try_from(source_schema.row_options())?.into_inner(); @@ -314,7 +296,6 @@ pub(crate) async fn try_bind_columns_from_source( let res = match (&source_schema.format, &source_schema.row_encode) { (Format::Native, Encode::Native) => ( None, - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Native as i32, row_encode: EncodeType::Native as i32, @@ -322,10 +303,6 @@ pub(crate) async fn try_bind_columns_from_source( }, ), (Format::Plain, Encode::Protobuf) => { - if sql_defined_schema { - return Err(RwError::from(ProtocolError( - "User-defined schema is not allowed with FORMAT PLAIN ENCODE PROTOBUF. Please refer to https://www.risingwave.dev/docs/current/sql-create-source/#protobuf for more information.".to_string()))); - }; let (row_schema_location, use_schema_registry) = get_schema_location(&mut options)?; let protobuf_schema = ProtobufSchema { message_name: consume_string_from_options(&mut options, MESSAGE_NAME_KEY)?, @@ -340,7 +317,6 @@ pub(crate) async fn try_bind_columns_from_source( extract_protobuf_table_schema(&protobuf_schema, with_properties.clone()) .await?, ), - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Plain as i32, row_encode: EncodeType::Protobuf as i32, @@ -357,19 +333,8 @@ pub(crate) async fn try_bind_columns_from_source( } (Format::Plain, Encode::Json) => { let schema_config = get_json_schema_location(&mut options)?; - if schema_config.is_some() && sql_defined_schema { - return Err(RwError::from(ProtocolError( - "User-defined schema is not allowed with schema registry.".to_string(), - ))); - } - if schema_config.is_none() && sql_defined_columns.is_empty() { - return Err(RwError::from(InvalidInputSyntax( - "schema definition is required for ENCODE JSON".to_owned(), - ))); - } ( extract_json_table_schema(&schema_config, with_properties).await?, - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Plain as i32, row_encode: EncodeType::Json as i32, @@ -384,10 +349,7 @@ pub(crate) async fn try_bind_columns_from_source( row_schema_location, use_schema_registry, }; - if sql_defined_schema { - return Err(RwError::from(ProtocolError( - "User-defined schema is not allowed with FORMAT PLAIN ENCODE AVRO. Please refer to https://www.risingwave.dev/docs/current/sql-create-source/#avro for more information.".to_string()))); - } + let key_message_name = get_key_message_name(&mut options); let message_name = try_consume_string_from_options(&mut options, MESSAGE_NAME_KEY); let name_strategy = @@ -405,7 +367,6 @@ pub(crate) async fn try_bind_columns_from_source( }; ( Some(extract_avro_table_schema(&stream_source_info, with_properties).await?), - sql_defined_pk_names, stream_source_info, ) } @@ -425,7 +386,6 @@ pub(crate) async fn try_bind_columns_from_source( } ( None, - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Plain as i32, row_encode: EncodeType::Csv as i32, @@ -435,48 +395,20 @@ pub(crate) async fn try_bind_columns_from_source( }, ) } - (Format::Plain, Encode::Bytes) => { - if !sql_defined_schema || sql_defined_columns.len() != 1 { - return Err(RwError::from(ProtocolError( - "BYTES format only accepts one column".to_string(), - ))); - } - - match sql_defined_columns[0].data_type { - Some(ast::DataType::Bytea) => {} - _ => { - return Err(RwError::from(ProtocolError( - "BYTES format only accepts BYTEA type".to_string(), - ))) - } - } - - ( - None, - sql_defined_pk_names, - StreamSourceInfo { - format: FormatType::Plain as i32, - row_encode: EncodeType::Bytes as i32, - ..Default::default() - }, - ) - } + (Format::Plain, Encode::Bytes) => ( + None, + StreamSourceInfo { + format: FormatType::Plain as i32, + row_encode: EncodeType::Bytes as i32, + ..Default::default() + }, + ), (Format::Upsert, Encode::Json) => { let schema_config = get_json_schema_location(&mut options)?; let columns = extract_json_table_schema(&schema_config, with_properties).await?; - let (columns, pk_names) = if !sql_defined_pk { - let mut columns = match columns { - None => bind_sql_columns(sql_defined_columns)?, - Some(columns) => columns, - }; - add_upsert_default_key_column(&mut columns); - (Some(columns), vec![DEFAULT_KEY_COLUMN_NAME.into()]) - } else { - (columns, sql_defined_pk_names) - }; + ( columns, - pk_names, StreamSourceInfo { format: FormatType::Upsert as i32, row_encode: EncodeType::Json as i32, @@ -491,10 +423,6 @@ pub(crate) async fn try_bind_columns_from_source( row_schema_location, use_schema_registry, }; - if sql_defined_schema { - return Err(RwError::from(ProtocolError( - "User-defined schema is not allowed with row format upsert avro. Please refer to https://www.risingwave.dev/docs/current/sql-create-source/#avro for more information.".to_string()))); - } let name_strategy = get_sr_name_strategy_check(&mut options, avro_schema.use_schema_registry)? @@ -502,57 +430,25 @@ pub(crate) async fn try_bind_columns_from_source( let key_message_name = get_key_message_name(&mut options); let message_name = try_consume_string_from_options(&mut options, MESSAGE_NAME_KEY); - if sql_defined_pk { - if sql_defined_pk_names.len() != 1 { - return Err(RwError::from(ProtocolError( - "upsert avro supports only one primary key column.".to_string(), - ))); - } - let upsert_avro_primary_key = sql_defined_pk_names[0].clone(); - - let stream_source_info = StreamSourceInfo { - key_message_name, - format: FormatType::Upsert as i32, - row_encode: EncodeType::Avro as i32, - row_schema_location: avro_schema.row_schema_location.0.clone(), - use_schema_registry: avro_schema.use_schema_registry, - proto_message_name: message_name.unwrap_or(AstString("".into())).0, - upsert_avro_primary_key, - name_strategy, - ..Default::default() - }; - let columns = - extract_avro_table_schema(&stream_source_info, with_properties).await?; + let stream_source_info = StreamSourceInfo { + key_message_name, + format: FormatType::Upsert as i32, + row_encode: EncodeType::Avro as i32, + row_schema_location: avro_schema.row_schema_location.0.clone(), + use_schema_registry: avro_schema.use_schema_registry, + proto_message_name: message_name.unwrap_or(AstString("".into())).0, + name_strategy, + ..Default::default() + }; + let columns = extract_avro_table_schema(&stream_source_info, with_properties).await?; - (Some(columns), sql_defined_pk_names, stream_source_info) - } else { - let stream_source_info = StreamSourceInfo { - format: FormatType::Upsert as i32, - row_encode: EncodeType::Avro as i32, - row_schema_location: avro_schema.row_schema_location.0.clone(), - use_schema_registry: avro_schema.use_schema_registry, - proto_message_name: message_name.unwrap_or(AstString("".into())).0, - name_strategy, - key_message_name, - ..Default::default() - }; - let (columns, pk_from_avro) = - extract_upsert_avro_table_schema(&stream_source_info, with_properties).await?; - (Some(columns), pk_from_avro, stream_source_info) - } + (Some(columns), stream_source_info) } (Format::Debezium, Encode::Json) => { - if !sql_defined_pk { - return Err(RwError::from(ProtocolError( - "Primary key must be specified when creating source with format debezium." - .to_string(), - ))); - } let schema_config = get_json_schema_location(&mut options)?; ( extract_json_table_schema(&schema_config, with_properties).await?, - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Debezium as i32, row_encode: EncodeType::Json as i32, @@ -571,11 +467,6 @@ pub(crate) async fn try_bind_columns_from_source( let avro_schema = DebeziumAvroSchema { row_schema_location, }; - if sql_defined_schema { - return Err(RwError::from(ProtocolError( - "User-defined schema is not allowed with row format debezium avro.".to_string(), - ))); - } // no need to check whether works schema registry because debezium avro always work with // schema registry @@ -597,121 +488,22 @@ pub(crate) async fn try_bind_columns_from_source( let full_columns = extract_debezium_avro_table_schema(&stream_source_info, with_properties).await?; - let pk_names = if sql_defined_pk { - sql_defined_pk_names - } else { - let pk_names = - extract_debezium_avro_table_pk_columns(&stream_source_info, with_properties) - .await?; - // extract pk(s) from schema registry - for pk_name in &pk_names { - full_columns - .iter() - .find(|c: &&ColumnCatalog| c.name().eq(pk_name)) - .ok_or_else(|| { - RwError::from(ProtocolError(format!( - "avro's key column {} not exists in avro's row schema", - pk_name - ))) - })?; - } - pk_names - }; - (Some(full_columns), pk_names, stream_source_info) - } - (Format::DebeziumMongo, Encode::Json) => { - let mut columns = vec![ - ColumnCatalog { - column_desc: ColumnDesc { - data_type: DataType::Varchar, - column_id: 0.into(), - name: "_id".to_string(), - field_descs: vec![], - type_name: "".to_string(), - generated_or_default_column: None, - description: None, - }, - is_hidden: false, - }, - ColumnCatalog { - column_desc: ColumnDesc { - data_type: DataType::Jsonb, - column_id: 0.into(), - name: "payload".to_string(), - field_descs: vec![], - type_name: "".to_string(), - generated_or_default_column: None, - description: None, - }, - is_hidden: false, - }, - ]; - if sql_defined_schema { - let non_generated_sql_defined_columns = - non_generated_sql_columns(sql_defined_columns); - if non_generated_sql_defined_columns.len() != 2 - && non_generated_sql_defined_columns[0].name.real_value() != columns[0].name() - && non_generated_sql_defined_columns[1].name.real_value() != columns[1].name() - { - return Err(RwError::from(ProtocolError( - "the not generated columns of the source with row format DebeziumMongoJson - must be (_id [Jsonb | Varchar | Int32 | Int64], payload jsonb)." - .to_string(), - ))); - } - if let Some(key_data_type) = &non_generated_sql_defined_columns[0].data_type { - let key_data_type = bind_data_type(key_data_type)?; - match key_data_type { - DataType::Jsonb | DataType::Varchar | DataType::Int32 | DataType::Int64 => { - columns[0].column_desc.data_type = key_data_type; - } - _ => { - return Err(RwError::from(ProtocolError( - "the `_id` column of the source with row format DebeziumMongoJson - must be [Jsonb | Varchar | Int32 | Int64]" - .to_string(), - ))); - } - } - } - if let Some(value_data_type) = &non_generated_sql_defined_columns[1].data_type { - if !matches!(bind_data_type(value_data_type)?, DataType::Jsonb) { - return Err(RwError::from(ProtocolError( - "the `payload` column of the source with row format DebeziumMongoJson - must be Jsonb datatype" - .to_string(), - ))); - } - } - } - let pk_names = if sql_defined_pk { - sql_defined_pk_names - } else { - vec!["_id".to_string()] - }; - ( - Some(columns), - pk_names, - StreamSourceInfo { - format: FormatType::DebeziumMongo as i32, - row_encode: EncodeType::Json as i32, - ..Default::default() - }, - ) + (Some(full_columns), stream_source_info) } + (Format::DebeziumMongo, Encode::Json) => ( + None, + StreamSourceInfo { + format: FormatType::DebeziumMongo as i32, + row_encode: EncodeType::Json as i32, + ..Default::default() + }, + ), (Format::Maxwell, Encode::Json) => { - if !sql_defined_pk { - return Err(RwError::from(ProtocolError( - "Primary key must be specified when creating source with FORMAT MAXWELL ENCODE JSON." - .to_string(), - ))); - } let schema_config = get_json_schema_location(&mut options)?; ( extract_json_table_schema(&schema_config, with_properties).await?, - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Maxwell as i32, row_encode: EncodeType::Json as i32, @@ -722,16 +514,9 @@ pub(crate) async fn try_bind_columns_from_source( } (Format::Canal, Encode::Json) => { - if !sql_defined_pk { - return Err(RwError::from(ProtocolError( - "Primary key must be specified when creating source with row format cannal_json." - .to_string(), - ))); - } let schema_config = get_json_schema_location(&mut options)?; ( extract_json_table_schema(&schema_config, with_properties).await?, - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Canal as i32, row_encode: EncodeType::Json as i32, @@ -762,6 +547,225 @@ pub(crate) async fn try_bind_columns_from_source( Ok(res) } +/// Bind columns from both source and sql defined. +pub(crate) fn bind_all_columns( + source_schema: &ConnectorSchema, + cols_from_source: Option>, + cols_from_sql: Vec, + col_defs_from_sql: &[ColumnDef], +) -> Result> { + if let Some(cols_from_source) = cols_from_source { + if cols_from_sql.is_empty() { + Ok(cols_from_source) + } else { + // TODO(yuhao): https://github.com/risingwavelabs/risingwave/issues/12209 + Err(RwError::from(ProtocolError( + format!("User-defined schema from SQL is not allowed with FORMAT {} ENCODE {}. \ + Please refer to https://www.risingwave.dev/docs/current/sql-create-source/ for more information.", source_schema.format, source_schema.row_encode)))) + } + } else { + // FIXME(yuhao): cols_from_sql should be None is no `()` is given. + if cols_from_sql.is_empty() { + return Err(RwError::from(ProtocolError( + "Schema definition is required, either from SQL or schema registry.".to_string(), + ))); + } + match (&source_schema.format, &source_schema.row_encode) { + (Format::DebeziumMongo, Encode::Json) => { + let mut columns = vec![ + ColumnCatalog { + column_desc: ColumnDesc { + data_type: DataType::Varchar, + column_id: 0.into(), + name: "_id".to_string(), + field_descs: vec![], + type_name: "".to_string(), + generated_or_default_column: None, + description: None, + }, + is_hidden: false, + }, + ColumnCatalog { + column_desc: ColumnDesc { + data_type: DataType::Jsonb, + column_id: 0.into(), + name: "payload".to_string(), + field_descs: vec![], + type_name: "".to_string(), + generated_or_default_column: None, + description: None, + }, + is_hidden: false, + }, + ]; + let non_generated_sql_defined_columns = + non_generated_sql_columns(col_defs_from_sql); + if non_generated_sql_defined_columns.len() != 2 + || non_generated_sql_defined_columns[0].name.real_value() != columns[0].name() + || non_generated_sql_defined_columns[1].name.real_value() != columns[1].name() + { + return Err(RwError::from(ProtocolError( + "the not generated columns of the source with row format DebeziumMongoJson + must be (_id [Jsonb | Varchar | Int32 | Int64], payload jsonb)." + .to_string(), + ))); + } + // ok to unwrap since it was checked at `bind_sql_columns` + let key_data_type = bind_data_type( + non_generated_sql_defined_columns[0] + .data_type + .as_ref() + .unwrap(), + )?; + match key_data_type { + DataType::Jsonb | DataType::Varchar | DataType::Int32 | DataType::Int64 => { + columns[0].column_desc.data_type = key_data_type.clone(); + } + _ => { + return Err(RwError::from(ProtocolError( + "the `_id` column of the source with row format DebeziumMongoJson + must be [Jsonb | Varchar | Int32 | Int64]" + .to_string(), + ))); + } + } + + // ok to unwrap since it was checked at `bind_sql_columns` + let value_data_type = bind_data_type( + non_generated_sql_defined_columns[1] + .data_type + .as_ref() + .unwrap(), + )?; + if !matches!(value_data_type, DataType::Jsonb) { + return Err(RwError::from(ProtocolError( + "the `payload` column of the source with row format DebeziumMongoJson + must be Jsonb datatype" + .to_string(), + ))); + } + Ok(columns) + } + (Format::Plain, Encode::Bytes) => { + if cols_from_sql.len() != 1 || cols_from_sql[0].data_type() != &DataType::Bytea { + return Err(RwError::from(ProtocolError( + "ENCODE BYTES only accepts one BYTEA type column".to_string(), + ))); + } + Ok(cols_from_sql) + } + (_, _) => Ok(cols_from_sql), + } + } +} + +/// Bind column from source. Add key column to table columns if necessary. +/// Return (columns, pks) +pub(crate) async fn bind_source_pk( + source_schema: &ConnectorSchema, + source_info: &StreamSourceInfo, + columns: &mut Vec, + sql_defined_pk_names: Vec, + with_properties: &HashMap, +) -> Result> { + let sql_defined_pk = !sql_defined_pk_names.is_empty(); + + let res = match (&source_schema.format, &source_schema.row_encode) { + (Format::Native, Encode::Native) | (Format::Plain, _) => sql_defined_pk_names, + (Format::Upsert, Encode::Json) => { + if sql_defined_pk { + sql_defined_pk_names + } else { + add_upsert_default_key_column(columns); + vec![DEFAULT_KEY_COLUMN_NAME.into()] + } + } + (Format::Upsert, Encode::Avro) => { + if sql_defined_pk { + if sql_defined_pk_names.len() != 1 { + return Err(RwError::from(ProtocolError( + "upsert avro supports only one primary key column.".to_string(), + ))); + } + sql_defined_pk_names + } else if let Some(extracted_pk_names) = + extract_upsert_avro_table_pk_columns(source_info, with_properties).await? + { + extracted_pk_names + } else { + // For upsert avro, if we can't extract pk from schema, use message key as primary key + add_upsert_default_key_column(columns); + vec![DEFAULT_KEY_COLUMN_NAME.into()] + } + } + + (Format::Debezium, Encode::Json) => { + if !sql_defined_pk { + return Err(RwError::from(ProtocolError( + "Primary key must be specified when creating source with FORMAT DEBEZIUM." + .to_string(), + ))); + } + sql_defined_pk_names + } + (Format::Debezium, Encode::Avro) => { + if sql_defined_pk { + sql_defined_pk_names + } else { + let pk_names = + extract_debezium_avro_table_pk_columns(source_info, with_properties).await?; + // extract pk(s) from schema registry + for pk_name in &pk_names { + columns + .iter() + .find(|c: &&ColumnCatalog| c.name().eq(pk_name)) + .ok_or_else(|| { + RwError::from(ProtocolError(format!( + "avro's key column {} not exists in avro's row schema", + pk_name + ))) + })?; + } + pk_names + } + } + (Format::DebeziumMongo, Encode::Json) => { + if sql_defined_pk { + sql_defined_pk_names + } else { + vec!["_id".to_string()] + } + } + + (Format::Maxwell, Encode::Json) => { + if !sql_defined_pk { + return Err(RwError::from(ProtocolError( + "Primary key must be specified when creating source with FORMAT MAXWELL ENCODE JSON." + .to_string(), + ))); + } + sql_defined_pk_names + } + + (Format::Canal, Encode::Json) => { + if !sql_defined_pk { + return Err(RwError::from(ProtocolError( + "Primary key must be specified when creating source with FORMAT CANAL ENCODE JSON." + .to_string(), + ))); + } + sql_defined_pk_names + } + (format, encoding) => { + return Err(RwError::from(ProtocolError(format!( + "Unknown combination {:?} {:?}", + format, encoding + )))); + } + }; + Ok(res) +} + // Add a hidden column `_rw_kafka_timestamp` to each message from Kafka source. fn check_and_add_timestamp_column( with_properties: &HashMap, @@ -1070,14 +1074,26 @@ pub async fn handle_create_source( validate_compatibility(&source_schema, &mut with_properties)?; ensure_table_constraints_supported(&stmt.constraints)?; - let pk_names = bind_pk_names(&stmt.columns, &stmt.constraints)?; + let sql_pk_names = bind_sql_pk_names(&stmt.columns, &stmt.constraints)?; - let (columns_from_resolve_source, pk_names, source_info) = - try_bind_columns_from_source(&source_schema, pk_names, &stmt.columns, &with_properties) - .await?; + let (columns_from_resolve_source, source_info) = + bind_columns_from_source(&source_schema, &with_properties).await?; let columns_from_sql = bind_sql_columns(&stmt.columns)?; - let mut columns = columns_from_resolve_source.unwrap_or(columns_from_sql); + let mut columns = bind_all_columns( + &source_schema, + columns_from_resolve_source, + columns_from_sql, + &stmt.columns, + )?; + let pk_names = bind_source_pk( + &source_schema, + &source_info, + &mut columns, + sql_pk_names, + &with_properties, + ) + .await?; check_and_add_timestamp_column(&with_properties, &mut columns); diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index e412658cb712f..efa4278bcbc99 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -43,8 +43,8 @@ use crate::catalog::table_catalog::TableVersion; use crate::catalog::{check_valid_column_name, CatalogError, ColumnId}; use crate::expr::{Expr, ExprImpl, ExprRewriter, InlineNowProcTime}; use crate::handler::create_source::{ - bind_source_watermark, check_source_schema, try_bind_columns_from_source, - validate_compatibility, UPSTREAM_SOURCE_KEY, + bind_all_columns, bind_columns_from_source, bind_source_pk, bind_source_watermark, + check_source_schema, validate_compatibility, UPSTREAM_SOURCE_KEY, }; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::LogicalSource; @@ -346,7 +346,7 @@ pub fn ensure_table_constraints_supported(table_constraints: &[TableConstraint]) Ok(()) } -pub fn bind_pk_names( +pub fn bind_sql_pk_names( columns_defs: &[ColumnDef], table_constraints: &[TableConstraint], ) -> Result> { @@ -457,13 +457,27 @@ pub(crate) async fn gen_create_table_plan_with_source( validate_compatibility(&source_schema, &mut properties)?; ensure_table_constraints_supported(&constraints)?; - let pk_names = bind_pk_names(&column_defs, &constraints)?; - let (columns_from_resolve_source, pk_names, mut source_info) = - try_bind_columns_from_source(&source_schema, pk_names, &column_defs, &properties).await?; + let sql_pk_names = bind_sql_pk_names(&column_defs, &constraints)?; + + let (columns_from_resolve_source, mut source_info) = + bind_columns_from_source(&source_schema, &properties).await?; let columns_from_sql = bind_sql_columns(&column_defs)?; - let mut columns = columns_from_resolve_source.unwrap_or(columns_from_sql); + let mut columns = bind_all_columns( + &source_schema, + columns_from_resolve_source, + columns_from_sql, + &column_defs, + )?; + let pk_names = bind_source_pk( + &source_schema, + &source_info, + &mut columns, + sql_pk_names, + &properties, + ) + .await?; for c in &mut columns { c.column_desc.column_id = col_id_gen.generate(c.name()) @@ -605,7 +619,7 @@ pub(crate) fn gen_create_table_plan_without_bind( version: Option, ) -> Result<(PlanRef, Option, PbTable)> { ensure_table_constraints_supported(&constraints)?; - let pk_names = bind_pk_names(&column_defs, &constraints)?; + let pk_names = bind_sql_pk_names(&column_defs, &constraints)?; let (mut columns, pk_column_ids, row_id_index) = bind_pk_on_relation(columns, pk_names)?; let watermark_descs = bind_source_watermark( @@ -976,7 +990,7 @@ mod tests { c.column_desc.column_id = col_id_gen.generate(c.name()) } ensure_table_constraints_supported(&constraints)?; - let pk_names = bind_pk_names(&column_defs, &constraints)?; + let pk_names = bind_sql_pk_names(&column_defs, &constraints)?; let (_, pk_column_ids, _) = bind_pk_on_relation(columns, pk_names)?; Ok(pk_column_ids) })(); diff --git a/src/sqlparser/src/ast/ddl.rs b/src/sqlparser/src/ast/ddl.rs index 8cdc4ac2aa865..ae497d777c4d0 100644 --- a/src/sqlparser/src/ast/ddl.rs +++ b/src/sqlparser/src/ast/ddl.rs @@ -363,6 +363,12 @@ impl ColumnDef { options, } } + + pub fn is_generated(&self) -> bool { + self.options + .iter() + .any(|option| matches!(option.option, ColumnOption::GeneratedColumns(_))) + } } impl fmt::Display for ColumnDef { From a9d9bc980248d0bc4a81d5ab29d4d029494d6eef Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Mon, 30 Oct 2023 16:58:18 +0800 Subject: [PATCH 03/14] fix(expr): `null` input of `~` / `!~` (#13145) --- e2e_test/batch/basic/pg_regrex_match_op.slt.part | 10 ++++++++++ proto/expr.proto | 1 + src/expr/impl/src/scalar/regexp.rs | 9 +++++++++ .../planner_test/tests/testdata/output/subquery.yaml | 4 ++-- src/frontend/src/binder/expr/binary_op.rs | 9 +++------ src/frontend/src/expr/pure.rs | 1 + 6 files changed, 26 insertions(+), 8 deletions(-) diff --git a/e2e_test/batch/basic/pg_regrex_match_op.slt.part b/e2e_test/batch/basic/pg_regrex_match_op.slt.part index 0988f078d3401..cd12a5296b868 100644 --- a/e2e_test/batch/basic/pg_regrex_match_op.slt.part +++ b/e2e_test/batch/basic/pg_regrex_match_op.slt.part @@ -7,3 +7,13 @@ query T select 'foobarbequebazilbarfbonk' !~ '(b[^b]+)(b[^b]+)'; ---- f + +query T +select 'foobarbequebazilbarfbonk' ~ null; +---- +NULL + +query T +select null !~ '(b[^b]+)(b[^b]+)'; +---- +NULL diff --git a/proto/expr.proto b/proto/expr.proto index fecefc12b4ee7..4437a548beec3 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -107,6 +107,7 @@ message ExprNode { REGEXP_REPLACE = 280; REGEXP_COUNT = 281; REGEXP_SPLIT_TO_ARRAY = 282; + REGEXP_EQ = 283; POW = 233; EXP = 234; CHR = 235; diff --git a/src/expr/impl/src/scalar/regexp.rs b/src/expr/impl/src/scalar/regexp.rs index d919334fe2c98..64234a8384dfe 100644 --- a/src/expr/impl/src/scalar/regexp.rs +++ b/src/expr/impl/src/scalar/regexp.rs @@ -134,6 +134,15 @@ impl FromStr for RegexpOptions { } } +#[function( + // source ~ pattern + "regexp_eq(varchar, varchar) -> boolean", + prebuild = "RegexpContext::from_pattern($1)?" +)] +fn regexp_eq(text: &str, regex: &RegexpContext) -> bool { + regex.regex.is_match(text).unwrap() +} + #[function( // regexp_match(source, pattern) "regexp_match(varchar, varchar) -> varchar[]", diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 69b6e21592c0a..309892e692b02 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -237,7 +237,7 @@ logical_plan: |- LogicalProject { exprs: [rw_schemas.name, rw_tables.name, Case(($expr1 = 'r':Varchar), 'table':Varchar, ($expr1 = 'v':Varchar), 'view':Varchar, ($expr1 = 'm':Varchar), 'materialized view':Varchar, ($expr1 = 'i':Varchar), 'index':Varchar, ($expr1 = 'S':Varchar), 'sequence':Varchar, ($expr1 = 's':Varchar), 'special':Varchar, ($expr1 = 't':Varchar), 'TOAST table':Varchar, ($expr1 = 'f':Varchar), 'foreign table':Varchar, ($expr1 = 'p':Varchar), 'partitioned table':Varchar, ($expr1 = 'I':Varchar), 'partitioned index':Varchar) as $expr3, rw_users.name] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalFilter { predicate: In($expr1, 'r':Varchar, 'p':Varchar, 'v':Varchar, 'm':Varchar, 'S':Varchar, 'f':Varchar, '':Varchar) AND (rw_schemas.name <> 'pg_catalog':Varchar) AND IsNull(RegexpMatch(rw_schemas.name, '^pg_toast':Varchar)) AND (rw_schemas.name <> 'information_schema':Varchar) } + ├─LogicalFilter { predicate: In($expr1, 'r':Varchar, 'p':Varchar, 'v':Varchar, 'm':Varchar, 'S':Varchar, 'f':Varchar, '':Varchar) AND (rw_schemas.name <> 'pg_catalog':Varchar) AND Not(RegexpEq(rw_schemas.name, '^pg_toast':Varchar)) AND (rw_schemas.name <> 'information_schema':Varchar) } │ └─LogicalJoin { type: LeftOuter, on: (rw_schemas.id = rw_tables.schema_id), output: all } │ ├─LogicalShare { id: 16 } │ │ └─LogicalProject { exprs: [rw_tables.id, rw_tables.name, rw_tables.schema_id, rw_tables.owner, Case(('table':Varchar = 'table':Varchar), 'r':Varchar, ('table':Varchar = 'system table':Varchar), 'r':Varchar, ('table':Varchar = 'index':Varchar), 'i':Varchar, ('table':Varchar = 'view':Varchar), 'v':Varchar, ('table':Varchar = 'materialized view':Varchar), 'm':Varchar) as $expr1, 0:Int32, 0:Int32, Array as $expr2] } @@ -302,7 +302,7 @@ │ │ └─BatchFilter { predicate: true } │ │ └─BatchScan { table: rw_views, columns: [rw_views.name, rw_views.schema_id, rw_views.owner], distribution: Single } │ └─BatchExchange { order: [], dist: HashShard(rw_schemas.id) } - │ └─BatchFilter { predicate: (rw_schemas.name <> 'pg_catalog':Varchar) AND IsNull(RegexpMatch(rw_schemas.name, '^pg_toast':Varchar)) AND (rw_schemas.name <> 'information_schema':Varchar) } + │ └─BatchFilter { predicate: (rw_schemas.name <> 'pg_catalog':Varchar) AND Not(RegexpEq(rw_schemas.name, '^pg_toast':Varchar)) AND (rw_schemas.name <> 'information_schema':Varchar) } │ └─BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name], distribution: Single } └─BatchExchange { order: [], dist: HashShard(rw_users.id) } └─BatchProject { exprs: [rw_users.name, rw_users.id] } diff --git a/src/frontend/src/binder/expr/binary_op.rs b/src/frontend/src/binder/expr/binary_op.rs index 352d2bfbfd246..33f6f366e6496 100644 --- a/src/frontend/src/binder/expr/binary_op.rs +++ b/src/frontend/src/binder/expr/binary_op.rs @@ -142,13 +142,10 @@ impl Binder { } } } - BinaryOperator::PGRegexMatch => { - func_types.push(ExprType::IsNotNull); - ExprType::RegexpMatch - } + BinaryOperator::PGRegexMatch => ExprType::RegexpEq, BinaryOperator::PGRegexNotMatch => { - func_types.push(ExprType::IsNull); - ExprType::RegexpMatch + func_types.push(ExprType::Not); + ExprType::RegexpEq } _ => { return Err( diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index 7a63c7f95ae99..cebd59322c2b9 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -108,6 +108,7 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::RegexpReplace | expr_node::Type::RegexpCount | expr_node::Type::RegexpSplitToArray + | expr_node::Type::RegexpEq | expr_node::Type::Pow | expr_node::Type::Exp | expr_node::Type::Ln From c0ab3ca01c05a95c4d1b07be22684f6b1fbffea3 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Mon, 30 Oct 2023 17:46:14 +0800 Subject: [PATCH 04/14] feat(expr): support jsonb `-` and `#-` operator (#13118) Signed-off-by: Runji Wang --- Cargo.lock | 4 +- proto/expr.proto | 7 + src/common/src/types/jsonb.rs | 8 + src/expr/impl/src/scalar/jsonb_delete.rs | 369 ++++++++++++++++++++++ src/expr/impl/src/scalar/mod.rs | 1 + src/frontend/src/binder/expr/binary_op.rs | 1 + src/frontend/src/binder/expr/function.rs | 2 + src/frontend/src/expr/pure.rs | 1 + src/sqlparser/src/ast/operator.rs | 2 + src/sqlparser/src/parser.rs | 2 + src/sqlparser/src/tokenizer.rs | 4 + src/tests/regress/data/sql/jsonb.sql | 91 +++--- 12 files changed, 444 insertions(+), 48 deletions(-) create mode 100644 src/expr/impl/src/scalar/jsonb_delete.rs diff --git a/Cargo.lock b/Cargo.lock index 498002620379d..92bbc847031cc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4137,9 +4137,9 @@ dependencies = [ [[package]] name = "jsonbb" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44376417b2ff0cd879b5c84976fa9e0855c316321b4e0502e33e52963bf84f74" +checksum = "efd95430271266a57cbb8fd31115559c853fcaa5f367d32c4720034f7bd37b7f" dependencies = [ "bytes", "serde", diff --git a/proto/expr.proto b/proto/expr.proto index 4437a548beec3..0d29cabd494fb 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -237,6 +237,13 @@ message ExprNode { JSONB_EXISTS_ANY = 611; // jsonb ?& text[] JSONB_EXISTS_ALL = 612; + // see SUBTRACT for: + // jsonb - text -> jsonb + // jsonb - text[] -> jsonb + // jsonb - integer -> jsonb + // + // jsonb #- text[] -> jsonb + JSONB_DELETE_PATH = 615; // Non-pure functions below (> 1000) // ------------------------ diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index 664af6c0b1921..15345ac7cfe20 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -293,6 +293,14 @@ impl<'a> JsonbRef<'a> { self.0.as_null().is_some() } + /// Returns true if this is a jsonb null, boolean, number or string. + pub fn is_scalar(&self) -> bool { + matches!( + self.0, + ValueRef::Null | ValueRef::Bool(_) | ValueRef::Number(_) | ValueRef::String(_) + ) + } + /// Returns true if this is a jsonb array. pub fn is_array(&self) -> bool { matches!(self.0, ValueRef::Array(_)) diff --git a/src/expr/impl/src/scalar/jsonb_delete.rs b/src/expr/impl/src/scalar/jsonb_delete.rs new file mode 100644 index 0000000000000..dff60f34f722b --- /dev/null +++ b/src/expr/impl/src/scalar/jsonb_delete.rs @@ -0,0 +1,369 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashSet; + +use jsonbb::{Value, ValueRef}; +use risingwave_common::types::{JsonbRef, JsonbVal, ListRef}; +use risingwave_expr::{function, ExprError, Result}; + +/// Removes a key (and its value) from a JSON object, or matching string value(s) from a JSON array. +/// +/// Examples: +/// +/// ```slt +/// # remove key from object +/// query T +/// SELECT '{"a": "b", "c": "d"}'::jsonb - 'a'; +/// ---- +/// {"c": "d"} +/// +/// # remove matching value from array +/// query T +/// SELECT '["a", "b", "c", "b"]'::jsonb - 'b'; +/// ---- +/// ["a", "c"] +/// +/// query error cannot delete from scalar +/// SELECT '1'::jsonb - 'b'; +/// ``` +#[function("subtract(jsonb, varchar) -> jsonb")] +fn jsonb_remove(v: JsonbRef<'_>, key: &str) -> Result { + match v.into() { + ValueRef::Object(obj) => Ok(JsonbVal::from(Value::object( + obj.iter().filter(|(k, _)| *k != key), + ))), + ValueRef::Array(arr) => Ok(JsonbVal::from(Value::array( + arr.iter().filter(|value| value.as_str() != Some(key)), + ))), + _ => Err(ExprError::InvalidParam { + name: "jsonb", + reason: "cannot delete from scalar".into(), + }), + } +} + +/// Deletes all matching keys or array elements from the left operand. +/// +/// Examples: +/// +/// ```slt +/// query T +/// SELECT '{"a": "b", "c": "d"}'::jsonb - '{a,c}'::text[]; +/// ---- +/// {} +/// +/// query error cannot delete from scalar +/// SELECT '1'::jsonb - '{a,c}'::text[]; +/// ``` +#[function("subtract(jsonb, varchar[]) -> jsonb")] +fn jsonb_remove_keys(v: JsonbRef<'_>, keys: ListRef<'_>) -> Result { + let keys_set: HashSet<&str> = keys.iter().flatten().map(|s| s.into_utf8()).collect(); + + match v.into() { + ValueRef::Object(obj) => Ok(JsonbVal::from(Value::object( + obj.iter().filter(|(k, _)| !keys_set.contains(*k)), + ))), + ValueRef::Array(arr) => { + Ok(JsonbVal::from(Value::array(arr.iter().filter( + |value| match value.as_str() { + Some(s) => !keys_set.contains(s), + None => true, + }, + )))) + } + _ => Err(ExprError::InvalidParam { + name: "jsonb", + reason: "cannot delete from scalar".into(), + }), + } +} + +/// Deletes the array element with the specified index (negative integers count from the end). +/// Throws an error if JSON value is not an array. +/// +/// Examples: +/// +/// ```slt +/// query T +/// SELECT '["a", "b"]'::jsonb - 1; +/// ---- +/// ["a"] +/// +/// query T +/// SELECT '["a", "b"]'::jsonb - -1; +/// ---- +/// ["a"] +/// +/// query T +/// SELECT '["a", "b"]'::jsonb - 2; +/// ---- +/// ["a", "b"] +/// +/// query T +/// SELECT '["a", "b"]'::jsonb - -3; +/// ---- +/// ["a", "b"] +/// +/// query error cannot delete from scalar +/// SELECT '1'::jsonb - 1; +/// +/// query error cannot delete from object using integer index +/// SELECT '{"a": 1}'::jsonb - 1; +/// ``` +#[function("subtract(jsonb, int4) -> jsonb")] +fn jsonb_remove_index(v: JsonbRef<'_>, index: i32) -> Result { + let array = match v.into() { + ValueRef::Array(array) => array, + ValueRef::Object(_) => { + return Err(ExprError::InvalidParam { + name: "jsonb", + reason: "cannot delete from object using integer index".into(), + }) + } + _ => { + return Err(ExprError::InvalidParam { + name: "jsonb", + reason: "cannot delete from scalar".into(), + }) + } + }; + let Some(idx) = normalize_array_index(array.len(), index) else { + // out of bounds index returns original value + return Ok(JsonbVal::from(v)); + }; + Ok(JsonbVal::from(Value::array( + array + .iter() + .enumerate() + .filter(|&(i, _)| i != idx) + .map(|(_, v)| v), + ))) +} + +/// Deletes the field or array element at the specified path, where path elements can be +/// either field keys or array indexes. +/// +/// Examples: +/// +/// ```slt +/// # Basic test case +/// query T +/// SELECT '["a", {"b":1}]'::jsonb #- '{1,b}'; +/// ---- +/// ["a", {}] +/// +/// # Invalid path +/// query error path element at position 1 is null +/// SELECT '["a", {"b":1}]'::jsonb #- array[null]; +/// +/// # Removing non-existent key from an object +/// query T +/// SELECT '{"a": 1, "b": 2}'::jsonb #- '{c}'; +/// ---- +/// {"a": 1, "b": 2} +/// +/// # Removing an existing key from an object +/// query T +/// SELECT '{"a": 1, "b": 2}'::jsonb #- '{a}'; +/// ---- +/// {"b": 2} +/// +/// # Removing an item from an array by positive index +/// query T +/// SELECT '["a", "b", "c"]'::jsonb #- '{1}'; +/// ---- +/// ["a", "c"] +/// +/// # Removing an item from an array by negative index +/// query T +/// SELECT '["a", "b", "c"]'::jsonb #- '{-1}'; +/// ---- +/// ["a", "b"] +/// +/// # Removing a non-existent index from an array +/// query T +/// SELECT '["a", "b", "c"]'::jsonb #- '{3}'; +/// ---- +/// ["a", "b", "c"] +/// +/// # Path element is not an integer for array +/// query error path element at position 1 is not an integer: "a" +/// SELECT '["a", "b", "c"]'::jsonb #- '{a}'; +/// +/// # Path to deeply nested value +/// query T +/// SELECT '{"a": {"b": {"c": [1, 2, 3]}}}'::jsonb #- '{a,b,c,1}'; +/// ---- +/// {"a": {"b": {"c": [1, 3]}}} +/// +/// # Path terminates early (before reaching the final depth of the JSON) +/// query T +/// SELECT '{"a": {"b": {"c": [1, 2, 3]}}}'::jsonb #- '{a}'; +/// ---- +/// {} +/// +/// # Removing non-existent path in nested structure +/// query T +/// SELECT '{"a": {"b": {"c": [1, 2, 3]}}}'::jsonb #- '{a,x}'; +/// ---- +/// {"a": {"b": {"c": [1, 2, 3]}}} +/// +/// # Path is longer than the depth of the JSON structure +/// query T +/// SELECT '{"a": 1}'::jsonb #- '{a,b}'; +/// ---- +/// {"a": 1} +/// +/// # Edge case: Removing root +/// query T +/// SELECT '{"a": 1}'::jsonb #- '{}'; +/// ---- +/// {"a": 1} +/// +/// # Edge case: Empty array +/// query T +/// SELECT '[]'::jsonb #- '{a}'; +/// ---- +/// [] +/// +/// # Edge case: Empty object +/// query T +/// SELECT '{}'::jsonb #- '{null}'; +/// ---- +/// {} +/// +/// query error cannot delete path in scalar +/// SELECT '1'::jsonb #- '{}'; +/// ``` +#[function("jsonb_delete_path(jsonb, varchar[]) -> jsonb")] +fn jsonb_delete_path(v: JsonbRef<'_>, path: ListRef<'_>) -> Result { + if v.is_scalar() { + return Err(ExprError::InvalidParam { + name: "jsonb", + reason: "cannot delete path in scalar".into(), + }); + } + if path.is_empty() { + return Ok(JsonbVal::from(v)); + } + let jsonb: ValueRef<'_> = v.into(); + let mut builder = jsonbb::Builder::>::with_capacity(jsonb.capacity()); + jsonbb_remove_path(jsonb, path, 0, &mut builder)?; + Ok(JsonbVal::from(builder.finish())) +} + +// Recursively remove `path[i..]` from `jsonb` and write the result to `builder`. +// Panics if `i` is out of bounds. +fn jsonbb_remove_path( + jsonb: ValueRef<'_>, + path: ListRef<'_>, + i: usize, + builder: &mut jsonbb::Builder, +) -> Result<()> { + match jsonb { + ValueRef::Object(obj) => { + if obj.is_empty() { + builder.add_value(jsonb); + return Ok(()); + } + let key = path + .get(i) + .unwrap() + .ok_or_else(|| ExprError::InvalidParam { + name: "path", + reason: format!("path element at position {} is null", i + 1).into(), + })? + .into_utf8(); + if !obj.contains_key(key) { + builder.add_value(jsonb); + return Ok(()); + } + builder.begin_object(); + for (k, v) in obj.iter() { + if k != key { + builder.add_string(k); + builder.add_value(v); + continue; + } + if i != path.len() - 1 { + builder.add_string(k); + // recursively remove path[i+1..] from v + jsonbb_remove_path(v, path, i + 1, builder)?; + } + } + builder.end_object(); + Ok(()) + } + ValueRef::Array(array) => { + if array.is_empty() { + builder.add_value(jsonb); + return Ok(()); + } + let key = path + .get(i) + .unwrap() + .ok_or_else(|| ExprError::InvalidParam { + name: "path", + reason: format!("path element at position {} is null", i + 1).into(), + })? + .into_utf8(); + let idx = key.parse::().map_err(|_| ExprError::InvalidParam { + name: "path", + reason: format!( + "path element at position {} is not an integer: \"{}\"", + i + 1, + key + ) + .into(), + })?; + let Some(idx) = normalize_array_index(array.len(), idx) else { + // out of bounds index returns original value + builder.add_value(jsonb); + return Ok(()); + }; + builder.begin_array(); + for (j, v) in array.iter().enumerate() { + if j != idx { + builder.add_value(v); + continue; + } + if i != path.len() - 1 { + // recursively remove path[i+1..] from v + jsonbb_remove_path(v, path, i + 1, builder)?; + } + } + builder.end_array(); + Ok(()) + } + _ => { + builder.add_value(jsonb); + Ok(()) + } + } +} + +/// Normalizes an array index to `0..len`. +/// Negative indices count from the end. i.e. `-len..0 => 0..len`. +/// Returns `None` if index is out of bounds. +fn normalize_array_index(len: usize, index: i32) -> Option { + if index < -(len as i32) || index >= (len as i32) { + return None; + } + if index >= 0 { + Some(index as usize) + } else { + Some((len as i32 + index) as usize) + } +} diff --git a/src/expr/impl/src/scalar/mod.rs b/src/expr/impl/src/scalar/mod.rs index d1b89d3c75d6c..564091897ae1a 100644 --- a/src/expr/impl/src/scalar/mod.rs +++ b/src/expr/impl/src/scalar/mod.rs @@ -45,6 +45,7 @@ mod int256; mod jsonb_access; mod jsonb_concat; mod jsonb_contains; +mod jsonb_delete; mod jsonb_info; mod jsonb_object; mod length; diff --git a/src/frontend/src/binder/expr/binary_op.rs b/src/frontend/src/binder/expr/binary_op.rs index 33f6f366e6496..8718ea74e5b5d 100644 --- a/src/frontend/src/binder/expr/binary_op.rs +++ b/src/frontend/src/binder/expr/binary_op.rs @@ -91,6 +91,7 @@ impl Binder { BinaryOperator::PGBitwiseShiftRight => ExprType::BitwiseShiftRight, BinaryOperator::Arrow => ExprType::JsonbAccess, BinaryOperator::LongArrow => ExprType::JsonbAccessStr, + BinaryOperator::HashMinus => ExprType::JsonbDeletePath, BinaryOperator::HashArrow => ExprType::JsonbAccessMulti, BinaryOperator::HashLongArrow => ExprType::JsonbAccessMultiStr, BinaryOperator::Prefix => ExprType::StartsWith, diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 6dfab2c3bc283..a32e07f72ce51 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -884,6 +884,8 @@ impl Binder { ("jsonb_exists", raw_call(ExprType::JsonbExists)), ("jsonb_exists_any", raw_call(ExprType::JsonbExistsAny)), ("jsonb_exists_all", raw_call(ExprType::JsonbExistsAll)), + ("jsonb_delete", raw_call(ExprType::Subtract)), + ("jsonb_delete_path", raw_call(ExprType::JsonbDeletePath)), // Functions that return a constant value ("pi", pi()), // greatest and least diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index cebd59322c2b9..94d38dcff56f1 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -182,6 +182,7 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::JsonbArrayLength | expr_node::Type::JsonbObject | expr_node::Type::JsonbPretty + | expr_node::Type::JsonbDeletePath | expr_node::Type::JsonbContains | expr_node::Type::JsonbContained | expr_node::Type::JsonbExists diff --git a/src/sqlparser/src/ast/operator.rs b/src/sqlparser/src/ast/operator.rs index 147a78d0b7174..9929d56b28679 100644 --- a/src/sqlparser/src/ast/operator.rs +++ b/src/sqlparser/src/ast/operator.rs @@ -99,6 +99,7 @@ pub enum BinaryOperator { LongArrow, HashArrow, HashLongArrow, + HashMinus, Contains, Contained, Exists, @@ -148,6 +149,7 @@ impl fmt::Display for BinaryOperator { BinaryOperator::LongArrow => "->>", BinaryOperator::HashArrow => "#>", BinaryOperator::HashLongArrow => "#>>", + BinaryOperator::HashMinus => "#-", BinaryOperator::Contains => "@>", BinaryOperator::Contained => "<@", BinaryOperator::Exists => "?", diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index d73ec295b0f90..87af26bfb1750 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -1397,6 +1397,7 @@ impl Parser { Token::LongArrow => Some(BinaryOperator::LongArrow), Token::HashArrow => Some(BinaryOperator::HashArrow), Token::HashLongArrow => Some(BinaryOperator::HashLongArrow), + Token::HashMinus => Some(BinaryOperator::HashMinus), Token::AtArrow => Some(BinaryOperator::Contains), Token::ArrowAt => Some(BinaryOperator::Contained), Token::QuestionMark => Some(BinaryOperator::Exists), @@ -1741,6 +1742,7 @@ impl Parser { | Token::LongArrow | Token::HashArrow | Token::HashLongArrow + | Token::HashMinus | Token::AtArrow | Token::ArrowAt | Token::QuestionMark diff --git a/src/sqlparser/src/tokenizer.rs b/src/sqlparser/src/tokenizer.rs index 914f0e5f0f8d4..4fafde820f414 100644 --- a/src/sqlparser/src/tokenizer.rs +++ b/src/sqlparser/src/tokenizer.rs @@ -164,6 +164,8 @@ pub enum Token { HashArrow, /// `#>>`, extract JSON sub-object at the specified path as text in PostgreSQL HashLongArrow, + /// `#-`, delete a key from a JSON object in PostgreSQL + HashMinus, /// `@>`, does the left JSON value contain the right JSON path/value entries at the top level AtArrow, /// `<@`, does the right JSON value contain the left JSON path/value entries at the top level @@ -241,6 +243,7 @@ impl fmt::Display for Token { Token::LongArrow => f.write_str("->>"), Token::HashArrow => f.write_str("#>"), Token::HashLongArrow => f.write_str("#>>"), + Token::HashMinus => f.write_str("#-"), Token::AtArrow => f.write_str("@>"), Token::ArrowAt => f.write_str("<@"), Token::QuestionMark => f.write_str("?"), @@ -761,6 +764,7 @@ impl<'a> Tokenizer<'a> { '#' => { chars.next(); // consume the '#' match chars.peek() { + Some('-') => self.consume_and_return(chars, Token::HashMinus), Some('>') => { chars.next(); // consume first '>' match chars.peek() { diff --git a/src/tests/regress/data/sql/jsonb.sql b/src/tests/regress/data/sql/jsonb.sql index 69bd9a928a0f5..8ae2b473b0dd4 100644 --- a/src/tests/regress/data/sql/jsonb.sql +++ b/src/tests/regress/data/sql/jsonb.sql @@ -1090,31 +1090,30 @@ select jsonb_pretty('{"a":["b", "c"], "d": {"e":"f"}}'); --@ select pg_column_size('{"aa":1, "b":2}'::jsonb || '{}'::jsonb) = pg_column_size('{"aa":1, "b":2}'::jsonb); --@ select pg_column_size('{}'::jsonb || '{"aa":1, "b":2}'::jsonb) = pg_column_size('{"aa":1, "b":2}'::jsonb); ---@ select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'a'); ---@ select jsonb_delete('{"a":null , "b":2, "c":3}'::jsonb, 'a'); ---@ select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'b'); ---@ select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'c'); ---@ select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'd'); ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - 'a'; ---@ select '{"a":null , "b":2, "c":3}'::jsonb - 'a'; ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - 'b'; ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - 'c'; ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - 'd'; +select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'a'); +select jsonb_delete('{"a":null , "b":2, "c":3}'::jsonb, 'a'); +select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'b'); +select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'c'); +select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'd'); +select '{"a":1 , "b":2, "c":3}'::jsonb - 'a'; +select '{"a":null , "b":2, "c":3}'::jsonb - 'a'; +select '{"a":1 , "b":2, "c":3}'::jsonb - 'b'; +select '{"a":1 , "b":2, "c":3}'::jsonb - 'c'; +select '{"a":1 , "b":2, "c":3}'::jsonb - 'd'; --@ select pg_column_size('{"a":1 , "b":2, "c":3}'::jsonb - 'b') = pg_column_size('{"a":1, "b":2}'::jsonb); ---@ ---@ select '["a","b","c"]'::jsonb - 3; ---@ select '["a","b","c"]'::jsonb - 2; ---@ select '["a","b","c"]'::jsonb - 1; ---@ select '["a","b","c"]'::jsonb - 0; ---@ select '["a","b","c"]'::jsonb - -1; ---@ select '["a","b","c"]'::jsonb - -2; ---@ select '["a","b","c"]'::jsonb - -3; ---@ select '["a","b","c"]'::jsonb - -4; ---@ ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - '{b}'::text[]; ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - '{c,b}'::text[]; ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - '{}'::text[]; ---@ + +select '["a","b","c"]'::jsonb - 3; +select '["a","b","c"]'::jsonb - 2; +select '["a","b","c"]'::jsonb - 1; +select '["a","b","c"]'::jsonb - 0; +select '["a","b","c"]'::jsonb - -1; +select '["a","b","c"]'::jsonb - -2; +select '["a","b","c"]'::jsonb - -3; +select '["a","b","c"]'::jsonb - -4; +select '{"a":1 , "b":2, "c":3}'::jsonb - '{b}'::text[]; +select '{"a":1 , "b":2, "c":3}'::jsonb - '{c,b}'::text[]; +select '{"a":1 , "b":2, "c":3}'::jsonb - '{}'::text[]; + --@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{n}', '[1,2,3]'); --@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '[1,2,3]'); --@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{d,1,0}', '[1,2,3]'); @@ -1127,28 +1126,28 @@ select jsonb_pretty('{"a":["b", "c"], "d": {"e":"f"}}'); --@ --@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '"test"'); --@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '{"f": "test"}'); ---@ ---@ select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{n}'); ---@ select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{b,-1}'); ---@ select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{d,1,0}'); ---@ ---@ select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{n}'; ---@ select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{b,-1}'; ---@ select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{b,-1e}'; -- invalid array subscript ---@ select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{d,1,0}'; ---@ ---@ ---@ -- empty structure and error conditions for delete and replace ---@ ---@ select '"a"'::jsonb - 'a'; -- error ---@ select '{}'::jsonb - 'a'; ---@ select '[]'::jsonb - 'a'; ---@ select '"a"'::jsonb - 1; -- error ---@ select '{}'::jsonb - 1; -- error ---@ select '[]'::jsonb - 1; ---@ select '"a"'::jsonb #- '{a}'; -- error ---@ select '{}'::jsonb #- '{a}'; ---@ select '[]'::jsonb #- '{a}'; + +select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{n}'); +select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{b,-1}'); +select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{d,1,0}'); + +select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{n}'; +select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{b,-1}'; +select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{b,-1e}'; -- invalid array subscript +select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{d,1,0}'; + + +-- empty structure and error conditions for delete and replace + +select '"a"'::jsonb - 'a'; -- error +select '{}'::jsonb - 'a'; +select '[]'::jsonb - 'a'; +select '"a"'::jsonb - 1; -- error +select '{}'::jsonb - 1; -- error +select '[]'::jsonb - 1; +select '"a"'::jsonb #- '{a}'; -- error +select '{}'::jsonb #- '{a}'; +select '[]'::jsonb #- '{a}'; --@ select jsonb_set('"a"','{a}','"b"'); --error --@ select jsonb_set('{}','{a}','"b"', false); --@ select jsonb_set('[]','{1}','"b"', false); From 4fde65f08098e7c6d7272131ea56259d636a6b38 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Oct 2023 10:16:35 +0000 Subject: [PATCH 05/14] chore(deps): Bump simd-json from 0.12.0 to 0.13.3 (#13119) Signed-off-by: dependabot[bot] Signed-off-by: Runji Wang Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Runji Wang --- Cargo.lock | 29 ++++++++++++++++--- src/connector/Cargo.toml | 2 +- .../src/parser/canal/simd_json_parser.rs | 3 +- src/connector/src/parser/common.rs | 3 +- .../src/parser/debezium/mongo_json_parser.rs | 3 +- .../src/parser/debezium/simd_json_parser.rs | 3 +- src/connector/src/parser/unified/json.rs | 5 +++- 7 files changed, 38 insertions(+), 10 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 92bbc847031cc..73e8eab121d60 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6721,6 +6721,26 @@ dependencies = [ "bitflags 1.3.2", ] +[[package]] +name = "ref-cast" +version = "1.0.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acde58d073e9c79da00f2b5b84eed919c8326832648a5b109b3fce1bb1175280" +dependencies = [ + "ref-cast-impl", +] + +[[package]] +name = "ref-cast-impl" +version = "1.0.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f7473c2cfcf90008193dd0e3e16599455cb601a9fce322b5bb55de799664925" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.37", +] + [[package]] name = "regex" version = "1.10.0" @@ -9130,13 +9150,14 @@ dependencies = [ [[package]] name = "simd-json" -version = "0.12.0" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0f07a84c7456b901b8dd2c1d44caca8b0fd2c2616206ee5acc9d9da61e8d9ec" +checksum = "d123f285a3635e423ec2ef5b67e0168dcf86c0d62fffbcea88fcd1c926e47413" dependencies = [ "getrandom", "halfbrown", "lexical-core", + "ref-cast", "serde", "serde_json", "simdutf8", @@ -10596,9 +10617,9 @@ checksum = "d92ccd67fb88503048c01b59152a04effd0782d035a83a6d256ce6085f08f4a3" [[package]] name = "value-trait" -version = "0.6.1" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09a5b6c8ceb01263b969cac48d4a6705134d490ded13d889e52c0cfc80c6945e" +checksum = "ea87257cfcbedcb9444eda79c59fdfea71217e6305afee8ee33f500375c2ac97" dependencies = [ "float-cmp", "halfbrown", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 8331adb8dc20a..fe05cfaf523bd 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -108,7 +108,7 @@ serde = { version = "1", features = ["derive", "rc"] } serde_derive = "1" serde_json = "1" serde_with = { version = "3", features = ["json"] } -simd-json = "0.12.0" +simd-json = "0.13.3" strum = "0.25" strum_macros = "0.25" tempfile = "3" diff --git a/src/connector/src/parser/canal/simd_json_parser.rs b/src/connector/src/parser/canal/simd_json_parser.rs index f238af2053ceb..bea0561330774 100644 --- a/src/connector/src/parser/canal/simd_json_parser.rs +++ b/src/connector/src/parser/canal/simd_json_parser.rs @@ -15,7 +15,8 @@ use itertools::Itertools; use risingwave_common::error::ErrorCode::{self, ProtocolError}; use risingwave_common::error::{Result, RwError}; -use simd_json::{BorrowedValue, Mutable, ValueAccess}; +use simd_json::prelude::{MutableObject, ValueAsScalar, ValueObjectAccess}; +use simd_json::BorrowedValue; use crate::only_parse_payload; use crate::parser::canal::operators::*; diff --git a/src/connector/src/parser/common.rs b/src/connector/src/parser/common.rs index 5a288dfd80b8d..cc30fabe0f475 100644 --- a/src/connector/src/parser/common.rs +++ b/src/connector/src/parser/common.rs @@ -12,7 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use simd_json::{BorrowedValue, ValueAccess}; +use simd_json::prelude::ValueAsContainer; +use simd_json::BorrowedValue; /// Get a value from a json object by key, case insensitive. /// diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index 1217a49c42c9c..e4baacb895b62 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -17,7 +17,8 @@ use std::fmt::Debug; use risingwave_common::error::ErrorCode::{self, ProtocolError}; use risingwave_common::error::{Result, RwError}; use risingwave_common::types::DataType; -use simd_json::{BorrowedValue, Mutable}; +use simd_json::prelude::MutableObject; +use simd_json::BorrowedValue; use crate::only_parse_payload; use crate::parser::unified::debezium::{DebeziumChangeEvent, MongoProjection}; diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index ca4313c43edcb..9ee966456799b 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -15,7 +15,8 @@ use std::fmt::Debug; use risingwave_common::error::{ErrorCode, Result, RwError}; -use simd_json::{BorrowedValue, Mutable}; +use simd_json::prelude::MutableObject; +use simd_json::BorrowedValue; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; use crate::parser::unified::AccessImpl; diff --git a/src/connector/src/parser/unified/json.rs b/src/connector/src/parser/unified/json.rs index 79590749351d2..b770649b052c9 100644 --- a/src/connector/src/parser/unified/json.rs +++ b/src/connector/src/parser/unified/json.rs @@ -23,7 +23,10 @@ use risingwave_common::types::{ DataType, Date, Decimal, Int256, Interval, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, }; use risingwave_common::util::iter_util::ZipEqFast; -use simd_json::{BorrowedValue, ValueAccess, ValueType}; +use simd_json::prelude::{ + TypedValue, ValueAsContainer, ValueAsScalar, ValueObjectAccess, ValueTryAsScalar, +}; +use simd_json::{BorrowedValue, ValueType}; use super::{Access, AccessError, AccessResult}; use crate::parser::common::json_object_get_case_insensitive; From 9e06e8828fdee621423a44dabfffa4e2453b4e09 Mon Sep 17 00:00:00 2001 From: emile-00 <106993396+emile-00@users.noreply.github.com> Date: Mon, 30 Oct 2023 11:15:29 -0700 Subject: [PATCH 06/14] chore: Update README architecture image (#13158) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 091bccc0de9b4..c6d4a8b14c8ba 100644 --- a/README.md +++ b/README.md @@ -56,7 +56,7 @@ RisingWave is a distributed SQL streaming database that enables cost-efficient and reliable processing of streaming data. -![RisingWave](https://github.com/risingwavelabs/risingwave-docs/blob/0f7e1302b22493ba3c1c48e78810750ce9a5ff42/docs/images/archi_simple.png) +![RisingWave](https://github.com/risingwavelabs/risingwave-docs/blob/main/docs/images/new_archi_grey.png) ## Try it out in 5 minutes **Mac** From 16aabb212bd467c2d4bc14c9eb5208ee89a238d3 Mon Sep 17 00:00:00 2001 From: Yingjun Wu Date: Mon, 30 Oct 2023 11:48:09 -0700 Subject: [PATCH 07/14] Add docker info in README.md (#13160) --- README.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/README.md b/README.md index c6d4a8b14c8ba..51ee4b19a5cc0 100644 --- a/README.md +++ b/README.md @@ -40,6 +40,12 @@ > Slack + + Docker + Date: Tue, 31 Oct 2023 09:04:57 +0800 Subject: [PATCH 08/14] chore(deps): bump reqsign to 0.14.3 and remove patches (#13157) Signed-off-by: TennyZhuang --- Cargo.lock | 32 ++++++++++++++++++++++---------- Cargo.toml | 5 ----- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 73e8eab121d60..477cbf02bc3c0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2551,8 +2551,9 @@ checksum = "86e3bdc80eee6e16b2b6b0f87fbc98c04bee3455e35174c0de1a125d0688c632" [[package]] name = "dlv-list" -version = "0.5.1" -source = "git+https://github.com/sgodwincs/dlv-list-rs.git?rev=5bbc5d0#5bbc5d0cc84f257e173d851f8dc1674fb6e46f95" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "442039f5147480ba31067cb00ada1adae6892028e40e45fc5de7b7df6dcc1b5f" dependencies = [ "const-random", ] @@ -5524,11 +5525,12 @@ dependencies = [ [[package]] name = "ordered-multimap" -version = "0.6.0" -source = "git+https://github.com/risingwavelabs/ordered-multimap-rs.git?rev=19c743f#19c743f3e3d106c99ba37628f06a2ca6faa2284f" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4d6a8c22fc714f0c2373e6091bf6f5e9b37b1bc0b1184874b7e0a4e303d318f" dependencies = [ "dlv-list", - "hashbrown 0.13.2", + "hashbrown 0.14.0", ] [[package]] @@ -6574,6 +6576,16 @@ dependencies = [ "serde", ] +[[package]] +name = "quick-xml" +version = "0.31.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1004a344b30a54e2ee58d66a71b32d2db2feb0a31f9a2d302bf0536f15de2a33" +dependencies = [ + "memchr", + "serde", +] + [[package]] name = "quote" version = "1.0.33" @@ -6802,9 +6814,9 @@ dependencies = [ [[package]] name = "reqsign" -version = "0.14.1" +version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3228e570df74d69d3d3236a71371f1edd748a3e4eb728ea1f29d403bc10fc727" +checksum = "1ad14258ddd8ef6e564d57a94613e138cc9c21ef8a1fec547206d853213c7959" dependencies = [ "anyhow", "async-trait", @@ -6819,7 +6831,7 @@ dependencies = [ "log", "once_cell", "percent-encoding", - "quick-xml 0.29.0", + "quick-xml 0.31.0", "rand", "reqwest", "rsa", @@ -8353,9 +8365,9 @@ dependencies = [ [[package]] name = "rust-ini" -version = "0.19.0" +version = "0.20.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e2a3bcec1f113553ef1c88aae6c020a369d03d55b58de9869a0908930385091" +checksum = "3e0698206bcb8882bf2a9ecb4c1e7785db57ff052297085a6efd4fe42302068a" dependencies = [ "cfg-if", "ordered-multimap", diff --git a/Cargo.toml b/Cargo.toml index f0fbb8485d318..a312a5fe045af 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -243,8 +243,3 @@ tokio-retry = { git = "https://github.com/madsim-rs/rust-tokio-retry.git", rev = tokio-postgres = { git = "https://github.com/madsim-rs/rust-postgres.git", rev = "ac00d88" } # patch: unlimit 4MB message size for grpc client etcd-client = { git = "https://github.com/risingwavelabs/etcd-client.git", rev = "4e84d40" } - -# Patch for coverage_attribute. -# https://github.com/sgodwincs/dlv-list-rs/pull/19#issuecomment-1774786289 -dlv-list = { git = "https://github.com/sgodwincs/dlv-list-rs.git", rev = "5bbc5d0" } -ordered-multimap = { git = "https://github.com/risingwavelabs/ordered-multimap-rs.git", rev = "19c743f" } From 9e334d73ab8a1170074e44c0f14c7a0b403506ef Mon Sep 17 00:00:00 2001 From: Tao Wu Date: Tue, 31 Oct 2023 11:00:11 +0800 Subject: [PATCH 09/14] chore: use dockerhub's image by default (#13067) --- README.md | 2 +- ci/scripts/integration-tests.sh | 8 +++----- docker/docker-compose.yml | 17 ++++++----------- .../docker-compose.yml | 4 ---- integration_tests/citus-cdc/docker-compose.yml | 4 ---- .../elasticsearch-sink/docker-compose.yml | 4 ---- .../feature-store/docker-compose.yml | 4 ---- .../iceberg-sink/docker-compose.yml | 4 ---- integration_tests/mysql-cdc/docker-compose.yml | 4 ---- integration_tests/mysql-sink/docker-compose.yml | 4 ---- .../postgres-cdc/docker-compose.yml | 4 ---- .../postgres-sink/docker-compose.yml | 4 ---- .../tidb-cdc-sink/docker-compose.yml | 4 ---- 13 files changed, 10 insertions(+), 57 deletions(-) diff --git a/README.md b/README.md index 51ee4b19a5cc0..c2f26ebc1a29b 100644 --- a/README.md +++ b/README.md @@ -45,7 +45,7 @@ target="_blank" > Docker - + Date: Tue, 31 Oct 2023 11:45:22 +0800 Subject: [PATCH 10/14] chore(test): ban `AggKind::InternalLastSeenValue` for sqlsmith (#13164) --- src/tests/sqlsmith/src/sql_gen/types.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/tests/sqlsmith/src/sql_gen/types.rs b/src/tests/sqlsmith/src/sql_gen/types.rs index 06d170e604ace..5ef762f9951ad 100644 --- a/src/tests/sqlsmith/src/sql_gen/types.rs +++ b/src/tests/sqlsmith/src/sql_gen/types.rs @@ -173,6 +173,7 @@ pub(crate) static AGG_FUNC_TABLE: LazyLock Date: Tue, 31 Oct 2023 11:59:08 +0800 Subject: [PATCH 11/14] chore(ci): add skip-ci for `main-cron` (#13126) --- ci/workflows/main-cron.yml | 39 +++++++++++++++++++++++++++++++++++++- 1 file changed, 38 insertions(+), 1 deletion(-) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index d8e78952c141f..6f58222424e18 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -7,6 +7,7 @@ auto-retry: &auto-retry steps: - label: "build" command: "ci/scripts/build.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-build") key: "build" plugins: - docker-compose#v4.9.0: @@ -18,6 +19,7 @@ steps: - label: "build other components" command: "ci/scripts/build-other.sh" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-build-other") key: "build-other" plugins: - seek-oss/aws-sm#v2.3.1: @@ -34,6 +36,7 @@ steps: - label: "build (deterministic simulation)" command: "ci/scripts/build-simulation.sh" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-build-simulation") key: "build-simulation" plugins: - docker-compose#v4.9.0: @@ -45,6 +48,7 @@ steps: - label: "docslt" command: "ci/scripts/docslt.sh" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-docslt") key: "docslt" plugins: - docker-compose#v4.9.0: @@ -56,6 +60,7 @@ steps: - label: "end-to-end test (release)" command: "ci/scripts/cron-e2e-test.sh -p ci-release -m ci-3streaming-2serving-3fe" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-test") depends_on: - "build" - "docslt" @@ -70,6 +75,7 @@ steps: - label: "end-to-end test (parallel) (release)" command: "ci/scripts/e2e-test-parallel.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-parallel-tests") depends_on: - "build" - "docslt" @@ -90,6 +96,7 @@ steps: - label: "end-to-end test (parallel, in-memory) (release)" command: "ci/scripts/e2e-test-parallel-in-memory.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-parallel-in-memory-tests") depends_on: - "build" - "docslt" @@ -104,6 +111,7 @@ steps: - label: "end-to-end source test (release)" command: "ci/scripts/e2e-source-test.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-source-tests") depends_on: - "build" - "build-other" @@ -118,6 +126,7 @@ steps: - label: "end-to-end sink test (release)" command: "ci/scripts/e2e-sink-test.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-sink-tests") depends_on: - "build" - "build-other" @@ -132,6 +141,7 @@ steps: - label: "fuzz test" command: "ci/scripts/cron-fuzz-test.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-sqlsmith-fuzzing-tests") depends_on: - "build" - "build-simulation" @@ -149,6 +159,7 @@ steps: # This ensures our `main-cron` workflow will be stable. - label: "unit test" command: "ci/scripts/unit-test.sh" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-unit-test") plugins: - ./ci/plugins/swapfile - seek-oss/aws-sm#v2.3.1: @@ -164,6 +175,7 @@ steps: - label: "unit test (deterministic simulation)" command: "MADSIM_TEST_NUM=100 timeout 15m ci/scripts/deterministic-unit-test.sh" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-unit-test-deterministic-simulation") plugins: - docker-compose#v4.9.0: run: rw-build-env @@ -174,6 +186,7 @@ steps: - label: "integration test (deterministic simulation) - scale" command: "TEST_NUM=60 ci/scripts/deterministic-it-test.sh scale::" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation") depends_on: "build-simulation" plugins: - docker-compose#v4.9.0: @@ -185,6 +198,7 @@ steps: - label: "integration test (deterministic simulation) - recovery" command: "TEST_NUM=60 ci/scripts/deterministic-it-test.sh recovery::" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation") depends_on: "build-simulation" plugins: - docker-compose#v4.9.0: @@ -196,6 +210,7 @@ steps: - label: "integration test (deterministic simulation) - others" command: "TEST_NUM=30 ci/scripts/deterministic-it-test.sh backfill_tests:: storage:: sink::" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation") depends_on: "build-simulation" plugins: - docker-compose#v4.9.0: @@ -207,6 +222,7 @@ steps: - label: "end-to-end test (deterministic simulation)" command: "TEST_NUM=64 timeout 55m ci/scripts/deterministic-e2e-test.sh" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-test-deterministic-simulation") depends_on: "build-simulation" plugins: - seek-oss/aws-sm#v2.3.1: @@ -224,6 +240,7 @@ steps: - label: "recovery test (deterministic simulation)" command: "TEST_NUM=12 KILL_RATE=1.0 timeout 55m ci/scripts/deterministic-recovery-test.sh" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-recovery-test-deterministic-simulation") depends_on: "build-simulation" plugins: - docker-compose#v4.9.0: @@ -236,6 +253,7 @@ steps: - label: "misc check" command: "ci/scripts/misc-check.sh" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-misc-check") plugins: - docker-compose#v4.9.0: run: rw-build-env @@ -247,6 +265,7 @@ steps: - label: "end-to-end iceberg sink test (release)" command: "ci/scripts/e2e-iceberg-sink-test.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests") depends_on: - "build" - "build-other" @@ -261,6 +280,7 @@ steps: - label: "end-to-end iceberg sink v2 test (release)" command: "ci/scripts/e2e-iceberg-sink-v2-test.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-iceberg-sink-v2-tests") depends_on: - "build" - "build-other" @@ -275,6 +295,7 @@ steps: - label: "e2e java-binding test (release)" command: "ci/scripts/java-binding-test.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-java-binding-tests") depends_on: - "build" - "build-other" @@ -291,6 +312,7 @@ steps: - label: "S3 source check on AWS (json parser)" command: "ci/scripts/s3-source-test.sh -p ci-release -s run.py" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") depends_on: build plugins: - seek-oss/aws-sm#v2.3.1: @@ -308,6 +330,7 @@ steps: - label: "S3 source check on AWS (json parser)" command: "ci/scripts/s3-source-test.sh -p ci-release -s json_file.py" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") depends_on: build plugins: - seek-oss/aws-sm#v2.3.1: @@ -325,6 +348,7 @@ steps: - label: "S3 source check on AWS (csv parser)" command: "ci/scripts/s3-source-test.sh -p ci-release -s run_csv.py" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") depends_on: build plugins: - seek-oss/aws-sm#v2.3.1: @@ -342,6 +366,7 @@ steps: - label: "S3_v2 source check on AWS (json parser)" command: "ci/scripts/s3-source-test.sh -p ci-release -s 'fs_source_v2.py json'" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") depends_on: build plugins: - seek-oss/aws-sm#v2.3.1: @@ -359,6 +384,7 @@ steps: - label: "S3_v2 source check on AWS (csv parser)" command: "ci/scripts/s3-source-test.sh -p ci-release -s 'fs_source_v2.py csv_without_header'" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") depends_on: build plugins: - seek-oss/aws-sm#v2.3.1: @@ -376,6 +402,7 @@ steps: - label: "S3 source on OpenDAL fs engine" command: "ci/scripts/s3-source-test-for-opendal-fs-engine.sh -p ci-release -s run" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") depends_on: build plugins: - seek-oss/aws-sm#v2.3.1: @@ -393,6 +420,7 @@ steps: - label: "pulsar source check" command: "ci/scripts/pulsar-source-test.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-pulsar-source-tests") depends_on: - build - build-other @@ -413,6 +441,7 @@ steps: - label: "micro benchmark" command: "ci/scripts/run-micro-benchmarks.sh" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-micro-benchmarks") key: "run-micro-benchmarks" plugins: - docker-compose#v4.9.0: @@ -423,7 +452,7 @@ steps: retry: *auto-retry - label: "upload micro-benchmark" - if: build.branch == "main" || build.pull_request.labels includes "ci/upload-micro-benchmark" + if: build.branch == "main" || (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-micro-benchmarks") command: - "BUILDKITE_BUILD_NUMBER=$BUILDKITE_BUILD_NUMBER ci/scripts/upload-micro-bench-results.sh" depends_on: "run-micro-benchmarks" @@ -444,6 +473,7 @@ steps: # Backwards compatibility tests - label: "Backwards compatibility tests" command: "RW_COMMIT=$BUILDKITE_COMMIT ci/scripts/backwards-compat-test.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-backwards-compat-tests") depends_on: - "build" plugins: @@ -458,6 +488,7 @@ steps: # Sqlsmith differential testing - label: "Sqlsmith Differential Testing" command: "ci/scripts/sqlsmith-differential-test.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-sqlsmith-differential-tests") depends_on: - "build" plugins: @@ -470,6 +501,7 @@ steps: - label: "Backfill tests" command: "ci/scripts/backfill-test.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-backfill-tests") depends_on: - "build" plugins: @@ -483,6 +515,7 @@ steps: - label: "e2e standalone binary test" command: "ci/scripts/e2e-test.sh -p ci-release -m standalone" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-standalone-tests") depends_on: - "build" - "build-other" @@ -498,6 +531,7 @@ steps: - label: "end-to-end test for opendal (parallel)" command: "ci/scripts/e2e-test-parallel-for-opendal.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-parallel-tests-for-opendal") depends_on: - "build" - "docslt" @@ -512,6 +546,7 @@ steps: - label: "end-to-end clickhouse sink test" command: "ci/scripts/e2e-clickhouse-sink-test.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-clickhouse-sink-tests") depends_on: - "build" - "build-other" @@ -526,6 +561,7 @@ steps: - label: "end-to-end pulsar sink test" command: "ci/scripts/e2e-pulsar-sink-test.sh -p ci-release" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-pulsar-sink-tests") depends_on: - "build" - "build-other" @@ -540,6 +576,7 @@ steps: - label: "connector node integration test Java {{matrix.java_version}}" command: "ci/scripts/connector-node-integration-test.sh -p ci-release -v {{matrix.java_version}}" + if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-connector-node-integration-tests") depends_on: - "build" - "build-other" From c583e2c6c054764249acf484438c7bf7197765f4 Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 31 Oct 2023 12:16:40 +0800 Subject: [PATCH 12/14] chore: cargo +nightly fmt (#13162) --- src/batch/src/executor/project_set.rs | 18 +- src/batch/src/executor/top_n.rs | 4 +- src/common/src/cache.rs | 8 +- src/common/src/types/interval.rs | 8 +- src/common/src/util/column_index_mapping.rs | 6 +- src/compute/tests/cdc_tests.rs | 6 +- src/connector/src/parser/mod.rs | 19 +- src/connector/src/parser/mysql.rs | 4 +- src/connector/src/parser/unified/debezium.rs | 62 +++--- src/connector/src/parser/unified/upsert.rs | 4 +- src/connector/src/parser/upsert_parser.rs | 4 +- src/connector/src/sink/encoder/template.rs | 11 +- src/connector/src/sink/log_store.rs | 4 +- src/connector/src/source/external.rs | 4 +- .../src/source/google_pubsub/source/reader.rs | 4 +- .../src/source/kafka/private_link.rs | 8 +- src/ctl/src/cmd_impl/meta/reschedule.rs | 4 +- src/ctl/src/cmd_impl/meta/serving.rs | 4 +- src/ctl/src/cmd_impl/scale/resize.rs | 4 +- src/expr/core/src/expr/build.rs | 4 +- src/expr/core/src/expr/expr_coalesce.rs | 4 +- .../impl/src/aggregate/percentile_cont.rs | 29 +-- .../impl/src/aggregate/percentile_disc.rs | 22 ++- src/expr/impl/src/scalar/to_timestamp.rs | 4 +- src/expr/macro/src/gen.rs | 28 ++- src/expr/macro/src/parse.rs | 4 +- src/frontend/src/binder/expr/column.rs | 6 +- src/frontend/src/binder/expr/function.rs | 19 +- src/frontend/src/binder/relation/join.rs | 4 +- src/frontend/src/binder/relation/mod.rs | 21 +- .../src/binder/relation/table_or_source.rs | 8 +- src/frontend/src/binder/select.rs | 90 ++++++--- src/frontend/src/expr/function_call.rs | 8 +- src/frontend/src/expr/mod.rs | 16 +- src/frontend/src/handler/alter_user.rs | 4 +- src/frontend/src/handler/create_user.rs | 4 +- src/frontend/src/handler/extended_handle.rs | 8 +- .../plan_expr_rewriter/cse_rewriter.rs | 4 +- .../optimizer/plan_node/generic/hop_window.rs | 4 +- .../src/optimizer/plan_node/logical_join.rs | 5 +- .../plan_node/logical_over_window.rs | 185 ++++++++---------- .../src/optimizer/plan_node/logical_source.rs | 4 +- .../optimizer/plan_node/stream_group_topn.rs | 4 +- .../optimizer/plan_node/stream_hash_join.rs | 4 +- .../src/optimizer/rule/except_merge_rule.rs | 4 +- .../optimizer/rule/index_delta_join_rule.rs | 7 +- .../optimizer/rule/intersect_merge_rule.rs | 4 +- .../rule/over_window_to_topn_rule.rs | 4 +- .../src/optimizer/rule/union_merge_rule.rs | 4 +- src/frontend/src/planner/query.rs | 4 +- src/frontend/src/planner/select.rs | 8 +- .../src/scheduler/distributed/stage.rs | 5 +- src/frontend/src/scheduler/local.rs | 13 +- src/frontend/src/utils/condition.rs | 4 +- .../src/utils/stream_graph_formatter.rs | 71 +++---- .../picker/space_reclaim_compaction_picker.rs | 8 +- src/meta/src/hummock/manager/mod.rs | 2 +- src/meta/src/manager/catalog/fragment.rs | 13 +- src/meta/src/manager/catalog/mod.rs | 3 +- src/meta/src/rpc/ddl_controller.rs | 8 +- src/meta/src/rpc/election/etcd.rs | 4 +- src/meta/src/stream/stream_manager.rs | 6 +- src/object_store/src/object/mem.rs | 4 +- .../opendal_engine/opendal_object_store.rs | 4 +- src/object_store/src/object/s3.rs | 4 +- .../src/config/provide_expander.rs | 4 +- src/risedevtool/src/preflight_check.rs | 5 +- src/risedevtool/src/task/compactor_service.rs | 4 +- .../src/task/compute_node_service.rs | 4 +- src/risedevtool/src/task/frontend_service.rs | 4 +- src/risedevtool/src/task/meta_node_service.rs | 4 +- src/rpc_client/src/meta_client.rs | 3 +- src/sqlparser/src/tokenizer.rs | 8 +- .../src/hummock/compactor/compactor_runner.rs | 16 +- .../src/hummock/event_handler/uploader.rs | 4 +- .../hummock/iterator/delete_range_iterator.rs | 5 +- src/storage/src/hummock/mod.rs | 10 +- .../src/hummock/sstable/multi_builder.rs | 17 +- src/storage/src/hummock/sstable_store.rs | 4 +- src/storage/src/memory.rs | 4 +- .../src/common/log_store_impl/in_mem.rs | 4 +- .../log_store_impl/kv_log_store/buffer.rs | 8 +- .../log_store_impl/kv_log_store/serde.rs | 4 +- src/stream/src/common/table/state_table.rs | 30 ++- .../executor/backfill/arrangement_backfill.rs | 18 +- .../src/executor/backfill/cdc_backfill.rs | 52 +++-- src/stream/src/executor/hash_agg.rs | 11 +- src/stream/src/executor/hash_join.rs | 3 +- .../src/executor/over_window/general.rs | 4 +- src/stream/src/executor/project_set.rs | 18 +- .../executor/source/state_table_handler.rs | 7 +- src/stream/src/executor/temporal_join.rs | 6 +- src/stream/src/executor/top_n/top_n_cache.rs | 42 ++-- src/stream/src/executor/top_n/top_n_state.rs | 8 +- src/stream/src/executor/watermark_filter.rs | 9 +- .../src/executor/wrapper/epoch_check.rs | 4 +- .../src/from_proto/source/trad_source.rs | 17 +- src/stream/src/task/stream_manager.rs | 4 +- src/tests/regress/src/schedule.rs | 12 +- src/tests/sqlsmith/src/runner.rs | 14 +- src/tests/sqlsmith/tests/frontend/mod.rs | 16 +- src/utils/runtime/src/logger.rs | 4 +- src/utils/runtime/src/panic_hook.rs | 4 +- 103 files changed, 793 insertions(+), 465 deletions(-) diff --git a/src/batch/src/executor/project_set.rs b/src/batch/src/executor/project_set.rs index fa3dfac917e8a..1df7f9e246d74 100644 --- a/src/batch/src/executor/project_set.rs +++ b/src/batch/src/executor/project_set.rs @@ -92,11 +92,15 @@ impl ProjectSetExecutor { // for each column for (item, value) in results.iter_mut().zip_eq_fast(&mut row[1..]) { *value = match item { - Either::Left(state) => if let Some((i, value)) = state.peek() && i == row_idx { - valid = true; - value - } else { - None + Either::Left(state) => { + if let Some((i, value)) = state.peek() + && i == row_idx + { + valid = true; + value + } else { + None + } } Either::Right(array) => array.value_at(row_idx), }; @@ -110,7 +114,9 @@ impl ProjectSetExecutor { } // move to the next row for item in &mut results { - if let Either::Left(state) = item && matches!(state.peek(), Some((i, _)) if i == row_idx) { + if let Either::Left(state) = item + && matches!(state.peek(), Some((i, _)) if i == row_idx) + { state.next().await?; } } diff --git a/src/batch/src/executor/top_n.rs b/src/batch/src/executor/top_n.rs index cffbae855de61..b43f1bc07c24d 100644 --- a/src/batch/src/executor/top_n.rs +++ b/src/batch/src/executor/top_n.rs @@ -180,7 +180,9 @@ impl TopNHeap { let mut ties_with_peek = vec![]; // pop all the ties with peek ties_with_peek.push(self.heap.pop().unwrap()); - while let Some(e) = self.heap.peek() && e.encoded_row == peek.encoded_row { + while let Some(e) = self.heap.peek() + && e.encoded_row == peek.encoded_row + { ties_with_peek.push(self.heap.pop().unwrap()); } self.heap.push(elem); diff --git a/src/common/src/cache.rs b/src/common/src/cache.rs index 5f80592fed27e..f6af1ec60c0da 100644 --- a/src/common/src/cache.rs +++ b/src/common/src/cache.rs @@ -757,7 +757,9 @@ impl LruCache { shard.release(handle) }; // do not deallocate data with holding mutex. - if let Some((key, value)) = data && let Some(listener) = &self.listener { + if let Some((key, value)) = data + && let Some(listener) = &self.listener + { listener.on_release(key, value); } } @@ -819,7 +821,9 @@ impl LruCache { shard.erase(hash, key) }; // do not deallocate data with holding mutex. - if let Some((key, value)) = data && let Some(listener) = &self.listener { + if let Some((key, value)) = data + && let Some(listener) = &self.listener + { listener.on_release(key, value); } } diff --git a/src/common/src/types/interval.rs b/src/common/src/types/interval.rs index aca4d090bcac2..c921905d8d9f0 100644 --- a/src/common/src/types/interval.rs +++ b/src/common/src/types/interval.rs @@ -1386,7 +1386,9 @@ impl Interval { fn parse_postgres(s: &str) -> Result { use DateTimeField::*; let mut tokens = parse_interval(s)?; - if tokens.len() % 2 != 0 && let Some(TimeStrToken::Num(_)) = tokens.last() { + if tokens.len() % 2 != 0 + && let Some(TimeStrToken::Num(_)) = tokens.last() + { tokens.push(TimeStrToken::TimeUnit(DateTimeField::Second)); } if tokens.len() % 2 != 0 { @@ -1394,7 +1396,9 @@ impl Interval { } let mut token_iter = tokens.into_iter(); let mut result = Interval::from_month_day_usec(0, 0, 0); - while let Some(num) = token_iter.next() && let Some(interval_unit) = token_iter.next() { + while let Some(num) = token_iter.next() + && let Some(interval_unit) = token_iter.next() + { match (num, interval_unit) { (TimeStrToken::Num(num), TimeStrToken::TimeUnit(interval_unit)) => { result = (|| match interval_unit { diff --git a/src/common/src/util/column_index_mapping.rs b/src/common/src/util/column_index_mapping.rs index 2c12dc47efb11..212c07df1e285 100644 --- a/src/common/src/util/column_index_mapping.rs +++ b/src/common/src/util/column_index_mapping.rs @@ -67,8 +67,10 @@ impl ColIndexMapping { return false; } for (src, tar) in self.map.iter().enumerate() { - if let Some(tar_value) = tar && src == *tar_value { - continue + if let Some(tar_value) = tar + && src == *tar_value + { + continue; } else { return false; } diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index 6a50b8410bbd4..fff56a17d9117 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -323,9 +323,11 @@ async fn consume_message_stream(mut stream: BoxedMessageStream) -> StreamResult< println!("[mv] chunk: {:#?}", c); } Message::Barrier(b) => { - if let Some(m) = b.mutation && matches!(*m, Mutation::Stop(_)) { + if let Some(m) = b.mutation + && matches!(*m, Mutation::Stop(_)) + { println!("encounter stop barrier"); - break + break; } } } diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index c7b8bf702e1cc..bdbb110daf7fc 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -169,12 +169,19 @@ impl MessageMeta<'_> { SourceColumnType::Offset => Datum::Some(self.offset.into()).into(), // Extract custom meta data per connector. SourceColumnType::Meta if let SourceMeta::Kafka(kafka_meta) = self.meta => { - assert_eq!(desc.name.as_str(), KAFKA_TIMESTAMP_COLUMN_NAME, "unexpected meta column name"); - kafka_meta.timestamp.map(|ts| { - risingwave_common::cast::i64_to_timestamptz(ts) - .unwrap() - .to_scalar_value() - }).into() + assert_eq!( + desc.name.as_str(), + KAFKA_TIMESTAMP_COLUMN_NAME, + "unexpected meta column name" + ); + kafka_meta + .timestamp + .map(|ts| { + risingwave_common::cast::i64_to_timestamptz(ts) + .unwrap() + .to_scalar_value() + }) + .into() } // For other cases, return `None`. diff --git a/src/connector/src/parser/mysql.rs b/src/connector/src/parser/mysql.rs index 58be305a69118..0a0f8f52e90b2 100644 --- a/src/connector/src/parser/mysql.rs +++ b/src/connector/src/parser/mysql.rs @@ -143,7 +143,9 @@ mod tests { }); pin_mut!(row_stream); while let Some(row) = row_stream.next().await { - if let Ok(ro) = row && ro.is_some() { + if let Ok(ro) = row + && ro.is_some() + { let owned_row = ro.unwrap(); let d = owned_row.datum_at(2); if let Some(scalar) = d { diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index e16df28aebdf2..e392e31e3644d 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -145,42 +145,36 @@ pub fn extract_bson_id(id_type: &DataType, bson_doc: &serde_json::Value) -> anyh .get("_id") .ok_or_else(|| anyhow::format_err!("Debezuim Mongo requires document has a `_id` field"))?; let id: Datum = match id_type { - DataType::Jsonb => ScalarImpl::Jsonb(id_field.clone().into()).into(), - DataType::Varchar => match id_field { - serde_json::Value::String(s) => Some(ScalarImpl::Utf8(s.clone().into())), - serde_json::Value::Object(obj) if obj.contains_key("$oid") => Some(ScalarImpl::Utf8( - obj["$oid"].as_str().to_owned().unwrap_or_default().into(), - )), - _ => anyhow::bail!( - "Can not convert bson {:?} to {:?}", - id_field, id_type - ), - }, - DataType::Int32 => { - if let serde_json::Value::Object(ref obj) = id_field && obj.contains_key("$numberInt") { - let int_str = obj["$numberInt"].as_str().unwrap_or_default(); - Some(ScalarImpl::Int32(int_str.parse().unwrap_or_default())) - } else { - anyhow::bail!( - "Can not convert bson {:?} to {:?}", - id_field, id_type - ) + DataType::Jsonb => ScalarImpl::Jsonb(id_field.clone().into()).into(), + DataType::Varchar => match id_field { + serde_json::Value::String(s) => Some(ScalarImpl::Utf8(s.clone().into())), + serde_json::Value::Object(obj) if obj.contains_key("$oid") => Some(ScalarImpl::Utf8( + obj["$oid"].as_str().to_owned().unwrap_or_default().into(), + )), + _ => anyhow::bail!("Can not convert bson {:?} to {:?}", id_field, id_type), + }, + DataType::Int32 => { + if let serde_json::Value::Object(ref obj) = id_field + && obj.contains_key("$numberInt") + { + let int_str = obj["$numberInt"].as_str().unwrap_or_default(); + Some(ScalarImpl::Int32(int_str.parse().unwrap_or_default())) + } else { + anyhow::bail!("Can not convert bson {:?} to {:?}", id_field, id_type) + } } - } - DataType::Int64 => { - if let serde_json::Value::Object(ref obj) = id_field && obj.contains_key("$numberLong") - { - let int_str = obj["$numberLong"].as_str().unwrap_or_default(); - Some(ScalarImpl::Int64(int_str.parse().unwrap_or_default())) - } else { - anyhow::bail!( - "Can not convert bson {:?} to {:?}", - id_field, id_type - ) + DataType::Int64 => { + if let serde_json::Value::Object(ref obj) = id_field + && obj.contains_key("$numberLong") + { + let int_str = obj["$numberLong"].as_str().unwrap_or_default(); + Some(ScalarImpl::Int64(int_str.parse().unwrap_or_default())) + } else { + anyhow::bail!("Can not convert bson {:?} to {:?}", id_field, id_type) + } } - } - _ => unreachable!("DebeziumMongoJsonParser::new must ensure _id column datatypes."), -}; + _ => unreachable!("DebeziumMongoJsonParser::new must ensure _id column datatypes."), + }; Ok(id) } impl MongoProjection { diff --git a/src/connector/src/parser/unified/upsert.rs b/src/connector/src/parser/unified/upsert.rs index a0be4f050b9c9..2697d4bdf8151 100644 --- a/src/connector/src/parser/unified/upsert.rs +++ b/src/connector/src/parser/unified/upsert.rs @@ -114,7 +114,9 @@ where other => return other, }; - if let Some(key_as_column_name) = &self.key_as_column_name && name == key_as_column_name { + if let Some(key_as_column_name) = &self.key_as_column_name + && name == key_as_column_name + { return self.access(&["key"], Some(type_expected)); } diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index f9ce0caa7e254..71210b9e4b8f8 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -102,7 +102,9 @@ impl UpsertParser { row_op = row_op.with_key(self.key_builder.generate_accessor(data).await?); } // Empty payload of kafka is Some(vec![]) - if let Some(data) = payload && !data.is_empty() { + if let Some(data) = payload + && !data.is_empty() + { row_op = row_op.with_value(self.payload_builder.generate_accessor(data).await?); change_event_op = ChangeEventOperation::Upsert; } diff --git a/src/connector/src/sink/encoder/template.rs b/src/connector/src/sink/encoder/template.rs index 97d8271f9e83a..1f70836ab453d 100644 --- a/src/connector/src/sink/encoder/template.rs +++ b/src/connector/src/sink/encoder/template.rs @@ -50,9 +50,14 @@ impl TemplateEncoder { )); } for capture in re.captures_iter(format) { - if let Some(inner_content) = capture.get(1) && !set.contains(inner_content.as_str()){ - return Err(SinkError::Redis(format!("Can't find field({:?}) in key_format or value_format",inner_content.as_str()))) - } + if let Some(inner_content) = capture.get(1) + && !set.contains(inner_content.as_str()) + { + return Err(SinkError::Redis(format!( + "Can't find field({:?}) in key_format or value_format", + inner_content.as_str() + ))); + } } Ok(()) } diff --git a/src/connector/src/sink/log_store.rs b/src/connector/src/sink/log_store.rs index f7d99141139f5..49e28bce2f795 100644 --- a/src/connector/src/sink/log_store.rs +++ b/src/connector/src/sink/log_store.rs @@ -400,7 +400,9 @@ impl<'a, F: TryFuture + Unpin + 'static> DeliveryFutureManagerAddFuture pub async fn await_one_delivery(&mut self) -> Result<(), F::Error> { for (_, item) in &mut self.0.items { - if let DeliveryFutureManagerItem::Chunk {futures, ..} = item && let Some(mut delivery_future) = futures.pop_front() { + if let DeliveryFutureManagerItem::Chunk { futures, .. } = item + && let Some(mut delivery_future) = futures.pop_front() + { self.0.future_count -= 1; return poll_fn(|cx| delivery_future.try_poll_unpin(cx)).await; } else { diff --git a/src/connector/src/source/external.rs b/src/connector/src/source/external.rs index 9eff3991a4d4a..953277ba36106 100644 --- a/src/connector/src/source/external.rs +++ b/src/connector/src/source/external.rs @@ -286,7 +286,9 @@ impl ExternalTableReader for MySqlExternalTableReader { impl MySqlExternalTableReader { pub fn new(properties: HashMap, rw_schema: Schema) -> ConnectorResult { - if let Some(field) = rw_schema.fields.last() && field.name.as_str() != OFFSET_COLUMN_NAME { + if let Some(field) = rw_schema.fields.last() + && field.name.as_str() != OFFSET_COLUMN_NAME + { return Err(ConnectorError::Config(anyhow!( "last column of schema must be `_rw_offset`" ))); diff --git a/src/connector/src/source/google_pubsub/source/reader.rs b/src/connector/src/source/google_pubsub/source/reader.rs index dfe95eeb1b808..8241e1657c496 100644 --- a/src/connector/src/source/google_pubsub/source/reader.rs +++ b/src/connector/src/source/google_pubsub/source/reader.rs @@ -96,7 +96,9 @@ impl CommonSplitReader for PubsubSplitReader { yield chunk; // Stop if we've approached the stop_offset - if let Some(stop_offset) = self.stop_offset && latest_offset >= stop_offset { + if let Some(stop_offset) = self.stop_offset + && latest_offset >= stop_offset + { return Ok(()); } } diff --git a/src/connector/src/source/kafka/private_link.rs b/src/connector/src/source/kafka/private_link.rs index 573e14c3e073f..5a6688a4cf8e9 100644 --- a/src/connector/src/source/kafka/private_link.rs +++ b/src/connector/src/source/kafka/private_link.rs @@ -120,7 +120,9 @@ impl PrivateLinkConsumerContext { impl ClientContext for PrivateLinkConsumerContext { /// this func serves as a callback when `poll` is completed. fn stats(&self, statistics: Statistics) { - if let Some(metrics) = &self.metrics && let Some(id) = &self.identifier { + if let Some(metrics) = &self.metrics + && let Some(id) = &self.identifier + { metrics.report(id.as_str(), &statistics); } } @@ -160,7 +162,9 @@ impl PrivateLinkProducerContext { impl ClientContext for PrivateLinkProducerContext { fn stats(&self, statistics: Statistics) { - if let Some(metrics) = &self.metrics && let Some(id) = &self.identifier { + if let Some(metrics) = &self.metrics + && let Some(id) = &self.identifier + { metrics.report(id.as_str(), &statistics); } } diff --git a/src/ctl/src/cmd_impl/meta/reschedule.rs b/src/ctl/src/cmd_impl/meta/reschedule.rs index 6d7765e7b1a22..737d5c2e88b07 100644 --- a/src/ctl/src/cmd_impl/meta/reschedule.rs +++ b/src/ctl/src/cmd_impl/meta/reschedule.rs @@ -273,7 +273,9 @@ pub async fn unregister_workers( .ok() .or_else(|| worker_index_by_host.get(&worker).cloned()); - if let Some(worker_id) = worker_id && worker_ids.contains(&worker_id) { + if let Some(worker_id) = worker_id + && worker_ids.contains(&worker_id) + { if !target_worker_ids.insert(worker_id) { println!("Warn: {} and {} are the same worker", worker, worker_id); } diff --git a/src/ctl/src/cmd_impl/meta/serving.rs b/src/ctl/src/cmd_impl/meta/serving.rs index 867317c0915b4..cff3c6f911282 100644 --- a/src/ctl/src/cmd_impl/meta/serving.rs +++ b/src/ctl/src/cmd_impl/meta/serving.rs @@ -82,7 +82,9 @@ pub async fn list_serving_fragment_mappings(context: &CtlContext) -> anyhow::Res ) .into(), ); - if let Some(w) = worker && let Some(addr) = w.host.as_ref() { + if let Some(w) = worker + && let Some(addr) = w.host.as_ref() + { row.add_cell(format!("id: {}; {}:{}", w.id, addr.host, addr.port).into()); } else { row.add_cell("".into()); diff --git a/src/ctl/src/cmd_impl/scale/resize.rs b/src/ctl/src/cmd_impl/scale/resize.rs index 786d0fa4c83b7..59c2280d17873 100644 --- a/src/ctl/src/cmd_impl/scale/resize.rs +++ b/src/ctl/src/cmd_impl/scale/resize.rs @@ -393,7 +393,9 @@ pub async fn update_schedulability( .ok() .or_else(|| worker_index_by_host.get(&worker).cloned()); - if let Some(worker_id) = worker_id && worker_ids.contains(&worker_id){ + if let Some(worker_id) = worker_id + && worker_ids.contains(&worker_id) + { if !target_worker_ids.insert(worker_id) { println!("Warn: {} and {} are the same worker", worker, worker_id); } diff --git a/src/expr/core/src/expr/build.rs b/src/expr/core/src/expr/build.rs index 7dffbcd42d66b..f0fd3397c4fa8 100644 --- a/src/expr/core/src/expr/build.rs +++ b/src/expr/core/src/expr/build.rs @@ -349,7 +349,9 @@ pub(crate) fn lexer(input: &str) -> Vec { ':' => Token::Colon, '$' => { let mut number = String::new(); - while let Some(c) = chars.peek() && c.is_ascii_digit() { + while let Some(c) = chars.peek() + && c.is_ascii_digit() + { number.push(chars.next().unwrap()); } let index = number.parse::().expect("Invalid number"); diff --git a/src/expr/core/src/expr/expr_coalesce.rs b/src/expr/core/src/expr/expr_coalesce.rs index 71c7392c7ec37..b7916f414136c 100644 --- a/src/expr/core/src/expr/expr_coalesce.rs +++ b/src/expr/core/src/expr/expr_coalesce.rs @@ -56,7 +56,9 @@ impl Expression for CoalesceExpression { } let mut builder = self.return_type.create_array_builder(len); for (i, sel) in selection.iter().enumerate() { - if init_vis.is_set(i) && let Some(child_idx) = sel { + if init_vis.is_set(i) + && let Some(child_idx) = sel + { builder.append(children_array[*child_idx].value_at(i)); } else { builder.append_null() diff --git a/src/expr/impl/src/aggregate/percentile_cont.rs b/src/expr/impl/src/aggregate/percentile_cont.rs index 46002d1f596f7..eeaa257627cd7 100644 --- a/src/expr/impl/src/aggregate/percentile_cont.rs +++ b/src/expr/impl/src/aggregate/percentile_cont.rs @@ -118,19 +118,22 @@ impl AggregateFunction for PercentileCont { async fn get_result(&self, state: &AggregateState) -> Result { let state = &state.downcast_ref::().0; - Ok(if let Some(fraction) = self.fraction && !state.is_empty() { - let rn = fraction * (state.len() - 1) as f64; - let crn = f64::ceil(rn); - let frn = f64::floor(rn); - let result = if crn == frn { - state[crn as usize] + Ok( + if let Some(fraction) = self.fraction + && !state.is_empty() + { + let rn = fraction * (state.len() - 1) as f64; + let crn = f64::ceil(rn); + let frn = f64::floor(rn); + let result = if crn == frn { + state[crn as usize] + } else { + (crn - rn) * state[frn as usize] + (rn - frn) * state[crn as usize] + }; + Some(result.into()) } else { - (crn - rn) * state[frn as usize] - + (rn - frn) * state[crn as usize] - }; - Some(result.into()) - } else { - None - }) + None + }, + ) } } diff --git a/src/expr/impl/src/aggregate/percentile_disc.rs b/src/expr/impl/src/aggregate/percentile_disc.rs index c9143dcf8e640..80ebbfd24e544 100644 --- a/src/expr/impl/src/aggregate/percentile_disc.rs +++ b/src/expr/impl/src/aggregate/percentile_disc.rs @@ -143,15 +143,19 @@ impl AggregateFunction for PercentileDisc { async fn get_result(&self, state: &AggregateState) -> Result { let state = &state.downcast_ref::().0; - Ok(if let Some(fractions) = self.fractions && !state.is_empty() { - let idx = if fractions == 0.0 { - 0 + Ok( + if let Some(fractions) = self.fractions + && !state.is_empty() + { + let idx = if fractions == 0.0 { + 0 + } else { + f64::ceil(fractions * state.len() as f64) as usize - 1 + }; + Some(state[idx].clone()) } else { - f64::ceil(fractions * state.len() as f64) as usize - 1 - }; - Some(state[idx].clone()) - } else { - None - }) + None + }, + ) } } diff --git a/src/expr/impl/src/scalar/to_timestamp.rs b/src/expr/impl/src/scalar/to_timestamp.rs index bc93720373c74..e4ef9edc235eb 100644 --- a/src/expr/impl/src/scalar/to_timestamp.rs +++ b/src/expr/impl/src/scalar/to_timestamp.rs @@ -108,7 +108,9 @@ fn build_dummy(_return_type: DataType, _children: Vec) -> Resul )] pub fn to_date(s: &str, tmpl: &ChronoPattern) -> Result { let mut parsed = parse(s, tmpl)?; - if let Some(year) = &mut parsed.year && *year < 0 { + if let Some(year) = &mut parsed.year + && *year < 0 + { *year += 1; } Ok(parsed.to_naive_date()?.into()) diff --git a/src/expr/macro/src/gen.rs b/src/expr/macro/src/gen.rs index 454d2a3169137..5056d5dc24d11 100644 --- a/src/expr/macro/src/gen.rs +++ b/src/expr/macro/src/gen.rs @@ -609,14 +609,18 @@ impl FunctionAttr { .collect_vec(); let downcast_state = if custom_state.is_some() { quote! { let mut state: &mut #state_type = state0.downcast_mut(); } - } else if let Some(s) = &self.state && s == "ref" { + } else if let Some(s) = &self.state + && s == "ref" + { quote! { let mut state: Option<#state_type> = state0.as_datum_mut().as_ref().map(|x| x.as_scalar_ref_impl().try_into().unwrap()); } } else { quote! { let mut state: Option<#state_type> = state0.as_datum_mut().take().map(|s| s.try_into().unwrap()); } }; let restore_state = if custom_state.is_some() { quote! {} - } else if let Some(s) = &self.state && s == "ref" { + } else if let Some(s) = &self.state + && s == "ref" + { quote! { *state0.as_datum_mut() = state.map(|x| x.to_owned_scalar().into()); } } else { quote! { *state0.as_datum_mut() = state.map(|s| s.into()); } @@ -694,10 +698,14 @@ impl FunctionAttr { let first_state = if self.init_state.is_some() { // for count, the state will never be None quote! { unreachable!() } - } else if let Some(s) = &self.state && s == "ref" { + } else if let Some(s) = &self.state + && s == "ref" + { // for min/max/first/last, the state is the first value quote! { Some(v0) } - } else if let AggregateFnOrImpl::Impl(impl_) = user_fn && impl_.create_state.is_some() { + } else if let AggregateFnOrImpl::Impl(impl_) = user_fn + && impl_.create_state.is_some() + { // use user-defined create_state function quote! {{ let state = self.function.create_state(); @@ -727,7 +735,9 @@ impl FunctionAttr { }; let get_result = if custom_state.is_some() { quote! { Ok(Some(state.downcast_ref::<#state_type>().into())) } - } else if let AggregateFnOrImpl::Impl(impl_) = user_fn && impl_.finalize.is_some() { + } else if let AggregateFnOrImpl::Impl(impl_) = user_fn + && impl_.finalize.is_some() + { quote! { let state = match state.as_datum() { Some(s) => s.as_scalar_ref_impl().try_into().unwrap(), @@ -1109,8 +1119,12 @@ fn data_type(ty: &str) -> TokenStream2 { /// output_types("struct") -> ["varchar", "jsonb"] /// ``` fn output_types(ty: &str) -> Vec<&str> { - if let Some(s) = ty.strip_prefix("struct<") && let Some(args) = s.strip_suffix('>') { - args.split(',').map(|s| s.split_whitespace().nth(1).unwrap()).collect() + if let Some(s) = ty.strip_prefix("struct<") + && let Some(args) = s.strip_suffix('>') + { + args.split(',') + .map(|s| s.split_whitespace().nth(1).unwrap()) + .collect() } else { vec![ty] } diff --git a/src/expr/macro/src/parse.rs b/src/expr/macro/src/parse.rs index 8e2e8c6d0b2f1..fc9e4d45437e2 100644 --- a/src/expr/macro/src/parse.rs +++ b/src/expr/macro/src/parse.rs @@ -314,7 +314,9 @@ fn strip_iterator(ty: &syn::Type) -> Option<&syn::Type> { return None; }; for arg in &angle_bracketed.args { - if let syn::GenericArgument::AssocType(b) = arg && b.ident == "Item" { + if let syn::GenericArgument::AssocType(b) = arg + && b.ident == "Item" + { return Some(&b.ty); } } diff --git a/src/frontend/src/binder/expr/column.rs b/src/frontend/src/binder/expr/column.rs index 41ca86919defe..4c1a426950b63 100644 --- a/src/frontend/src/binder/expr/column.rs +++ b/src/frontend/src/binder/expr/column.rs @@ -148,8 +148,10 @@ impl Binder { // FIXME: The type of `CTID` should be `tid`. // FIXME: The `CTID` column should be unique, so literal may break something. // FIXME: At least we should add a notice here. - if let ErrorCode::ItemNotFound(_) = err && column_name == "ctid" { - return Ok(Literal::new(Some("".into()), DataType::Varchar).into()) + if let ErrorCode::ItemNotFound(_) = err + && column_name == "ctid" + { + return Ok(Literal::new(Some("".into()), DataType::Varchar).into()); } Err(err.into()) } diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index a32e07f72ce51..145b11581be90 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -93,7 +93,9 @@ impl Binder { }; // agg calls - if f.over.is_none() && let Ok(kind) = function_name.parse() { + if f.over.is_none() + && let Ok(kind) = function_name.parse() + { return self.bind_agg(f, kind); } @@ -154,11 +156,12 @@ impl Binder { // user defined function // TODO: resolve schema name https://github.com/risingwavelabs/risingwave/issues/12422 - if let Ok(schema) = self.first_valid_schema() && - let Some(func) = schema.get_function_by_name_args( + if let Ok(schema) = self.first_valid_schema() + && let Some(func) = schema.get_function_by_name_args( &function_name, &inputs.iter().map(|arg| arg.return_type()).collect_vec(), - ) { + ) + { use crate::catalog::function_catalog::FunctionKind::*; match &func.kind { Scalar { .. } => return Ok(UserDefinedFunction::new(func.clone(), inputs).into()), @@ -360,8 +363,12 @@ impl Binder { // check signature and do implicit cast match (kind, direct_args.as_mut_slice(), args.as_mut_slice()) { (AggKind::PercentileCont | AggKind::PercentileDisc, [fraction], [arg]) => { - if fraction.cast_implicit_mut(DataType::Float64).is_ok() && let Ok(casted) = fraction.fold_const() { - if let Some(ref casted) = casted && !(0.0..=1.0).contains(&casted.as_float64().0) { + if fraction.cast_implicit_mut(DataType::Float64).is_ok() + && let Ok(casted) = fraction.fold_const() + { + if let Some(ref casted) = casted + && !(0.0..=1.0).contains(&casted.as_float64().0) + { return Err(ErrorCode::InvalidInputSyntax(format!( "direct arg in `{}` must between 0.0 and 1.0", kind diff --git a/src/frontend/src/binder/relation/join.rs b/src/frontend/src/binder/relation/join.rs index eb4ce96f9ab3f..10c7cd2b646e2 100644 --- a/src/frontend/src/binder/relation/join.rs +++ b/src/frontend/src/binder/relation/join.rs @@ -193,7 +193,9 @@ impl Binder { // TODO: is it ok to ignore quote style? // If we have a `USING` constraint, we only bind the columns appearing in the // constraint. - if let Some(cols) = &using_columns && !cols.contains(&column) { + if let Some(cols) = &using_columns + && !cols.contains(&column) + { continue; } let indices_l = match old_context.get_unqualified_indices(&column.real_value()) diff --git a/src/frontend/src/binder/relation/mod.rs b/src/frontend/src/binder/relation/mod.rs index a6a1a8d2b02f2..856f221ec9855 100644 --- a/src/frontend/src/binder/relation/mod.rs +++ b/src/frontend/src/binder/relation/mod.rs @@ -193,11 +193,13 @@ impl Binder { let schema_name = identifiers.pop().map(|ident| ident.real_value()); let database_name = identifiers.pop().map(|ident| ident.real_value()); - if let Some(database_name) = database_name && database_name != db_name { + if let Some(database_name) = database_name + && database_name != db_name + { return Err(ResolveQualifiedNameError::new( formatted_name, - ResolveQualifiedNameErrorKind::NotCurrentDatabase) - ); + ResolveQualifiedNameErrorKind::NotCurrentDatabase, + )); } Ok((schema_name, name)) @@ -330,7 +332,9 @@ impl Binder { for_system_time_as_of_proctime: bool, ) -> Result { let (schema_name, table_name) = Self::resolve_schema_qualified_name(&self.db_name, name)?; - if schema_name.is_none() && let Some(item) = self.context.cte_to_relation.get(&table_name) { + if schema_name.is_none() + && let Some(item) = self.context.cte_to_relation.get(&table_name) + { // Handles CTE let (share_id, query, mut original_alias) = item.deref().clone(); @@ -341,9 +345,7 @@ impl Binder { original_alias.columns = original_alias .columns .into_iter() - .zip_longest( - from_alias.columns - ) + .zip_longest(from_alias.columns) .map(EitherOrBoth::into_right) .collect(); } @@ -360,7 +362,10 @@ impl Binder { )?; // Share the CTE. - let input_relation = Relation::Subquery(Box::new(BoundSubquery { query, lateral: false })); + let input_relation = Relation::Subquery(Box::new(BoundSubquery { + query, + lateral: false, + })); let share_relation = Relation::Share(Box::new(BoundShare { share_id, input: input_relation, diff --git a/src/frontend/src/binder/relation/table_or_source.rs b/src/frontend/src/binder/relation/table_or_source.rs index b05b5db42b300..cd2d2ef45efab 100644 --- a/src/frontend/src/binder/relation/table_or_source.rs +++ b/src/frontend/src/binder/relation/table_or_source.rs @@ -145,9 +145,11 @@ impl Binder { let user_name = &self.auth_context.user_name; for path in self.search_path.path() { - if is_system_schema(path) && - let Ok(sys_table_catalog) = - self.catalog.get_sys_table_by_name(&self.db_name, path, table_name) { + if is_system_schema(path) + && let Ok(sys_table_catalog) = + self.catalog + .get_sys_table_by_name(&self.db_name, path, table_name) + { return Ok(resolve_sys_table_relation(sys_table_catalog)); } else { let schema_name = if path == USER_NAME_WILD_CARD { diff --git a/src/frontend/src/binder/select.rs b/src/frontend/src/binder/select.rs index ac1a53e75f63f..ceb7d55312f46 100644 --- a/src/frontend/src/binder/select.rs +++ b/src/frontend/src/binder/select.rs @@ -227,21 +227,50 @@ impl Binder { self.context.clause = Some(Clause::GroupBy); // Only support one grouping item in group by clause - let group_by = if select.group_by.len() == 1 && let Expr::GroupingSets(grouping_sets) = &select.group_by[0] { - GroupBy::GroupingSets(self.bind_grouping_items_expr_in_select(grouping_sets.clone(), &out_name_to_index, &select_items)?) - } else if select.group_by.len() == 1 && let Expr::Rollup(rollup) = &select.group_by[0] { - GroupBy::Rollup(self.bind_grouping_items_expr_in_select(rollup.clone(), &out_name_to_index, &select_items)?) - } else if select.group_by.len() == 1 && let Expr::Cube(cube) = &select.group_by[0] { - GroupBy::Cube(self.bind_grouping_items_expr_in_select(cube.clone(), &out_name_to_index, &select_items)?) + let group_by = if select.group_by.len() == 1 + && let Expr::GroupingSets(grouping_sets) = &select.group_by[0] + { + GroupBy::GroupingSets(self.bind_grouping_items_expr_in_select( + grouping_sets.clone(), + &out_name_to_index, + &select_items, + )?) + } else if select.group_by.len() == 1 + && let Expr::Rollup(rollup) = &select.group_by[0] + { + GroupBy::Rollup(self.bind_grouping_items_expr_in_select( + rollup.clone(), + &out_name_to_index, + &select_items, + )?) + } else if select.group_by.len() == 1 + && let Expr::Cube(cube) = &select.group_by[0] + { + GroupBy::Cube(self.bind_grouping_items_expr_in_select( + cube.clone(), + &out_name_to_index, + &select_items, + )?) } else { - if select.group_by.iter().any(|expr| matches!(expr, Expr::GroupingSets(_)) || matches!(expr, Expr::Rollup(_)) || matches!(expr, Expr::Cube(_))) { - return Err(ErrorCode::BindError("Only support one grouping item in group by clause".to_string()).into()); + if select.group_by.iter().any(|expr| { + matches!(expr, Expr::GroupingSets(_)) + || matches!(expr, Expr::Rollup(_)) + || matches!(expr, Expr::Cube(_)) + }) { + return Err(ErrorCode::BindError( + "Only support one grouping item in group by clause".to_string(), + ) + .into()); } - GroupBy::GroupKey(select - .group_by - .into_iter() - .map(|expr| self.bind_group_by_expr_in_select(expr, &out_name_to_index, &select_items)) - .try_collect()?) + GroupBy::GroupKey( + select + .group_by + .into_iter() + .map(|expr| { + self.bind_group_by_expr_in_select(expr, &out_name_to_index, &select_items) + }) + .try_collect()?, + ) }; self.context.clause = None; @@ -795,7 +824,9 @@ impl Binder { let mut bound_exprs = vec![]; for expr in exprs { let expr_impl = match expr { - Expr::Identifier(name) if let Some(index) = name_to_index.get(&name.real_value()) => { + Expr::Identifier(name) + if let Some(index) = name_to_index.get(&name.real_value()) => + { match *index { usize::MAX => { return Err(ErrorCode::BindError(format!( @@ -809,24 +840,21 @@ impl Binder { } } } - Expr::Value(Value::Number(number)) => { - match number.parse::() { - Ok(index) if 1 <= index && index <= select_items.len() => { - let idx_from_0 = index - 1; - InputRef::new(idx_from_0, select_items[idx_from_0].return_type()).into() - } - _ => { - return Err(ErrorCode::InvalidInputSyntax(format!( - "Invalid ordinal number in DISTINCT ON: {}", - number - )) - .into()) - } + Expr::Value(Value::Number(number)) => match number.parse::() { + Ok(index) if 1 <= index && index <= select_items.len() => { + let idx_from_0 = index - 1; + InputRef::new(idx_from_0, select_items[idx_from_0].return_type()) + .into() } - } - expr => { - self.bind_expr(expr)? - } + _ => { + return Err(ErrorCode::InvalidInputSyntax(format!( + "Invalid ordinal number in DISTINCT ON: {}", + number + )) + .into()) + } + }, + expr => self.bind_expr(expr)?, }; bound_exprs.push(expr_impl); } diff --git a/src/frontend/src/expr/function_call.rs b/src/frontend/src/expr/function_call.rs index f5e618892fc5e..7f2c84c6cce78 100644 --- a/src/frontend/src/expr/function_call.rs +++ b/src/frontend/src/expr/function_call.rs @@ -111,12 +111,16 @@ impl FunctionCall { target: DataType, allows: CastContext, ) -> Result<(), CastError> { - if let ExprImpl::Parameter(expr) = child && !expr.has_infer() { + if let ExprImpl::Parameter(expr) = child + && !expr.has_infer() + { // Always Ok below. Safe to mutate `expr` (from `child`). expr.cast_infer_type(target); return Ok(()); } - if let ExprImpl::FunctionCall(func) = child && func.func_type == ExprType::Row { + if let ExprImpl::FunctionCall(func) = child + && func.func_type == ExprType::Row + { // Row function will have empty fields in Datatype::Struct at this point. Therefore, // we will need to take some special care to generate the cast types. For normal struct // types, they will be handled in `cast_ok`. diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index 6eec2983f5c91..27e781a88690b 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -822,13 +822,19 @@ impl ExprImpl { match expr_type { ExprType::Add | ExprType::Subtract => { let (_, lhs, rhs) = function_call.clone().decompose_as_binary(); - if let ExprImpl::InputRef(input_ref) = &lhs && rhs.is_const() { + if let ExprImpl::InputRef(input_ref) = &lhs + && rhs.is_const() + { // Currently we will return `None` for non-literal because the result of the expression might be '1 day'. However, there will definitely exist false positives such as '1 second + 1 second'. // We will treat the expression as an input offset when rhs is `null`. - if rhs.return_type() == DataType::Interval && rhs.as_literal().map_or(true, |literal| literal.get_data().as_ref().map_or(false, |scalar| { - let interval = scalar.as_interval(); - interval.months() != 0 || interval.days() != 0 - })) { + if rhs.return_type() == DataType::Interval + && rhs.as_literal().map_or(true, |literal| { + literal.get_data().as_ref().map_or(false, |scalar| { + let interval = scalar.as_interval(); + interval.months() != 0 || interval.days() != 0 + }) + }) + { None } else { Some((input_ref.index(), Some((expr_type, rhs)))) diff --git a/src/frontend/src/handler/alter_user.rs b/src/frontend/src/handler/alter_user.rs index 0d83c3ae867d5..b72d8d32ae8be 100644 --- a/src/frontend/src/handler/alter_user.rs +++ b/src/frontend/src/handler/alter_user.rs @@ -102,7 +102,9 @@ fn alter_prost_user_info( } UserOption::Password(opt) => { // TODO: Behaviour of PostgreSQL: Notice when password is empty string. - if let Some(password) = opt && !password.0.is_empty() { + if let Some(password) = opt + && !password.0.is_empty() + { user_info.auth_info = encrypted_password(&user_info.name, &password.0); } else { user_info.auth_info = None; diff --git a/src/frontend/src/handler/create_user.rs b/src/frontend/src/handler/create_user.rs index 8659e1b647c33..0bac084db2c80 100644 --- a/src/frontend/src/handler/create_user.rs +++ b/src/frontend/src/handler/create_user.rs @@ -85,7 +85,9 @@ fn make_prost_user_info( } UserOption::Password(opt) => { // TODO: Behaviour of PostgreSQL: Notice when password is empty string. - if let Some(password) = opt && !password.0.is_empty() { + if let Some(password) = opt + && !password.0.is_empty() + { user_info.auth_info = encrypted_password(&user_info.name, &password.0); } } diff --git a/src/frontend/src/handler/extended_handle.rs b/src/frontend/src/handler/extended_handle.rs index bcd65782d59a2..1c0f0d36f0cbf 100644 --- a/src/frontend/src/handler/extended_handle.rs +++ b/src/frontend/src/handler/extended_handle.rs @@ -100,7 +100,9 @@ pub fn handle_parse( Ok(PrepareStatement::PureStatement(statement)) } Statement::CreateTable { query, .. } => { - if let Some(query) = query && have_parameter_in_query(query) { + if let Some(query) = query + && have_parameter_in_query(query) + { Err(ErrorCode::NotImplemented( "CREATE TABLE AS SELECT with parameters".to_string(), None.into(), @@ -111,7 +113,9 @@ pub fn handle_parse( } } Statement::CreateSink { stmt } => { - if let CreateSink::AsQuery(query) = &stmt.sink_from && have_parameter_in_query(query) { + if let CreateSink::AsQuery(query) = &stmt.sink_from + && have_parameter_in_query(query) + { Err(ErrorCode::NotImplemented( "CREATE SINK AS SELECT with parameters".to_string(), None.into(), diff --git a/src/frontend/src/optimizer/plan_expr_rewriter/cse_rewriter.rs b/src/frontend/src/optimizer/plan_expr_rewriter/cse_rewriter.rs index d027bf9273696..bc59b0c89a8fa 100644 --- a/src/frontend/src/optimizer/plan_expr_rewriter/cse_rewriter.rs +++ b/src/frontend/src/optimizer/plan_expr_rewriter/cse_rewriter.rs @@ -36,7 +36,9 @@ impl CseRewriter { impl ExprRewriter for CseRewriter { fn rewrite_function_call(&mut self, func_call: FunctionCall) -> ExprImpl { - if let Some(count) = self.expr_counter.counter.get(&func_call) && *count > 1 { + if let Some(count) = self.expr_counter.counter.get(&func_call) + && *count > 1 + { if let Some(expr) = self.cse_mapping.get(&func_call) { let expr: ExprImpl = ExprImpl::InputRef(expr.clone().into()); return expr; diff --git a/src/frontend/src/optimizer/plan_node/generic/hop_window.rs b/src/frontend/src/optimizer/plan_node/generic/hop_window.rs index 9bd0dec4b70cc..36095041b6380 100644 --- a/src/frontend/src/optimizer/plan_node/generic/hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/generic/hop_window.rs @@ -106,7 +106,9 @@ impl GenericPlanNode for HopWindow { internal2output.try_map(self.internal_window_end_col_idx()), ) }; - if let Some(start_idx) = start_idx_in_output && let Some(end_idx) = end_idx_in_output { + if let Some(start_idx) = start_idx_in_output + && let Some(end_idx) = end_idx_in_output + { fd_set.add_functional_dependency_by_column_indices(&[start_idx], &[end_idx]); fd_set.add_functional_dependency_by_column_indices(&[end_idx], &[start_idx]); } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index c594ededa40cf..35a15a5d64792 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -977,7 +977,10 @@ impl LogicalJoin { // Use primary table. let mut result_plan = self.to_stream_temporal_join(predicate.clone(), ctx); // Return directly if this temporal join can match the pk of its right table. - if let Ok(temporal_join) = &result_plan && temporal_join.eq_join_predicate().eq_indexes().len() == logical_scan.primary_key().len() { + if let Ok(temporal_join) = &result_plan + && temporal_join.eq_join_predicate().eq_indexes().len() + == logical_scan.primary_key().len() + { return result_plan; } let indexes = logical_scan.indexes(); diff --git a/src/frontend/src/optimizer/plan_node/logical_over_window.rs b/src/frontend/src/optimizer/plan_node/logical_over_window.rs index a78a145ab1997..6193c072563c6 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_window.rs @@ -115,45 +115,44 @@ impl<'a> LogicalOverWindowBuilder<'a> { match agg_kind { AggKind::Avg => { assert_eq!(args.len(), 1); - let left_ref = ExprImpl::from(self.push_window_func(WindowFunction::new( + let left_ref = ExprImpl::from(self.push_window_func(WindowFunction::new( WindowFuncKind::Aggregate(AggKind::Sum), partition_by.clone(), order_by.clone(), args.clone(), frame.clone(), - )?)).cast_explicit(return_type)?; + )?)) + .cast_explicit(return_type)?; let right_ref = ExprImpl::from(self.push_window_func(WindowFunction::new( - WindowFuncKind::Aggregate(AggKind::Count), - partition_by, - order_by, - args, - frame, - )?)); - - let new_expr = ExprImpl::from( - FunctionCall::new(ExprType::Divide, vec![left_ref, right_ref])?, - ); + WindowFuncKind::Aggregate(AggKind::Count), + partition_by, + order_by, + args, + frame, + )?)); + + let new_expr = ExprImpl::from(FunctionCall::new( + ExprType::Divide, + vec![left_ref, right_ref], + )?); Ok(new_expr) } - AggKind::StddevPop - | AggKind::StddevSamp - | AggKind::VarPop - | AggKind::VarSamp => { + AggKind::StddevPop | AggKind::StddevSamp | AggKind::VarPop | AggKind::VarSamp => { let input = args.first().unwrap(); - let squared_input_expr = ExprImpl::from( - FunctionCall::new( - ExprType::Multiply, - vec![input.clone(), input.clone()], - )?, - ); - - let sum_of_squares_expr = ExprImpl::from(self.push_window_func(WindowFunction::new( - WindowFuncKind::Aggregate(AggKind::Sum), - partition_by.clone(), - order_by.clone(), - vec![squared_input_expr], - frame.clone(), - )?)).cast_explicit(return_type.clone())?; + let squared_input_expr = ExprImpl::from(FunctionCall::new( + ExprType::Multiply, + vec![input.clone(), input.clone()], + )?); + + let sum_of_squares_expr = + ExprImpl::from(self.push_window_func(WindowFunction::new( + WindowFuncKind::Aggregate(AggKind::Sum), + partition_by.clone(), + order_by.clone(), + vec![squared_input_expr], + frame.clone(), + )?)) + .cast_explicit(return_type.clone())?; let sum_expr = ExprImpl::from(self.push_window_func(WindowFunction::new( WindowFuncKind::Aggregate(AggKind::Sum), @@ -161,7 +160,8 @@ impl<'a> LogicalOverWindowBuilder<'a> { order_by.clone(), args.clone(), frame.clone(), - )?)).cast_explicit(return_type.clone())?; + )?)) + .cast_explicit(return_type.clone())?; let count_expr = ExprImpl::from(self.push_window_func(WindowFunction::new( WindowFuncKind::Aggregate(AggKind::Count), @@ -171,32 +171,26 @@ impl<'a> LogicalOverWindowBuilder<'a> { frame, )?)); - let square_of_sum_expr = ExprImpl::from( - FunctionCall::new( - ExprType::Multiply, - vec![sum_expr.clone(), sum_expr], - )?, - ); - - let numerator_expr = ExprImpl::from( - FunctionCall::new( - ExprType::Subtract, - vec![ - sum_of_squares_expr, - ExprImpl::from( - FunctionCall::new( - ExprType::Divide, - vec![square_of_sum_expr, count_expr.clone()], - )?, - ), - ], - )?, - ); + let square_of_sum_expr = ExprImpl::from(FunctionCall::new( + ExprType::Multiply, + vec![sum_expr.clone(), sum_expr], + )?); + + let numerator_expr = ExprImpl::from(FunctionCall::new( + ExprType::Subtract, + vec![ + sum_of_squares_expr, + ExprImpl::from(FunctionCall::new( + ExprType::Divide, + vec![square_of_sum_expr, count_expr.clone()], + )?), + ], + )?); let denominator_expr = match agg_kind { AggKind::StddevPop | AggKind::VarPop => count_expr.clone(), - AggKind::StddevSamp | AggKind::VarSamp => ExprImpl::from( - FunctionCall::new( + AggKind::StddevSamp | AggKind::VarSamp => { + ExprImpl::from(FunctionCall::new( ExprType::Subtract, vec![ count_expr.clone(), @@ -205,17 +199,15 @@ impl<'a> LogicalOverWindowBuilder<'a> { DataType::Int64, )), ], - )?, - ), + )?) + } _ => unreachable!(), }; - let mut target_expr = ExprImpl::from( - FunctionCall::new( - ExprType::Divide, - vec![numerator_expr, denominator_expr], - )?, - ); + let mut target_expr = ExprImpl::from(FunctionCall::new( + ExprType::Divide, + vec![numerator_expr, denominator_expr], + )?); if matches!(agg_kind, AggKind::StddevPop | AggKind::StddevSamp) { target_expr = ExprImpl::from( @@ -224,31 +216,24 @@ impl<'a> LogicalOverWindowBuilder<'a> { } match agg_kind { - AggKind::VarPop | AggKind::StddevPop => { - Ok(target_expr) - } + AggKind::VarPop | AggKind::StddevPop => Ok(target_expr), AggKind::StddevSamp | AggKind::VarSamp => { - let less_than_expr = ExprImpl::from( - FunctionCall::new( - ExprType::LessThanOrEqual, - vec![ - count_expr, - ExprImpl::from(Literal::new( - Datum::from(ScalarImpl::Int64(1)), - DataType::Int64, - )), - ], - )?, - ); - let null_expr = - ExprImpl::from(Literal::new(None, return_type)); - - let case_expr = ExprImpl::from( - FunctionCall::new( - ExprType::Case, - vec![less_than_expr, null_expr, target_expr], - )?, - ); + let less_than_expr = ExprImpl::from(FunctionCall::new( + ExprType::LessThanOrEqual, + vec![ + count_expr, + ExprImpl::from(Literal::new( + Datum::from(ScalarImpl::Int64(1)), + DataType::Int64, + )), + ], + )?); + let null_expr = ExprImpl::from(Literal::new(None, return_type)); + + let case_expr = ExprImpl::from(FunctionCall::new( + ExprType::Case, + vec![less_than_expr, null_expr, target_expr], + )?); Ok(case_expr) } _ => unreachable!(), @@ -307,21 +292,19 @@ impl<'a> OverWindowProjectBuilder<'a> { window_function: &WindowFunction, ) -> std::result::Result<(), ErrorCode> { if let WindowFuncKind::Aggregate(agg_kind) = window_function.kind - && matches!( - agg_kind, - AggKind::StddevPop - | AggKind::StddevSamp - | AggKind::VarPop - | AggKind::VarSamp - ) - { - let input = window_function.args.iter().exactly_one().unwrap(); - let squared_input_expr = ExprImpl::from( - FunctionCall::new(ExprType::Multiply, vec![input.clone(), input.clone()]) - .unwrap(), - ); - self.builder.add_expr(&squared_input_expr).map_err(|err| ErrorCode::NotImplemented(format!("{err} inside args"), None.into()))?; - } + && matches!( + agg_kind, + AggKind::StddevPop | AggKind::StddevSamp | AggKind::VarPop | AggKind::VarSamp + ) + { + let input = window_function.args.iter().exactly_one().unwrap(); + let squared_input_expr = ExprImpl::from( + FunctionCall::new(ExprType::Multiply, vec![input.clone(), input.clone()]).unwrap(), + ); + self.builder.add_expr(&squared_input_expr).map_err(|err| { + ErrorCode::NotImplemented(format!("{err} inside args"), None.into()) + })?; + } for arg in &window_function.args { self.builder.add_expr(arg).map_err(|err| { ErrorCode::NotImplemented(format!("{err} inside args"), None.into()) diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index cac051957b0a5..542178a830b73 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -571,7 +571,9 @@ impl ToStream for LogicalSource { } assert!(!(self.core.gen_row_id && self.core.for_table)); - if let Some(row_id_index) = self.core.row_id_index && self.core.gen_row_id { + if let Some(row_id_index) = self.core.row_id_index + && self.core.gen_row_id + { plan = StreamRowIdGen::new(plan, row_id_index).into(); } Ok(plan) diff --git a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs index d0c3077f83286..ca208f94df05a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs @@ -57,7 +57,9 @@ impl StreamGroupTopN { let mut stream_key = core .stream_key() .expect("logical node should have stream key here"); - if let Some(vnode_col_idx) = vnode_col_idx && stream_key.len() > 1 { + if let Some(vnode_col_idx) = vnode_col_idx + && stream_key.len() > 1 + { // The output stream key of `GroupTopN` is a union of group key and input stream key, // while vnode is calculated from a subset of input stream key. So we can safely remove // the vnode column from output stream key. While at meanwhile we cannot leave the stream key diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs index 36aff15d96055..1dc35bb909905 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -161,7 +161,9 @@ impl StreamHashJoin { ) }; let mut is_valuable_inequality = do_state_cleaning; - if let Some(internal) = internal && !watermark_columns.contains(internal) { + if let Some(internal) = internal + && !watermark_columns.contains(internal) + { watermark_columns.insert(internal); is_valuable_inequality = true; } diff --git a/src/frontend/src/optimizer/rule/except_merge_rule.rs b/src/frontend/src/optimizer/rule/except_merge_rule.rs index 0979aaf262f06..77b4c827d07c3 100644 --- a/src/frontend/src/optimizer/rule/except_merge_rule.rs +++ b/src/frontend/src/optimizer/rule/except_merge_rule.rs @@ -26,7 +26,9 @@ impl Rule for ExceptMergeRule { let top_except_inputs = top_except.inputs(); let (left_most_input, remain_vec) = top_except_inputs.split_at(1); - if let Some(bottom_except) = left_most_input[0].as_logical_except() && bottom_except.all() == top_all { + if let Some(bottom_except) = left_most_input[0].as_logical_except() + && bottom_except.all() == top_all + { let mut new_inputs = vec![]; new_inputs.extend(bottom_except.inputs()); new_inputs.extend(remain_vec.iter().cloned()); diff --git a/src/frontend/src/optimizer/rule/index_delta_join_rule.rs b/src/frontend/src/optimizer/rule/index_delta_join_rule.rs index 30435d635568b..5ae42877f6454 100644 --- a/src/frontend/src/optimizer/rule/index_delta_join_rule.rs +++ b/src/frontend/src/optimizer/rule/index_delta_join_rule.rs @@ -122,11 +122,8 @@ impl Rule for IndexDeltaJoinRule { if chain_type != table_scan.chain_type() { Some( - StreamTableScan::new_with_chain_type( - table_scan.core().clone(), - chain_type, - ) - .into(), + StreamTableScan::new_with_chain_type(table_scan.core().clone(), chain_type) + .into(), ) } else { Some(table_scan.clone().into()) diff --git a/src/frontend/src/optimizer/rule/intersect_merge_rule.rs b/src/frontend/src/optimizer/rule/intersect_merge_rule.rs index c21ea85e6baf5..3110b9e3881b6 100644 --- a/src/frontend/src/optimizer/rule/intersect_merge_rule.rs +++ b/src/frontend/src/optimizer/rule/intersect_merge_rule.rs @@ -24,7 +24,9 @@ impl Rule for IntersectMergeRule { let mut new_inputs = vec![]; let mut has_merge = false; for input in top_intersect.inputs() { - if let Some(bottom_intersect) = input.as_logical_intersect() && bottom_intersect.all() == top_all { + if let Some(bottom_intersect) = input.as_logical_intersect() + && bottom_intersect.all() == top_all + { new_inputs.extend(bottom_intersect.inputs()); has_merge = true; } else { diff --git a/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs b/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs index 93637d3ba8193..496a51d6d9f3d 100644 --- a/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs +++ b/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs @@ -161,7 +161,9 @@ fn handle_rank_preds(rank_preds: &[ExprImpl], window_func_pos: usize) -> Option< assert_eq!(input_ref.index, window_func_pos); let v = v.cast_implicit(DataType::Int64).ok()?.fold_const().ok()??; let v = *v.as_int64(); - if let Some(eq) = eq && eq != v { + if let Some(eq) = eq + && eq != v + { tracing::warn!( "Failed to optimize rank predicate with conflicting equal conditions." ); diff --git a/src/frontend/src/optimizer/rule/union_merge_rule.rs b/src/frontend/src/optimizer/rule/union_merge_rule.rs index 169c9b72c530f..cd7199c2a5050 100644 --- a/src/frontend/src/optimizer/rule/union_merge_rule.rs +++ b/src/frontend/src/optimizer/rule/union_merge_rule.rs @@ -24,7 +24,9 @@ impl Rule for UnionMergeRule { let mut new_inputs = vec![]; let mut has_merge = false; for input in top_union.inputs() { - if let Some(bottom_union) = input.as_logical_union() && bottom_union.all() == top_all { + if let Some(bottom_union) = input.as_logical_union() + && bottom_union.all() == top_all + { new_inputs.extend(bottom_union.inputs()); has_merge = true; } else { diff --git a/src/frontend/src/planner/query.rs b/src/frontend/src/planner/query.rs index a5f9651c4141e..d00ce93598a1e 100644 --- a/src/frontend/src/planner/query.rs +++ b/src/frontend/src/planner/query.rs @@ -56,7 +56,9 @@ impl Planner { } let mut out_fields = FixedBitSet::with_capacity(plan.schema().len()); out_fields.insert_range(..plan.schema().len() - extra_order_exprs_len); - if let Some(field) = plan.schema().fields.get(0) && field.name == "projected_row_id" { + if let Some(field) = plan.schema().fields.get(0) + && field.name == "projected_row_id" + { // Do not output projected_row_id hidden column. out_fields.set(0, false); } diff --git a/src/frontend/src/planner/select.rs b/src/frontend/src/planner/select.rs index 96b32680309df..5266ce55710e3 100644 --- a/src/frontend/src/planner/select.rs +++ b/src/frontend/src/planner/select.rs @@ -67,7 +67,9 @@ impl Planner { exprs.iter().map(|expr| (expr.clone(), false)).collect(); let mut uncovered_distinct_on_exprs_cnt = distinct_on_exprs.len(); let mut order_iter = order.iter().map(|o| &select_items[o.column_index]); - while uncovered_distinct_on_exprs_cnt > 0 && let Some(order_expr) = order_iter.next() { + while uncovered_distinct_on_exprs_cnt > 0 + && let Some(order_expr) = order_iter.next() + { match distinct_on_exprs.get_mut(order_expr) { Some(has_been_covered) => { if !*has_been_covered { @@ -179,7 +181,9 @@ impl Planner { if let BoundDistinct::Distinct = distinct { let fields = root.schema().fields(); - let group_key = if let Some(field) = fields.get(0) && field.name == "projected_row_id" { + let group_key = if let Some(field) = fields.get(0) + && field.name == "projected_row_id" + { // Do not group by projected_row_id hidden column. (1..fields.len()).collect() } else { diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index 7c3030370d56f..cee3db2986cf5 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -344,7 +344,10 @@ impl StageRunner { // the task. // We schedule the task to the worker node that owns the data partition. let parallel_unit_ids = vnode_bitmaps.keys().cloned().collect_vec(); - let workers = self.worker_node_manager.manager.get_workers_by_parallel_unit_ids(¶llel_unit_ids)?; + let workers = self + .worker_node_manager + .manager + .get_workers_by_parallel_unit_ids(¶llel_unit_ids)?; for (i, (parallel_unit_id, worker)) in parallel_unit_ids .into_iter() .zip_eq_fast(workers.into_iter()) diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index 28cfa25b70bf1..d3d558ef4eff2 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -282,7 +282,10 @@ impl LocalQueryExecution { // `exchange_source`. let (parallel_unit_ids, vnode_bitmaps): (Vec<_>, Vec<_>) = vnode_bitmaps.clone().into_iter().unzip(); - let workers = self.worker_node_manager.manager.get_workers_by_parallel_unit_ids(¶llel_unit_ids)?; + let workers = self + .worker_node_manager + .manager + .get_workers_by_parallel_unit_ids(¶llel_unit_ids)?; for (idx, (worker_node, partition)) in (workers.into_iter().zip_eq_fast(vnode_bitmaps.into_iter())).enumerate() { @@ -355,8 +358,12 @@ impl LocalQueryExecution { sources.push(exchange_source); } } else { - let second_stage_plan_node = - self.convert_plan_node(&second_stage.root, &mut None, None, next_executor_id)?; + let second_stage_plan_node = self.convert_plan_node( + &second_stage.root, + &mut None, + None, + next_executor_id, + )?; let second_stage_plan_fragment = PlanFragment { root: Some(second_stage_plan_node), exchange_info: Some(ExchangeInfo { diff --git a/src/frontend/src/utils/condition.rs b/src/frontend/src/utils/condition.rs index 5305f9f1f356a..332dda739bb3a 100644 --- a/src/frontend/src/utils/condition.rs +++ b/src/frontend/src/utils/condition.rs @@ -889,7 +889,9 @@ impl Condition { } // remove all constant boolean `true` res.retain(|expr| { - if let Some(v) = try_get_bool_constant(expr) && v { + if let Some(v) = try_get_bool_constant(expr) + && v + { false } else { true diff --git a/src/frontend/src/utils/stream_graph_formatter.rs b/src/frontend/src/utils/stream_graph_formatter.rs index 2e9e6d1bb01ec..500167f9380de 100644 --- a/src/frontend/src/utils/stream_graph_formatter.rs +++ b/src/frontend/src/utils/stream_graph_formatter.rs @@ -261,57 +261,58 @@ impl StreamGraphFormatter { self.pretty_add_table(node.get_state_table().unwrap()), )); } - stream_node::NodeBody::Chain(node) => { - fields.push(( - "state table", - self.pretty_add_table(node.get_state_table().unwrap()), - )) - } - stream_node::NodeBody::Sort(node) => { + stream_node::NodeBody::Chain(node) => fields.push(( + "state table", + self.pretty_add_table(node.get_state_table().unwrap()), + )), + stream_node::NodeBody::Sort(node) => { fields.push(( "state table", self.pretty_add_table(node.get_state_table().unwrap()), )); } stream_node::NodeBody::WatermarkFilter(node) => { - let vec = node.tables.iter().map(|tb| self.pretty_add_table(tb) ).collect_vec(); - fields.push(("state tables", Pretty::Array(vec) - )); + let vec = node + .tables + .iter() + .map(|tb| self.pretty_add_table(tb)) + .collect_vec(); + fields.push(("state tables", Pretty::Array(vec))); } stream_node::NodeBody::EowcOverWindow(node) => { fields.push(( - "state table", - self.pretty_add_table(node.get_state_table().unwrap()), + "state table", + self.pretty_add_table(node.get_state_table().unwrap()), )); } - stream_node::NodeBody::OverWindow(node) =>{ + stream_node::NodeBody::OverWindow(node) => { fields.push(( "state table", self.pretty_add_table(node.get_state_table().unwrap()), )); } - stream_node::NodeBody::Project(_) | - stream_node::NodeBody::Filter(_) | - stream_node::NodeBody::StatelessSimpleAgg(_) | - stream_node::NodeBody::HopWindow(_) | - stream_node::NodeBody::Merge(_) | - stream_node::NodeBody::Exchange(_) | - stream_node::NodeBody::BatchPlan(_) | - stream_node::NodeBody::Lookup(_) | - stream_node::NodeBody::LookupUnion(_) | - stream_node::NodeBody::Union(_) | - stream_node::NodeBody::DeltaIndexJoin(_) | - stream_node::NodeBody::Sink(_) | - stream_node::NodeBody::Expand(_) | - stream_node::NodeBody::ProjectSet(_) | - stream_node::NodeBody::Dml(_) | - stream_node::NodeBody::RowIdGen(_) | - stream_node::NodeBody::TemporalJoin(_) | - stream_node::NodeBody::BarrierRecv(_) | - stream_node::NodeBody::Values(_) | - stream_node::NodeBody::Source(_) | - stream_node::NodeBody::StreamFsFetch(_) | - stream_node::NodeBody::NoOp(_) => {} + stream_node::NodeBody::Project(_) + | stream_node::NodeBody::Filter(_) + | stream_node::NodeBody::StatelessSimpleAgg(_) + | stream_node::NodeBody::HopWindow(_) + | stream_node::NodeBody::Merge(_) + | stream_node::NodeBody::Exchange(_) + | stream_node::NodeBody::BatchPlan(_) + | stream_node::NodeBody::Lookup(_) + | stream_node::NodeBody::LookupUnion(_) + | stream_node::NodeBody::Union(_) + | stream_node::NodeBody::DeltaIndexJoin(_) + | stream_node::NodeBody::Sink(_) + | stream_node::NodeBody::Expand(_) + | stream_node::NodeBody::ProjectSet(_) + | stream_node::NodeBody::Dml(_) + | stream_node::NodeBody::RowIdGen(_) + | stream_node::NodeBody::TemporalJoin(_) + | stream_node::NodeBody::BarrierRecv(_) + | stream_node::NodeBody::Values(_) + | stream_node::NodeBody::Source(_) + | stream_node::NodeBody::StreamFsFetch(_) + | stream_node::NodeBody::NoOp(_) => {} }; if self.verbose { diff --git a/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs index 7dc7a4688e644..64b2e24c756d5 100644 --- a/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs @@ -63,12 +63,16 @@ impl SpaceReclaimCompactionPicker { ) -> Option { assert!(!levels.levels.is_empty()); let mut select_input_ssts = vec![]; - if let Some(l0) = levels.l0.as_ref() && state.last_level == 0 { + if let Some(l0) = levels.l0.as_ref() + && state.last_level == 0 + { // only pick trivial reclaim sstables because this kind of task could be optimized and do not need send to compactor. for level in &l0.sub_levels { for sst in &level.table_infos { let exist_count = self.exist_table_count(sst); - if exist_count == sst.table_ids.len() || level_handlers[0].is_pending_compact( &sst.sst_id) { + if exist_count == sst.table_ids.len() + || level_handlers[0].is_pending_compact(&sst.sst_id) + { if !select_input_ssts.is_empty() { break; } diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 1b3a284e9ccc9..678374b8c571a 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -353,7 +353,7 @@ impl HummockManager { && !env.opts.do_not_config_object_storage_lifecycle { let is_bucket_expiration_configured = s3.inner().configure_bucket_lifecycle().await; - if is_bucket_expiration_configured{ + if is_bucket_expiration_configured { return Err(ObjectError::internal("Cluster cannot start with object expiration configured for bucket because RisingWave data will be lost when object expiration kicks in. Please disable object expiration and restart the cluster.") .into()); diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index 8b26b8afa11d9..c3445da8cc57a 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -78,7 +78,9 @@ impl FragmentManagerCore { .filter(|tf| tf.state() != State::Initial) .flat_map(|table_fragments| { table_fragments.fragments.values().filter_map(|fragment| { - if let Some(id_filter) = id_filter.as_ref() && !id_filter.contains(&fragment.fragment_id) { + if let Some(id_filter) = id_filter.as_ref() + && !id_filter.contains(&fragment.fragment_id) + { return None; } let parallelism = match fragment.vnode_mapping.as_ref() { @@ -687,7 +689,9 @@ impl FragmentManager { .with_context(|| format!("table_fragment not exist: id={}", table_id))?; for status in table_fragment.actor_status.values_mut() { - if let Some(pu) = &status.parallel_unit && migration_plan.parallel_unit_plan.contains_key(&pu.id) { + if let Some(pu) = &status.parallel_unit + && migration_plan.parallel_unit_plan.contains_key(&pu.id) + { status.parallel_unit = Some(migration_plan.parallel_unit_plan[&pu.id].clone()); } } @@ -717,8 +721,9 @@ impl FragmentManager { .values() .filter(|tf| { for status in tf.actor_status.values() { - if let Some(pu) = &status.parallel_unit && - migration_plan.parallel_unit_plan.contains_key(&pu.id) { + if let Some(pu) = &status.parallel_unit + && migration_plan.parallel_unit_plan.contains_key(&pu.id) + { return true; } } diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index d2007dcab45d6..781dd244c1c57 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -742,7 +742,8 @@ impl CatalogManager { fn assert_table_creating(tables: &BTreeMap, table: &Table) { if let Some(t) = tables.get(&table.id) && let Ok(StreamJobStatus::Creating) = t.get_stream_job_status() - {} else { + { + } else { panic!("Table must be in creating procedure: {table:#?}") } } diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 8f6e7c0be6915..c08281a2f59ed 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -402,11 +402,15 @@ impl DdlController { async fn delete_vpc_endpoint(&self, connection: &Connection) -> MetaResult<()> { // delete AWS vpc endpoint if let Some(connection::Info::PrivateLinkService(svc)) = &connection.info - && svc.get_provider()? == PbPrivateLinkProvider::Aws { + && svc.get_provider()? == PbPrivateLinkProvider::Aws + { if let Some(aws_cli) = self.aws_client.as_ref() { aws_cli.delete_vpc_endpoint(&svc.endpoint_id).await?; } else { - warn!("AWS client is not initialized, skip deleting vpc endpoint {}", svc.endpoint_id); + warn!( + "AWS client is not initialized, skip deleting vpc endpoint {}", + svc.endpoint_id + ); } } Ok(()) diff --git a/src/meta/src/rpc/election/etcd.rs b/src/meta/src/rpc/election/etcd.rs index f30d8253cb95d..dbcd62abfdbb7 100644 --- a/src/meta/src/rpc/election/etcd.rs +++ b/src/meta/src/rpc/election/etcd.rs @@ -94,7 +94,9 @@ impl ElectionClient for EtcdElectionClient { let (mut keeper, mut resp_stream) = self.client.keep_alive(lease_id).await?; let _resp = keeper.keep_alive().await?; let resp = resp_stream.message().await?; - if let Some(resp) = resp && resp.ttl() <= 0 { + if let Some(resp) = resp + && resp.ttl() <= 0 + { tracing::info!("lease {} expired or revoked, re-granting", lease_id); if restored_leader { tracing::info!("restored leader lease {} lost", lease_id); diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 77a784c64ac09..003252582d6c5 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -126,7 +126,11 @@ impl CreatingStreamingJobInfo { && let Some(shutdown_tx) = job.shutdown_tx.take() { let (tx, rx) = oneshot::channel(); - if shutdown_tx.send(CreatingState::Canceling { finish_tx: tx }).await.is_ok() { + if shutdown_tx + .send(CreatingState::Canceling { finish_tx: tx }) + .await + .is_ok() + { receivers.insert(job_id, rx); } else { tracing::warn!("failed to send canceling state"); diff --git a/src/object_store/src/object/mem.rs b/src/object_store/src/object/mem.rs index 02ee6c744d33a..0cd266abb293d 100644 --- a/src/object_store/src/object/mem.rs +++ b/src/object_store/src/object/mem.rs @@ -237,7 +237,9 @@ impl InMemObjectStore { .map(|(_, obj)| obj) .ok_or_else(|| Error::not_found(format!("no object at path '{}'", path)))?; - if let Some(end) = range.end() && end > obj.len() { + if let Some(end) = range.end() + && end > obj.len() + { return Err(Error::other("bad block offset and size").into()); } diff --git a/src/object_store/src/object/opendal_engine/opendal_object_store.rs b/src/object_store/src/object/opendal_engine/opendal_object_store.rs index ff682946b0651..204d9cac25753 100644 --- a/src/object_store/src/object/opendal_engine/opendal_object_store.rs +++ b/src/object_store/src/object/opendal_engine/opendal_object_store.rs @@ -86,7 +86,9 @@ impl ObjectStore for OpendalObjectStore { .await? }; - if let Some(len) = range.len() && len != data.len() { + if let Some(len) = range.len() + && len != data.len() + { return Err(ObjectError::internal(format!( "mismatched size: expected {}, found {} when reading {} at {:?}", len, diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index 89f9aa5a053d5..6dce56ed4c759 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -386,7 +386,9 @@ impl ObjectStore for S3ObjectStore { ) .await?; - if let Some(len) = range.len() && len != val.len() { + if let Some(len) = range.len() + && len != val.len() + { return Err(ObjectError::internal(format!( "mismatched size: expected {}, found {} when reading {} at {:?}", len, diff --git a/src/risedevtool/src/config/provide_expander.rs b/src/risedevtool/src/config/provide_expander.rs index 9948c81b0336c..2860ffd6c9850 100644 --- a/src/risedevtool/src/config/provide_expander.rs +++ b/src/risedevtool/src/config/provide_expander.rs @@ -70,7 +70,9 @@ impl ProvideExpander { .into_hash() .ok_or_else(|| anyhow!("expect a hashmap"))?; let map = map.into_iter().map(|(k, v)| { - if let Some(k) = k.as_str() && k.starts_with("provide-") { + if let Some(k) = k.as_str() + && k.starts_with("provide-") + { let array = v .as_vec() .ok_or_else(|| anyhow!("expect an array of provide-"))?; diff --git a/src/risedevtool/src/preflight_check.rs b/src/risedevtool/src/preflight_check.rs index 47fb8235495fc..e29e3f4709c93 100644 --- a/src/risedevtool/src/preflight_check.rs +++ b/src/risedevtool/src/preflight_check.rs @@ -26,7 +26,10 @@ fn preflight_check_proxy() -> Result<()> { || env::var("all_proxy").is_ok() || env::var("ALL_PROXY").is_ok() { - if let Ok(x) = env::var("no_proxy") && x.contains("127.0.0.1") && x.contains("::1") { + if let Ok(x) = env::var("no_proxy") + && x.contains("127.0.0.1") + && x.contains("::1") + { println!( "[{}] {} - You are using proxies for all RisingWave components. Please make sure that `no_proxy` is set for all worker nodes within the cluster.", style("risedev-preflight-check").bold(), diff --git a/src/risedevtool/src/task/compactor_service.rs b/src/risedevtool/src/task/compactor_service.rs index adecc007b8207..c7dcbb8c1179c 100644 --- a/src/risedevtool/src/task/compactor_service.rs +++ b/src/risedevtool/src/task/compactor_service.rs @@ -34,7 +34,9 @@ impl CompactorService { fn compactor(&self) -> Result { let prefix_bin = env::var("PREFIX_BIN")?; - if let Ok(x) = env::var("ENABLE_ALL_IN_ONE") && x == "true" { + if let Ok(x) = env::var("ENABLE_ALL_IN_ONE") + && x == "true" + { Ok(Command::new( Path::new(&prefix_bin).join("risingwave").join("compactor"), )) diff --git a/src/risedevtool/src/task/compute_node_service.rs b/src/risedevtool/src/task/compute_node_service.rs index ced6bec115f6a..20d01f33f53d1 100644 --- a/src/risedevtool/src/task/compute_node_service.rs +++ b/src/risedevtool/src/task/compute_node_service.rs @@ -34,7 +34,9 @@ impl ComputeNodeService { fn compute_node(&self) -> Result { let prefix_bin = env::var("PREFIX_BIN")?; - if let Ok(x) = env::var("ENABLE_ALL_IN_ONE") && x == "true" { + if let Ok(x) = env::var("ENABLE_ALL_IN_ONE") + && x == "true" + { Ok(Command::new( Path::new(&prefix_bin) .join("risingwave") diff --git a/src/risedevtool/src/task/frontend_service.rs b/src/risedevtool/src/task/frontend_service.rs index cf0213028e465..b19167abbfdd1 100644 --- a/src/risedevtool/src/task/frontend_service.rs +++ b/src/risedevtool/src/task/frontend_service.rs @@ -35,7 +35,9 @@ impl FrontendService { fn frontend(&self) -> Result { let prefix_bin = env::var("PREFIX_BIN")?; - if let Ok(x) = env::var("ENABLE_ALL_IN_ONE") && x == "true" { + if let Ok(x) = env::var("ENABLE_ALL_IN_ONE") + && x == "true" + { Ok(Command::new( Path::new(&prefix_bin) .join("risingwave") diff --git a/src/risedevtool/src/task/meta_node_service.rs b/src/risedevtool/src/task/meta_node_service.rs index 2494a9eceaf16..df48b59c2f1f5 100644 --- a/src/risedevtool/src/task/meta_node_service.rs +++ b/src/risedevtool/src/task/meta_node_service.rs @@ -35,7 +35,9 @@ impl MetaNodeService { fn meta_node(&self) -> Result { let prefix_bin = env::var("PREFIX_BIN")?; - if let Ok(x) = env::var("ENABLE_ALL_IN_ONE") && x == "true" { + if let Ok(x) = env::var("ENABLE_ALL_IN_ONE") + && x == "true" + { Ok(Command::new( Path::new(&prefix_bin).join("risingwave").join("meta-node"), )) diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index b8603fbe46e62..b58a13027261b 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -247,7 +247,8 @@ impl MetaClient { }) .await?; if let Some(status) = &add_worker_resp.status - && status.code() == risingwave_pb::common::status::Code::UnknownWorker { + && status.code() == risingwave_pb::common::status::Code::UnknownWorker + { tracing::error!("invalid worker: {}", status.message); std::process::exit(1); } diff --git a/src/sqlparser/src/tokenizer.rs b/src/sqlparser/src/tokenizer.rs index 4fafde820f414..1f3b99314d143 100644 --- a/src/sqlparser/src/tokenizer.rs +++ b/src/sqlparser/src/tokenizer.rs @@ -1019,7 +1019,9 @@ impl<'a> Tokenizer<'a> { ) -> Result<(), String> { let mut unicode_seq: String = String::with_capacity(len); for _ in 0..len { - if let Some(c) = chars.peek() && c.is_ascii_hexdigit() { + if let Some(c) = chars.peek() + && c.is_ascii_hexdigit() + { unicode_seq.push(chars.next().unwrap()); } else { break; @@ -1063,7 +1065,9 @@ impl<'a> Tokenizer<'a> { let mut unicode_seq: String = String::with_capacity(3); unicode_seq.push(digit); for _ in 0..2 { - if let Some(c) = chars.peek() && matches!(*c, '0'..='7') { + if let Some(c) = chars.peek() + && matches!(*c, '0'..='7') + { unicode_seq.push(chars.next().unwrap()); } else { break; diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 1925acbce7534..e46965b16e5fc 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -701,7 +701,9 @@ where while iter.is_valid() { progress_key_num += 1; - if let Some(task_progress) = task_progress.as_ref() && progress_key_num >= PROGRESS_KEY_INTERVAL { + if let Some(task_progress) = task_progress.as_ref() + && progress_key_num >= PROGRESS_KEY_INTERVAL + { task_progress.inc_progress_key(progress_key_num); progress_key_num = 0; } @@ -751,7 +753,9 @@ where } del_iter.next(); progress_key_num += 1; - if let Some(task_progress) = task_progress.as_ref() && progress_key_num >= PROGRESS_KEY_INTERVAL { + if let Some(task_progress) = task_progress.as_ref() + && progress_key_num >= PROGRESS_KEY_INTERVAL + { task_progress.inc_progress_key(progress_key_num); progress_key_num = 0; } @@ -858,14 +862,18 @@ where .await?; del_iter.next(); progress_key_num += 1; - if let Some(task_progress) = task_progress.as_ref() && progress_key_num >= PROGRESS_KEY_INTERVAL { + if let Some(task_progress) = task_progress.as_ref() + && progress_key_num >= PROGRESS_KEY_INTERVAL + { task_progress.inc_progress_key(progress_key_num); progress_key_num = 0; } } } - if let Some(task_progress) = task_progress.as_ref() && progress_key_num > 0 { + if let Some(task_progress) = task_progress.as_ref() + && progress_key_num > 0 + { // Avoid losing the progress_key_num in the last Interval task_progress.inc_progress_key(progress_key_num); } diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index a07da55fb7046..3b9e9dc587fe1 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -1333,7 +1333,9 @@ mod tests { assert_eq!(epoch, uploader.max_sealed_epoch); // check sealed data has two imms let imms_by_epoch = uploader.sealed_data.imms_by_epoch(); - if let Some((e, imms)) = imms_by_epoch.last_key_value() && *e == epoch{ + if let Some((e, imms)) = imms_by_epoch.last_key_value() + && *e == epoch + { assert_eq!(2, imms.len()); } diff --git a/src/storage/src/hummock/iterator/delete_range_iterator.rs b/src/storage/src/hummock/iterator/delete_range_iterator.rs index 7936fb994a92a..4d943ce63d3fc 100644 --- a/src/storage/src/hummock/iterator/delete_range_iterator.rs +++ b/src/storage/src/hummock/iterator/delete_range_iterator.rs @@ -291,7 +291,10 @@ impl DeleteRangeIterator for ForwardMergeRangeIterator { async { self.tmp_buffer .push(self.heap.pop().expect("no inner iter")); - while let Some(node) = self.heap.peek() && node.is_valid() && node.next_extended_user_key() == self.tmp_buffer[0].next_extended_user_key() { + while let Some(node) = self.heap.peek() + && node.is_valid() + && node.next_extended_user_key() == self.tmp_buffer[0].next_extended_user_key() + { self.tmp_buffer.push(self.heap.pop().unwrap()); } for node in &self.tmp_buffer { diff --git a/src/storage/src/hummock/mod.rs b/src/storage/src/hummock/mod.rs index 60553b5aa09a3..11af5e7deaea2 100644 --- a/src/storage/src/hummock/mod.rs +++ b/src/storage/src/hummock/mod.rs @@ -77,7 +77,15 @@ pub async fn get_from_sstable_info( // Bloom filter key is the distribution key, which is no need to be the prefix of pk, and do not // contain `TablePrefix` and `VnodePrefix`. if let Some(hash) = dist_key_hash - && !hit_sstable_bloom_filter(sstable.value(), &(Bound::Included(full_key.user_key), Bound::Included(full_key.user_key)), hash, local_stats) + && !hit_sstable_bloom_filter( + sstable.value(), + &( + Bound::Included(full_key.user_key), + Bound::Included(full_key.user_key), + ), + hash, + local_stats, + ) { if !read_options.ignore_range_tombstone { let delete_epoch = get_min_delete_range_epoch_from_sstable( diff --git a/src/storage/src/hummock/sstable/multi_builder.rs b/src/storage/src/hummock/sstable/multi_builder.rs index 9bee67e78ca68..4baabb4fdafe6 100644 --- a/src/storage/src/hummock/sstable/multi_builder.rs +++ b/src/storage/src/hummock/sstable/multi_builder.rs @@ -212,9 +212,17 @@ where } } } - if need_seal_current && let Some(event) = builder.last_range_tombstone() && event.new_epoch != HummockEpoch::MAX { + if need_seal_current + && let Some(event) = builder.last_range_tombstone() + && event.new_epoch != HummockEpoch::MAX + { last_range_tombstone_epoch = event.new_epoch; - if event.event_key.left_user_key.as_ref().eq(&full_key.user_key) { + if event + .event_key + .left_user_key + .as_ref() + .eq(&full_key.user_key) + { // If the last range tombstone equals the new key, we can not create new file because we must keep the new key in origin file. need_seal_current = false; } else { @@ -295,7 +303,10 @@ where /// Add kv pair to sstable. pub async fn add_monotonic_delete(&mut self, event: MonotonicDeleteEvent) -> HummockResult<()> { - if let Some(builder) = self.current_builder.as_mut() && builder.reach_capacity() && event.new_epoch != HummockEpoch::MAX { + if let Some(builder) = self.current_builder.as_mut() + && builder.reach_capacity() + && event.new_epoch != HummockEpoch::MAX + { if builder.last_range_tombstone_epoch() != HummockEpoch::MAX { builder.add_monotonic_delete(MonotonicDeleteEvent { event_key: event.event_key.clone(), diff --git a/src/storage/src/hummock/sstable_store.rs b/src/storage/src/hummock/sstable_store.rs index 73d6110cacd29..25ac82636c77d 100644 --- a/src/storage/src/hummock/sstable_store.rs +++ b/src/storage/src/hummock/sstable_store.rs @@ -821,7 +821,9 @@ impl SstableWriter for StreamingUploadWriter { self.sstable_store.insert_meta_cache(self.object_id, meta); // Add block cache. - if let CachePolicy::Fill(fill_high_priority_cache) = self.policy && !self.blocks.is_empty() { + if let CachePolicy::Fill(fill_high_priority_cache) = self.policy + && !self.blocks.is_empty() + { for (block_idx, block) in self.blocks.into_iter().enumerate() { self.sstable_store.block_cache.insert( self.object_id, diff --git a/src/storage/src/memory.rs b/src/storage/src/memory.rs index ac52b65e5488f..c454ad94339dc 100644 --- a/src/storage/src/memory.rs +++ b/src/storage/src/memory.rs @@ -524,7 +524,9 @@ impl RangeKvStateStore { } last_user_key = Some(key.user_key.clone()); } - if let Some(limit) = limit && data.len() >= limit { + if let Some(limit) = limit + && data.len() >= limit + { break; } } diff --git a/src/stream/src/common/log_store_impl/in_mem.rs b/src/stream/src/common/log_store_impl/in_mem.rs index 35040be82c93b..8ff3e05d0bea8 100644 --- a/src/stream/src/common/log_store_impl/in_mem.rs +++ b/src/stream/src/common/log_store_impl/in_mem.rs @@ -222,7 +222,9 @@ impl LogReader for BoundedInMemLogStoreReader { next_epoch, } = &self.epoch_progress { - if let TruncateOffset::Barrier {epoch} = offset && epoch == *sealed_epoch { + if let TruncateOffset::Barrier { epoch } = offset + && epoch == *sealed_epoch + { let sealed_epoch = *sealed_epoch; self.epoch_progress = Consuming(*next_epoch); self.truncated_epoch_tx diff --git a/src/stream/src/common/log_store_impl/kv_log_store/buffer.rs b/src/stream/src/common/log_store_impl/kv_log_store/buffer.rs index ed1c495c81d75..d0773b008c16a 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/buffer.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/buffer.rs @@ -250,7 +250,9 @@ impl LogStoreBufferSender { pub(crate) fn pop_truncation(&self, curr_epoch: u64) -> Option { let mut inner = self.buffer.inner(); let mut ret = None; - while let Some((epoch, _)) = inner.truncation_list.front() && *epoch < curr_epoch { + while let Some((epoch, _)) = inner.truncation_list.front() + && *epoch < curr_epoch + { ret = inner.truncation_list.pop_front(); } ret @@ -380,7 +382,9 @@ impl LogStoreBufferReceiver { } } if let Some((epoch, seq_id)) = latest_offset { - if let Some((prev_epoch, ref mut prev_seq_id)) = inner.truncation_list.back_mut() && *prev_epoch == epoch { + if let Some((prev_epoch, ref mut prev_seq_id)) = inner.truncation_list.back_mut() + && *prev_epoch == epoch + { *prev_seq_id = seq_id; } else { inner.truncation_list.push_back((epoch, seq_id)); diff --git a/src/stream/src/common/log_store_impl/kv_log_store/serde.rs b/src/stream/src/common/log_store_impl/kv_log_store/serde.rs index d3102aa936fad..ba69209887b67 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/serde.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/serde.rs @@ -586,7 +586,9 @@ impl LogStoreRowOpStream { .pop() .expect("have check non-empty"); self.row_streams.push(stream.into_future()); - while let Some((stream_epoch, _)) = self.not_started_streams.last() && *stream_epoch == epoch { + while let Some((stream_epoch, _)) = self.not_started_streams.last() + && *stream_epoch == epoch + { let (_, stream) = self.not_started_streams.pop().expect("should not be empty"); self.row_streams.push(stream.into_future()); } diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 37e788a3e7abd..f0ef04ab7ec91 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -852,14 +852,14 @@ where match op { Op::Insert | Op::UpdateInsert => { if USE_WATERMARK_CACHE && let Some(ref pk) = key { - self.watermark_cache.insert(pk); - } + self.watermark_cache.insert(pk); + } self.insert_inner(TableKey(key_bytes), value); } Op::Delete | Op::UpdateDelete => { if USE_WATERMARK_CACHE && let Some(ref pk) = key { - self.watermark_cache.delete(pk); - } + self.watermark_cache.delete(pk); + } self.delete_inner(TableKey(key_bytes), value); } } @@ -870,14 +870,14 @@ where match op { Op::Insert | Op::UpdateInsert => { if USE_WATERMARK_CACHE && let Some(ref pk) = key { - self.watermark_cache.insert(pk); - } + self.watermark_cache.insert(pk); + } self.insert_inner(TableKey(key_bytes), value); } Op::Delete | Op::UpdateDelete => { if USE_WATERMARK_CACHE && let Some(ref pk) = key { - self.watermark_cache.delete(pk); - } + self.watermark_cache.delete(pk); + } self.delete_inner(TableKey(key_bytes), value); } } @@ -1026,11 +1026,21 @@ where }); // Compute Delete Ranges - if should_clean_watermark && let Some(watermark_suffix) = watermark_suffix && let Some(first_byte) = watermark_suffix.first() { + if should_clean_watermark + && let Some(watermark_suffix) = watermark_suffix + && let Some(first_byte) = watermark_suffix.first() + { trace!(table_id = %self.table_id, watermark = ?watermark_suffix, vnodes = ?{ self.vnodes.iter_vnodes().collect_vec() }, "delete range"); - if prefix_serializer.as_ref().unwrap().get_order_types().first().unwrap().is_ascending() { + if prefix_serializer + .as_ref() + .unwrap() + .get_order_types() + .first() + .unwrap() + .is_ascending() + { // We either serialize null into `0u8`, data into `(1u8 || scalar)`, or serialize null // into `1u8`, data into `(0u8 || scalar)`. We do not want to delete null // here, so `range_begin_suffix` cannot be `vec![]` when null is represented as `0u8`. diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index ae5e8696de6c3..0bd6e47841584 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -450,7 +450,9 @@ where while let Some(Ok(msg)) = upstream.next().await { if let Some(msg) = mapping_message(msg, &self.output_indices) { // If not finished then we need to update state, otherwise no need. - if let Message::Barrier(barrier) = &msg && !is_completely_finished { + if let Message::Barrier(barrier) = &msg + && !is_completely_finished + { // If snapshot was empty, we do not need to backfill, // but we still need to persist the finished state. // We currently persist it on the second barrier here rather than first. @@ -458,9 +460,13 @@ where // since it expects to have been initialized in previous epoch // (there's no epoch before the first epoch). if is_snapshot_empty { - let finished_state = construct_initial_finished_state(pk_in_output_indices.len()); + let finished_state = + construct_initial_finished_state(pk_in_output_indices.len()); for vnode in upstream_table.vnodes().iter_vnodes() { - backfill_state.update_progress(vnode, BackfillProgressPerVnode::InProgress(finished_state.clone())); + backfill_state.update_progress( + vnode, + BackfillProgressPerVnode::InProgress(finished_state.clone()), + ); } } @@ -471,9 +477,11 @@ where &backfill_state, &mut committed_progress, &mut temporary_state, - ).await?; + ) + .await?; - self.progress.finish(barrier.epoch.curr, total_snapshot_processed_rows); + self.progress + .finish(barrier.epoch.curr, total_snapshot_processed_rows); yield msg; break; } diff --git a/src/stream/src/executor/backfill/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc_backfill.rs index c17aad1d2d62d..e3121e1bd0a7b 100644 --- a/src/stream/src/executor/backfill/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc_backfill.rs @@ -509,35 +509,31 @@ impl CdcBackfillExecutor { .await?; if let Some(SplitImpl::MysqlCdc(split)) = cdc_split.as_mut() - && let Some(s) = split.mysql_split.as_mut() { - let start_offset = - last_binlog_offset.as_ref().map(|cdc_offset| { - let source_offset = - if let CdcOffset::MySql(o) = cdc_offset - { - DebeziumSourceOffset { - file: Some(o.filename.clone()), - pos: Some(o.position), - ..Default::default() - } - } else { - DebeziumSourceOffset::default() - }; - - let mut server = "RW_CDC_".to_string(); - server.push_str( - upstream_table_id.to_string().as_str(), - ); - DebeziumOffset { - source_partition: hashmap! { - "server".to_string() => server - }, - source_offset, - // upstream heartbeat event would not emit to the cdc backfill executor, - // since we don't parse heartbeat event in the source parser. - is_heartbeat: false, + && let Some(s) = split.mysql_split.as_mut() + { + let start_offset = last_binlog_offset.as_ref().map(|cdc_offset| { + let source_offset = if let CdcOffset::MySql(o) = cdc_offset { + DebeziumSourceOffset { + file: Some(o.filename.clone()), + pos: Some(o.position), + ..Default::default() } - }); + } else { + DebeziumSourceOffset::default() + }; + + let mut server = "RW_CDC_".to_string(); + server.push_str(upstream_table_id.to_string().as_str()); + DebeziumOffset { + source_partition: hashmap! { + "server".to_string() => server + }, + source_offset, + // upstream heartbeat event would not emit to the cdc backfill executor, + // since we don't parse heartbeat event in the source parser. + is_heartbeat: false, + } + }); // persist the last binlog offset into split state s.inner.start_offset = start_offset.map(|o| { diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index 8d02cc328fa43..2a321dcc2b64c 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -384,7 +384,9 @@ impl HashAggExecutor { .zip_eq_fast(&mut this.storages) .zip_eq_fast(call_visibilities.iter()) { - if let AggStateStorage::MaterializedInput { table, mapping } = storage && !call.distinct { + if let AggStateStorage::MaterializedInput { table, mapping } = storage + && !call.distinct + { let chunk = chunk.project_with_vis(mapping.upstream_columns(), visibility.clone()); table.write_chunk(chunk); } @@ -413,8 +415,11 @@ impl HashAggExecutor { .zip_eq_fast(&mut this.storages) .zip_eq_fast(visibilities.iter()) { - if let AggStateStorage::MaterializedInput { table, mapping } = storage && call.distinct { - let chunk = chunk.project_with_vis(mapping.upstream_columns(), visibility.clone()); + if let AggStateStorage::MaterializedInput { table, mapping } = storage + && call.distinct + { + let chunk = + chunk.project_with_vis(mapping.upstream_columns(), visibility.clone()); table.write_chunk(chunk); } } diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index 75414fe24a379..93f1734a3ec0e 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -912,7 +912,8 @@ impl HashJoinExecutor input_watermark.val = value.unwrap(), diff --git a/src/stream/src/executor/over_window/general.rs b/src/stream/src/executor/over_window/general.rs index 9e66835b54b05..c9717f9defe61 100644 --- a/src/stream/src/executor/over_window/general.rs +++ b/src/stream/src/executor/over_window/general.rs @@ -389,7 +389,9 @@ impl OverWindowExecutor { } // Update recently accessed range for later shrinking cache. - if !this.cache_policy.is_full() && let Some(accessed_range) = accessed_range { + if !this.cache_policy.is_full() + && let Some(accessed_range) = accessed_range + { match vars.recently_accessed_ranges.entry(part_key) { btree_map::Entry::Vacant(vacant) => { vacant.insert(accessed_range); diff --git a/src/stream/src/executor/project_set.rs b/src/stream/src/executor/project_set.rs index ff3214db88eaa..e1000122af247 100644 --- a/src/stream/src/executor/project_set.rs +++ b/src/stream/src/executor/project_set.rs @@ -189,11 +189,15 @@ impl Inner { // for each column for (item, value) in results.iter_mut().zip_eq_fast(&mut row[1..]) { *value = match item { - Either::Left(state) => if let Some((i, value)) = state.peek() && i == row_idx { - valid = true; - value - } else { - None + Either::Left(state) => { + if let Some((i, value)) = state.peek() + && i == row_idx + { + valid = true; + value + } else { + None + } } Either::Right(array) => array.value_at(row_idx), }; @@ -211,7 +215,9 @@ impl Inner { } // move to the next row for item in &mut results { - if let Either::Left(state) = item && matches!(state.peek(), Some((i, _)) if i == row_idx) { + if let Either::Left(state) = item + && matches!(state.peek(), Some((i, _)) if i == row_idx) + { state.next().await?; } } diff --git a/src/stream/src/executor/source/state_table_handler.rs b/src/stream/src/executor/source/state_table_handler.rs index d742e72a4c7a9..1705350426968 100644 --- a/src/stream/src/executor/source/state_table_handler.rs +++ b/src/stream/src/executor/source/state_table_handler.rs @@ -226,10 +226,13 @@ impl SourceStateTableHandler { Some(row) => match row.datum_at(1) { Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { let mut split_impl = SplitImpl::restore_from_json(jsonb_ref.to_owned_scalar())?; - if let SplitImpl::MysqlCdc(ref mut split) = split_impl && let Some(mysql_split) = split.mysql_split.as_mut() { + if let SplitImpl::MysqlCdc(ref mut split) = split_impl + && let Some(mysql_split) = split.mysql_split.as_mut() + { // if the snapshot_done is not set, we should check whether the backfill is finished if !mysql_split.inner.snapshot_done { - mysql_split.inner.snapshot_done = self.recover_cdc_snapshot_state(split_id).await?; + mysql_split.inner.snapshot_done = + self.recover_cdc_snapshot_state(split_id).await?; } } Some(split_impl) diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs index 82c1e56649672..ddfcfd6b8e041 100644 --- a/src/stream/src/executor/temporal_join.rs +++ b/src/stream/src/executor/temporal_join.rs @@ -444,7 +444,8 @@ impl TemporalJoinExecutor }; if key.null_bitmap().is_subset(&null_matched) && let join_entry = self.right_table.lookup(&key, epoch).await? - && !join_entry.is_empty() { + && !join_entry.is_empty() + { for right_row in join_entry.cached.values() { // check join condition let ok = if let Some(ref mut cond) = self.condition { @@ -458,7 +459,8 @@ impl TemporalJoinExecutor }; if ok { - if let Some(chunk) = builder.append_row(op, left_row, right_row) { + if let Some(chunk) = builder.append_row(op, left_row, right_row) + { yield Message::Chunk(chunk); } } diff --git a/src/stream/src/executor/top_n/top_n_cache.rs b/src/stream/src/executor/top_n/top_n_cache.rs index b8275eba52b16..a1b7e26e8ae3a 100644 --- a/src/stream/src/executor/top_n/top_n_cache.rs +++ b/src/stream/src/executor/top_n/top_n_cache.rs @@ -232,7 +232,9 @@ impl TopNCache { return; } // For direct insert, we need to check if the key is smaller than the largest key - if let Some(high_last) = self.high.last_key_value() && cache_key <= *high_last.0 { + if let Some(high_last) = self.high.last_key_value() + && cache_key <= *high_last.0 + { debug_assert!(cache_key != *high_last.0, "cache_key should be unique"); self.high.insert(cache_key, row); } @@ -260,15 +262,16 @@ impl TopNCacheTrait for TopNCache { self.low.insert(cache_key, (&row).into()); return; } - let elem_to_compare_with_middle = - if let Some(low_last) = self.low.last_entry() && cache_key <= *low_last.key() { - // Take the last element of `cache.low` and insert input row to it. - let low_last = low_last.remove_entry(); - self.low.insert(cache_key, (&row).into()); - low_last - } else { - (cache_key, (&row).into()) - }; + let elem_to_compare_with_middle = if let Some(low_last) = self.low.last_entry() + && cache_key <= *low_last.key() + { + // Take the last element of `cache.low` and insert input row to it. + let low_last = low_last.remove_entry(); + self.low.insert(cache_key, (&row).into()); + low_last + } else { + (cache_key, (&row).into()) + }; if !self.is_middle_cache_full() { self.middle.insert( @@ -586,15 +589,16 @@ impl AppendOnlyTopNCacheTrait for TopNCache { return Ok(()); } - let elem_to_insert_into_middle = - if let Some(low_last) = self.low.last_entry() && &cache_key <= low_last.key() { - // Take the last element of `cache.low` and insert input row to it. - let low_last = low_last.remove_entry(); - self.low.insert(cache_key, row_ref.into()); - low_last - } else { - (cache_key, row_ref.into()) - }; + let elem_to_insert_into_middle = if let Some(low_last) = self.low.last_entry() + && &cache_key <= low_last.key() + { + // Take the last element of `cache.low` and insert input row to it. + let low_last = low_last.remove_entry(); + self.low.insert(cache_key, row_ref.into()); + low_last + } else { + (cache_key, row_ref.into()) + }; if !self.is_middle_cache_full() { self.middle.insert( diff --git a/src/stream/src/executor/top_n/top_n_state.rs b/src/stream/src/executor/top_n/top_n_state.rs index 841e7f5bb50d7..7214eb91064bc 100644 --- a/src/stream/src/executor/top_n/top_n_state.rs +++ b/src/stream/src/executor/top_n/top_n_state.rs @@ -136,7 +136,9 @@ impl ManagedTopNState { while let Some(item) = state_table_iter.next().await { // Note(bugen): should first compare with start key before constructing TopNStateRow. let topn_row = self.get_topn_row(item?.into_owned_row(), group_key.len()); - if let Some(start_key) = start_key.as_ref() && &topn_row.cache_key <= start_key { + if let Some(start_key) = start_key.as_ref() + && &topn_row.cache_key <= start_key + { continue; } // let row= &topn_row.row; @@ -225,7 +227,9 @@ impl ManagedTopNState { topn_cache.high_capacity > 0, "topn cache high_capacity should always > 0" ); - while !topn_cache.is_high_cache_full() && let Some(item) = state_table_iter.next().await { + while !topn_cache.is_high_cache_full() + && let Some(item) = state_table_iter.next().await + { let topn_row = self.get_topn_row(item?.into_owned_row(), group_key.len()); topn_cache .high diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index 5e5454cecff93..c8899553ac46a 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -217,7 +217,9 @@ impl WatermarkFilterExecutor { if watermark.col_idx == event_time_col_idx { tracing::warn!("WatermarkFilterExecutor received a watermark on the event it is filtering."); let watermark = watermark.val; - if let Some(cur_watermark) = current_watermark.clone() && cur_watermark.default_cmp(&watermark).is_lt() { + if let Some(cur_watermark) = current_watermark.clone() + && cur_watermark.default_cmp(&watermark).is_lt() + { current_watermark = Some(watermark.clone()); idle_input = false; yield Message::Watermark(Watermark::new( @@ -267,7 +269,10 @@ impl WatermarkFilterExecutor { let global_max_watermark = Self::get_global_max_watermark(&table).await?; - current_watermark = if let Some(global_max_watermark) = global_max_watermark.clone() && let Some(watermark) = current_watermark.clone(){ + current_watermark = if let Some(global_max_watermark) = + global_max_watermark.clone() + && let Some(watermark) = current_watermark.clone() + { Some(cmp::max_by( watermark, global_max_watermark, diff --git a/src/stream/src/executor/wrapper/epoch_check.rs b/src/stream/src/executor/wrapper/epoch_check.rs index 3b2975d08366b..fb3a24b3dda2b 100644 --- a/src/stream/src/executor/wrapper/epoch_check.rs +++ b/src/stream/src/executor/wrapper/epoch_check.rs @@ -47,7 +47,9 @@ pub async fn epoch_check(info: Arc, input: impl MessageStream) { ); } - if let Some(last_epoch) = last_epoch && !b.is_with_stop_mutation() { + if let Some(last_epoch) = last_epoch + && !b.is_with_stop_mutation() + { assert_eq!( b.epoch.prev, last_epoch, diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index b87ce5ff39dc7..b105837d9b093 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -160,8 +160,11 @@ impl ExecutorBuilder for SourceExecutorBuilder { ); let table_type = ExternalTableType::from_properties(&source.properties); - if table_type.can_backfill() && let Some(table_desc) = source_info.upstream_table.clone() { - let upstream_table_name = SchemaTableName::from_properties(&source.properties); + if table_type.can_backfill() + && let Some(table_desc) = source_info.upstream_table.clone() + { + let upstream_table_name = + SchemaTableName::from_properties(&source.properties); let table_pk_indices = table_desc .pk .iter() @@ -173,7 +176,8 @@ impl ExecutorBuilder for SourceExecutorBuilder { .map(|desc| OrderType::from_protobuf(desc.get_order_type().unwrap())) .collect_vec(); - let table_reader = table_type.create_table_reader(source.properties.clone(), schema.clone())?; + let table_reader = table_type + .create_table_reader(source.properties.clone(), schema.clone())?; let external_table = ExternalStorageTable::new( TableId::new(source.source_id), upstream_table_name, @@ -188,18 +192,19 @@ impl ExecutorBuilder for SourceExecutorBuilder { let source_state_handler = SourceStateTableHandler::from_table_catalog( source.state_table.as_ref().unwrap(), store.clone(), - ).await; + ) + .await; let cdc_backfill = CdcBackfillExecutor::new( params.actor_context.clone(), external_table, Box::new(source_exec), - (0..source.columns.len()).collect_vec(), // eliminate the last column (_rw_offset) + (0..source.columns.len()).collect_vec(), /* eliminate the last column (_rw_offset) */ None, schema.clone(), params.pk_indices, params.executor_stats, source_state_handler, - source_ctrl_opts.chunk_size + source_ctrl_opts.chunk_size, ); cdc_backfill.boxed() } else { diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index f54eb9921f77c..4386413029ae1 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -821,7 +821,9 @@ impl LocalStreamManagerCore { let mut actor_infos = self.context.actor_infos.write(); for actor in new_actor_infos { let ret = actor_infos.insert(actor.get_actor_id(), actor.clone()); - if let Some(prev_actor) = ret && actor != &prev_actor { + if let Some(prev_actor) = ret + && actor != &prev_actor + { bail!( "actor info mismatch when broadcasting {}", actor.get_actor_id() diff --git a/src/tests/regress/src/schedule.rs b/src/tests/regress/src/schedule.rs index 357cab473d135..c33cf5e91ed5a 100644 --- a/src/tests/regress/src/schedule.rs +++ b/src/tests/regress/src/schedule.rs @@ -329,12 +329,16 @@ impl TestCase { // Find the matching output line, and collect lines before the next matching line. let mut expected_output = vec![]; - while let Some(line) = expected_lines.next() && line != original_input_line { + while let Some(line) = expected_lines.next() + && line != original_input_line + { expected_output.push(line); } let mut actual_output = vec![]; - while let Some(line) = actual_lines.next() && line != input_line { + while let Some(line) = actual_lines.next() + && line != input_line + { actual_output.push(line); } @@ -371,7 +375,9 @@ fn compare_output(query: &[&str], expected: &[String], actual: &[String]) -> boo eq }; - if let Some(l) = query.last() && l.starts_with(PREFIX_IGNORE) { + if let Some(l) = query.last() + && l.starts_with(PREFIX_IGNORE) + { return true; } if !expected.is_empty() diff --git a/src/tests/sqlsmith/src/runner.rs b/src/tests/sqlsmith/src/runner.rs index 5efc793cdd95c..cebd50f7e008e 100644 --- a/src/tests/sqlsmith/src/runner.rs +++ b/src/tests/sqlsmith/src/runner.rs @@ -473,7 +473,9 @@ fn validate_response( Ok(rows) => Ok((0, rows)), Err(e) => { // Permit runtime errors conservatively. - if let Some(e) = e.as_db_error() && is_permissible_error(&e.to_string()) { + if let Some(e) = e.as_db_error() + && is_permissible_error(&e.to_string()) + { tracing::info!("[SKIPPED ERROR]: {:#?}", e); return Ok((1, vec![])); } @@ -509,16 +511,20 @@ async fn run_query_inner( ), }; if let Err(e) = &response - && let Some(e) = e.as_db_error() { + && let Some(e) = e.as_db_error() + { if is_recovery_in_progress_error(&e.to_string()) { let tries = 5; let interval = 1; - for _ in 0..tries { // retry 5 times + for _ in 0..tries { + // retry 5 times sleep(Duration::from_secs(interval)).await; let query_task = client.simple_query(query); let response = timeout(Duration::from_secs(timeout_duration), query_task).await; match response { - Ok(Ok(r)) => { return Ok((0, r)); } + Ok(Ok(r)) => { + return Ok((0, r)); + } Err(_) => bail!( "[UNEXPECTED ERROR] Query timeout after {timeout_duration}s:\n{:?}", query diff --git a/src/tests/sqlsmith/tests/frontend/mod.rs b/src/tests/sqlsmith/tests/frontend/mod.rs index a0ab1d59cf58e..8f681ab38a956 100644 --- a/src/tests/sqlsmith/tests/frontend/mod.rs +++ b/src/tests/sqlsmith/tests/frontend/mod.rs @@ -149,7 +149,9 @@ async fn test_stream_query( setup_sql: &str, ) -> Result<()> { let mut rng; - if let Ok(x) = env::var("RW_RANDOM_SEED_SQLSMITH") && x == "true" { + if let Ok(x) = env::var("RW_RANDOM_SEED_SQLSMITH") + && x == "true" + { rng = SmallRng::from_entropy(); } else { rng = SmallRng::seed_from_u64(seed); @@ -205,7 +207,9 @@ fn test_batch_query( setup_sql: &str, ) -> Result<()> { let mut rng; - if let Ok(x) = env::var("RW_RANDOM_SEED_SQLSMITH") && x == "true" { + if let Ok(x) = env::var("RW_RANDOM_SEED_SQLSMITH") + && x == "true" + { rng = SmallRng::from_entropy(); } else { rng = SmallRng::seed_from_u64(seed); @@ -248,7 +252,9 @@ async fn setup_sqlsmith_with_seed_inner(seed: u64) -> Result { let session = frontend.session_ref(); let mut rng; - if let Ok(x) = env::var("RW_RANDOM_SEED_SQLSMITH") && x == "true" { + if let Ok(x) = env::var("RW_RANDOM_SEED_SQLSMITH") + && x == "true" + { rng = SmallRng::from_entropy(); } else { rng = SmallRng::seed_from_u64(seed); @@ -266,7 +272,9 @@ async fn setup_sqlsmith_with_seed_inner(seed: u64) -> Result { /// Otherwise no error: skip status: false. fn validate_result(result: Result) -> Result { if let Err(e) = result { - if let Some(s) = e.message() && is_permissible_error(s) { + if let Some(s) = e.message() + && is_permissible_error(s) + { return Ok(true); } else { return Err(e); diff --git a/src/utils/runtime/src/logger.rs b/src/utils/runtime/src/logger.rs index 916dd93d7a32b..c4fe90c294081 100644 --- a/src/utils/runtime/src/logger.rs +++ b/src/utils/runtime/src/logger.rs @@ -188,7 +188,9 @@ pub fn init_risingwave_logger(settings: LoggerSettings) { } // Overrides from env var. - if let Ok(rust_log) = std::env::var(EnvFilter::DEFAULT_ENV) && !rust_log.is_empty() { + if let Ok(rust_log) = std::env::var(EnvFilter::DEFAULT_ENV) + && !rust_log.is_empty() + { let rust_log_targets: Targets = rust_log.parse().expect("failed to parse `RUST_LOG`"); if let Some(default_level) = rust_log_targets.default_level() { filter = filter.with_default(default_level); diff --git a/src/utils/runtime/src/panic_hook.rs b/src/utils/runtime/src/panic_hook.rs index 848e7df8509c7..0a43fac48f191 100644 --- a/src/utils/runtime/src/panic_hook.rs +++ b/src/utils/runtime/src/panic_hook.rs @@ -15,7 +15,9 @@ /// Set panic hook to abort the process if we're not catching unwind, without losing the information /// of stack trace and await-tree. pub fn set_panic_hook() { - if let Ok(limit) = rlimit::Resource::CORE.get_soft() && limit > 0 { + if let Ok(limit) = rlimit::Resource::CORE.get_soft() + && limit > 0 + { tracing::info!(limit, "coredump on panic is likely to be enabled"); }; From 9616cbf88e7f488849004b7e3f14eb7880e87287 Mon Sep 17 00:00:00 2001 From: Dylan Date: Tue, 31 Oct 2023 13:53:43 +0800 Subject: [PATCH 13/14] build(release): release risingwave all-in-one (#13133) Co-authored-by: Jianwei Huang <1223644280@qq.com> --- ci/scripts/release.sh | 13 ++++++++++--- src/jni_core/src/jvm_runtime.rs | 9 +++++---- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/ci/scripts/release.sh b/ci/scripts/release.sh index 08e5794f173cd..eb05e4aa63871 100755 --- a/ci/scripts/release.sh +++ b/ci/scripts/release.sh @@ -66,9 +66,15 @@ fi echo "--- Build connector node" cd ${REPO_ROOT}/java && mvn -B package -Dmaven.test.skip=true -Dno-build-rust -cd ${REPO_ROOT} && mv ${REPO_ROOT}/java/connector-node/assembly/target/risingwave-connector-1.0.0.tar.gz risingwave-connector-"${BUILDKITE_TAG}".tar.gz if [[ -n "${BUILDKITE_TAG}" ]]; then + echo "--- Collect all release assets" + cd ${REPO_ROOT} && mkdir release-assets && cd release-assets + cp -r ${REPO_ROOT}/target/release/* . + mv ${REPO_ROOT}/java/connector-node/assembly/target/risingwave-connector-1.0.0.tar.gz risingwave-connector-"${BUILDKITE_TAG}".tar.gz + tar -zxvf risingwave-connector-"${BUILDKITE_TAG}".tar.gz libs + ls -l + echo "--- Install gh cli" yum install -y dnf dnf install -y 'dnf-command(config-manager)' @@ -90,8 +96,9 @@ if [[ -n "${BUILDKITE_TAG}" ]]; then tar -czvf risectl-"${BUILDKITE_TAG}"-x86_64-unknown-linux.tar.gz risectl gh release upload "${BUILDKITE_TAG}" risectl-"${BUILDKITE_TAG}"-x86_64-unknown-linux.tar.gz - echo "--- Release build and upload risingwave connector node jar asset" - gh release upload "${BUILDKITE_TAG}" risingwave-connector-"${BUILDKITE_TAG}".tar.gz + echo "--- Release upload risingwave-all-in-one asset" + tar -czvf risingwave-"${BUILDKITE_TAG}"-x86_64-unknown-linux-all-in-one.tar.gz risingwave libs + gh release upload "${BUILDKITE_TAG}" risingwave-"${BUILDKITE_TAG}"-x86_64-unknown-linux-all-in-one.tar.gz fi diff --git a/src/jni_core/src/jvm_runtime.rs b/src/jni_core/src/jvm_runtime.rs index bd1f068b6eaee..9ba8a2a533683 100644 --- a/src/jni_core/src/jvm_runtime.rs +++ b/src/jni_core/src/jvm_runtime.rs @@ -48,12 +48,13 @@ impl JavaVmWrapper { let libs_path = if let Ok(libs_path) = std::env::var("CONNECTOR_LIBS_PATH") { libs_path } else { - return Err(ErrorCode::InternalError( - "environment variable CONNECTOR_LIBS_PATH is not specified".to_string(), - ) - .into()); + tracing::warn!("environment variable CONNECTOR_LIBS_PATH is not specified, so use default path `./libs` instead"); + let path = std::env::current_exe()?.parent().unwrap().join("./libs"); + path.to_str().unwrap().into() }; + tracing::info!("libs_path = {}", libs_path); + let dir = Path::new(&libs_path); if !dir.is_dir() { From f18e73e369ca2c863eebb10cb3f0b37833a5cd93 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Tue, 31 Oct 2023 14:07:08 +0800 Subject: [PATCH 14/14] perf(over window): incremental aggregation (new) (#13038) Signed-off-by: Richard Chien --- Cargo.lock | 1 + src/batch/src/executor/sort_over_window.rs | 5 +- src/expr/core/Cargo.toml | 1 + src/expr/core/src/window_function/call.rs | 3 +- .../src/window_function/state/aggregate.rs | 153 ++++++--- .../core/src/window_function/state/buffer.rs | 297 +++++++++++++----- .../core/src/window_function/state/mod.rs | 8 +- .../src/window_function/state/row_number.rs | 74 +++-- src/expr/core/src/window_function/states.rs | 45 ++- src/stream/src/executor/over_window/eowc.rs | 8 +- .../src/executor/over_window/general.rs | 5 +- .../integration_tests/eowc_over_window.rs | 29 +- 12 files changed, 447 insertions(+), 182 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 477cbf02bc3c0..1da55d7e68e4a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7487,6 +7487,7 @@ dependencies = [ "downcast-rs", "easy-ext", "either", + "enum-as-inner", "expect-test", "futures-async-stream", "futures-util", diff --git a/src/batch/src/executor/sort_over_window.rs b/src/batch/src/executor/sort_over_window.rs index c8b6c7ef9388c..21bfc8aa6b177 100644 --- a/src/batch/src/executor/sort_over_window.rs +++ b/src/batch/src/executor/sort_over_window.rs @@ -191,12 +191,11 @@ impl SortOverWindowExecutor { } } for row in rows.drain(..) { - if let Some(chunk) = - chunk_builder.append_one_row(row.chain(OwnedRow::new(states.curr_output()?))) + if let Some(chunk) = chunk_builder + .append_one_row(row.chain(OwnedRow::new(states.slide_no_evict_hint()?))) { yield chunk; } - states.just_slide_forward(); } } } diff --git a/src/expr/core/Cargo.toml b/src/expr/core/Cargo.toml index ada9a3639525c..ab8dd697e220d 100644 --- a/src/expr/core/Cargo.toml +++ b/src/expr/core/Cargo.toml @@ -31,6 +31,7 @@ ctor = "0.2" downcast-rs = "1.2" easy-ext = "1" either = "1" +enum-as-inner = "0.6" futures-async-stream = { workspace = true } futures-util = "0.3" itertools = "0.11" diff --git a/src/expr/core/src/window_function/call.rs b/src/expr/core/src/window_function/call.rs index a74beb672fd4f..ac7c1a2b78dd7 100644 --- a/src/expr/core/src/window_function/call.rs +++ b/src/expr/core/src/window_function/call.rs @@ -15,6 +15,7 @@ use std::cmp::Ordering; use std::fmt::Display; +use enum_as_inner::EnumAsInner; use risingwave_common::bail; use risingwave_common::types::DataType; use risingwave_pb::expr::window_frame::{PbBound, PbExclusion}; @@ -267,7 +268,7 @@ impl FrameBound { } } -#[derive(Debug, Copy, Clone, Eq, PartialEq, Hash, Default)] +#[derive(Debug, Copy, Clone, Eq, PartialEq, Hash, Default, EnumAsInner)] pub enum FrameExclusion { CurrentRow, // Group, diff --git a/src/expr/core/src/window_function/state/aggregate.rs b/src/expr/core/src/window_function/state/aggregate.rs index 7deee85693ef2..38958b50b8c38 100644 --- a/src/expr/core/src/window_function/state/aggregate.rs +++ b/src/expr/core/src/window_function/state/aggregate.rs @@ -15,7 +15,7 @@ use std::collections::BTreeSet; use futures_util::FutureExt; -use risingwave_common::array::{DataChunk, StreamChunk}; +use risingwave_common::array::{DataChunk, Op, StreamChunk}; use risingwave_common::estimate_size::{EstimateSize, KvSize}; use risingwave_common::types::{DataType, Datum}; use risingwave_common::util::iter_util::ZipEqFast; @@ -24,12 +24,16 @@ use smallvec::SmallVec; use super::buffer::WindowBuffer; use super::{StateEvictHint, StateKey, StatePos, WindowState}; -use crate::aggregate::{build_append_only, AggArgs, AggCall, BoxedAggregateFunction}; +use crate::aggregate::{ + AggArgs, AggCall, AggregateFunction, AggregateState as AggImplState, BoxedAggregateFunction, +}; +use crate::sig::FUNCTION_REGISTRY; use crate::window_function::{WindowFuncCall, WindowFuncKind}; use crate::Result; pub struct AggregateState { - agg_call: AggCall, + agg_func: BoxedAggregateFunction, + agg_impl: AggImpl, arg_data_types: Vec, buffer: WindowBuffer>, buffer_heap_size: KvSize, @@ -58,13 +62,54 @@ impl AggregateState { distinct: false, direct_args: vec![], }; + let agg_func_sig = FUNCTION_REGISTRY + .get_aggregate( + agg_kind, + &arg_data_types, + &call.return_type, + false, // means prefer retractable version + ) + .expect("the agg func must exist"); + let agg_func = agg_func_sig.build_aggregate(&agg_call)?; + let (agg_impl, enable_delta) = + if !agg_func_sig.append_only && call.frame.exclusion.is_no_others() { + let init_state = agg_func.create_state(); + (AggImpl::Incremental(init_state), true) + } else { + (AggImpl::Full, false) + }; Ok(Self { - agg_call, + agg_func, + agg_impl, arg_data_types, - buffer: WindowBuffer::new(call.frame.clone()), + buffer: WindowBuffer::new(call.frame.clone(), enable_delta), buffer_heap_size: KvSize::new(), }) } + + fn slide_inner(&mut self) -> StateEvictHint { + let removed_keys: BTreeSet<_> = self + .buffer + .slide() + .map(|(k, v)| { + v.iter().for_each(|arg| { + self.buffer_heap_size.sub_val(arg); + }); + self.buffer_heap_size.sub_val(&k); + k + }) + .collect(); + if removed_keys.is_empty() { + StateEvictHint::CannotEvict( + self.buffer + .smallest_key() + .expect("sliding without removing, must have some entry in the buffer") + .clone(), + ) + } else { + StateEvictHint::CanEvict(removed_keys) + } + } } impl WindowState for AggregateState { @@ -84,36 +129,35 @@ impl WindowState for AggregateState { } } - fn curr_output(&self) -> Result { + fn slide(&mut self) -> Result<(Datum, StateEvictHint)> { let wrapper = AggregatorWrapper { - agg: build_append_only(&self.agg_call)?, + agg_func: self.agg_func.as_ref(), arg_data_types: &self.arg_data_types, }; - wrapper.aggregate(self.buffer.curr_window_values().map(SmallVec::as_slice)) + let output = match self.agg_impl { + AggImpl::Full => wrapper.aggregate(self.buffer.curr_window_values()), + AggImpl::Incremental(ref mut state) => { + wrapper.update(state, self.buffer.consume_curr_window_values_delta()) + } + }?; + let evict_hint = self.slide_inner(); + Ok((output, evict_hint)) } - fn slide_forward(&mut self) -> StateEvictHint { - let removed_keys: BTreeSet<_> = self - .buffer - .slide() - .map(|(k, v)| { - v.iter().for_each(|arg| { - self.buffer_heap_size.sub_val(arg); - }); - self.buffer_heap_size.sub_val(&k); - k - }) - .collect(); - if removed_keys.is_empty() { - StateEvictHint::CannotEvict( - self.buffer - .smallest_key() - .expect("sliding without removing, must have some entry in the buffer") - .clone(), - ) - } else { - StateEvictHint::CanEvict(removed_keys) - } + fn slide_no_output(&mut self) -> Result { + match self.agg_impl { + AggImpl::Full => {} + AggImpl::Incremental(ref mut state) => { + // for incremental agg, we need to update the state even if the caller doesn't need + // the output + let wrapper = AggregatorWrapper { + agg_func: self.agg_func.as_ref(), + arg_data_types: &self.arg_data_types, + }; + wrapper.update(state, self.buffer.consume_curr_window_values_delta())?; + } + }; + Ok(self.slide_inner()) } } @@ -125,41 +169,62 @@ impl EstimateSize for AggregateState { } } +enum AggImpl { + Incremental(AggImplState), + Full, +} + struct AggregatorWrapper<'a> { - agg: BoxedAggregateFunction, + agg_func: &'a dyn AggregateFunction, arg_data_types: &'a [DataType], } impl AggregatorWrapper<'_> { - fn aggregate<'a>(&'a self, values: impl Iterator) -> Result { - // TODO(rc): switch to a better general version of aggregator implementation + fn aggregate(&self, values: impl IntoIterator) -> Result + where + V: AsRef<[Datum]>, + { + let mut state = self.agg_func.create_state(); + self.update( + &mut state, + values.into_iter().map(|args| (Op::Insert, args)), + ) + } + fn update( + &self, + state: &mut AggImplState, + delta: impl IntoIterator, + ) -> Result + where + V: AsRef<[Datum]>, + { let mut args_builders = self .arg_data_types .iter() .map(|data_type| data_type.create_array_builder(0 /* bad! */)) .collect::>(); - let mut n_values = 0; - for value in values { - n_values += 1; - for (builder, datum) in args_builders.iter_mut().zip_eq_fast(value.iter()) { + let mut ops = Vec::new(); + let mut n_rows = 0; + for (op, value) in delta { + n_rows += 1; + ops.push(op); + for (builder, datum) in args_builders.iter_mut().zip_eq_fast(value.as_ref()) { builder.append(datum); } } - let columns = args_builders .into_iter() .map(|builder| builder.finish().into()) .collect::>(); - let chunk = StreamChunk::from(DataChunk::new(columns, n_values)); + let chunk = StreamChunk::from_parts(ops, DataChunk::new(columns, n_rows)); - let mut state = self.agg.create_state(); - self.agg - .update(&mut state, &chunk) + self.agg_func + .update(state, &chunk) .now_or_never() .expect("we don't support UDAF currently, so the function should return immediately")?; - self.agg - .get_result(&state) + self.agg_func + .get_result(state) .now_or_never() .expect("we don't support UDAF currently, so the function should return immediately") } diff --git a/src/expr/core/src/window_function/state/buffer.rs b/src/expr/core/src/window_function/state/buffer.rs index a375c7bfec225..fa684b9049459 100644 --- a/src/expr/core/src/window_function/state/buffer.rs +++ b/src/expr/core/src/window_function/state/buffer.rs @@ -15,7 +15,8 @@ use std::collections::VecDeque; use std::ops::Range; -use either::Either; +use risingwave_common::array::Op; +use smallvec::{smallvec, SmallVec}; use crate::window_function::{Frame, FrameBounds, FrameExclusion}; @@ -26,12 +27,13 @@ struct Entry { // TODO(rc): May be a good idea to extract this into a separate crate. /// A common sliding window buffer. -pub struct WindowBuffer { +pub struct WindowBuffer { frame: Frame, buffer: VecDeque>, curr_idx: usize, left_idx: usize, // inclusive, note this can be > `curr_idx` right_excl_idx: usize, // exclusive, note this can be <= `curr_idx` + curr_delta: Option>, } /// Note: A window frame can be pure preceding, pure following, or acrossing the _current row_. @@ -41,15 +43,24 @@ pub struct CurrWindow<'a, K> { pub following_saturated: bool, } -impl WindowBuffer { - pub fn new(frame: Frame) -> Self { +impl WindowBuffer { + pub fn new(frame: Frame, enable_delta: bool) -> Self { assert!(frame.bounds.is_valid()); + if enable_delta { + // TODO(rc): currently only support `FrameExclusion::NoOthers` for delta + assert!(frame.exclusion.is_no_others()); + } Self { frame, buffer: Default::default(), curr_idx: 0, left_idx: 0, right_excl_idx: 0, + curr_delta: if enable_delta { + Some(Default::default()) + } else { + None + }, } } @@ -64,7 +75,10 @@ impl WindowBuffer { } else { // FIXME(rc): Clippy rule `clippy::nonminimal_bool` is misreporting that // the following can be simplified. - // assert!(self.curr_idx >= self.left_idx); + #[allow(clippy::nonminimal_bool)] + { + assert!(self.curr_idx >= self.left_idx); + } self.curr_idx - self.left_idx >= start_off.unsigned_abs() } } else { @@ -84,9 +98,12 @@ impl WindowBuffer { true // pure preceding frame, always following-saturated } else { // FIXME(rc): Ditto. - // assert!(self.right_excl_idx > 0); - // assert!(self.right_excl_idx > self.curr_idx); - // assert!(self.right_excl_idx <= self.buffer.len()); + #[allow(clippy::nonminimal_bool)] + { + assert!(self.right_excl_idx > 0); + assert!(self.right_excl_idx > self.curr_idx); + assert!(self.right_excl_idx <= self.buffer.len()); + } self.right_excl_idx - 1 - self.curr_idx >= end_off as usize } } else { @@ -110,28 +127,43 @@ impl WindowBuffer { } } + fn curr_window_outer(&self) -> Range { + self.left_idx..self.right_excl_idx + } + + fn curr_window_exclusion(&self) -> Range { + // TODO(rc): should intersect with `curr_window_outer` to be more accurate + match self.frame.exclusion { + FrameExclusion::CurrentRow => self.curr_idx..self.curr_idx + 1, + FrameExclusion::NoOthers => self.curr_idx..self.curr_idx, + } + } + + fn curr_window_ranges(&self) -> (Range, Range) { + let selection = self.curr_window_outer(); + let exclusion = self.curr_window_exclusion(); + range_except(selection, exclusion) + } + /// Iterate over values in the current window. pub fn curr_window_values(&self) -> impl Iterator { assert!(self.left_idx <= self.right_excl_idx); assert!(self.right_excl_idx <= self.buffer.len()); - let selection = self.left_idx..self.right_excl_idx; - if selection.is_empty() { - return Either::Left(std::iter::empty()); - } + let (left, right) = self.curr_window_ranges(); + self.buffer + .range(left) + .chain(self.buffer.range(right)) + .map(|Entry { value, .. }| value) + } - let exclusion = match self.frame.exclusion { - FrameExclusion::CurrentRow => self.curr_idx..self.curr_idx + 1, - FrameExclusion::NoOthers => self.curr_idx..self.curr_idx, - }; - let (left, right) = range_except(selection, exclusion); - - Either::Right( - self.buffer - .range(left) - .chain(self.buffer.range(right)) - .map(|Entry { value, .. }| value), - ) + /// Consume the delta of values comparing the current window to the previous window. + /// The delta is not guaranteed to be sorted, especially when frame exclusion is not `NoOthers`. + pub fn consume_curr_window_values_delta(&mut self) -> impl Iterator + '_ { + self.curr_delta + .as_mut() + .expect("delta mode should be enabled") + .drain(..) } fn recalculate_left_right(&mut self) { @@ -175,10 +207,29 @@ impl WindowBuffer { } } + fn maintain_delta(&mut self, old_outer: Range, new_outer: Range) { + debug_assert!(self.frame.exclusion.is_no_others()); + + let (outer_removed, outer_added) = range_diff(old_outer.clone(), new_outer.clone()); + let delta = self.curr_delta.as_mut().unwrap(); + for idx in outer_removed.iter().cloned().flatten() { + delta.push((Op::Delete, self.buffer[idx].value.clone())); + } + for idx in outer_added.iter().cloned().flatten() { + delta.push((Op::Insert, self.buffer[idx].value.clone())); + } + } + /// Append a key-value pair to the buffer. pub fn append(&mut self, key: K, value: V) { + let old_outer = self.curr_window_outer(); + self.buffer.push_back(Entry { key, value }); - self.recalculate_left_right() + self.recalculate_left_right(); + + if self.curr_delta.is_some() { + self.maintain_delta(old_outer, self.curr_window_outer()); + } } /// Get the smallest key that is still kept in the buffer. @@ -190,8 +241,15 @@ impl WindowBuffer { /// Slide the current window forward. /// Returns the keys that are removed from the buffer. pub fn slide(&mut self) -> impl Iterator + '_ { + let old_outer = self.curr_window_outer(); + self.curr_idx += 1; self.recalculate_left_right(); + + if self.curr_delta.is_some() { + self.maintain_delta(old_outer, self.curr_window_outer()); + } + let min_needed_idx = std::cmp::min(self.left_idx, self.curr_idx); self.curr_idx -= min_needed_idx; self.left_idx -= min_needed_idx; @@ -205,7 +263,12 @@ impl WindowBuffer { /// Calculate range (A - B), the result might be the union of two ranges when B is totally included /// in the A. fn range_except(a: Range, b: Range) -> (Range, Range) { - if a.end <= b.start || b.end <= a.start { + #[allow(clippy::if_same_then_else)] // for better readability + if a.is_empty() { + (0..0, 0..0) + } else if b.is_empty() { + (a, 0..0) + } else if a.end <= b.start || b.end <= a.start { // a: [ ) // b: [ ) // or @@ -233,31 +296,129 @@ fn range_except(a: Range, b: Range) -> (Range, Range } } +/// Calculate the difference of two ranges A and B, return (removed ranges, added ranges). +/// Note this is quite different from [`range_except`]. +#[allow(clippy::type_complexity)] // looks complex but it's not +fn range_diff( + a: Range, + b: Range, +) -> (SmallVec<[Range; 2]>, SmallVec<[Range; 2]>) { + if a.start == b.start { + match a.end.cmp(&b.end) { + std::cmp::Ordering::Equal => { + // a: [ ) + // b: [ ) + (smallvec![], smallvec![]) + } + std::cmp::Ordering::Less => { + // a: [ ) + // b: [ ) + (smallvec![], smallvec![a.end..b.end]) + } + std::cmp::Ordering::Greater => { + // a: [ ) + // b: [ ) + (smallvec![b.end..a.end], smallvec![]) + } + } + } else if a.end == b.end { + debug_assert!(a.start != b.start); + if a.start < b.start { + // a: [ ) + // b: [ ) + (smallvec![a.start..b.start], smallvec![]) + } else { + // a: [ ) + // b: [ ) + (smallvec![], smallvec![b.start..a.start]) + } + } else { + debug_assert!(a.start != b.start && a.end != b.end); + if a.end <= b.start || b.end <= a.start { + // a: [ ) + // b: [ [ ) + // or + // a: [ ) + // b: [ ) ) + (smallvec![a], smallvec![b]) + } else if b.start < a.start && a.end < b.end { + // a: [ ) + // b: [ ) + (smallvec![], smallvec![b.start..a.start, a.end..b.end]) + } else if a.start < b.start && b.end < a.end { + // a: [ ) + // b: [ ) + (smallvec![a.start..b.start, b.end..a.end], smallvec![]) + } else if a.end < b.end { + // a: [ ) + // b: [ ) + (smallvec![a.start..b.start], smallvec![a.end..b.end]) + } else { + // a: [ ) + // b: [ ) + (smallvec![b.end..a.end], smallvec![b.start..a.start]) + } + } +} + #[cfg(test)] mod tests { + use std::collections::HashSet; + use itertools::Itertools; use super::*; use crate::window_function::{Frame, FrameBound}; + #[test] + fn test_range_diff() { + fn test( + a: Range, + b: Range, + expected_removed: impl IntoIterator, + expected_added: impl IntoIterator, + ) { + let (removed, added) = range_diff(a, b); + let removed_set = removed.into_iter().flatten().collect::>(); + let added_set = added.into_iter().flatten().collect::>(); + let expected_removed_set = expected_removed.into_iter().collect::>(); + let expected_added_set = expected_added.into_iter().collect::>(); + assert_eq!(removed_set, expected_removed_set); + assert_eq!(added_set, expected_added_set); + } + + test(0..0, 0..0, [], []); + test(0..1, 0..1, [], []); + test(0..1, 0..2, [], [1]); + test(0..2, 0..1, [1], []); + test(0..2, 1..2, [0], []); + test(1..2, 0..2, [], [0]); + test(0..1, 1..2, [0], [1]); + test(0..1, 2..3, [0], [2]); + test(1..2, 0..1, [1], [0]); + test(2..3, 0..1, [2], [0]); + test(0..3, 1..2, [0, 2], []); + test(1..2, 0..3, [], [0, 2]); + test(0..3, 2..4, [0, 1], [3]); + test(2..4, 0..3, [3], [0, 1]); + } + #[test] fn test_rows_frame_unbounded_preceding_to_current_row() { - let mut buffer = WindowBuffer::new(Frame::rows( - FrameBound::UnboundedPreceding, - FrameBound::CurrentRow, - )); + let mut buffer = WindowBuffer::new( + Frame::rows(FrameBound::UnboundedPreceding, FrameBound::CurrentRow), + true, + ); let window = buffer.curr_window(); assert!(window.key.is_none()); assert!(!window.preceding_saturated); assert!(!window.following_saturated); - buffer.append(1, "hello"); let window = buffer.curr_window(); assert_eq!(window.key, Some(&1)); assert!(!window.preceding_saturated); // unbounded preceding is never saturated assert!(window.following_saturated); - buffer.append(2, "world"); let window = buffer.curr_window(); assert_eq!(window.key, Some(&1)); @@ -267,7 +428,6 @@ mod tests { buffer.curr_window_values().cloned().collect_vec(), vec!["hello"] ); - let removed_keys = buffer.slide().map(|(k, _)| k).collect_vec(); assert!(removed_keys.is_empty()); // unbouded preceding, nothing can ever be removed let window = buffer.curr_window(); @@ -279,16 +439,15 @@ mod tests { #[test] fn test_rows_frame_preceding_to_current_row() { - let mut buffer = WindowBuffer::new(Frame::rows( - FrameBound::Preceding(1), - FrameBound::CurrentRow, - )); + let mut buffer = WindowBuffer::new( + Frame::rows(FrameBound::Preceding(1), FrameBound::CurrentRow), + true, + ); let window = buffer.curr_window(); assert!(window.key.is_none()); assert!(!window.preceding_saturated); assert!(!window.following_saturated); - buffer.append(1, "hello"); let window = buffer.curr_window(); assert_eq!(window.key, Some(&1)); @@ -298,13 +457,11 @@ mod tests { buffer.curr_window_values().cloned().collect_vec(), vec!["hello"] ); - buffer.append(2, "world"); let window = buffer.curr_window(); assert_eq!(window.key, Some(&1)); assert!(!window.preceding_saturated); assert!(window.following_saturated); - let removed_keys = buffer.slide().map(|(k, _)| k).collect_vec(); assert!(removed_keys.is_empty()); let window = buffer.curr_window(); @@ -312,7 +469,6 @@ mod tests { assert!(window.preceding_saturated); assert!(window.following_saturated); assert_eq!(buffer.smallest_key(), Some(&1)); - buffer.append(3, "!"); let window = buffer.curr_window(); assert_eq!(window.key, Some(&2)); @@ -322,10 +478,10 @@ mod tests { #[test] fn test_rows_frame_preceding_to_preceding() { - let mut buffer = WindowBuffer::new(Frame::rows( - FrameBound::Preceding(2), - FrameBound::Preceding(1), - )); + let mut buffer = WindowBuffer::new( + Frame::rows(FrameBound::Preceding(2), FrameBound::Preceding(1)), + true, + ); buffer.append(1, "RisingWave"); let window = buffer.curr_window(); @@ -333,11 +489,9 @@ mod tests { assert!(!window.preceding_saturated); assert!(window.following_saturated); assert!(buffer.curr_window_values().collect_vec().is_empty()); - let removed_keys = buffer.slide().map(|(k, _)| k).collect_vec(); assert!(removed_keys.is_empty()); assert_eq!(buffer.smallest_key(), Some(&1)); - buffer.append(2, "is the best"); let window = buffer.curr_window(); assert_eq!(window.key, Some(&2)); @@ -347,11 +501,9 @@ mod tests { buffer.curr_window_values().cloned().collect_vec(), vec!["RisingWave"] ); - let removed_keys = buffer.slide().map(|(k, _)| k).collect_vec(); assert!(removed_keys.is_empty()); assert_eq!(buffer.smallest_key(), Some(&1)); - buffer.append(3, "streaming platform"); let window = buffer.curr_window(); assert_eq!(window.key, Some(&3)); @@ -361,7 +513,6 @@ mod tests { buffer.curr_window_values().cloned().collect_vec(), vec!["RisingWave", "is the best"] ); - let removed_keys = buffer.slide().map(|(k, _)| k).collect_vec(); assert_eq!(removed_keys, vec![1]); assert_eq!(buffer.smallest_key(), Some(&2)); @@ -369,10 +520,10 @@ mod tests { #[test] fn test_rows_frame_current_row_to_unbounded_following() { - let mut buffer = WindowBuffer::new(Frame::rows( - FrameBound::CurrentRow, - FrameBound::UnboundedFollowing, - )); + let mut buffer = WindowBuffer::new( + Frame::rows(FrameBound::CurrentRow, FrameBound::UnboundedFollowing), + true, + ); buffer.append(1, "RisingWave"); let window = buffer.curr_window(); @@ -383,7 +534,6 @@ mod tests { buffer.curr_window_values().cloned().collect_vec(), vec!["RisingWave"] ); - buffer.append(2, "is the best"); let window = buffer.curr_window(); assert_eq!(window.key, Some(&1)); @@ -393,7 +543,6 @@ mod tests { buffer.curr_window_values().cloned().collect_vec(), vec!["RisingWave", "is the best"] ); - let removed_keys = buffer.slide().map(|(k, _)| k).collect_vec(); assert_eq!(removed_keys, vec![1]); assert_eq!(buffer.smallest_key(), Some(&2)); @@ -409,10 +558,10 @@ mod tests { #[test] fn test_rows_frame_current_row_to_following() { - let mut buffer = WindowBuffer::new(Frame::rows( - FrameBound::CurrentRow, - FrameBound::Following(1), - )); + let mut buffer = WindowBuffer::new( + Frame::rows(FrameBound::CurrentRow, FrameBound::Following(1)), + true, + ); buffer.append(1, "RisingWave"); let window = buffer.curr_window(); @@ -423,7 +572,6 @@ mod tests { buffer.curr_window_values().cloned().collect_vec(), vec!["RisingWave"] ); - buffer.append(2, "is the best"); let window = buffer.curr_window(); assert_eq!(window.key, Some(&1)); @@ -433,7 +581,6 @@ mod tests { buffer.curr_window_values().cloned().collect_vec(), vec!["RisingWave", "is the best"] ); - buffer.append(3, "streaming platform"); let window = buffer.curr_window(); assert_eq!(window.key, Some(&1)); @@ -443,7 +590,6 @@ mod tests { buffer.curr_window_values().cloned().collect_vec(), vec!["RisingWave", "is the best"] ); - let removed_keys = buffer.slide().map(|(k, _)| k).collect_vec(); assert_eq!(removed_keys, vec![1]); let window = buffer.curr_window(); @@ -458,10 +604,10 @@ mod tests { #[test] fn test_rows_frame_following_to_following() { - let mut buffer = WindowBuffer::new(Frame::rows( - FrameBound::Following(1), - FrameBound::Following(2), - )); + let mut buffer = WindowBuffer::new( + Frame::rows(FrameBound::Following(1), FrameBound::Following(2)), + true, + ); buffer.append(1, "RisingWave"); let window = buffer.curr_window(); @@ -469,7 +615,6 @@ mod tests { assert!(window.preceding_saturated); assert!(!window.following_saturated); assert!(buffer.curr_window_values().collect_vec().is_empty()); - buffer.append(2, "is the best"); let window = buffer.curr_window(); assert_eq!(window.key, Some(&1)); @@ -479,7 +624,6 @@ mod tests { buffer.curr_window_values().cloned().collect_vec(), vec!["is the best"] ); - buffer.append(3, "streaming platform"); let window = buffer.curr_window(); assert_eq!(window.key, Some(&1)); @@ -489,7 +633,6 @@ mod tests { buffer.curr_window_values().cloned().collect_vec(), vec!["is the best", "streaming platform"] ); - let removed_keys = buffer.slide().map(|(k, _)| k).collect_vec(); assert_eq!(removed_keys, vec![1]); let window = buffer.curr_window(); @@ -504,11 +647,14 @@ mod tests { #[test] fn test_rows_frame_exclude_current_row() { - let mut buffer = WindowBuffer::new(Frame::rows_with_exclusion( - FrameBound::UnboundedPreceding, - FrameBound::CurrentRow, - FrameExclusion::CurrentRow, - )); + let mut buffer = WindowBuffer::new( + Frame::rows_with_exclusion( + FrameBound::UnboundedPreceding, + FrameBound::CurrentRow, + FrameExclusion::CurrentRow, + ), + false, + ); buffer.append(1, "hello"); assert!(buffer @@ -516,7 +662,6 @@ mod tests { .cloned() .collect_vec() .is_empty()); - buffer.append(2, "world"); let _ = buffer.slide(); assert_eq!( diff --git a/src/expr/core/src/window_function/state/mod.rs b/src/expr/core/src/window_function/state/mod.rs index 971fb97f66cdc..927f5aaf6e0c0 100644 --- a/src/expr/core/src/window_function/state/mod.rs +++ b/src/expr/core/src/window_function/state/mod.rs @@ -101,11 +101,11 @@ pub trait WindowState: EstimateSize { /// Get the current window frame position. fn curr_window(&self) -> StatePos<'_>; - /// Get the window function result of current window frame. - fn curr_output(&self) -> Result; + /// Slide the window frame forward and collect the output and evict hint. Similar to `Iterator::next`. + fn slide(&mut self) -> Result<(Datum, StateEvictHint)>; - /// Slide the window frame forward. - fn slide_forward(&mut self) -> StateEvictHint; + /// Slide the window frame forward and collect the evict hint. Don't calculate the output if possible. + fn slide_no_output(&mut self) -> Result; } pub fn create_window_state(call: &WindowFuncCall) -> Result> { diff --git a/src/expr/core/src/window_function/state/row_number.rs b/src/expr/core/src/window_function/state/row_number.rs index fd485292c9382..6a2759d69308c 100644 --- a/src/expr/core/src/window_function/state/row_number.rs +++ b/src/expr/core/src/window_function/state/row_number.rs @@ -36,6 +36,19 @@ impl RowNumberState { curr_row_number: 1, } } + + fn slide_inner(&mut self) -> StateEvictHint { + self.curr_row_number += 1; + self.buffer + .pop_front() + .expect("should not slide forward when the current window is not ready"); + // can't evict any state key in EOWC mode, because we can't recover from previous output now + StateEvictHint::CannotEvict( + self.first_key + .clone() + .expect("should have appended some rows"), + ) + } } impl WindowState for RowNumberState { @@ -54,25 +67,18 @@ impl WindowState for RowNumberState { } } - fn curr_output(&self) -> Result { - if self.curr_window().is_ready { - Ok(Some(self.curr_row_number.into())) + fn slide(&mut self) -> Result<(Datum, StateEvictHint)> { + let output = if self.curr_window().is_ready { + Some(self.curr_row_number.into()) } else { - Ok(None) - } + None + }; + let evict_hint = self.slide_inner(); + Ok((output, evict_hint)) } - fn slide_forward(&mut self) -> StateEvictHint { - self.curr_row_number += 1; - self.buffer - .pop_front() - .expect("should not slide forward when the current window is not ready"); - // can't evict any state key in EOWC mode, because we can't recover from previous output now - StateEvictHint::CannotEvict( - self.first_key - .clone() - .expect("should have appended some rows"), - ) + fn slide_no_output(&mut self) -> Result { + Ok(self.slide_inner()) } } @@ -92,6 +98,24 @@ mod tests { } } + #[test] + #[should_panic(expected = "should not slide forward when the current window is not ready")] + fn test_row_number_state_bad_use() { + let call = WindowFuncCall { + kind: WindowFuncKind::RowNumber, + args: AggArgs::None, + return_type: DataType::Int64, + frame: Frame::rows( + FrameBound::UnboundedPreceding, + FrameBound::UnboundedFollowing, + ), + }; + let mut state = RowNumberState::new(&call); + assert!(state.curr_window().key.is_none()); + assert!(!state.curr_window().is_ready); + _ = state.slide() + } + #[test] fn test_row_number_state() { let call = WindowFuncCall { @@ -106,24 +130,23 @@ mod tests { let mut state = RowNumberState::new(&call); assert!(state.curr_window().key.is_none()); assert!(!state.curr_window().is_ready); - assert!(state.curr_output().unwrap().is_none()); state.append(create_state_key(100), SmallVec::new()); assert_eq!(state.curr_window().key.unwrap(), &create_state_key(100)); assert!(state.curr_window().is_ready); - assert_eq!(state.curr_output().unwrap().unwrap(), 1i64.into()); - state.append(create_state_key(103), SmallVec::new()); - state.append(create_state_key(102), SmallVec::new()); - assert_eq!(state.curr_window().key.unwrap(), &create_state_key(100)); - let evict_hint = state.slide_forward(); + let (output, evict_hint) = state.slide().unwrap(); + assert_eq!(output.unwrap(), 1i64.into()); match evict_hint { StateEvictHint::CannotEvict(state_key) => { assert_eq!(state_key, create_state_key(100)); } _ => unreachable!(), } + assert!(!state.curr_window().is_ready); + state.append(create_state_key(103), SmallVec::new()); + state.append(create_state_key(102), SmallVec::new()); assert_eq!(state.curr_window().key.unwrap(), &create_state_key(103)); - assert_eq!(state.curr_output().unwrap().unwrap(), 2i64.into()); - let evict_hint = state.slide_forward(); + let (output, evict_hint) = state.slide().unwrap(); + assert_eq!(output.unwrap(), 2i64.into()); match evict_hint { StateEvictHint::CannotEvict(state_key) => { assert_eq!(state_key, create_state_key(100)); @@ -131,6 +154,7 @@ mod tests { _ => unreachable!(), } assert_eq!(state.curr_window().key.unwrap(), &create_state_key(102)); - assert_eq!(state.curr_output().unwrap().unwrap(), 3i64.into()); + let (output, _) = state.slide().unwrap(); + assert_eq!(output.unwrap(), 3i64.into()); } } diff --git a/src/expr/core/src/window_function/states.rs b/src/expr/core/src/window_function/states.rs index d039eb323e101..3a506d165c356 100644 --- a/src/expr/core/src/window_function/states.rs +++ b/src/expr/core/src/window_function/states.rs @@ -48,28 +48,43 @@ impl WindowStates { self.0.iter().all(|state| state.curr_window().is_ready) } - /// Get the current output of all windows. - pub fn curr_output(&self) -> Result> { + /// Slide all windows forward and collect the output and evict hints. + pub fn slide(&mut self) -> Result<(Vec, StateEvictHint)> { debug_assert!(self.are_aligned()); - self.0.iter().map(|state| state.curr_output()).try_collect() + let mut output = Vec::with_capacity(self.0.len()); + let mut evict_hint: Option = None; + for state in &mut self.0 { + let (x_output, x_evict) = state.slide()?; + output.push(x_output); + evict_hint = match evict_hint { + Some(evict_hint) => Some(evict_hint.merge(x_evict)), + None => Some(x_evict), + }; + } + Ok(( + output, + evict_hint.expect("# of evict hints = # of window states"), + )) } - /// Slide all windows forward and collect the evict hints. - pub fn slide_forward(&mut self) -> StateEvictHint { + /// Slide all windows forward and collect the output, ignoring the evict hints. + pub fn slide_no_evict_hint(&mut self) -> Result> { debug_assert!(self.are_aligned()); - self.0 - .iter_mut() - .map(|state| state.slide_forward()) - .reduce(StateEvictHint::merge) - .expect("# of evict hints = # of window states") + let mut output = Vec::with_capacity(self.0.len()); + for state in &mut self.0 { + let (x_output, _) = state.slide()?; + output.push(x_output); + } + Ok(output) } - /// Slide all windows forward, ignoring the evict hints. - pub fn just_slide_forward(&mut self) { + /// Slide all windows forward, ignoring the output and evict hints. + pub fn just_slide(&mut self) -> Result<()> { debug_assert!(self.are_aligned()); - self.0 - .iter_mut() - .for_each(|state| _ = state.slide_forward()); + for state in &mut self.0 { + state.slide_no_output()?; + } + Ok(()) } } diff --git a/src/stream/src/executor/over_window/eowc.rs b/src/stream/src/executor/over_window/eowc.rs index b5da45edd47e5..fa20e3b49d970 100644 --- a/src/stream/src/executor/over_window/eowc.rs +++ b/src/stream/src/executor/over_window/eowc.rs @@ -241,7 +241,7 @@ impl EowcOverWindowExecutor { // Ignore ready windows (all ready windows were outputted before). while partition.states.are_ready() { - partition.states.just_slide_forward(); + partition.states.just_slide()?; partition.curr_row_buffer.pop_front(); } @@ -276,7 +276,8 @@ impl EowcOverWindowExecutor { &encoded_partition_key, ) .await?; - let mut partition = vars.partitions.get_mut(&encoded_partition_key).unwrap(); + let partition: &mut Partition = + &mut vars.partitions.get_mut(&encoded_partition_key).unwrap(); // Materialize input to state table. this.state_table.insert(input_row); @@ -314,7 +315,7 @@ impl EowcOverWindowExecutor { // The partition is ready to output, so we can produce a row. // Get all outputs. - let ret_values = partition.states.curr_output()?; + let (ret_values, evict_hint) = partition.states.slide()?; let curr_row = partition .curr_row_buffer .pop_front() @@ -330,7 +331,6 @@ impl EowcOverWindowExecutor { } // Evict unneeded rows from state table. - let evict_hint = partition.states.slide_forward(); if let StateEvictHint::CanEvict(keys_to_evict) = evict_hint { for key in keys_to_evict { let order_key = memcmp_encoding::decode_row( diff --git a/src/stream/src/executor/over_window/general.rs b/src/stream/src/executor/over_window/general.rs index c9717f9defe61..38c959039bf56 100644 --- a/src/stream/src/executor/over_window/general.rs +++ b/src/stream/src/executor/over_window/general.rs @@ -512,7 +512,7 @@ impl OverWindowExecutor { // Slide to the first affected key. We can safely compare to `Some(first_curr_key)` here // because it must exist in the states, by the definition of affected range. while states.curr_key() != Some(first_curr_key.as_normal_expect()) { - states.just_slide_forward(); + states.just_slide()?; } let mut curr_key_cursor = part_with_delta.find(first_curr_key).unwrap(); assert_eq!( @@ -525,7 +525,7 @@ impl OverWindowExecutor { let (key, row) = curr_key_cursor .key_value() .expect("cursor must be valid until `last_curr_key`"); - let output = states.curr_output()?; + let output = states.slide_no_evict_hint()?; let new_row = OwnedRow::new( row.as_inner() .iter() @@ -554,7 +554,6 @@ impl OverWindowExecutor { } } - states.just_slide_forward(); curr_key_cursor.move_next(); key != last_curr_key diff --git a/src/stream/tests/integration_tests/eowc_over_window.rs b/src/stream/tests/integration_tests/eowc_over_window.rs index 9407b6013dc03..7334654d8dd50 100644 --- a/src/stream/tests/integration_tests/eowc_over_window.rs +++ b/src/stream/tests/integration_tests/eowc_over_window.rs @@ -186,13 +186,17 @@ async fn test_over_window_aggregate() { check_with_script( || create_executor(calls.clone(), store.clone()), r###" -- !barrier 1 -- !chunk |2 - I T I i - + 1 p1 100 10 - + 1 p1 101 16 - + 4 p1 102 20 -"###, + - !barrier 1 + - !chunk |2 + I T I i + + 1 p1 100 10 + + 1 p1 101 16 + + 4 p1 102 20 + - !chunk |2 + I T I i + + 2 p1 103 30 + + 6 p1 104 11 + "###, expect![[r#" - input: !barrier 1 output: @@ -209,6 +213,17 @@ async fn test_over_window_aggregate() { | + | 1 | p1 | 100 | 10 | 26 | | + | 1 | p1 | 101 | 16 | 46 | +---+---+----+-----+----+----+ + - input: !chunk |- + +---+---+----+-----+----+ + | + | 2 | p1 | 103 | 30 | + | + | 6 | p1 | 104 | 11 | + +---+---+----+-----+----+ + output: + - !chunk |- + +---+---+----+-----+----+----+ + | + | 4 | p1 | 102 | 20 | 66 | + | + | 2 | p1 | 103 | 30 | 61 | + +---+---+----+-----+----+----+ "#]], SnapshotOptions::default(), )