From 8d0e869197a0e092d29a084ba4a4c6e1b127c36d Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Fri, 10 Mar 2023 16:00:53 +0800 Subject: [PATCH] refactor(common): unify order-related types (#8449) Signed-off-by: Richard Chien Co-authored-by: Bugen Zhao --- src/batch/benches/sort.rs | 14 +- src/batch/benches/top_n.rs | 14 +- src/batch/src/executor/group_top_n.rs | 36 +-- .../executor/join/distributed_lookup_join.rs | 2 +- .../src/executor/join/local_lookup_join.rs | 18 +- src/batch/src/executor/merge_sort_exchange.rs | 22 +- src/batch/src/executor/order_by.rs | 146 ++++----- src/batch/src/executor/row_seq_scan.rs | 10 +- src/batch/src/executor/top_n.rs | 48 +-- src/common/benches/bench_row.rs | 4 +- src/common/src/catalog/physical_table.rs | 8 +- .../src/util/encoding_for_comparison.rs | 45 +-- src/common/src/util/ordered/mod.rs | 14 +- src/common/src/util/ordered/serde.rs | 24 +- src/common/src/util/sort_util.rs | 189 +++++++++--- src/compute/tests/integration_tests.rs | 10 +- src/connector/src/sink/catalog/desc.rs | 4 +- src/connector/src/sink/catalog/mod.rs | 6 +- src/ctl/src/cmd_impl/table/scan.rs | 8 +- src/expr/src/vector_op/agg/aggregator.rs | 13 +- src/expr/src/vector_op/agg/array_agg.rs | 18 +- src/expr/src/vector_op/agg/string_agg.rs | 22 +- .../planner_test/tests/testdata/agg.yaml | 38 --- .../tests/testdata/batch_dist_agg.yaml | 12 +- .../planner_test/tests/testdata/nexmark.yaml | 8 +- .../tests/testdata/nexmark_source.yaml | 8 +- .../tests/testdata/over_window_function.yaml | 18 +- .../tests/testdata/stream_dist_agg.yaml | 130 ++++---- .../testdata/subquery_expr_correlated.yaml | 8 +- src/frontend/src/binder/expr/order_by.rs | 33 +-- src/frontend/src/binder/query.rs | 18 +- src/frontend/src/catalog/index_catalog.rs | 9 +- src/frontend/src/catalog/table_catalog.rs | 25 +- src/frontend/src/expr/order_by_expr.rs | 16 +- src/frontend/src/handler/create_index.rs | 34 +-- src/frontend/src/handler/describe.rs | 12 +- .../src/optimizer/plan_node/batch_limit.rs | 2 +- .../src/optimizer/plan_node/batch_sort_agg.rs | 13 +- .../src/optimizer/plan_node/derive.rs | 16 +- .../src/optimizer/plan_node/generic/agg.rs | 103 +------ .../plan_node/generic/dynamic_filter.rs | 2 +- .../src/optimizer/plan_node/generic/source.rs | 2 +- .../src/optimizer/plan_node/generic/top_n.rs | 14 +- .../src/optimizer/plan_node/logical_agg.rs | 47 ++- .../optimizer/plan_node/logical_over_agg.rs | 18 +- .../src/optimizer/plan_node/logical_scan.rs | 27 +- .../src/optimizer/plan_node/logical_topn.rs | 16 +- .../src/optimizer/plan_node/stream.rs | 10 +- .../optimizer/plan_node/stream_materialize.rs | 7 +- .../src/optimizer/plan_node/stream_sink.rs | 9 +- .../plan_node/stream_watermark_filter.rs | 2 +- src/frontend/src/optimizer/plan_node/utils.rs | 15 +- src/frontend/src/optimizer/property/order.rs | 279 ++++-------------- .../rule/apply_agg_transpose_rule.rs | 2 +- .../optimizer/rule/index_delta_join_rule.rs | 4 +- .../optimizer/rule/index_selection_rule.rs | 26 +- .../optimizer/rule/min_max_on_index_rule.rs | 54 ++-- .../optimizer/rule/over_agg_to_topn_rule.rs | 10 +- .../src/optimizer/rule/top_n_on_index_rule.rs | 44 +-- src/frontend/src/planner/query.rs | 6 +- src/frontend/src/planner/select.rs | 9 +- src/frontend/src/planner/set_expr.rs | 4 +- .../src/utils/column_index_mapping.rs | 33 +-- .../hummock_sdk/src/filter_key_extractor.rs | 14 +- src/stream/src/common/table/state_table.rs | 4 +- .../src/common/table/test_state_table.rs | 20 +- .../src/common/table/test_storage_table.rs | 8 +- src/stream/src/common/table/test_utils.rs | 6 +- .../src/executor/aggregation/agg_call.rs | 4 +- .../src/executor/aggregation/distinct.rs | 6 +- src/stream/src/executor/aggregation/minput.rs | 68 ++--- src/stream/src/executor/aggregation/value.rs | 4 +- src/stream/src/executor/backfill.rs | 10 +- src/stream/src/executor/dynamic_filter.rs | 4 +- src/stream/src/executor/global_simple_agg.rs | 8 +- src/stream/src/executor/hash_agg.rs | 20 +- src/stream/src/executor/hash_join.rs | 16 +- src/stream/src/executor/integration_tests.rs | 10 +- src/stream/src/executor/local_simple_agg.rs | 8 +- src/stream/src/executor/lookup/impl_.rs | 10 +- src/stream/src/executor/lookup/sides.rs | 4 +- src/stream/src/executor/lookup/tests.rs | 12 +- .../src/executor/managed_state/join/mod.rs | 2 +- .../managed_state/top_n/top_n_state.rs | 12 +- src/stream/src/executor/mview/materialize.rs | 32 +- src/stream/src/executor/mview/test_utils.rs | 4 +- src/stream/src/executor/sort.rs | 2 +- src/stream/src/executor/sort_buffer.rs | 2 +- .../src/executor/source/source_executor.rs | 4 +- src/stream/src/executor/test_utils.rs | 10 +- src/stream/src/executor/top_n/group_top_n.rs | 46 +-- .../executor/top_n/group_top_n_appendonly.rs | 12 +- .../src/executor/top_n/top_n_appendonly.rs | 30 +- src/stream/src/executor/top_n/top_n_plain.rs | 64 ++-- src/stream/src/executor/top_n/utils.rs | 12 +- src/stream/src/executor/watermark_filter.rs | 2 +- src/stream/src/from_proto/agg_common.rs | 11 +- src/stream/src/from_proto/batch_query.rs | 2 +- src/stream/src/from_proto/chain.rs | 4 +- src/stream/src/from_proto/group_top_n.rs | 14 +- .../src/from_proto/group_top_n_appendonly.rs | 14 +- src/stream/src/from_proto/lookup.rs | 6 +- src/stream/src/from_proto/mview.rs | 6 +- src/stream/src/from_proto/top_n.rs | 10 +- src/stream/src/from_proto/top_n_appendonly.rs | 10 +- 105 files changed, 1076 insertions(+), 1289 deletions(-) diff --git a/src/batch/benches/sort.rs b/src/batch/benches/sort.rs index f83af98369e9..f8de2fa730e9 100644 --- a/src/batch/benches/sort.rs +++ b/src/batch/benches/sort.rs @@ -18,7 +18,7 @@ use criterion::{criterion_group, criterion_main, BatchSize, BenchmarkId, Criteri use risingwave_batch::executor::{BoxedExecutor, SortExecutor}; use risingwave_common::enable_jemalloc_on_linux; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use tokio::runtime::Runtime; use utils::{create_input, execute_executor}; @@ -30,9 +30,9 @@ fn create_order_by_executor( single_column: bool, ) -> BoxedExecutor { const CHUNK_SIZE: usize = 1024; - let (child, order_pairs) = if single_column { + let (child, column_orders) = if single_column { let input = create_input(&[DataType::Int64], chunk_size, chunk_num); - (input, vec![OrderPair::new(0, OrderType::Ascending)]) + (input, vec![ColumnOrder::new(0, OrderType::ascending())]) } else { let input = create_input( &[ @@ -47,16 +47,16 @@ fn create_order_by_executor( ( input, vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Descending), - OrderPair::new(2, OrderType::Ascending), + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::descending()), + ColumnOrder::new(2, OrderType::ascending()), ], ) }; Box::new(SortExecutor::new( child, - order_pairs, + column_orders, "SortExecutor".into(), CHUNK_SIZE, )) diff --git a/src/batch/benches/top_n.rs b/src/batch/benches/top_n.rs index 91032866ea17..9d3a51a8a032 100644 --- a/src/batch/benches/top_n.rs +++ b/src/batch/benches/top_n.rs @@ -18,7 +18,7 @@ use criterion::{criterion_group, criterion_main, BatchSize, BenchmarkId, Criteri use risingwave_batch::executor::{BoxedExecutor, TopNExecutor}; use risingwave_common::enable_jemalloc_on_linux; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use tokio::runtime::Runtime; use utils::{create_input, execute_executor}; @@ -32,9 +32,9 @@ fn create_top_n_executor( limit: usize, ) -> BoxedExecutor { const CHUNK_SIZE: usize = 1024; - let (child, order_pairs) = if single_column { + let (child, column_orders) = if single_column { let input = create_input(&[DataType::Int64], chunk_size, chunk_num); - (input, vec![OrderPair::new(0, OrderType::Ascending)]) + (input, vec![ColumnOrder::new(0, OrderType::ascending())]) } else { let input = create_input( &[ @@ -49,16 +49,16 @@ fn create_top_n_executor( ( input, vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Descending), - OrderPair::new(2, OrderType::Ascending), + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::descending()), + ColumnOrder::new(2, OrderType::ascending()), ], ) }; Box::new(TopNExecutor::new( child, - order_pairs, + column_orders, offset, limit, false, diff --git a/src/batch/src/executor/group_top_n.rs b/src/batch/src/executor/group_top_n.rs index d925ad4e6081..20024c232758 100644 --- a/src/batch/src/executor/group_top_n.rs +++ b/src/batch/src/executor/group_top_n.rs @@ -27,7 +27,7 @@ use risingwave_common::types::DataType; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::encoding_for_comparison::encode_chunk; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::batch_plan::plan_node::NodeBody; use super::top_n::{HeapElem, TopNHeap}; @@ -41,7 +41,7 @@ use crate::task::BatchTaskContext; /// For each group, use a N-heap to store the smallest N rows. pub struct GroupTopNExecutor { child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, offset: usize, limit: usize, group_key: Vec, @@ -54,7 +54,7 @@ pub struct GroupTopNExecutor { pub struct GroupTopNExecutorBuilder { child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, offset: usize, limit: usize, group_key: Vec, @@ -70,7 +70,7 @@ impl HashKeyDispatcher for GroupTopNExecutorBuilder { fn dispatch_impl(self) -> Self::Output { Box::new(GroupTopNExecutor::::new( self.child, - self.order_pairs, + self.column_orders, self.offset, self.limit, self.with_ties, @@ -98,10 +98,10 @@ impl BoxedExecutorBuilder for GroupTopNExecutorBuilder { NodeBody::GroupTopN )?; - let order_pairs = top_n_node + let column_orders = top_n_node .column_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); let group_key = top_n_node @@ -117,7 +117,7 @@ impl BoxedExecutorBuilder for GroupTopNExecutorBuilder { let builder = Self { child, - order_pairs, + column_orders, offset: top_n_node.get_offset() as usize, limit: top_n_node.get_limit() as usize, group_key, @@ -135,7 +135,7 @@ impl GroupTopNExecutor { #[expect(clippy::too_many_arguments)] pub fn new( child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, offset: usize, limit: usize, with_ties: bool, @@ -146,7 +146,7 @@ impl GroupTopNExecutor { let schema = child.schema().clone(); Self { child, - order_pairs, + column_orders, offset, limit, with_ties, @@ -186,7 +186,7 @@ impl GroupTopNExecutor { let chunk = Arc::new(chunk?.compact()); let keys = K::build(self.group_key.as_slice(), &chunk)?; - for (row_id, (encoded_row, key)) in encode_chunk(&chunk, &self.order_pairs) + for (row_id, (encoded_row, key)) in encode_chunk(&chunk, &self.column_orders) .into_iter() .zip_eq_fast(keys.into_iter()) .enumerate() @@ -256,19 +256,19 @@ mod tests { 5 2 2 ", )); - let order_pairs = vec![ - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let top_n_executor = (GroupTopNExecutorBuilder { child: Box::new(mock_executor), - order_pairs, + column_orders, offset: 1, limit: 3, with_ties: false, diff --git a/src/batch/src/executor/join/distributed_lookup_join.rs b/src/batch/src/executor/join/distributed_lookup_join.rs index 0014da4ff48b..c23ade15f1e3 100644 --- a/src/batch/src/executor/join/distributed_lookup_join.rs +++ b/src/batch/src/executor/join/distributed_lookup_join.rs @@ -186,7 +186,7 @@ impl BoxedExecutorBuilder for DistributedLookupJoinExecutorBuilder { let order_types: Vec = table_desc .pk .iter() - .map(|order| OrderType::from_protobuf(&order.get_order_type().unwrap().direction())) + .map(|order| OrderType::from_protobuf(order.get_order_type().unwrap())) .collect(); let pk_indices = table_desc diff --git a/src/batch/src/executor/join/local_lookup_join.rs b/src/batch/src/executor/join/local_lookup_join.rs index 97e8278b1c3c..6fb7842444cd 100644 --- a/src/batch/src/executor/join/local_lookup_join.rs +++ b/src/batch/src/executor/join/local_lookup_join.rs @@ -465,7 +465,7 @@ mod tests { use risingwave_common::hash::HashKeyDispatcher; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; - use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::expr::{ new_binary_expr, BoxedExpression, InputRefExpression, LiteralExpression, }; @@ -557,20 +557,20 @@ mod tests { } fn create_order_by_executor(child: BoxedExecutor) -> BoxedExecutor { - let order_pairs = vec![ - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, ]; Box::new(SortExecutor::new( child, - order_pairs, + column_orders, "SortExecutor".into(), CHUNK_SIZE, )) diff --git a/src/batch/src/executor/merge_sort_exchange.rs b/src/batch/src/executor/merge_sort_exchange.rs index 1ca859cae29c..3108ed5a8f0d 100644 --- a/src/batch/src/executor/merge_sort_exchange.rs +++ b/src/batch/src/executor/merge_sort_exchange.rs @@ -20,7 +20,7 @@ use risingwave_common::array::DataChunk; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::{Result, RwError}; use risingwave_common::types::ToOwnedDatum; -use risingwave_common::util::sort_util::{HeapElem, OrderPair}; +use risingwave_common::util::sort_util::{ColumnOrder, HeapElem}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ExchangeSource as ProstExchangeSource; @@ -39,7 +39,7 @@ pub struct MergeSortExchangeExecutorImpl { context: C, /// keeps one data chunk of each source if any source_inputs: Vec>, - order_pairs: Arc>, + column_orders: Arc>, min_heap: BinaryHeap, proto_sources: Vec, sources: Vec, // impl @@ -76,7 +76,7 @@ impl MergeSortExchangeEx assert!(source_idx < self.source_inputs.len()); let chunk_ref = self.source_inputs[source_idx].as_ref().unwrap(); self.min_heap.push(HeapElem { - order_pairs: self.order_pairs.clone(), + column_orders: self.column_orders.clone(), chunk: chunk_ref.clone(), chunk_idx: source_idx, elem_idx: row_idx, @@ -191,12 +191,12 @@ impl BoxedExecutorBuilder for MergeSortExchangeExecutorBuilder { NodeBody::MergeSortExchange )?; - let order_pairs = sort_merge_node + let column_orders = sort_merge_node .column_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); - let order_pairs = Arc::new(order_pairs); + let column_orders = Arc::new(column_orders); let exchange_node = sort_merge_node.get_exchange()?; let proto_sources: Vec = exchange_node.get_sources().to_vec(); @@ -213,7 +213,7 @@ impl BoxedExecutorBuilder for MergeSortExchangeExecutorBuilder { Ok(Box::new(MergeSortExchangeExecutor:: { context: source.context().clone(), source_inputs: vec![None; num_sources], - order_pairs, + column_orders, min_heap: BinaryHeap::new(), proto_sources, sources: vec![], @@ -260,9 +260,9 @@ mod tests { proto_sources.push(ProstExchangeSource::default()); source_creators.push(fake_create_source.clone()); } - let order_pairs = Arc::new(vec![OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + let column_orders = Arc::new(vec![ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }]); let executor = Box::new(MergeSortExchangeExecutorImpl::< @@ -271,7 +271,7 @@ mod tests { > { context: ComputeNodeContext::for_test(), source_inputs: vec![None; proto_sources.len()], - order_pairs, + column_orders, min_heap: BinaryHeap::new(), proto_sources, sources: vec![], diff --git a/src/batch/src/executor/order_by.rs b/src/batch/src/executor/order_by.rs index d305e1e59ea7..ad7fe212feab 100644 --- a/src/batch/src/executor/order_by.rs +++ b/src/batch/src/executor/order_by.rs @@ -18,7 +18,7 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::{Result, RwError}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::encoding_for_comparison::encode_chunk; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::batch_plan::plan_node::NodeBody; use super::{BoxedDataChunkStream, BoxedExecutor, BoxedExecutorBuilder, Executor, ExecutorBuilder}; @@ -33,7 +33,7 @@ use crate::task::BatchTaskContext; /// 4. Build and yield data chunks according to the row order pub struct SortExecutor { child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, identity: String, schema: Schema, chunk_size: usize, @@ -64,14 +64,14 @@ impl BoxedExecutorBuilder for SortExecutor { let order_by_node = try_match_expand!(source.plan_node().get_node_body().unwrap(), NodeBody::Sort)?; - let order_pairs = order_by_node + let column_orders = order_by_node .column_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); Ok(Box::new(SortExecutor::new( child, - order_pairs, + column_orders, source.plan_node().get_identity().clone(), source.context.get_config().developer.batch_chunk_size, ))) @@ -91,7 +91,7 @@ impl SortExecutor { } for chunk in &chunks { - let encoded_chunk = encode_chunk(chunk, &self.order_pairs); + let encoded_chunk = encode_chunk(chunk, &self.column_orders); encoded_rows.extend( encoded_chunk .into_iter() @@ -117,14 +117,14 @@ impl SortExecutor { impl SortExecutor { pub fn new( child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, identity: String, chunk_size: usize, ) -> Self { let schema = child.schema().clone(); Self { child, - order_pairs, + column_orders, identity, schema, chunk_size, @@ -166,20 +166,20 @@ mod tests { 2 2 3 1", )); - let order_pairs = vec![ - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor2".to_string(), CHUNK_SIZE, )); @@ -216,19 +216,19 @@ mod tests { 2.2 -1.1 3.3 -2.2", )); - let order_pairs = vec![ - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor2".to_string(), CHUNK_SIZE, )); @@ -265,19 +265,19 @@ mod tests { 2.2 2.2 3.3 1.1", )); - let order_pairs = vec![ - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor2".to_string(), CHUNK_SIZE, )); @@ -335,23 +335,23 @@ mod tests { ); let mut mock_executor = MockExecutor::new(schema); mock_executor.add(input_chunk); - let order_pairs = vec![ - OrderPair { - column_idx: 2, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 2, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 1, - order_type: OrderType::Descending, + ColumnOrder { + column_index: 1, + order_type: OrderType::descending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor".to_string(), CHUNK_SIZE, )); @@ -408,23 +408,23 @@ mod tests { ); let mut mock_executor = MockExecutor::new(schema); mock_executor.add(input_chunk); - let order_pairs = vec![ - OrderPair { - column_idx: 0, - order_type: OrderType::Descending, + let column_orders = vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::descending(), }, - OrderPair { - column_idx: 1, - order_type: OrderType::Descending, + ColumnOrder { + column_index: 1, + order_type: OrderType::descending(), }, - OrderPair { - column_idx: 2, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 2, + order_type: OrderType::ascending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor".to_string(), CHUNK_SIZE, )); @@ -501,23 +501,23 @@ mod tests { ); let mut mock_executor = MockExecutor::new(schema); mock_executor.add(input_chunk); - let order_pairs = vec![ - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 2, - order_type: OrderType::Descending, + ColumnOrder { + column_index: 2, + order_type: OrderType::descending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor".to_string(), CHUNK_SIZE, )); @@ -680,19 +680,19 @@ mod tests { ); let mut mock_executor = MockExecutor::new(schema); mock_executor.add(input_chunk); - let order_pairs = vec![ - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 1, - order_type: OrderType::Descending, + ColumnOrder { + column_index: 1, + order_type: OrderType::descending(), }, ]; let order_by_executor = Box::new(SortExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, "SortExecutor".to_string(), CHUNK_SIZE, )); diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index 57fcb99623e1..d8ac75300809 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -26,7 +26,7 @@ use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, Datum}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::select_all; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::{Direction, OrderType}; use risingwave_common::util::value_encoding::deserialize_datum; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{scan_range, ScanRange as ProstScanRange}; @@ -193,7 +193,7 @@ impl BoxedExecutorBuilder for RowSeqScanExecutorBuilder { let order_types: Vec = table_desc .pk .iter() - .map(|order| OrderType::from_protobuf(&order.get_order_type().unwrap().direction())) + .map(|order| OrderType::from_protobuf(order.get_order_type().unwrap())) .collect(); let pk_indices = table_desc @@ -406,9 +406,9 @@ impl RowSeqScanExecutor { } = scan_range; let (start_bound, end_bound) = - match table.pk_serializer().get_order_types()[pk_prefix.len()] { - OrderType::Ascending => (next_col_bounds.0, next_col_bounds.1), - OrderType::Descending => (next_col_bounds.1, next_col_bounds.0), + match table.pk_serializer().get_order_types()[pk_prefix.len()].direction() { + Direction::Ascending => (next_col_bounds.0, next_col_bounds.1), + Direction::Descending => (next_col_bounds.1, next_col_bounds.0), }; // Range Scan. diff --git a/src/batch/src/executor/top_n.rs b/src/batch/src/executor/top_n.rs index a3a557189dc9..37aaba8f8b95 100644 --- a/src/batch/src/executor/top_n.rs +++ b/src/batch/src/executor/top_n.rs @@ -24,7 +24,7 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::{Result, RwError}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::encoding_for_comparison::encode_chunk; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::batch_plan::plan_node::NodeBody; use crate::executor::{ @@ -37,7 +37,7 @@ use crate::task::BatchTaskContext; /// Use a N-heap to store the smallest N rows. pub struct TopNExecutor { child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, offset: usize, limit: usize, with_ties: bool, @@ -57,14 +57,14 @@ impl BoxedExecutorBuilder for TopNExecutor { let top_n_node = try_match_expand!(source.plan_node().get_node_body().unwrap(), NodeBody::TopN)?; - let order_pairs = top_n_node + let column_orders = top_n_node .column_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); Ok(Box::new(Self::new( child, - order_pairs, + column_orders, top_n_node.get_offset() as usize, top_n_node.get_limit() as usize, top_n_node.get_with_ties(), @@ -77,7 +77,7 @@ impl BoxedExecutorBuilder for TopNExecutor { impl TopNExecutor { pub fn new( child: BoxedExecutor, - order_pairs: Vec, + column_orders: Vec, offset: usize, limit: usize, with_ties: bool, @@ -87,7 +87,7 @@ impl TopNExecutor { let schema = child.schema().clone(); Self { child, - order_pairs, + column_orders, offset, limit, with_ties, @@ -218,7 +218,7 @@ impl TopNExecutor { #[for_await] for chunk in self.child.execute() { let chunk = Arc::new(chunk?.compact()); - for (row_id, encoded_row) in encode_chunk(&chunk, &self.order_pairs) + for (row_id, encoded_row) in encode_chunk(&chunk, &self.column_orders) .into_iter() .enumerate() { @@ -275,19 +275,19 @@ mod tests { 4 2 5 1", )); - let order_pairs = vec![ - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let top_n_executor = Box::new(TopNExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, 1, 3, false, @@ -332,19 +332,19 @@ mod tests { 4 2 5 1", )); - let order_pairs = vec![ - OrderPair { - column_idx: 1, - order_type: OrderType::Ascending, + let column_orders = vec![ + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, - OrderPair { - column_idx: 0, - order_type: OrderType::Ascending, + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, ]; let top_n_executor = Box::new(TopNExecutor::new( Box::new(mock_executor), - order_pairs, + column_orders, 1, 0, false, diff --git a/src/common/benches/bench_row.rs b/src/common/benches/bench_row.rs index e1d43e1fbe05..fad19b9861fa 100644 --- a/src/common/benches/bench_row.rs +++ b/src/common/benches/bench_row.rs @@ -59,7 +59,7 @@ impl Case { fn memcmp_encode(c: &Case) -> Vec> { let serde = OrderedRowSerde::new( c.schema.to_vec(), - vec![OrderType::Descending; c.schema.len()], + vec![OrderType::descending(); c.schema.len()], ); let mut array = vec![]; for row in &c.rows { @@ -92,7 +92,7 @@ fn column_aware_encode(c: &Case) -> Vec> { fn memcmp_decode(c: &Case, bytes: &Vec>) -> Result>> { let serde = OrderedRowSerde::new( c.schema.to_vec(), - vec![OrderType::Descending; c.schema.len()], + vec![OrderType::descending(); c.schema.len()], ); let mut res = vec![]; if c.column_ids == c.needed_ids { diff --git a/src/common/src/catalog/physical_table.rs b/src/common/src/catalog/physical_table.rs index 780ffaf51006..4e163a9a7199 100644 --- a/src/common/src/catalog/physical_table.rs +++ b/src/common/src/catalog/physical_table.rs @@ -19,7 +19,7 @@ use risingwave_pb::common::PbColumnOrder; use risingwave_pb::plan_common::StorageTableDesc; use super::{ColumnDesc, ColumnId, TableId}; -use crate::util::sort_util::OrderPair; +use crate::util::sort_util::ColumnOrder; /// Includes necessary information for compute node to access data of the table. /// @@ -29,7 +29,7 @@ pub struct TableDesc { /// Id of the table, to find in storage. pub table_id: TableId, /// The key used to sort in storage. - pub pk: Vec, + pub pk: Vec, /// All columns in the table, noticed it is NOT sorted by columnId in the vec. pub columns: Vec, /// Distribution keys of this table, which corresponds to the corresponding column of the @@ -66,13 +66,13 @@ impl TableDesc { } pub fn order_column_indices(&self) -> Vec { - self.pk.iter().map(|col| (col.column_idx)).collect() + self.pk.iter().map(|col| (col.column_index)).collect() } pub fn order_column_ids(&self) -> Vec { self.pk .iter() - .map(|col| self.columns[col.column_idx].column_id) + .map(|col| self.columns[col.column_index].column_id) .collect() } diff --git a/src/common/src/util/encoding_for_comparison.rs b/src/common/src/util/encoding_for_comparison.rs index 10e845a16218..409a3b2f4252 100644 --- a/src/common/src/util/encoding_for_comparison.rs +++ b/src/common/src/util/encoding_for_comparison.rs @@ -15,15 +15,16 @@ use itertools::Itertools; use super::iter_util::ZipEqFast; +use super::sort_util::Direction; use crate::array::{ArrayImpl, DataChunk}; use crate::error::Result; use crate::row::OwnedRow; use crate::types::{memcmp_serialize_datum_into, ScalarRefImpl}; -use crate::util::sort_util::{OrderPair, OrderType}; +use crate::util::sort_util::{ColumnOrder, OrderType}; fn encode_value(value: Option>, order: &OrderType) -> Result> { let mut serializer = memcomparable::Serializer::new(vec![]); - serializer.set_reverse(order == &OrderType::Descending); + serializer.set_reverse(order.direction() == Direction::Descending); memcmp_serialize_datum_into(value, &mut serializer)?; Ok(serializer.into_inner()) } @@ -41,10 +42,10 @@ fn encode_array(array: &ArrayImpl, order: &OrderType) -> Result>> { /// the datachunk. /// /// TODO: specify the order for `NULL`. -pub fn encode_chunk(chunk: &DataChunk, order_pairs: &[OrderPair]) -> Vec> { - let encoded_columns = order_pairs +pub fn encode_chunk(chunk: &DataChunk, column_orders: &[ColumnOrder]) -> Vec> { + let encoded_columns = column_orders .iter() - .map(|o| encode_array(chunk.column_at(o.column_idx).array_ref(), &o.order_type).unwrap()) + .map(|o| encode_array(chunk.column_at(o.column_index).array_ref(), &o.order_type).unwrap()) .collect_vec(); let mut encoded_chunk = vec![vec![]; chunk.capacity()]; @@ -57,10 +58,10 @@ pub fn encode_chunk(chunk: &DataChunk, order_pairs: &[OrderPair]) -> Vec encoded_chunk } -pub fn encode_row(row: &OwnedRow, order_pairs: &[OrderPair]) -> Vec { +pub fn encode_row(row: &OwnedRow, column_orders: &[ColumnOrder]) -> Vec { let mut encoded_row = vec![]; - order_pairs.iter().for_each(|o| { - let value = row[o.column_idx].as_ref(); + column_orders.iter().for_each(|o| { + let value = row[o.column_index].as_ref(); encoded_row .extend(encode_value(value.map(|x| x.as_scalar_ref_impl()), &o.order_type).unwrap()); }); @@ -75,7 +76,7 @@ mod tests { use crate::array::DataChunk; use crate::row::OwnedRow; use crate::types::{DataType, ScalarImpl}; - use crate::util::sort_util::{OrderPair, OrderType}; + use crate::util::sort_util::{ColumnOrder, OrderType}; #[test] fn test_encode_row() { @@ -90,20 +91,20 @@ mod tests { let row1 = OwnedRow::new(vec![v10, v11, v12]); let row2 = OwnedRow::new(vec![v20, v21, v22]); - let order_pairs = vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Descending), + let column_orders = vec![ + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::descending()), ]; - let encoded_row1 = encode_row(&row1, &order_pairs); + let encoded_row1 = encode_row(&row1, &column_orders); let encoded_v10 = encode_value( v10_cloned.as_ref().map(|x| x.as_scalar_ref_impl()), - &OrderType::Ascending, + &OrderType::ascending(), ) .unwrap(); let encoded_v11 = encode_value( v11_cloned.as_ref().map(|x| x.as_scalar_ref_impl()), - &OrderType::Descending, + &OrderType::descending(), ) .unwrap(); let concated_encoded_row1 = encoded_v10 @@ -112,7 +113,7 @@ mod tests { .collect_vec(); assert_eq!(encoded_row1, concated_encoded_row1); - let encoded_row2 = encode_row(&row2, &order_pairs); + let encoded_row2 = encode_row(&row2, &column_orders); assert!(encoded_row1 < encoded_row2); } @@ -131,14 +132,14 @@ mod tests { &[row1.clone(), row2.clone()], &[DataType::Int32, DataType::Varchar, DataType::Float32], ); - let order_pairs = vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Descending), + let column_orders = vec![ + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::descending()), ]; - let encoded_row1 = encode_row(&row1, &order_pairs); - let encoded_row2 = encode_row(&row2, &order_pairs); - let encoded_chunk = encode_chunk(&chunk, &order_pairs); + let encoded_row1 = encode_row(&row1, &column_orders); + let encoded_row2 = encode_row(&row2, &column_orders); + let encoded_chunk = encode_chunk(&chunk, &column_orders); assert_eq!(&encoded_chunk, &[encoded_row1, encoded_row2]); } } diff --git a/src/common/src/util/ordered/mod.rs b/src/common/src/util/ordered/mod.rs index ee3629b5803e..08e10965fda5 100644 --- a/src/common/src/util/ordered/mod.rs +++ b/src/common/src/util/ordered/mod.rs @@ -20,10 +20,12 @@ use OrderedDatum::{NormalOrder, ReversedOrder}; pub use self::serde::*; use super::iter_util::ZipEqFast; +use super::sort_util::Direction; use crate::row::OwnedRow; use crate::types::{memcmp_serialize_datum_into, Datum}; use crate::util::sort_util::OrderType; +// TODO(rc): support `NULLS FIRST | LAST` #[derive(Clone, Eq, PartialEq, Ord, PartialOrd)] pub enum OrderedDatum { NormalOrder(Datum), @@ -62,9 +64,9 @@ impl OrderedRow { row.into_inner() .into_iter() .zip_eq_fast(order_types.iter()) - .map(|(datum, order_type)| match order_type { - OrderType::Ascending => NormalOrder(datum), - OrderType::Descending => ReversedOrder(Reverse(datum)), + .map(|(datum, order_type)| match order_type.direction() { + Direction::Ascending => NormalOrder(datum), + Direction::Descending => ReversedOrder(Reverse(datum)), }) .collect::>(), ) @@ -142,9 +144,9 @@ mod tests { } const ORDER_TYPES: &[OrderType] = &[ - OrderType::Ascending, - OrderType::Descending, - OrderType::Ascending, + OrderType::ascending(), + OrderType::descending(), + OrderType::ascending(), ]; #[test] diff --git a/src/common/src/util/ordered/serde.rs b/src/common/src/util/ordered/serde.rs index ac878c02418d..2cff721756f4 100644 --- a/src/common/src/util/ordered/serde.rs +++ b/src/common/src/util/ordered/serde.rs @@ -21,7 +21,7 @@ use crate::types::{ memcmp_deserialize_datum_from, memcmp_serialize_datum_into, DataType, ToDatumRef, }; use crate::util::iter_util::{ZipEqDebug, ZipEqFast}; -use crate::util::sort_util::OrderType; +use crate::util::sort_util::{Direction, OrderType}; /// `OrderedRowSerde` is responsible for serializing and deserializing Ordered Row. #[derive(Clone)] @@ -66,7 +66,7 @@ impl OrderedRowSerde { ) { for (datum, order_type) in datum_refs.zip_eq_debug(self.order_types.iter()) { let mut serializer = memcomparable::Serializer::new(&mut append_to); - serializer.set_reverse(*order_type == OrderType::Descending); + serializer.set_reverse(order_type.direction() == Direction::Descending); memcmp_serialize_datum_into(datum, &mut serializer).unwrap(); } } @@ -75,7 +75,7 @@ impl OrderedRowSerde { let mut values = Vec::with_capacity(self.schema.len()); let mut deserializer = memcomparable::Deserializer::new(data); for (data_type, order_type) in self.schema.iter().zip_eq_fast(self.order_types.iter()) { - deserializer.set_reverse(*order_type == OrderType::Descending); + deserializer.set_reverse(order_type.direction() == Direction::Descending); let datum = memcmp_deserialize_datum_from(data_type, &mut deserializer)?; values.push(datum); } @@ -102,7 +102,7 @@ impl OrderedRowSerde { let order_type = &self.order_types[index]; let data = &key[len..]; let mut deserializer = memcomparable::Deserializer::new(data); - deserializer.set_reverse(*order_type == OrderType::Descending); + deserializer.set_reverse(order_type.direction() == Direction::Descending); len += ScalarImpl::encoding_data_size(data_type, &mut deserializer)?; } @@ -122,7 +122,7 @@ mod tests { #[test] fn test_ordered_row_serializer() { - let orders = vec![OrderType::Descending, OrderType::Ascending]; + let orders = vec![OrderType::descending(), OrderType::ascending()]; let data_types = vec![DataType::Int16, DataType::Varchar]; let serializer = OrderedRowSerde::new(data_types, orders); let row1 = OwnedRow::new(vec![Some(Int16(5)), Some(Utf8("abc".into()))]); @@ -148,7 +148,7 @@ mod tests { use crate::types::ScalarImpl::{self, *}; { // basic - let order_types = vec![OrderType::Descending, OrderType::Ascending]; + let order_types = vec![OrderType::descending(), OrderType::ascending()]; let schema = vec![DataType::Varchar, DataType::Int16]; let serde = OrderedRowSerde::new(schema, order_types); @@ -170,7 +170,7 @@ mod tests { { // decimal - let order_types = vec![OrderType::Descending, OrderType::Ascending]; + let order_types = vec![OrderType::descending(), OrderType::ascending()]; let schema = vec![DataType::Varchar, DataType::Decimal]; let serde = OrderedRowSerde::new(schema, order_types); @@ -201,7 +201,7 @@ mod tests { #[test] fn test_deserialize_with_column_indices() { - let order_types = vec![OrderType::Descending, OrderType::Ascending]; + let order_types = vec![OrderType::descending(), OrderType::ascending()]; let schema = vec![DataType::Varchar, DataType::Int16]; let serde = OrderedRowSerde::new(schema, order_types); @@ -218,7 +218,7 @@ mod tests { let row_0_idx_0_len = serde.deserialize_prefix_len(&array[0], 1).unwrap(); let schema = vec![DataType::Varchar]; - let order_types = vec![OrderType::Descending]; + let order_types = vec![OrderType::descending()]; let deserde = OrderedRowSerde::new(schema, order_types); let prefix_slice = &array[0][0..row_0_idx_0_len]; assert_eq!( @@ -230,7 +230,7 @@ mod tests { { let row_0_idx_1_len = serde.deserialize_prefix_len(&array[0], 2).unwrap(); - let order_types = vec![OrderType::Descending, OrderType::Ascending]; + let order_types = vec![OrderType::descending(), OrderType::ascending()]; let schema = vec![DataType::Varchar, DataType::Int16]; let deserde = OrderedRowSerde::new(schema, order_types); let prefix_slice = &array[0][0..row_0_idx_1_len]; @@ -245,7 +245,7 @@ mod tests { use crate::types::interval::IntervalUnit; use crate::types::OrderedF64; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let schema = vec![DataType::Int16]; let serde = OrderedRowSerde::new(schema, order_types); @@ -414,7 +414,7 @@ mod tests { { { // test varchar Descending - let order_types = vec![OrderType::Descending]; + let order_types = vec![OrderType::descending()]; let schema = vec![DataType::Varchar]; let serde = OrderedRowSerde::new(schema, order_types); let varchar = "abcdefghijklmnopq"; diff --git a/src/common/src/util/sort_util.rs b/src/common/src/util/sort_util.rs index 234a7c39261a..65bae034a26b 100644 --- a/src/common/src/util/sort_util.rs +++ b/src/common/src/util/sort_util.rs @@ -13,78 +13,179 @@ // limitations under the License. use std::cmp::{Ord, Ordering}; +use std::fmt; use std::sync::Arc; +use parse_display::Display; use risingwave_pb::common::{PbColumnOrder, PbDirection, PbOrderType}; use crate::array::{Array, ArrayImpl, DataChunk}; +use crate::catalog::{FieldDisplay, Schema}; use crate::error::ErrorCode::InternalError; use crate::error::Result; -#[derive(PartialEq, Eq, Hash, Copy, Clone, Debug)] -pub enum OrderType { +// TODO(rc): to support `NULLS FIRST | LAST`, we may need to hide this enum, forcing developers use +// `OrderType` instead. +#[derive(PartialEq, Eq, Hash, Copy, Clone, Debug, Display, Default)] +pub enum Direction { + #[default] + #[display("ASC")] Ascending, + #[display("DESC")] Descending, } -impl OrderType { - // TODO(rc): from `PbOrderType` - pub fn from_protobuf(order_type: &PbDirection) -> OrderType { +impl Direction { + pub fn from_protobuf(order_type: &PbDirection) -> Direction { match order_type { - PbDirection::Ascending => OrderType::Ascending, - PbDirection::Descending => OrderType::Descending, + PbDirection::Ascending => Direction::Ascending, + PbDirection::Descending => Direction::Descending, PbDirection::Unspecified => unreachable!(), } } - // TODO(rc): to `PbOrderType` pub fn to_protobuf(self) -> PbDirection { match self { - OrderType::Ascending => PbDirection::Ascending, - OrderType::Descending => PbDirection::Descending, + Direction::Ascending => PbDirection::Ascending, + Direction::Descending => PbDirection::Descending, + } + } +} + +#[allow(dead_code)] +#[derive(PartialEq, Eq, Hash, Copy, Clone, Debug, Display, Default)] +pub enum NullsAre { + #[default] + #[display("LARGEST")] + Largest, + #[display("SMALLEST")] + Smallest, +} + +#[derive(PartialEq, Eq, Hash, Copy, Clone, Debug, Default)] +pub struct OrderType { + direction: Direction, + // TODO(rc): enable `NULLS FIRST | LAST` + // nulls_are: NullsAre, +} + +impl OrderType { + pub fn from_protobuf(order_type: &PbOrderType) -> OrderType { + OrderType { + direction: Direction::from_protobuf(&order_type.direction()), + } + } + + pub fn to_protobuf(self) -> PbOrderType { + PbOrderType { + direction: self.direction.to_protobuf() as _, + } + } +} + +impl OrderType { + pub const fn new(direction: Direction) -> Self { + Self { direction } + } + + /// Create an ascending order type, with other options set to default. + pub const fn ascending() -> Self { + Self { + direction: Direction::Ascending, + } + } + + /// Create an descending order type, with other options set to default. + pub const fn descending() -> Self { + Self { + direction: Direction::Descending, } } + + /// Get the order direction. + pub fn direction(&self) -> Direction { + self.direction + } } -/// Column index with an order type (ASC or DESC). Used to represent a sort key (`Vec`). +impl fmt::Display for OrderType { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.direction) + } +} + +/// Column index with an order type (ASC or DESC). Used to represent a sort key +/// (`Vec`). /// /// Corresponds to protobuf [`PbColumnOrder`]. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct OrderPair { - pub column_idx: usize, +#[derive(Clone, PartialEq, Eq, Hash)] +pub struct ColumnOrder { + pub column_index: usize, pub order_type: OrderType, } -impl OrderPair { - pub fn new(column_idx: usize, order_type: OrderType) -> Self { +impl ColumnOrder { + pub fn new(column_index: usize, order_type: OrderType) -> Self { Self { - column_idx, + column_index, order_type, } } + /// Shift the column index with offset. + pub fn shift_with_offset(&mut self, offset: isize) { + self.column_index = (self.column_index as isize + offset) as usize; + } +} + +impl ColumnOrder { pub fn from_protobuf(column_order: &PbColumnOrder) -> Self { - OrderPair { - column_idx: column_order.column_index as _, - order_type: OrderType::from_protobuf( - &column_order.get_order_type().unwrap().direction(), - ), + ColumnOrder { + column_index: column_order.column_index as _, + order_type: OrderType::from_protobuf(column_order.get_order_type().unwrap()), } } pub fn to_protobuf(&self) -> PbColumnOrder { PbColumnOrder { - column_index: self.column_idx as _, - order_type: Some(PbOrderType { - direction: self.order_type.to_protobuf() as _, - }), + column_index: self.column_index as _, + order_type: Some(self.order_type.to_protobuf()), } } } +impl fmt::Display for ColumnOrder { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "${} {}", self.column_index, self.order_type) + } +} + +impl fmt::Debug for ColumnOrder { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self) + } +} + +pub struct ColumnOrderDisplay<'a> { + pub column_order: &'a ColumnOrder, + pub input_schema: &'a Schema, +} + +impl fmt::Display for ColumnOrderDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let that = self.column_order; + write!( + f, + "{} {}", + FieldDisplay(self.input_schema.fields.get(that.column_index).unwrap()), + that.order_type + ) + } +} + #[derive(Clone, Debug)] pub struct HeapElem { - pub order_pairs: Arc>, + pub column_orders: Arc>, pub chunk: DataChunk, pub chunk_idx: usize, pub elem_idx: usize, @@ -109,7 +210,7 @@ impl Ord for HeapElem { self.elem_idx, &other.chunk, other.elem_idx, - self.order_pairs.as_ref(), + self.column_orders.as_ref(), ) .unwrap() }; @@ -142,7 +243,7 @@ where (Some(_), None) => Ordering::Less, (None, Some(_)) => Ordering::Greater, }; - if *order_type == OrderType::Descending { + if order_type.direction == Direction::Descending { ord.reverse() } else { ord @@ -172,15 +273,15 @@ pub fn compare_rows_in_chunk( lhs_idx: usize, rhs_data_chunk: &DataChunk, rhs_idx: usize, - order_pairs: &[OrderPair], + column_orders: &[ColumnOrder], ) -> Result { - for order_pair in order_pairs.iter() { - let lhs_array = lhs_data_chunk.column_at(order_pair.column_idx).array(); - let rhs_array = rhs_data_chunk.column_at(order_pair.column_idx).array(); + for column_order in column_orders.iter() { + let lhs_array = lhs_data_chunk.column_at(column_order.column_index).array(); + let rhs_array = rhs_data_chunk.column_at(column_order.column_index).array(); macro_rules! gen_match { ( $( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { match (lhs_array.as_ref(), rhs_array.as_ref()) { - $((ArrayImpl::$variant_name(lhs_inner), ArrayImpl::$variant_name(rhs_inner)) => Ok(compare_values_in_array(lhs_inner, lhs_idx, rhs_inner, rhs_idx, &order_pair.order_type)),)* + $((ArrayImpl::$variant_name(lhs_inner), ArrayImpl::$variant_name(rhs_inner)) => Ok(compare_values_in_array(lhs_inner, lhs_idx, rhs_inner, rhs_idx, &column_order.order_type)),)* (l_arr, r_arr) => Err(InternalError(format!("Unmatched array types, lhs array is: {}, rhs array is: {}", l_arr.get_ident(), r_arr.get_ident()))), }? } @@ -199,7 +300,7 @@ mod tests { use itertools::Itertools; - use super::{OrderPair, OrderType}; + use super::{ColumnOrder, OrderType}; use crate::array::{DataChunk, ListValue, StructValue}; use crate::row::{OwnedRow, Row}; use crate::types::{DataType, ScalarImpl}; @@ -220,18 +321,18 @@ mod tests { &[row1, row2], &[DataType::Int32, DataType::Varchar, DataType::Float32], ); - let order_pairs = vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Descending), + let column_orders = vec![ + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::descending()), ]; assert_eq!( Ordering::Equal, - compare_rows_in_chunk(&chunk, 0, &chunk, 0, &order_pairs).unwrap() + compare_rows_in_chunk(&chunk, 0, &chunk, 0, &column_orders).unwrap() ); assert_eq!( Ordering::Less, - compare_rows_in_chunk(&chunk, 0, &chunk, 1, &order_pairs).unwrap() + compare_rows_in_chunk(&chunk, 0, &chunk, 1, &column_orders).unwrap() ); } @@ -282,8 +383,8 @@ mod tests { ]))), ]); - let order_pairs = (0..row1.len()) - .map(|i| OrderPair::new(i, OrderType::Ascending)) + let column_orders = (0..row1.len()) + .map(|i| ColumnOrder::new(i, OrderType::ascending())) .collect_vec(); let chunk = DataChunk::from_rows( @@ -309,11 +410,11 @@ mod tests { ); assert_eq!( Ordering::Equal, - compare_rows_in_chunk(&chunk, 0, &chunk, 0, &order_pairs).unwrap() + compare_rows_in_chunk(&chunk, 0, &chunk, 0, &column_orders).unwrap() ); assert_eq!( Ordering::Less, - compare_rows_in_chunk(&chunk, 0, &chunk, 1, &order_pairs).unwrap() + compare_rows_in_chunk(&chunk, 0, &chunk, 1, &column_orders).unwrap() ); } } diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index 9224c4433095..68da7aa01c14 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -38,7 +38,7 @@ use risingwave_common::test_prelude::DataChunkTestExt; use risingwave_common::types::{DataType, IntoOrdered}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_hummock_sdk::to_committed_batch_query_epoch; use risingwave_pb::catalog::StreamSourceInfo; use risingwave_pb::plan_common::RowFormatType as ProstRowFormatType; @@ -203,7 +203,7 @@ async fn test_table_materialize() -> StreamResult<()> { Box::new(row_id_gen_executor), memory_state_store.clone(), table_id, - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], all_column_ids.clone(), 4, Arc::new(AtomicU64::new(0)), @@ -251,7 +251,7 @@ async fn test_table_materialize() -> StreamResult<()> { memory_state_store.clone(), table_id, column_descs.clone(), - vec![OrderType::Ascending], + vec![OrderType::ascending()], vec![0], value_indices, ); @@ -436,7 +436,7 @@ async fn test_row_seq_scan() -> Result<()> { memory_state_store.clone(), TableId::from(0x42), column_descs.clone(), - vec![OrderType::Ascending], + vec![OrderType::ascending()], vec![0_usize], ) .await; @@ -444,7 +444,7 @@ async fn test_row_seq_scan() -> Result<()> { memory_state_store.clone(), TableId::from(0x42), column_descs.clone(), - vec![OrderType::Ascending], + vec![OrderType::ascending()], vec![0], vec![0, 1, 2], ); diff --git a/src/connector/src/sink/catalog/desc.rs b/src/connector/src/sink/catalog/desc.rs index 4a2a84ab9b67..7b975081c02a 100644 --- a/src/connector/src/sink/catalog/desc.rs +++ b/src/connector/src/sink/catalog/desc.rs @@ -16,7 +16,7 @@ use std::collections::BTreeMap; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, DatabaseId, SchemaId, TableId, UserId}; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::plan_common::ColumnDesc as ProstColumnDesc; use risingwave_pb::stream_plan::SinkDesc as ProstSinkDesc; @@ -38,7 +38,7 @@ pub struct SinkDesc { /// Primiary keys of the sink (connector). Now the sink does not care about a field's /// order (ASC/DESC). - pub pk: Vec, + pub pk: Vec, /// Primary key indices of the corresponding sink operator's output. pub stream_key: Vec, diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index 1e663b9c8240..d34e9c612f1a 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -18,7 +18,7 @@ use std::collections::HashMap; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, DatabaseId, SchemaId, TableId, UserId}; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::{Sink as ProstSink, SinkType as ProstSinkType}; #[derive(Clone, Copy, Debug, Default, Hash, PartialOrd, PartialEq, Eq)] @@ -115,7 +115,7 @@ pub struct SinkCatalog { /// Primiary keys of the sink (connector). Now the sink does not care about a field's /// order (ASC/DESC). - pub pk: Vec, + pub pk: Vec, /// Primary key indices of the corresponding sink operator's output. pub stream_key: Vec, @@ -181,7 +181,7 @@ impl From for SinkCatalog { .into_iter() .map(ColumnCatalog::from) .collect_vec(), - pk: pb.pk.iter().map(OrderPair::from_protobuf).collect_vec(), + pk: pb.pk.iter().map(ColumnOrder::from_protobuf).collect_vec(), stream_key: pb.stream_key.iter().map(|k| *k as _).collect_vec(), distribution_key: pb.distribution_key.iter().map(|k| *k as _).collect_vec(), properties: pb.properties.clone(), diff --git a/src/ctl/src/cmd_impl/table/scan.rs b/src/ctl/src/cmd_impl/table/scan.rs index 8f61089b582b..5a77b942ebac 100644 --- a/src/ctl/src/cmd_impl/table/scan.rs +++ b/src/ctl/src/cmd_impl/table/scan.rs @@ -63,8 +63,8 @@ pub async fn make_state_table(hummock: S, table: &TableCatalog) - .iter() .map(|x| x.column_desc.clone()) .collect(), - table.pk().iter().map(|x| x.direct.to_order()).collect(), - table.pk().iter().map(|x| x.index).collect(), + table.pk().iter().map(|x| x.order_type).collect(), + table.pk().iter().map(|x| x.column_index).collect(), Distribution::all_vnodes(table.distribution_key().to_vec()), // scan all vnodes Some(table.value_indices.clone()), ) @@ -85,8 +85,8 @@ pub fn make_storage_table(hummock: S, table: &TableCatalog) -> St .iter() .map(|x| x.column_desc.column_id) .collect(), - table.pk().iter().map(|x| x.direct.to_order()).collect(), - table.pk().iter().map(|x| x.index).collect(), + table.pk().iter().map(|x| x.order_type).collect(), + table.pk().iter().map(|x| x.column_index).collect(), Distribution::all_vnodes(table.distribution_key().to_vec()), TableOption::build_table_option(&HashMap::new()), table.value_indices.clone(), diff --git a/src/expr/src/vector_op/agg/aggregator.rs b/src/expr/src/vector_op/agg/aggregator.rs index 5e932dfe0d80..73b589b67831 100644 --- a/src/expr/src/vector_op/agg/aggregator.rs +++ b/src/expr/src/vector_op/agg/aggregator.rs @@ -18,7 +18,7 @@ use dyn_clone::DynClone; use risingwave_common::array::*; use risingwave_common::bail; use risingwave_common::types::*; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::expr::AggCall; use crate::expr::{build_from_prost, AggKind}; @@ -70,16 +70,15 @@ impl AggStateFactory { let return_type = DataType::from(prost.get_return_type()?); let agg_kind = AggKind::try_from(prost.get_type()?)?; let distinct = prost.distinct; - let order_pairs = prost + let column_orders = prost .get_order_by() .iter() .map(|col_order| { let col_idx = col_order.get_column_index() as usize; - let order_type = - OrderType::from_protobuf(&col_order.get_order_type().unwrap().direction()); + let order_type = OrderType::from_protobuf(col_order.get_order_type().unwrap()); // TODO(yuchao): `nulls first/last` is not supported yet, so it's ignore here, // see also `risingwave_common::util::sort_util::compare_values` - OrderPair::new(col_idx, order_type) + ColumnOrder::new(col_idx, order_type) }) .collect(); @@ -100,11 +99,11 @@ impl AggStateFactory { ); let agg_col_idx = agg_arg.get_index() as usize; let delim_col_idx = delim_arg.get_index() as usize; - create_string_agg_state(agg_col_idx, delim_col_idx, order_pairs)? + create_string_agg_state(agg_col_idx, delim_col_idx, column_orders)? } (AggKind::ArrayAgg, [arg]) => { let agg_col_idx = arg.get_index() as usize; - create_array_agg_state(return_type.clone(), agg_col_idx, order_pairs)? + create_array_agg_state(return_type.clone(), agg_col_idx, column_orders)? } (agg_kind, [arg]) => { // other unary agg call diff --git a/src/expr/src/vector_op/agg/array_agg.rs b/src/expr/src/vector_op/agg/array_agg.rs index 175c7c856410..4910e009362e 100644 --- a/src/expr/src/vector_op/agg/array_agg.rs +++ b/src/expr/src/vector_op/agg/array_agg.rs @@ -17,7 +17,7 @@ use risingwave_common::bail; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::{DataType, Datum, Scalar, ToOwnedDatum}; use risingwave_common::util::ordered::OrderedRow; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use crate::vector_op::agg::aggregator::Aggregator; use crate::Result; @@ -100,11 +100,11 @@ struct ArrayAggOrdered { } impl ArrayAggOrdered { - fn new(return_type: DataType, agg_col_idx: usize, order_pairs: Vec) -> Self { + fn new(return_type: DataType, agg_col_idx: usize, column_orders: Vec) -> Self { debug_assert!(matches!(return_type, DataType::List { datatype: _ })); - let (order_col_indices, order_types) = order_pairs + let (order_col_indices, order_types) = column_orders .into_iter() - .map(|p| (p.column_idx, p.order_type)) + .map(|p| (p.column_index, p.order_type)) .unzip(); ArrayAggOrdered { return_type, @@ -169,15 +169,15 @@ impl Aggregator for ArrayAggOrdered { pub fn create_array_agg_state( return_type: DataType, agg_col_idx: usize, - order_pairs: Vec, + column_orders: Vec, ) -> Result> { - if order_pairs.is_empty() { + if column_orders.is_empty() { Ok(Box::new(ArrayAggUnordered::new(return_type, agg_col_idx))) } else { Ok(Box::new(ArrayAggOrdered::new( return_type, agg_col_idx, - order_pairs, + column_orders, ))) } } @@ -274,8 +274,8 @@ mod tests { return_type.clone(), 0, vec![ - OrderPair::new(1, OrderType::Ascending), - OrderPair::new(0, OrderType::Descending), + ColumnOrder::new(1, OrderType::ascending()), + ColumnOrder::new(0, OrderType::descending()), ], )?; let mut builder = return_type.create_array_builder(0); diff --git a/src/expr/src/vector_op/agg/string_agg.rs b/src/expr/src/vector_op/agg/string_agg.rs index a607fe096a67..046d3288e86a 100644 --- a/src/expr/src/vector_op/agg/string_agg.rs +++ b/src/expr/src/vector_op/agg/string_agg.rs @@ -20,7 +20,7 @@ use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::ordered::OrderedRow; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use crate::vector_op::agg::aggregator::Aggregator; use crate::Result; @@ -128,10 +128,10 @@ struct StringAggOrdered { } impl StringAggOrdered { - fn new(agg_col_idx: usize, delim_col_idx: usize, order_pairs: Vec) -> Self { - let (order_col_indices, order_types) = order_pairs + fn new(agg_col_idx: usize, delim_col_idx: usize, column_orders: Vec) -> Self { + let (order_col_indices, order_types) = column_orders .into_iter() - .map(|p| (p.column_idx, p.order_type)) + .map(|p| (p.column_index, p.order_type)) .unzip(); Self { agg_col_idx, @@ -237,9 +237,9 @@ impl Aggregator for StringAggOrdered { pub fn create_string_agg_state( agg_col_idx: usize, delim_col_idx: usize, - order_pairs: Vec, + column_orders: Vec, ) -> Result> { - if order_pairs.is_empty() { + if column_orders.is_empty() { Ok(Box::new(StringAggUnordered::new( agg_col_idx, delim_col_idx, @@ -248,7 +248,7 @@ pub fn create_string_agg_state( Ok(Box::new(StringAggOrdered::new( agg_col_idx, delim_col_idx, - order_pairs, + column_orders, ))) } } @@ -256,7 +256,7 @@ pub fn create_string_agg_state( #[cfg(test)] mod tests { use risingwave_common::array::{DataChunk, DataChunkTestExt, Utf8ArrayBuilder}; - use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use super::*; @@ -315,9 +315,9 @@ mod tests { 1, 0, vec![ - OrderPair::new(2, OrderType::Ascending), - OrderPair::new(3, OrderType::Descending), - OrderPair::new(1, OrderType::Descending), + ColumnOrder::new(2, OrderType::ascending()), + ColumnOrder::new(3, OrderType::descending()), + ColumnOrder::new(1, OrderType::descending()), ], )?; let mut builder = ArrayBuilderImpl::Utf8(Utf8ArrayBuilder::new(0)); diff --git a/src/frontend/planner_test/tests/testdata/agg.yaml b/src/frontend/planner_test/tests/testdata/agg.yaml index d833613807ca..142273150759 100644 --- a/src/frontend/planner_test/tests/testdata/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/agg.yaml @@ -524,44 +524,6 @@ └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v1)] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } -- name: order by NULLS FIRST/LAST and default - sql: | - create table t(v1 int, v2 varchar, v3 int); - select sum(v1 order by v1, v2 NULLS FIRST, v3 NULLS LAST) as s1 from t; - logical_plan: | - LogicalProject { exprs: [sum(t.v1)] } - └─LogicalAgg { aggs: [sum(t.v1)] } - └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan_for_batch: | - LogicalAgg { aggs: [sum(t.v1)] } - └─LogicalScan { table: t, columns: [t.v1] } - stream_plan: | - StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [sum(sum(t.v1))] } - └─StreamGlobalSimpleAgg { aggs: [sum(sum(t.v1)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v1)] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } -- name: order by complex expressions - sql: | - create table t(v1 int, v2 varchar, v3 int); - select sum(v1 order by v1 + v3 ASC, length(v2) * v3 DESC NULLS FIRST) as s1 from t; - logical_plan: | - LogicalProject { exprs: [sum(t.v1)] } - └─LogicalAgg { aggs: [sum(t.v1)] } - └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } - optimized_logical_plan_for_batch: | - LogicalAgg { aggs: [sum(t.v1)] } - └─LogicalScan { table: t, columns: [t.v1] } - stream_plan: | - StreamMaterialize { columns: [s1], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [sum(sum(t.v1))] } - └─StreamGlobalSimpleAgg { aggs: [sum(sum(t.v1)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [sum(t.v1)] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: filter clause sql: | create table t(v1 int); diff --git a/src/frontend/planner_test/tests/testdata/batch_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/batch_dist_agg.yaml index 36d820523936..6735de3f1456 100644 --- a/src/frontend/planner_test/tests/testdata/batch_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/batch_dist_agg.yaml @@ -66,12 +66,12 @@ sql: | select string_agg(s, ',' order by o) as a1 from T; batch_plan: | - BatchSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.o, t.s], distribution: SomeShard } batch_local_plan: | - BatchSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.o, t.s], distribution: SomeShard } @@ -95,12 +95,12 @@ sql: | select count(v) as a1, string_agg(s, ',' order by o) as a2 from T; batch_plan: | - BatchSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } batch_local_plan: | - BatchSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } @@ -110,12 +110,12 @@ sql: | select max(v) as a1, string_agg(s, ',' order by o) as a2 from T; batch_plan: | - BatchSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } batch_local_plan: | - BatchSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index eb1c2f3ade96..ece902d6b240 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark.yaml @@ -561,7 +561,7 @@ LogicalProject { exprs: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid.auction, bid.bidder, bid.price, bid.date_time] } └─LogicalFilter { predicate: (ROW_NUMBER <= 1:Int32) } └─LogicalProject { exprs: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid.auction, bid.bidder, bid.price, bid.date_time, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY auction.id ORDER BY bid.price DESC NULLS FIRST, bid.date_time ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY auction.id ORDER BY bid.price DESC, bid.date_time ASC) } └─LogicalFilter { predicate: (auction.id = bid.auction) AND (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category] } @@ -899,7 +899,7 @@ LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra] } └─LogicalFilter { predicate: (ROW_NUMBER <= 1:Int32) } └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bid.bidder, bid.auction ORDER BY bid.date_time DESC NULLS FIRST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bid.bidder, bid.auction ORDER BY bid.date_time DESC) } └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id] } batch_plan: | BatchExchange { order: [], dist: Single } @@ -944,13 +944,13 @@ LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, ROW_NUMBER] } └─LogicalFilter { predicate: (ROW_NUMBER <= 10:Int32) } └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bid.auction ORDER BY bid.price DESC NULLS FIRST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bid.auction ORDER BY bid.price DESC) } └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id] } optimizer_error: | internal error: OverAgg can not be transformed. Plan: LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, ROW_NUMBER] } └─LogicalFilter { predicate: (ROW_NUMBER <= 10:Int32) } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bid.auction ORDER BY bid.price DESC NULLS FIRST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bid.auction ORDER BY bid.price DESC) } └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id] } - id: nexmark_q20 before: diff --git a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml index 56d83b4215be..773569f9982b 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -640,7 +640,7 @@ LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time] } └─LogicalFilter { predicate: (ROW_NUMBER <= 1:Int32) } └─LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY id ORDER BY price DESC NULLS FIRST, date_time ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY id ORDER BY price DESC, date_time ASC) } └─LogicalFilter { predicate: (id = auction) AND (date_time >= date_time) AND (date_time <= expires) } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } @@ -1000,7 +1000,7 @@ LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } └─LogicalFilter { predicate: (ROW_NUMBER <= 1:Int32) } └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bidder, auction ORDER BY date_time DESC NULLS FIRST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY bidder, auction ORDER BY date_time DESC) } └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: [(Unbounded, Unbounded)] } batch_plan: | BatchExchange { order: [], dist: Single } @@ -1047,13 +1047,13 @@ LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, ROW_NUMBER] } └─LogicalFilter { predicate: (ROW_NUMBER <= 10:Int32) } └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY auction ORDER BY price DESC NULLS FIRST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY auction ORDER BY price DESC) } └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: [(Unbounded, Unbounded)] } optimizer_error: | internal error: OverAgg can not be transformed. Plan: LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, ROW_NUMBER] } └─LogicalFilter { predicate: (ROW_NUMBER <= 10:Int32) } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY auction ORDER BY price DESC NULLS FIRST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY auction ORDER BY price DESC) } └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: [(Unbounded, Unbounded)] } - id: nexmark_q20 before: diff --git a/src/frontend/planner_test/tests/testdata/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/over_window_function.yaml index 7fc0a70c697d..f830ff4ad45b 100644 --- a/src/frontend/planner_test/tests/testdata/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/over_window_function.yaml @@ -18,7 +18,7 @@ select row_number() over(PARTITION BY x ORDER BY x) from t; logical_plan: | LogicalProject { exprs: [ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t._row_id] } - sql: | create table t(x int); @@ -36,7 +36,7 @@ create table t(x int); select 1+row_number() over(ORDER BY x) from t; planner_error: |- - Feature is not yet implemented: window function in expression: (1:Int32 + ROW_NUMBER() OVER(ORDER BY $0)) + Feature is not yet implemented: window function in expression: (1:Int32 + ROW_NUMBER() OVER(ORDER BY $0 ASC)) No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml - sql: | create table t(x int); @@ -44,7 +44,7 @@ logical_plan: | LogicalProject { exprs: [t.x] } └─LogicalProject { exprs: [t.x, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t._row_id] } - sql: | create table t(x int); @@ -72,13 +72,13 @@ LogicalProject { exprs: [t.x, ROW_NUMBER] } └─LogicalFilter { predicate: (ROW_NUMBER < 3:Int32) } └─LogicalProject { exprs: [t.x, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t._row_id] } optimizer_error: | internal error: OverAgg can not be transformed. Plan: LogicalProject { exprs: [t.x, ROW_NUMBER] } └─LogicalFilter { predicate: (ROW_NUMBER < 3:Int32) } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t._row_id] } - name: TopN without rank output sql: | @@ -90,7 +90,7 @@ LogicalProject { exprs: [t.x, t.y] } └─LogicalFilter { predicate: (ROW_NUMBER < 3:Int32) AND (t.x > t.y) } └─LogicalProject { exprs: [t.x, t.y, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.y ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.y ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } @@ -148,7 +148,7 @@ internal error: OverAgg can not be transformed. Plan: LogicalProject { exprs: [t.x, t.y] } └─LogicalFilter { predicate: (RANK <= 3:Int32) AND (RANK > 1:Int32) } - └─LogicalOverAgg { window_function: RANK() OVER(PARTITION BY t.y ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: RANK() OVER(PARTITION BY t.y ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } - sql: | create table t(x int, y int); @@ -241,7 +241,7 @@ internal error: OverAgg can not be transformed. Plan: LogicalProject { exprs: [t.x, t.y] } └─LogicalFilter { predicate: (3:Int32 < ROW_NUMBER) AND (ROW_NUMBER = 6:Int32) AND (ROW_NUMBER <= 5:Int32) } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.y ORDER BY t.x ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.y ORDER BY t.x ASC) } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } - id: create_bid sql: | @@ -324,7 +324,7 @@ LogicalProject { exprs: [t.x, t.y] } └─LogicalFilter { predicate: (ROW_NUMBER = 1:Int32) } └─LogicalProject { exprs: [t.x, t.y, ROW_NUMBER] } - └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.y ASC NULLS LAST) } + └─LogicalOverAgg { window_function: ROW_NUMBER() OVER(PARTITION BY t.x ORDER BY t.y ASC) } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } optimized_logical_plan_for_batch: | LogicalProject { exprs: [t.x, t.y] } diff --git a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml index 9863b74b79d7..cca0a2fa15ed 100644 --- a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml @@ -95,21 +95,21 @@ select string_agg(s, ',' order by v) as a1 from S; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } + └─BatchSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } └─BatchProject { exprs: [s.s, ',':Varchar, s.v] } └─BatchScan { table: s, columns: [s.v, s.s], distribution: Single } stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } - └─StreamGlobalSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } + └─StreamGlobalSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC)), count] } └─StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } └─StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST))] } - StreamGlobalSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } + StreamGlobalSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC)), count] } result table: 1, state tables: [0] StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } Chain { table: s, columns: [s.v, s.s, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } @@ -117,7 +117,7 @@ BatchPlanNode Table 0 { columns: [s_v, s_t__row_id, s_s, ',':Varchar], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [string_agg(s_s, ',':Varchar order_by(s_v ASC NULLS LAST)), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 1 { columns: [string_agg(s_s, ',':Varchar order_by(s_v ASC)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: extreme_on_T before: @@ -320,14 +320,14 @@ sql: | select string_agg(s, ',' order by o) as a1 from T; batch_plan: | - BatchSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.o, t.s], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - └─StreamGlobalSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └─StreamGlobalSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.s, ',':Varchar, t.o, t._row_id] } └─StreamTableScan { table: t, columns: [t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -335,8 +335,8 @@ Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - StreamGlobalSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + StreamGlobalSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } result table: 1, state tables: [0] StreamExchange Single from 1 @@ -347,7 +347,7 @@ BatchPlanNode Table 0 { columns: [t_o, t__row_id, t_s, ',':Varchar], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 1 { columns: [string_agg(t_s, ',':Varchar order_by(t_o ASC)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: string_agg_on_AO before: @@ -356,8 +356,8 @@ select string_agg(s, ',' order by o) as a1 from AO; stream_plan: | StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [ao.s, ',':Varchar, ao.o, ao._row_id] } └─StreamTableScan { table: ao, columns: [ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } @@ -365,8 +365,8 @@ Fragment 0 StreamMaterialize { columns: [a1], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -376,7 +376,7 @@ Upstream BatchPlanNode - Table 0 { columns: [string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: extreme_count_on_T before: @@ -453,14 +453,14 @@ sql: | select count(v) as a1, string_agg(s, ',' order by o) as a2 from T; batch_plan: | - BatchSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - └─StreamGlobalSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └─StreamGlobalSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.v, t.s, ',':Varchar, t.o, t._row_id] } └─StreamTableScan { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -468,8 +468,8 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - StreamGlobalSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + StreamGlobalSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } result table: 1, state tables: [0] StreamExchange Single from 1 @@ -480,7 +480,7 @@ BatchPlanNode Table 0 { columns: [t_o, t__row_id, t_s, ',':Varchar], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [count(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 1 { columns: [count(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: count_string_agg_on_AO before: @@ -489,8 +489,8 @@ select count(v) as a1, string_agg(s, ',' order by o) as a2 from AO; stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [ao.v, ao.s, ',':Varchar, ao.o, ao._row_id] } └─StreamTableScan { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } @@ -498,8 +498,8 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -509,7 +509,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_string_agg_on_T before: @@ -517,14 +517,14 @@ sql: | select max(v) as a1, string_agg(s, ',' order by o) as a2 from T; batch_plan: | - BatchSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + BatchSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - └─StreamGlobalSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └─StreamGlobalSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.v, t.s, ',':Varchar, t.o, t._row_id] } └─StreamTableScan { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -532,8 +532,8 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - StreamGlobalSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + StreamGlobalSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } result table: 2, state tables: [0, 1] StreamExchange Single from 1 @@ -545,7 +545,7 @@ Table 0 { columns: [t_v, t__row_id], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [t_o, t__row_id, t_s, ',':Varchar], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 2 { columns: [max(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 2 { columns: [max(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_string_agg_on_AO before: @@ -554,8 +554,8 @@ select max(v) as a1, string_agg(s, ',' order by o) as a2 from AO; stream_plan: | StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } - └─StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - └─StreamAppendOnlyGlobalSimpleAgg { aggs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [ao.v, ao.s, ',':Varchar, ao.o, ao._row_id] } └─StreamTableScan { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } @@ -563,8 +563,8 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], pk_columns: [], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST))] } - StreamAppendOnlyGlobalSimpleAgg { aggs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + StreamAppendOnlyGlobalSimpleAgg { aggs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -574,7 +574,7 @@ Upstream BatchPlanNode - Table 0 { columns: [max(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [max(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_on_T_by_k before: @@ -949,15 +949,15 @@ select string_agg(s, ',' order by o) as a1 from T group by k; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } - └─BatchHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST))] } + └─BatchProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └─BatchHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } └─BatchExchange { order: [], dist: HashShard(t.k) } └─BatchProject { exprs: [t.k, t.s, ',':Varchar, t.o] } └─BatchScan { table: t, columns: [t.k, t.o, t.s], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), t.k] } - └─StreamHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), t.k] } + └─StreamHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } └─StreamExchange { dist: HashShard(t.k) } └─StreamProject { exprs: [t.k, t.s, ',':Varchar, t.o, t._row_id] } └─StreamTableScan { table: t, columns: [t.k, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -965,8 +965,8 @@ Fragment 0 StreamMaterialize { columns: [a1, t.k(hidden)], pk_columns: [t.k], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), t.k] } - StreamHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), t.k] } + StreamHashAgg { group_key: [t.k], aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } result table: 1, state tables: [0] StreamExchange Hash([0]) from 1 @@ -977,7 +977,7 @@ BatchPlanNode Table 0 { columns: [t_k, t_o, t__row_id, t_s, ',':Varchar], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2, 3, 4], distribution key: [0] } - Table 1 { columns: [t_k, string_agg(t_s, ',':Varchar order_by(t_o ASC NULLS LAST)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [t_k, string_agg(t_s, ',':Varchar order_by(t_o ASC)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, t.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: string_agg_on_Tk_by_k before: @@ -986,22 +986,22 @@ select string_agg(s, ',' order by o) as a1 from Tk group by k; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST))] } - └─BatchSortAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST))] } + └─BatchProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC))] } + └─BatchSortAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC))] } └─BatchProject { exprs: [tk.k, tk.s, ',':Varchar, tk.o] } └─BatchScan { table: tk, columns: [tk.k, tk.o, tk.s], distribution: UpstreamHashShard(tk.k) } stream_plan: | StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), tk.k] } - └─StreamHashAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC)), tk.k] } + └─StreamHashAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC)), count] } └─StreamProject { exprs: [tk.k, tk.s, ',':Varchar, tk.o, tk.t._row_id] } └─StreamTableScan { table: tk, columns: [tk.k, tk.o, tk.s, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [a1, tk.k(hidden)], pk_columns: [tk.k], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), tk.k] } - StreamHashAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC)), tk.k] } + StreamHashAgg { group_key: [tk.k], aggs: [string_agg(tk.s, ',':Varchar order_by(tk.o ASC)), count] } result table: 1, state tables: [0] StreamProject { exprs: [tk.k, tk.s, ',':Varchar, tk.o, tk.t._row_id] } Chain { table: tk, columns: [tk.k, tk.o, tk.s, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } @@ -1009,7 +1009,7 @@ BatchPlanNode Table 0 { columns: [tk_k, tk_o, tk_t__row_id, tk_s, ',':Varchar], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2, 3, 4], distribution key: [0] } - Table 1 { columns: [tk_k, string_agg(tk_s, ',':Varchar order_by(tk_o ASC NULLS LAST)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [tk_k, string_agg(tk_s, ',':Varchar order_by(tk_o ASC)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, tk.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: string_agg_on_S_by_k before: @@ -1018,15 +1018,15 @@ select string_agg(s, ',' order by o) as a1 from S group by k; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST))] } - └─BatchHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST))] } + └─BatchProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC))] } + └─BatchHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC))] } └─BatchExchange { order: [], dist: HashShard(s.k) } └─BatchProject { exprs: [s.k, s.s, ',':Varchar, s.o] } └─BatchScan { table: s, columns: [s.k, s.o, s.s], distribution: Single } stream_plan: | StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), s.k] } - └─StreamHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC)), s.k] } + └─StreamHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC)), count] } └─StreamExchange { dist: HashShard(s.k) } └─StreamProject { exprs: [s.k, s.s, ',':Varchar, s.o, s.t._row_id] } └─StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], pk: [s.t._row_id], dist: Single } @@ -1034,8 +1034,8 @@ Fragment 0 StreamMaterialize { columns: [a1, s.k(hidden)], pk_columns: [s.k], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), s.k] } - StreamHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC)), s.k] } + StreamHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC)), count] } result table: 1, state tables: [0] StreamExchange Hash([0]) from 1 @@ -1046,7 +1046,7 @@ BatchPlanNode Table 0 { columns: [s_k, s_o, s_t__row_id, s_s, ',':Varchar], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [1, 2, 3, 4], distribution key: [0] } - Table 1 { columns: [s_k, string_agg(s_s, ',':Varchar order_by(s_o ASC NULLS LAST)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 1 { columns: [s_k, string_agg(s_s, ',':Varchar order_by(s_o ASC)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, s.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } - id: string_agg_on_AO_by_k before: @@ -1055,8 +1055,8 @@ select string_agg(s, ',' order by o) as a1 from AO group by k; stream_plan: | StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } - └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), ao.k] } - └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), ao.k] } + └─StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } └─StreamExchange { dist: HashShard(ao.k) } └─StreamProject { exprs: [ao.k, ao.s, ',':Varchar, ao.o, ao._row_id] } └─StreamTableScan { table: ao, columns: [ao.k, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } @@ -1064,8 +1064,8 @@ Fragment 0 StreamMaterialize { columns: [a1, ao.k(hidden)], pk_columns: [ao.k], pk_conflict: "no check" } materialized table: 4294967294 - StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), ao.k] } - StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC NULLS LAST)), count] } + StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), ao.k] } + StreamAppendOnlyHashAgg { group_key: [ao.k], aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 @@ -1075,5 +1075,5 @@ Upstream BatchPlanNode - Table 0 { columns: [ao_k, string_agg(ao_s, ',':Varchar order_by(ao_o ASC NULLS LAST)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [ao_k, string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 4294967294 { columns: [a1, ao.k], primary key: [$1 ASC], value indices: [0, 1], distribution key: [1] } diff --git a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml index 54af04a3ea8c..d45221df79b6 100644 --- a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml @@ -518,9 +518,9 @@ select count(*) from a where a.y = (select string_agg(x, ',' order by x) from b where b.z = a.z); optimized_logical_plan_for_batch: | LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND (a.y = string_agg(b.x, ',':Varchar order_by(b.x ASC NULLS LAST))), output: [] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND (a.y = string_agg(b.x, ',':Varchar order_by(b.x ASC))), output: [] } ├─LogicalScan { table: a, columns: [a.y, a.z] } - └─LogicalAgg { group_key: [a.z], aggs: [string_agg(b.x, ',':Varchar order_by(b.x ASC NULLS LAST))] } + └─LogicalAgg { group_key: [a.z], aggs: [string_agg(b.x, ',':Varchar order_by(b.x ASC))] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(a.z, b.z), output: [a.z, b.x, ',':Varchar] } ├─LogicalAgg { group_key: [a.z], aggs: [] } | └─LogicalScan { table: a, columns: [a.z] } @@ -728,9 +728,9 @@ CREATE TABLE strings(v1 VARCHAR); SELECT (SELECT STRING_AGG(v1, ',' ORDER BY t.v1) FROM strings) FROM strings AS t; optimized_logical_plan_for_batch: | - LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(strings.v1, strings.v1), output: [string_agg(strings.v1, ',':Varchar order_by(strings.v1 ASC NULLS LAST))] } + LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(strings.v1, strings.v1), output: [string_agg(strings.v1, ',':Varchar order_by(strings.v1 ASC))] } ├─LogicalScan { table: strings, columns: [strings.v1] } - └─LogicalAgg { group_key: [strings.v1], aggs: [string_agg(strings.v1, ',':Varchar order_by(strings.v1 ASC NULLS LAST))] } + └─LogicalAgg { group_key: [strings.v1], aggs: [string_agg(strings.v1, ',':Varchar order_by(strings.v1 ASC))] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(strings.v1, strings.v1), output: [strings.v1, strings.v1, ',':Varchar, strings.v1] } ├─LogicalAgg { group_key: [strings.v1], aggs: [] } | └─LogicalScan { table: strings, columns: [strings.v1] } diff --git a/src/frontend/src/binder/expr/order_by.rs b/src/frontend/src/binder/expr/order_by.rs index f1c848efc24f..cdf062cae76c 100644 --- a/src/frontend/src/binder/expr/order_by.rs +++ b/src/frontend/src/binder/expr/order_by.rs @@ -12,11 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; +use risingwave_common::error::{ErrorCode, Result}; +use risingwave_common::util::sort_util::OrderType; use risingwave_sqlparser::ast::OrderByExpr; use crate::expr::OrderByExpr as BoundOrderByExpr; -use crate::optimizer::property::Direction; use crate::Binder; impl Binder { @@ -34,23 +34,20 @@ impl Binder { nulls_first, }: OrderByExpr, ) -> Result { - let direction = match asc { - None | Some(true) => Direction::Asc, - Some(false) => Direction::Desc, + // TODO(rc): support `NULLS FIRST | LAST` + if nulls_first.is_some() { + return Err(ErrorCode::NotImplemented( + "NULLS FIRST or NULLS LAST".to_string(), + 4743.into(), + ) + .into()); + } + let order_type = match asc { + None => OrderType::default(), + Some(true) => OrderType::ascending(), + Some(false) => OrderType::descending(), }; - - let nulls_first = nulls_first.unwrap_or_else(|| match direction { - Direction::Asc => false, - Direction::Desc => true, - Direction::Any => unreachable!(), - }); - let expr = self.bind_expr(expr)?; - - Ok(BoundOrderByExpr { - expr, - direction, - nulls_first, - }) + Ok(BoundOrderByExpr { expr, order_type }) } } diff --git a/src/frontend/src/binder/query.rs b/src/frontend/src/binder/query.rs index f1d2c4573a4e..5daccb317fb7 100644 --- a/src/frontend/src/binder/query.rs +++ b/src/frontend/src/binder/query.rs @@ -18,18 +18,18 @@ use std::rc::Rc; use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_sqlparser::ast::{Cte, Expr, Fetch, OrderByExpr, Query, Value, With}; use crate::binder::{Binder, BoundSetExpr}; use crate::expr::{CorrelatedId, Depth, ExprImpl}; -use crate::optimizer::property::{Direction, FieldOrder}; /// A validated sql query, including order and union. /// An example of its relationship with `BoundSetExpr` and `BoundSelect` can be found here: #[derive(Debug, Clone)] pub struct BoundQuery { pub body: BoundSetExpr, - pub order: Vec, + pub order: Vec, pub limit: Option, pub offset: Option, pub with_ties: bool, @@ -207,7 +207,8 @@ impl Binder { name_to_index: &HashMap, extra_order_exprs: &mut Vec, visible_output_num: usize, - ) -> Result { + ) -> Result { + // TODO(rc): support `NULLS FIRST | LAST` if nulls_first.is_some() { return Err(ErrorCode::NotImplemented( "NULLS FIRST or NULLS LAST".to_string(), @@ -215,11 +216,12 @@ impl Binder { ) .into()); } - let direct = match asc { - None | Some(true) => Direction::Asc, - Some(false) => Direction::Desc, + let order_type = match asc { + None => OrderType::default(), + Some(true) => OrderType::ascending(), + Some(false) => OrderType::descending(), }; - let index = match expr { + let column_index = match expr { Expr::Identifier(name) if let Some(index) = name_to_index.get(&name.real_value()) => match *index != usize::MAX { true => *index, false => return Err(ErrorCode::BindError(format!("ORDER BY \"{}\" is ambiguous", name.real_value())).into()), @@ -239,7 +241,7 @@ impl Binder { visible_output_num + extra_order_exprs.len() - 1 } }; - Ok(FieldOrder { index, direct }) + Ok(ColumnOrder::new(column_index, order_type)) } fn bind_with(&mut self, with: With) -> Result<()> { diff --git a/src/frontend/src/catalog/index_catalog.rs b/src/frontend/src/catalog/index_catalog.rs index 0e5837a20d2c..2feec9388e3d 100644 --- a/src/frontend/src/catalog/index_catalog.rs +++ b/src/frontend/src/catalog/index_catalog.rs @@ -18,13 +18,13 @@ use std::sync::Arc; use itertools::Itertools; use risingwave_common::catalog::IndexId; use risingwave_common::types::DataType; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::Index as ProstIndex; use risingwave_pb::expr::expr_node::RexNode; use super::ColumnId; use crate::catalog::{DatabaseId, SchemaId, TableCatalog}; use crate::expr::{Expr, InputRef}; -use crate::optimizer::property::FieldOrder; #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct IndexCatalog { @@ -98,16 +98,13 @@ impl IndexCatalog { } } - pub fn primary_table_pk_ref_to_index_table(&self) -> Vec { + pub fn primary_table_pk_ref_to_index_table(&self) -> Vec { let mapping = self.primary_to_secondary_mapping(); self.primary_table .pk .iter() - .map(|x| FieldOrder { - index: *mapping.get(&x.index).unwrap(), - direct: x.direct, - }) + .map(|x| ColumnOrder::new(*mapping.get(&x.column_index).unwrap(), x.order_type)) .collect_vec() } diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 7c9e5ca29894..779409d4a125 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -19,13 +19,13 @@ use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, TableDesc, TableId, TableVersionId}; use risingwave_common::constants::hummock::TABLE_OPTION_DUMMY_RETENTION_SECOND; use risingwave_common::error::{ErrorCode, RwError}; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::table::{ OptionalAssociatedSourceId, TableType as ProstTableType, TableVersion as ProstTableVersion, }; use risingwave_pb::catalog::Table as ProstTable; use super::{ColumnId, ConflictBehaviorType, DatabaseId, FragmentId, RelationCatalog, SchemaId}; -use crate::optimizer::property::FieldOrder; use crate::user::UserId; use crate::WithOptions; @@ -75,7 +75,7 @@ pub struct TableCatalog { pub columns: Vec, /// Key used as materialize's storage key prefix, including MV order columns and stream_key. - pub pk: Vec, + pub pk: Vec, /// pk_indices of the corresponding materialize operator's output. pub stream_key: Vec, @@ -270,7 +270,7 @@ impl TableCatalog { } /// Get a reference to the table catalog's pk desc. - pub fn pk(&self) -> &[FieldOrder] { + pub fn pk(&self) -> &[ColumnOrder] { self.pk.as_ref() } @@ -278,7 +278,7 @@ impl TableCatalog { pub fn pk_column_ids(&self) -> Vec { self.pk .iter() - .map(|f| self.columns[f.index].column_id()) + .map(|x| self.columns[x.column_index].column_id()) .collect() } @@ -291,7 +291,7 @@ impl TableCatalog { TableDesc { table_id: self.id, - pk: self.pk.iter().map(FieldOrder::to_order_pair).collect(), + pk: self.pk.clone(), stream_key: self.stream_key.clone(), columns: self.columns.iter().map(|c| c.column_desc.clone()).collect(), distribution_key: self.distribution_key.clone(), @@ -395,7 +395,7 @@ impl From for TableCatalog { col_index.insert(col_id, idx); } - let pk = tb.pk.iter().map(FieldOrder::from_protobuf).collect(); + let pk = tb.pk.iter().map(ColumnOrder::from_protobuf).collect(); let mut watermark_columns = FixedBitSet::with_capacity(columns.len()); for idx in tb.watermark_indices { watermark_columns.insert(idx as _); @@ -455,6 +455,7 @@ mod tests { use risingwave_common::constants::hummock::PROPERTIES_RETENTION_SECOND_KEY; use risingwave_common::test_prelude::*; use risingwave_common::types::*; + use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::Table as ProstTable; use risingwave_pb::plan_common::{ ColumnCatalog as ProstColumnCatalog, ColumnDesc as ProstColumnDesc, @@ -462,7 +463,6 @@ mod tests { use super::*; use crate::catalog::table_catalog::{TableCatalog, TableType}; - use crate::optimizer::property::{Direction, FieldOrder}; use crate::WithOptions; #[test] @@ -499,11 +499,7 @@ mod tests { is_hidden: false, }, ], - pk: vec![FieldOrder { - index: 0, - direct: Direction::Asc, - } - .to_protobuf()], + pk: vec![ColumnOrder::new(0, OrderType::ascending()).to_protobuf()], stream_key: vec![0], dependent_relations: vec![], distribution_key: vec![], @@ -570,10 +566,7 @@ mod tests { } ], stream_key: vec![0], - pk: vec![FieldOrder { - index: 0, - direct: Direction::Asc, - }], + pk: vec![ColumnOrder::new(0, OrderType::ascending())], distribution_key: vec![], append_only: false, owner: risingwave_common::catalog::DEFAULT_SUPER_USER_ID, diff --git a/src/frontend/src/expr/order_by_expr.rs b/src/frontend/src/expr/order_by_expr.rs index 23be49e6b559..5bb76899086d 100644 --- a/src/frontend/src/expr/order_by_expr.rs +++ b/src/frontend/src/expr/order_by_expr.rs @@ -15,9 +15,9 @@ use std::fmt::Display; use itertools::Itertools; +use risingwave_common::util::sort_util::OrderType; use crate::expr::{ExprImpl, ExprMutator, ExprRewriter, ExprVisitor}; -use crate::optimizer::property::Direction; /// A sort expression in the `ORDER BY` clause. /// @@ -25,19 +25,12 @@ use crate::optimizer::property::Direction; #[derive(Clone, Eq, PartialEq, Hash)] pub struct OrderByExpr { pub expr: ExprImpl, - pub direction: Direction, - pub nulls_first: bool, + pub order_type: OrderType, } impl Display for OrderByExpr { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{:?}", self.expr)?; - if self.direction == Direction::Desc { - write!(f, " DESC")?; - } - if self.nulls_first { - write!(f, " NULLS FIRST")?; - } + write!(f, "{:?} {}", self.expr, self.order_type)?; Ok(()) } } @@ -72,8 +65,7 @@ impl OrderBy { .into_iter() .map(|e| OrderByExpr { expr: rewriter.rewrite_expr(e.expr), - direction: e.direction, - nulls_first: e.nulls_first, + order_type: e.order_type, }) .collect(), } diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index 8bd25c8becdd..55143647ab25 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -20,7 +20,7 @@ use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::catalog::{IndexId, TableDesc, TableId}; use risingwave_common::error::{ErrorCode, Result, RwError}; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::catalog::{Index as ProstIndex, Table as ProstTable}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_pb::user::grant_privilege::{Action, Object}; @@ -33,7 +33,7 @@ use crate::expr::{Expr, ExprImpl, InputRef}; use crate::handler::privilege::ObjectCheckItem; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::{Explain, LogicalProject, LogicalScan, StreamMaterialize}; -use crate::optimizer::property::{Distribution, FieldOrder, Order, RequiredDist}; +use crate::optimizer::property::{Distribution, Order, RequiredDist}; use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, PlanRoot}; use crate::scheduler::streaming_manager::CreatingStreamingJobInfo; use crate::session::SessionImpl; @@ -85,11 +85,11 @@ pub(crate) fn gen_create_index_plan( .map(|(x, y)| (y.name.clone(), x)) .collect::>(); - let to_order_pair = |(ident, order): &(Ident, OrderType)| { + let to_column_order = |(ident, order): &(Ident, OrderType)| { let x = ident.real_value(); table_desc_map .get(&x) - .map(|x| OrderPair::new(*x, *order)) + .map(|x| ColumnOrder::new(*x, *order)) .ok_or_else(|| ErrorCode::ItemNotFound(x).into()) }; @@ -103,7 +103,7 @@ pub(crate) fn gen_create_index_plan( let mut index_columns = columns .iter() - .map(to_order_pair) + .map(to_column_order) .try_collect::<_, Vec<_>, RwError>()?; let mut include_columns = if include.is_empty() { @@ -131,7 +131,7 @@ pub(crate) fn gen_create_index_plan( let mut set = HashSet::new(); index_columns = index_columns .into_iter() - .filter(|x| set.insert(x.column_idx)) + .filter(|x| set.insert(x.column_index)) .collect_vec(); // Remove include columns are already in index columns @@ -145,7 +145,7 @@ pub(crate) fn gen_create_index_plan( // Distributed by columns should be a prefix of index columns if !index_columns .iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .collect_vec() .starts_with(&distributed_by_columns) { @@ -205,7 +205,7 @@ pub(crate) fn gen_create_index_plan( .collect_vec(), original_columns: index_columns .iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .collect_vec() .iter() .chain(include_columns.iter()) @@ -269,7 +269,7 @@ fn assemble_materialize( table_desc: Rc, context: OptimizerContextRef, index_name: String, - index_columns: &[OrderPair], + index_columns: &[ColumnOrder], include_columns: &[usize], distributed_by_columns_len: usize, ) -> Result { @@ -290,7 +290,7 @@ fn assemble_materialize( let exprs = index_columns .iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .collect_vec() .iter() .chain(include_columns.iter()) @@ -307,7 +307,7 @@ fn assemble_materialize( let out_names: Vec = index_columns .iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .collect_vec() .iter() .chain(include_columns.iter()) @@ -323,10 +323,7 @@ fn assemble_materialize( index_columns .iter() .enumerate() - .map(|(i, order_pair)| match order_pair.order_type { - OrderType::Ascending => FieldOrder::ascending(i), - OrderType::Descending => FieldOrder::descending(i), - }) + .map(|(i, column_order)| ColumnOrder::new(i, column_order.order_type)) .collect(), ), project_required_cols, @@ -339,6 +336,7 @@ fn check_columns(columns: Vec) -> Result> { columns .into_iter() .map(|column| { + // TODO(rc): support `NULLS FIRST | LAST` if column.nulls_first.is_some() { return Err(ErrorCode::NotImplemented( "nulls_first not supported".into(), @@ -352,11 +350,11 @@ fn check_columns(columns: Vec) -> Result> { if let Expr::Identifier(ident) = column.expr { Ok::<(_, _), RwError>(( ident, - column.asc.map_or(OrderType::Ascending, |x| { + column.asc.map_or(OrderType::ascending(), |x| { if x { - OrderType::Ascending + OrderType::ascending() } else { - OrderType::Descending + OrderType::descending() } }), )) diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index 6c918ce88b6f..741716fddb75 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -22,6 +22,7 @@ use pgwire::types::Row; use risingwave_common::catalog::ColumnDesc; use risingwave_common::error::Result; use risingwave_common::types::DataType; +use risingwave_common::util::sort_util::Direction; use risingwave_sqlparser::ast::{display_comma_separated, ObjectName}; use super::RwPgResponse; @@ -29,7 +30,6 @@ use crate::binder::{Binder, Relation}; use crate::catalog::{CatalogError, IndexCatalog}; use crate::handler::util::col_descs_to_rows; use crate::handler::HandlerArgs; -use crate::optimizer::property::Direction; pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Result { let session = handler_args.session; @@ -60,7 +60,7 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res .table_catalog .pk() .iter() - .map(|idx| t.table_catalog.columns[idx.index].clone()) + .map(|x| t.table_catalog.columns[x.column_index].clone()) .collect_vec(); (t.table_catalog.columns, pk_column_catalogs, t.table_indexes) } @@ -117,10 +117,10 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res let index_columns_with_ordering = index_table .pk .iter() - .filter(|x| !index_table.columns[x.index].is_hidden) + .filter(|x| !index_table.columns[x.column_index].is_hidden) .map(|x| { - let index_column_name = index_table.columns[x.index].name().to_string(); - if Direction::Desc == x.direct { + let index_column_name = index_table.columns[x.column_index].name().to_string(); + if Direction::Descending == x.order_type.direction() { index_column_name + " DESC" } else { index_column_name @@ -131,7 +131,7 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res let pk_column_index_set = index_table .pk .iter() - .map(|x| x.index) + .map(|x| x.column_index) .collect::>(); let include_columns = index_table diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index 5c81dd6265fc..9a851dde7063 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -52,7 +52,7 @@ impl BatchLimit { let any_order = Order::any(); let ensure_single_dist = RequiredDist::single().enforce_if_not_satisfies( batch_partial_limit.into(), - if self.order().field_order.is_empty() { + if self.order().column_orders.is_empty() { &any_order } else { self.order() diff --git a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs index 4d124e598a5d..28973348ed96 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -49,21 +49,16 @@ impl BatchSortAgg { d => d.clone(), }; let input_order = Order { - field_order: input + column_orders: input .order() - .field_order + .column_orders .iter() - .filter(|field_ord| { - logical - .group_key() - .iter() - .any(|g_k| *g_k == field_ord.index) - }) + .filter(|o| logical.group_key().iter().any(|g_k| *g_k == o.column_index)) .cloned() .collect(), }; - assert_eq!(input_order.field_order.len(), logical.group_key().len()); + assert_eq!(input_order.column_orders.len(), logical.group_key().len()); let order = logical .i2o_col_mapping() diff --git a/src/frontend/src/optimizer/plan_node/derive.rs b/src/frontend/src/optimizer/plan_node/derive.rs index e2cf5a51a3ed..fe2bd3e2da32 100644 --- a/src/frontend/src/optimizer/plan_node/derive.rs +++ b/src/frontend/src/optimizer/plan_node/derive.rs @@ -18,9 +18,10 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, Schema, USER_COLUMN_ID_OFFSET}; use risingwave_common::error::{ErrorCode, Result}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use super::PlanRef; -use crate::optimizer::property::{Direction, FieldOrder, Order}; +use crate::optimizer::property::Order; pub(crate) fn derive_columns( input_schema: &Schema, @@ -79,7 +80,7 @@ pub(crate) fn derive_pk( input: PlanRef, user_order_by: Order, columns: &[ColumnCatalog], -) -> (Vec, Vec) { +) -> (Vec, Vec) { // Note(congyi): avoid pk duplication let stream_key = input.logical_pk().iter().copied().unique().collect_vec(); let schema = input.schema(); @@ -100,9 +101,9 @@ pub(crate) fn derive_pk( let mut in_order = FixedBitSet::with_capacity(schema.len()); let mut pk = vec![]; - for field in &user_order_by.field_order { - let idx = field.index; - pk.push(field.clone()); + for order in &user_order_by.column_orders { + let idx = order.column_index; + pk.push(order.clone()); in_order.insert(idx); } @@ -110,10 +111,7 @@ pub(crate) fn derive_pk( if in_order.contains(idx) { continue; } - pk.push(FieldOrder { - index: idx, - direct: Direction::Asc, - }); + pk.push(ColumnOrder::new(idx, OrderType::ascending())); in_order.insert(idx); } diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 5f43f535ecfd..750d15de651c 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -18,9 +18,8 @@ use std::fmt; use itertools::Itertools; use risingwave_common::catalog::{Field, FieldDisplay, Schema}; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::{ColumnOrder, ColumnOrderDisplay, OrderType}; use risingwave_expr::expr::AggKind; -use risingwave_pb::common::{PbColumnOrder, PbOrderType}; use risingwave_pb::expr::AggCall as ProstAggCall; use risingwave_pb::stream_plan::{agg_call_state, AggCallState as AggCallStateProst}; @@ -28,7 +27,6 @@ use super::super::utils::TableCatalogBuilder; use super::{stream, GenericPlanNode, GenericPlanRef}; use crate::expr::{Expr, ExprRewriter, InputRef, InputRefDisplay}; use crate::optimizer::optimizer_context::OptimizerContextRef; -use crate::optimizer::property::Direction; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::utils::{ ColIndexMapping, ColIndexMappingRewriteExt, Condition, ConditionDisplay, IndexRewriter, @@ -180,13 +178,13 @@ impl Agg { }; for &idx in &self.group_key { - add_column(idx, Some(OrderType::Ascending), false); + add_column(idx, Some(OrderType::ascending()), false); } for (order_type, idx) in sort_keys { add_column(idx, Some(order_type), true); } for &idx in &in_pks { - add_column(idx, Some(OrderType::Ascending), true); + add_column(idx, Some(OrderType::ascending()), true); } for idx in include_keys { add_column(idx, None, true); @@ -222,7 +220,7 @@ impl Agg { for &idx in &self.group_key { let tb_column_idx = internal_table_catalog_builder.add_column(&in_fields[idx]); internal_table_catalog_builder - .add_order_column(tb_column_idx, OrderType::Ascending); + .add_order_column(tb_column_idx, OrderType::ascending()); included_upstream_indices.push(idx); } @@ -272,15 +270,15 @@ impl Agg { let sort_keys = { match agg_call.agg_kind { AggKind::Min => { - vec![(OrderType::Ascending, agg_call.inputs[0].index)] + vec![(OrderType::ascending(), agg_call.inputs[0].index)] } AggKind::Max => { - vec![(OrderType::Descending, agg_call.inputs[0].index)] + vec![(OrderType::descending(), agg_call.inputs[0].index)] } AggKind::StringAgg | AggKind::ArrayAgg => agg_call .order_by .iter() - .map(|o| (o.direction.to_order(), o.input.index)) + .map(|o| (o.order_type, o.column_index)) .collect(), _ => unreachable!(), } @@ -351,7 +349,7 @@ impl Agg { let tb_column_idx = internal_table_catalog_builder.add_column(field); if tb_column_idx < self.group_key.len() { internal_table_catalog_builder - .add_order_column(tb_column_idx, OrderType::Ascending); + .add_order_column(tb_column_idx, OrderType::ascending()); } } internal_table_catalog_builder.set_read_prefix_len_hint(self.group_key.len()); @@ -400,7 +398,7 @@ impl Agg { .collect_vec(); for &idx in &key_cols { let table_col_idx = table_builder.add_column(&in_fields[idx]); - table_builder.add_order_column(table_col_idx, OrderType::Ascending); + table_builder.add_order_column(table_col_idx, OrderType::ascending()); } // Agg calls with same distinct column share the same dedup table, but they may have @@ -464,73 +462,6 @@ impl Agg { } } -/// Rewritten version of [`crate::expr::OrderByExpr`] which uses `InputRef` instead of `ExprImpl`. -/// Refer to [`LogicalAggBuilder::try_rewrite_agg_call`] for more details. -/// -/// TODO(yuchao): replace `PlanAggOrderByField` with enhanced `FieldOrder` -#[derive(Clone, PartialEq, Eq, Hash)] -pub struct PlanAggOrderByField { - pub input: InputRef, - pub direction: Direction, - pub nulls_first: bool, -} - -impl fmt::Debug for PlanAggOrderByField { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "{:?}", self.input)?; - match self.direction { - Direction::Asc => write!(f, " ASC")?, - Direction::Desc => write!(f, " DESC")?, - _ => {} - } - write!( - f, - " NULLS {}", - if self.nulls_first { "FIRST" } else { "LAST" } - )?; - Ok(()) - } -} - -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct PlanAggOrderByFieldDisplay<'a> { - pub plan_agg_order_by_field: &'a PlanAggOrderByField, - pub input_schema: &'a Schema, -} - -impl fmt::Display for PlanAggOrderByFieldDisplay<'_> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - let that = self.plan_agg_order_by_field; - InputRefDisplay { - input_ref: &that.input, - input_schema: self.input_schema, - } - .fmt(f)?; - match that.direction { - Direction::Asc => write!(f, " ASC")?, - Direction::Desc => write!(f, " DESC")?, - _ => {} - } - write!( - f, - " NULLS {}", - if that.nulls_first { "FIRST" } else { "LAST" } - )?; - Ok(()) - } -} - -impl PlanAggOrderByField { - fn to_protobuf(&self) -> PbColumnOrder { - PbColumnOrder { - column_index: self.input.index() as _, - order_type: Some(PbOrderType { - direction: self.direction.to_protobuf() as _, - }), - } - } -} - /// Rewritten version of [`AggCall`] which uses `InputRef` instead of `ExprImpl`. /// Refer to [`LogicalAggBuilder::try_rewrite_agg_call`] for more details. #[derive(Clone, PartialEq, Eq, Hash)] @@ -552,7 +483,7 @@ pub struct PlanAggCall { pub inputs: Vec, pub distinct: bool, - pub order_by: Vec, + pub order_by: Vec, /// Selective aggregation: only the input rows for which /// `filter` evaluates to `true` will be fed to the aggregate function. pub filter: Condition, @@ -598,7 +529,7 @@ impl PlanAggCall { // modify order_by exprs self.order_by.iter_mut().for_each(|x| { - x.input.index = mapping.map(x.input.index); + x.column_index = mapping.map(x.column_index); }); // modify filter @@ -614,11 +545,7 @@ impl PlanAggCall { return_type: Some(self.return_type.to_protobuf()), args: self.inputs.iter().map(InputRef::to_proto).collect(), distinct: self.distinct, - order_by: self - .order_by - .iter() - .map(PlanAggOrderByField::to_protobuf) - .collect(), + order_by: self.order_by.iter().map(ColumnOrder::to_protobuf).collect(), filter: self.filter.as_expr_unless_true().map(|x| x.to_expr_proto()), } } @@ -699,9 +626,9 @@ impl fmt::Debug for PlanAggCallDisplay<'_> { write!( f, " order_by({})", - that.order_by.iter().format_with(", ", |e, f| { - f(&PlanAggOrderByFieldDisplay { - plan_agg_order_by_field: e, + that.order_by.iter().format_with(", ", |o, f| { + f(&ColumnOrderDisplay { + column_order: o, input_schema: self.input_schema, }) }) diff --git a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs index 6fd34d7978ee..4ba5b9f88523 100644 --- a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs @@ -77,7 +77,7 @@ pub fn infer_left_internal_table_catalog( }); pk_indices.iter().for_each(|idx| { - internal_table_catalog_builder.add_order_column(*idx, OrderType::Ascending) + internal_table_catalog_builder.add_order_column(*idx, OrderType::ascending()) }); internal_table_catalog_builder.build(dist_keys) diff --git a/src/frontend/src/optimizer/plan_node/generic/source.rs b/src/frontend/src/optimizer/plan_node/generic/source.rs index 50c120daf710..3efbe4e60ea8 100644 --- a/src/frontend/src/optimizer/plan_node/generic/source.rs +++ b/src/frontend/src/optimizer/plan_node/generic/source.rs @@ -89,7 +89,7 @@ impl Source { let ordered_col_idx = builder.add_column(&key); builder.add_column(&value); - builder.add_order_column(ordered_col_idx, OrderType::Ascending); + builder.add_order_column(ordered_col_idx, OrderType::ascending()); builder.build(vec![]) } diff --git a/src/frontend/src/optimizer/plan_node/generic/top_n.rs b/src/frontend/src/optimizer/plan_node/generic/top_n.rs index 4ebdabb124b9..2c8d34833154 100644 --- a/src/frontend/src/optimizer/plan_node/generic/top_n.rs +++ b/src/frontend/src/optimizer/plan_node/generic/top_n.rs @@ -43,7 +43,7 @@ impl TopN { let schema = me.schema(); let pk_indices = me.logical_pk(); let columns_fields = schema.fields().to_vec(); - let field_order = &self.order.field_order; + let column_orders = &self.order.column_orders; let mut internal_table_catalog_builder = TableCatalogBuilder::new(me.ctx().with_options().internal_table_subset()); @@ -58,21 +58,21 @@ impl TopN { // does a prefix scanning with the group key, we can fetch the data in the // desired order. self.group_key.iter().for_each(|&idx| { - internal_table_catalog_builder.add_order_column(idx, OrderType::Ascending); + internal_table_catalog_builder.add_order_column(idx, OrderType::ascending()); order_cols.insert(idx); }); - field_order.iter().for_each(|field_order| { - if !order_cols.contains(&field_order.index) { + column_orders.iter().for_each(|order| { + if !order_cols.contains(&order.column_index) { internal_table_catalog_builder - .add_order_column(field_order.index, OrderType::from(field_order.direct)); - order_cols.insert(field_order.index); + .add_order_column(order.column_index, order.order_type); + order_cols.insert(order.column_index); } }); pk_indices.iter().for_each(|idx| { if !order_cols.contains(idx) { - internal_table_catalog_builder.add_order_column(*idx, OrderType::Ascending); + internal_table_catalog_builder.add_order_column(*idx, OrderType::ascending()); order_cols.insert(*idx); } }); diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index db92c173e237..b16806cf7655 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -19,11 +19,11 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::error::{ErrorCode, Result, TrackingIssue}; use risingwave_common::types::{DataType, Datum, OrderedF64, ScalarImpl}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::expr::AggKind; use super::generic::{ - self, AggCallState, GenericPlanNode, GenericPlanRef, PlanAggCall, PlanAggOrderByField, - ProjectBuilder, + self, AggCallState, GenericPlanNode, GenericPlanRef, PlanAggCall, ProjectBuilder, }; use super::{ BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, PlanBase, PlanRef, @@ -39,10 +39,7 @@ use crate::optimizer::plan_node::{ gen_filter_and_pushdown, BatchSortAgg, ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; -use crate::optimizer::property::Direction::{Asc, Desc}; -use crate::optimizer::property::{ - Distribution, FieldOrder, FunctionalDependencySet, Order, RequiredDist, -}; +use crate::optimizer::property::{Distribution, FunctionalDependencySet, Order, RequiredDist}; use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition, Substitute}; /// `LogicalAgg` groups input data by their group key and computes aggregation functions. @@ -334,29 +331,26 @@ impl LogicalAgg { // aggregation and use sort aggregation. The data type of the columns need to be int32 fn output_requires_order_on_group_keys(&self, required_order: &Order) -> (bool, Order) { let group_key_order = Order { - field_order: self + column_orders: self .group_key() .iter() .map(|group_by_idx| { - let direct = if required_order.field_order.contains(&FieldOrder { - index: *group_by_idx, - direct: Desc, - }) { + let order_type = if required_order + .column_orders + .contains(&ColumnOrder::new(*group_by_idx, OrderType::descending())) + { // If output requires descending order, use descending order - Desc + OrderType::descending() } else { // In all other cases use ascending order - Asc + OrderType::ascending() }; - FieldOrder { - index: *group_by_idx, - direct, - } + ColumnOrder::new(*group_by_idx, order_type) }) .collect(), }; return ( - !required_order.field_order.is_empty() + !required_order.column_orders.is_empty() && group_key_order.satisfies(required_order) && self.group_key().iter().all(|group_by_idx| { self.schema().fields().get(*group_by_idx).unwrap().data_type == DataType::Int32 @@ -373,9 +367,9 @@ impl LogicalAgg { new_logical .input() .order() - .field_order + .column_orders .iter() - .any(|field_order| field_order.index == *group_by_idx) + .any(|order| order.column_index == *group_by_idx) && new_logical .input() .schema() @@ -594,11 +588,7 @@ impl LogicalAggBuilder { .iter() .map(|e| { let index = self.input_proj_builder.add_expr(&e.expr)?; - Ok(PlanAggOrderByField { - input: InputRef::new(index, e.expr.return_type()), - direction: e.direction, - nulls_first: e.nulls_first, - }) + Ok(ColumnOrder::new(index, e.order_type)) }) .try_collect() .map_err(|err: &'static str| { @@ -1005,9 +995,8 @@ impl LogicalAgg { agg_call.inputs.iter_mut().for_each(|i| { *i = InputRef::new(input_col_change.map(i.index()), i.return_type()) }); - agg_call.order_by.iter_mut().for_each(|field| { - let i = &mut field.input; - *i = InputRef::new(input_col_change.map(i.index()), i.return_type()) + agg_call.order_by.iter_mut().for_each(|o| { + o.column_index = input_col_change.map(o.column_index); }); agg_call.filter = agg_call.filter.rewrite_expr(&mut input_col_change); agg_call @@ -1097,7 +1086,7 @@ impl ColPrunable for LogicalAgg { let index = index - self.group_key().len(); let agg_call = self.agg_calls()[index].clone(); tmp.extend(agg_call.inputs.iter().map(|x| x.index())); - tmp.extend(agg_call.order_by.iter().map(|x| x.input.index())); + tmp.extend(agg_call.order_by.iter().map(|x| x.column_index)); // collect columns used in aggregate filter expressions for i in &agg_call.filter.conjunctions { tmp.union_with(&i.collect_input_refs(input_cnt)); diff --git a/src/frontend/src/optimizer/plan_node/logical_over_agg.rs b/src/frontend/src/optimizer/plan_node/logical_over_agg.rs index f8215b97d7f8..043e459eb1ea 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_agg.rs @@ -19,8 +19,8 @@ use itertools::Itertools; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; +use risingwave_common::util::sort_util::{ColumnOrder, ColumnOrderDisplay}; -use super::generic::{PlanAggOrderByField, PlanAggOrderByFieldDisplay}; use super::{ gen_filter_and_pushdown, ColPrunable, ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, @@ -37,9 +37,7 @@ pub struct PlanWindowFunction { pub function_type: WindowFunctionType, pub return_type: DataType, pub partition_by: Vec, - /// TODO: rename & move `PlanAggOrderByField` so that it can be better shared like - /// [`crate::expr::OrderByExpr`] - pub order_by: Vec, + pub order_by: Vec, } struct PlanWindowFunctionDisplay<'a> { @@ -81,9 +79,9 @@ impl<'a> std::fmt::Debug for PlanWindowFunctionDisplay<'a> { write!( f, "{delim}ORDER BY {}", - window_function.order_by.iter().format_with(", ", |e, f| { - f(&PlanAggOrderByFieldDisplay { - plan_agg_order_by_field: e, + window_function.order_by.iter().format_with(", ", |o, f| { + f(&ColumnOrderDisplay { + column_order: o, input_schema: self.input_schema, }) }) @@ -194,11 +192,7 @@ impl LogicalOverAgg { .sort_exprs .into_iter() .map(|e| match e.expr.as_input_ref() { - Some(i) => Ok(PlanAggOrderByField { - input: *i.clone(), - direction: e.direction, - nulls_first: e.nulls_first, - }), + Some(i) => Ok(ColumnOrder::new(i.index(), e.order_type)), None => Err(ErrorCode::NotImplemented( "ORDER BY expression in window function".to_string(), None.into(), diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 76e7fab648a3..999449397d1a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -20,7 +20,7 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{ColumnDesc, Field, Schema, TableDesc}; use risingwave_common::error::{ErrorCode, Result, RwError}; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::{ @@ -36,8 +36,7 @@ use crate::optimizer::plan_node::{ BatchSeqScan, ColumnPruningContext, LogicalFilter, LogicalProject, LogicalValues, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; -use crate::optimizer::property::Direction::Asc; -use crate::optimizer::property::{FieldOrder, FunctionalDependencySet, Order}; +use crate::optimizer::property::{FunctionalDependencySet, Order}; use crate::optimizer::rule::IndexSelectionRule; use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition, ConditionDisplay}; @@ -225,12 +224,9 @@ impl LogicalScan { .iter() .map(|order| { let idx = id_to_tb_idx - .get(&self.table_desc().columns[order.column_idx].column_id) + .get(&self.table_desc().columns[order.column_index].column_id) .unwrap(); - match order.order_type { - OrderType::Ascending => FieldOrder::ascending(*idx), - OrderType::Descending => FieldOrder::descending(*idx), - } + ColumnOrder::new(*idx, order.order_type) }) .collect(), ); @@ -316,7 +312,7 @@ impl LogicalScan { self.core.chunk_size } - pub fn primary_key(&self) -> Vec { + pub fn primary_key(&self) -> Vec { self.core.table_desc.pk.clone() } @@ -568,19 +564,16 @@ impl LogicalScan { &self, required_order: &Order, ) -> Option> { - if required_order.field_order.is_empty() { + if required_order.column_orders.is_empty() { return None; } let index = self.indexes().iter().find(|idx| { Order { - field_order: idx + column_orders: idx .index_item .iter() - .map(|idx_item| FieldOrder { - index: idx_item.index, - direct: Asc, - }) + .map(|idx_item| ColumnOrder::new(idx_item.index, OrderType::ascending())) .collect(), } .satisfies(required_order) @@ -679,8 +672,8 @@ impl ToStream for LogicalScan { .pk .iter() .filter_map(|c| { - if !col_ids.contains(&self.table_desc().columns[c.column_idx].column_id) { - Some(c.column_idx) + if !col_ids.contains(&self.table_desc().columns[c.column_index].column_id) { + Some(c.column_index) } else { None } diff --git a/src/frontend/src/optimizer/plan_node/logical_topn.rs b/src/frontend/src/optimizer/plan_node/logical_topn.rs index fcaf1a6b32fd..a43448d72bc6 100644 --- a/src/frontend/src/optimizer/plan_node/logical_topn.rs +++ b/src/frontend/src/optimizer/plan_node/logical_topn.rs @@ -17,6 +17,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::error::{ErrorCode, Result, RwError}; +use risingwave_common::util::sort_util::ColumnOrder; use super::generic::GenericPlanNode; use super::{ @@ -29,7 +30,7 @@ use crate::optimizer::plan_node::{ BatchTopN, ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, StreamTopN, ToStreamContext, }; -use crate::optimizer::property::{Distribution, FieldOrder, Order, OrderDisplay, RequiredDist}; +use crate::optimizer::property::{Distribution, Order, OrderDisplay, RequiredDist}; use crate::planner::LIMIT_ALL_COUNT; use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition}; use crate::TableCatalog; @@ -287,9 +288,9 @@ impl ColPrunable for LogicalTopN { let order_required_cols = { let mut order_required_cols = FixedBitSet::with_capacity(self.input().schema().len()); self.topn_order() - .field_order + .column_orders .iter() - .for_each(|fo| order_required_cols.insert(fo.index)); + .for_each(|o| order_required_cols.insert(o.column_index)); order_required_cols }; let group_required_cols = { @@ -311,14 +312,11 @@ impl ColPrunable for LogicalTopN { self.input().schema().len(), ); let new_order = Order { - field_order: self + column_orders: self .topn_order() - .field_order + .column_orders .iter() - .map(|fo| FieldOrder { - index: mapping.map(fo.index), - direct: fo.direct, - }) + .map(|o| ColumnOrder::new(mapping.map(o.column_index), o.order_type)) .collect(), }; let new_group_key = self diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index b3b9ec8cb4c4..66d41d239600 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -18,7 +18,7 @@ use itertools::Itertools; use pb::stream_node as pb_node; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_pb::stream_plan as pb; use smallvec::SmallVec; @@ -29,7 +29,7 @@ use super::{generic, EqJoinPredicate, PlanNodeId}; use crate::expr::{Expr, ExprImpl}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::plan_tree_node_v2::PlanTreeNodeV2; -use crate::optimizer::property::{Distribution, FieldOrder}; +use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::TableCatalog; @@ -262,7 +262,7 @@ impl HashJoin { internal_table_catalog_builder.add_column(field); }); pk_indices.iter().for_each(|idx| { - internal_table_catalog_builder.add_order_column(*idx, OrderType::Ascending) + internal_table_catalog_builder.add_order_column(*idx, OrderType::ascending()) }); // Build degree table. @@ -273,7 +273,7 @@ impl HashJoin { pk_indices.iter().enumerate().for_each(|(order_idx, idx)| { degree_table_catalog_builder.add_column(&internal_columns_fields[*idx]); - degree_table_catalog_builder.add_order_column(order_idx, OrderType::Ascending); + degree_table_catalog_builder.add_order_column(order_idx, OrderType::ascending()); }); degree_table_catalog_builder.add_column(°ree_column_field); degree_table_catalog_builder @@ -671,7 +671,7 @@ pub fn to_stream_prost_body( // We don't need table id for materialize node in frontend. The id will be generated // on meta catalog service. table_id: 0, - column_orders: me.table.pk().iter().map(FieldOrder::to_protobuf).collect(), + column_orders: me.table.pk().iter().map(ColumnOrder::to_protobuf).collect(), table: Some(me.table.to_internal_table_prost()), handle_pk_conflict_behavior: 0, }) diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 22e281becebe..87809258e5e2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -20,6 +20,7 @@ use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, ConflictBehavior, TableId}; use risingwave_common::error::Result; use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::derive::derive_columns; @@ -28,7 +29,7 @@ use crate::catalog::table_catalog::{TableCatalog, TableType, TableVersion}; use crate::catalog::FragmentId; use crate::optimizer::plan_node::derive::derive_pk; use crate::optimizer::plan_node::{PlanBase, PlanNodeMeta}; -use crate::optimizer::property::{Distribution, FieldOrder, Order, RequiredDist}; +use crate::optimizer::property::{Distribution, Order, RequiredDist}; use crate::stream_fragmenter::BuildFragmentGraphState; /// Materializes a stream. @@ -232,7 +233,7 @@ impl fmt::Display for StreamMaterialize { let order_descs = table .pk .iter() - .map(|order| table.columns()[order.index].column_desc.name.clone()) + .map(|o| table.columns()[o.column_index].column_desc.name.clone()) .join(", "); let mut builder = f.debug_struct("StreamMaterialize"); @@ -307,7 +308,7 @@ impl StreamNode for StreamMaterialize { .table() .pk() .iter() - .map(FieldOrder::to_protobuf) + .map(ColumnOrder::to_protobuf) .collect(), table: Some(self.table().to_internal_table_prost()), handle_pk_conflict_behavior, diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index db1b07555047..2ff01cca373f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -116,7 +116,7 @@ impl StreamSink { name, definition, columns, - pk: pk.iter().map(|k| k.to_order_pair()).collect_vec(), + pk, stream_key, distribution_key, properties: properties.into_inner(), @@ -194,7 +194,12 @@ impl fmt::Display for StreamSink { builder.field( "pk", &IndicesDisplay { - indices: &self.sink_desc.pk.iter().map(|k| k.column_idx).collect_vec(), + indices: &self + .sink_desc + .pk + .iter() + .map(|k| k.column_index) + .collect_vec(), input_schema: &self.base.schema, }, ); diff --git a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs index c8ff1b50c426..a0d5a5468a7b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs @@ -132,7 +132,7 @@ pub fn infer_internal_table_catalog(watermark_type: DataType) -> TableCatalog { let ordered_col_idx = builder.add_column(&key); builder.add_column(&value); - builder.add_order_column(ordered_col_idx, OrderType::Ascending); + builder.add_order_column(ordered_col_idx, OrderType::ascending()); builder.set_vnode_col_idx(0); builder.set_value_indices(vec![1]); diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index 4a14bfb57f1b..ad46f2f387f6 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -18,18 +18,17 @@ use std::{fmt, vec}; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, Field, Schema}; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use crate::catalog::table_catalog::TableType; use crate::catalog::{FragmentId, TableCatalog, TableId}; -use crate::optimizer::property::{Direction, FieldOrder}; use crate::utils::WithOptions; #[derive(Default)] pub struct TableCatalogBuilder { /// All columns in this table columns: Vec, - pk: Vec, + pk: Vec, properties: WithOptions, value_indices: Option>, vnode_col_idx: Option, @@ -72,14 +71,8 @@ impl TableCatalogBuilder { /// Check whether need to add a ordered column. Different from value, order desc equal pk in /// semantics and they are encoded as storage key. - pub fn add_order_column(&mut self, index: usize, order_type: OrderType) { - self.pk.push(FieldOrder { - index, - direct: match order_type { - OrderType::Ascending => Direction::Asc, - OrderType::Descending => Direction::Desc, - }, - }); + pub fn add_order_column(&mut self, column_index: usize, order_type: OrderType) { + self.pk.push(ColumnOrder::new(column_index, order_type)); } pub fn set_read_prefix_len_hint(&mut self, read_prefix_len_hint: usize) { diff --git a/src/frontend/src/optimizer/property/order.rs b/src/frontend/src/optimizer/property/order.rs index 1717cbce0759..7b242a7d0279 100644 --- a/src/frontend/src/optimizer/property/order.rs +++ b/src/frontend/src/optimizer/property/order.rs @@ -15,47 +15,47 @@ use std::fmt; use itertools::Itertools; -use parse_display::Display; -use risingwave_common::catalog::{FieldDisplay, Schema}; +use risingwave_common::catalog::Schema; use risingwave_common::error::Result; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; -use risingwave_pb::common::{PbColumnOrder, PbDirection, PbOrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, ColumnOrderDisplay}; +use risingwave_pb::common::PbColumnOrder; use super::super::plan_node::*; use crate::optimizer::PlanRef; +// TODO(rc): use this type to replace all `Vec` #[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] pub struct Order { - pub field_order: Vec, + pub column_orders: Vec, } impl Order { - pub const fn new(field_order: Vec) -> Self { - Self { field_order } + pub const fn new(column_orders: Vec) -> Self { + Self { column_orders } } pub fn to_protobuf(&self) -> Vec { - self.field_order + self.column_orders .iter() - .map(FieldOrder::to_protobuf) + .map(ColumnOrder::to_protobuf) .collect_vec() } pub fn len(&self) -> usize { - self.field_order.len() + self.column_orders.len() } } impl fmt::Display for Order { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.write_str("[")?; - for (i, field_order) in self.field_order.iter().enumerate() { + write!(f, "[")?; + for (i, column_order) in self.column_orders.iter().enumerate() { if i > 0 { - f.write_str(", ")?; + write!(f, ", ")?; } - field_order.fmt(f)?; + write!(f, "{}", column_order)?; } - f.write_str("]") + write!(f, "]") } } @@ -64,173 +64,29 @@ pub struct OrderDisplay<'a> { pub input_schema: &'a Schema, } -impl OrderDisplay<'_> { +impl fmt::Display for OrderDisplay<'_> { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let that = self.order; - f.write_str("[")?; - for (i, field_order) in that.field_order.iter().enumerate() { + write!(f, "[")?; + for (i, column_order) in that.column_orders.iter().enumerate() { if i > 0 { - f.write_str(", ")?; - } - FieldOrderDisplay { - field_order, - input_schema: self.input_schema, + write!(f, ", ")?; } - .fmt(f)?; - } - f.write_str("]") - } -} - -impl fmt::Display for OrderDisplay<'_> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.fmt(f) - } -} - -impl fmt::Debug for OrderDisplay<'_> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.fmt(f) - } -} - -#[derive(Clone, PartialEq, Eq, Hash)] -pub struct FieldOrder { - pub index: usize, - pub direct: Direction, -} - -impl std::fmt::Debug for FieldOrder { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "${} {}", self.index, self.direct) - } -} - -pub struct FieldOrderDisplay<'a> { - pub field_order: &'a FieldOrder, - pub input_schema: &'a Schema, -} - -impl FieldOrderDisplay<'_> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - let that = self.field_order; - write!( - f, - "{} {}", - FieldDisplay(self.input_schema.fields.get(that.index).unwrap()), - that.direct - ) - } -} - -impl fmt::Debug for FieldOrderDisplay<'_> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.fmt(f) - } -} - -impl FieldOrder { - pub fn ascending(index: usize) -> Self { - Self { - index, - direct: Direction::Asc, - } - } - - pub fn descending(index: usize) -> Self { - Self { - index, - direct: Direction::Desc, - } - } - - pub fn to_protobuf(&self) -> PbColumnOrder { - PbColumnOrder { - column_index: self.index as _, - order_type: Some(PbOrderType { - direction: self.direct.to_protobuf() as _, - }), - } - } - - pub fn from_protobuf(column_order: &PbColumnOrder) -> Self { - Self { - index: column_order.column_index as _, - direct: Direction::from_protobuf(&column_order.get_order_type().unwrap().direction()), - } - } - - // TODO(rc): unify them - pub fn to_order_pair(&self) -> OrderPair { - OrderPair { - column_idx: self.index, - order_type: self.direct.to_order(), - } - } -} - -impl fmt::Display for FieldOrder { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "${} {}", self.index, self.direct) - } -} - -#[derive(Debug, Display, Clone, Eq, PartialEq, Copy, Hash)] -#[display(style = "UPPERCASE")] -pub enum Direction { - Asc, - Desc, - Any, // only used in order requirement -} - -impl From for OrderType { - fn from(dir: Direction) -> Self { - match dir { - Direction::Asc => OrderType::Ascending, - Direction::Desc => OrderType::Descending, - Direction::Any => OrderType::Ascending, - } - } -} - -impl Direction { - pub fn to_protobuf(self) -> PbDirection { - match self { - Self::Asc => PbDirection::Ascending, - Self::Desc => PbDirection::Descending, - _ => unimplemented!(), - } - } - - pub fn from_protobuf(order_type: &PbDirection) -> Self { - match order_type { - PbDirection::Ascending => Self::Asc, - PbDirection::Descending => Self::Desc, - PbDirection::Unspecified => unreachable!(), - } - } - - // TODO(rc): unify them - pub fn to_order(self) -> OrderType { - match self { - Self::Asc => OrderType::Ascending, - Self::Desc => OrderType::Descending, - _ => unreachable!(), - } - } -} - -impl Direction { - pub fn satisfies(&self, other: &Direction) -> bool { - match other { - Direction::Any => true, - _ => self == other, + write!( + f, + "{}", + ColumnOrderDisplay { + column_order, + input_schema: self.input_schema, + } + )?; } + write!(f, "]") } } const ANY_ORDER: Order = Order { - field_order: vec![], + column_orders: vec![], }; impl Order { @@ -248,12 +104,12 @@ impl Order { } pub fn satisfies(&self, other: &Order) -> bool { - if self.field_order.len() < other.field_order.len() { + if self.column_orders.len() < other.column_orders.len() { return false; } #[expect(clippy::disallowed_methods)] - for (order, other_order) in self.field_order.iter().zip(other.field_order.iter()) { - if order.index != other_order.index || !order.direct.satisfies(&other_order.direct) { + for (order, other_order) in self.column_orders.iter().zip(other.column_orders.iter()) { + if order != other_order { return false; } } @@ -267,65 +123,55 @@ impl Order { #[inline(always)] pub fn is_any(&self) -> bool { - self.field_order.is_empty() + self.column_orders.is_empty() } } #[cfg(test)] mod tests { - use super::{Direction, FieldOrder, Order}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; + + use super::Order; #[test] fn test_order_satisfy() { let o1 = Order { - field_order: vec![ - FieldOrder { - index: 0, - direct: Direction::Asc, + column_orders: vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, - FieldOrder { - index: 1, - direct: Direction::Desc, + ColumnOrder { + column_index: 1, + order_type: OrderType::descending(), }, - FieldOrder { - index: 2, - direct: Direction::Asc, + ColumnOrder { + column_index: 2, + order_type: OrderType::ascending(), }, ], }; let o2 = Order { - field_order: vec![ - FieldOrder { - index: 0, - direct: Direction::Asc, + column_orders: vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, - FieldOrder { - index: 1, - direct: Direction::Desc, + ColumnOrder { + column_index: 1, + order_type: OrderType::descending(), }, ], }; let o3 = Order { - field_order: vec![ - FieldOrder { - index: 0, - direct: Direction::Asc, + column_orders: vec![ + ColumnOrder { + column_index: 0, + order_type: OrderType::ascending(), }, - FieldOrder { - index: 1, - direct: Direction::Asc, - }, - ], - }; - let o4 = Order { - field_order: vec![ - FieldOrder { - index: 0, - direct: Direction::Asc, - }, - FieldOrder { - index: 1, - direct: Direction::Any, + ColumnOrder { + column_index: 1, + order_type: OrderType::ascending(), }, ], }; @@ -336,10 +182,5 @@ mod tests { assert!(!o2.satisfies(&o3)); assert!(!o3.satisfies(&o2)); - - assert!(o3.satisfies(&o4)); - assert!(o3.satisfies(&o4)); - assert!(!o4.satisfies(&o2)); - assert!(!o4.satisfies(&o3)); } } diff --git a/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs index 1c25ba4c7b8c..f966ef824d71 100644 --- a/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs @@ -130,7 +130,7 @@ impl Rule for ApplyAggTransposeRule { agg_call .order_by .iter_mut() - .for_each(|o| o.input.shift_with_offset(offset)); + .for_each(|o| o.shift_with_offset(offset)); agg_call.filter = agg_call.filter.clone().rewrite_expr(&mut shift_index); }); if is_scalar_agg { 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 d5ebadc51bde..e7939d4dc06d 100644 --- a/src/frontend/src/optimizer/rule/index_delta_join_rule.rs +++ b/src/frontend/src/optimizer/rule/index_delta_join_rule.rs @@ -81,7 +81,7 @@ impl Rule for IndexDeltaJoinRule { .index_table .pk .iter() - .map(|x| x.index) + .map(|x| x.column_index) .take(index.index_table.distribution_key.len()) .collect_vec(); @@ -107,7 +107,7 @@ impl Rule for IndexDeltaJoinRule { && primary_table_distribution_key == join_indices { // Check join key is prefix of primary table order key let primary_table_order_key_prefix = primary_table.table_desc().pk.iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .take(primary_table_distribution_key.len()) .collect_vec(); diff --git a/src/frontend/src/optimizer/rule/index_selection_rule.rs b/src/frontend/src/optimizer/rule/index_selection_rule.rs index 909b47d0f67f..61c277c06679 100644 --- a/src/frontend/src/optimizer/rule/index_selection_rule.rs +++ b/src/frontend/src/optimizer/rule/index_selection_rule.rs @@ -207,10 +207,14 @@ impl IndexSelectionRule { .zip_eq_fast(index.primary_table.pk.iter()) .map(|(x, y)| { Self::create_null_safe_equal_expr( - x.index, - index.index_table.columns[x.index].data_type().clone(), - y.index + index.index_item.len(), - index.primary_table.columns[y.index].data_type().clone(), + x.column_index, + index.index_table.columns[x.column_index] + .data_type() + .clone(), + y.column_index + index.index_item.len(), + index.primary_table.columns[y.column_index] + .data_type() + .clone(), ) }) .chain(new_predicate.into_iter()) @@ -303,8 +307,8 @@ impl IndexSelectionRule { Self::create_null_safe_equal_expr( x, schema.fields[x].data_type.clone(), - y.column_idx + index_access_len, - primary_table_desc.columns[y.column_idx].data_type.clone(), + y.column_index + index_access_len, + primary_table_desc.columns[y.column_index].data_type.clone(), ) }) .chain(new_predicate.into_iter()) @@ -477,7 +481,7 @@ impl IndexSelectionRule { match p2s_mapping.get(column_index.as_ref().unwrap()) { None => continue, // not found, prune this index Some(&idx) => { - if index.index_table.pk()[0].index != idx { + if index.index_table.pk()[0].column_index != idx { // not match, prune this index continue; } @@ -502,7 +506,7 @@ impl IndexSelectionRule { let primary_table_desc = logical_scan.table_desc(); if let Some(idx) = column_index { assert_eq!(conjunctions.len(), 1); - if primary_table_desc.pk[0].column_idx != idx { + if primary_table_desc.pk[0].column_index != idx { return result; } } @@ -513,7 +517,7 @@ impl IndexSelectionRule { primary_table_desc .pk .iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .collect_vec(), primary_table_desc.clone().into(), vec![], @@ -561,7 +565,7 @@ impl IndexSelectionRule { index .primary_table_pk_ref_to_index_table() .iter() - .map(|x| x.index) + .map(|x| x.column_index) .collect_vec(), index.index_table.table_desc().into(), vec![], @@ -688,7 +692,7 @@ impl<'a> TableScanIoEstimator<'a> { table_desc .pk .iter() - .map(|x| &table_desc.columns[x.column_idx]), + .map(|x| &table_desc.columns[x.column_index]), ) .map(|x| TableScanIoEstimator::estimate_data_type_size(&x.data_type)) .sum::() diff --git a/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs b/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs index 375d481ef124..285bb733dc54 100644 --- a/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs +++ b/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs @@ -21,7 +21,7 @@ use std::collections::BTreeMap; use itertools::Itertools; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::expr::AggKind; use super::{BoxedRule, Rule}; @@ -29,7 +29,7 @@ use crate::expr::{ExprImpl, ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ LogicalAgg, LogicalFilter, LogicalLimit, LogicalScan, PlanAggCall, PlanTreeNodeUnary, }; -use crate::optimizer::property::{Direction, FieldOrder, Order}; +use crate::optimizer::property::Order; use crate::optimizer::PlanRef; use crate::utils::Condition; @@ -65,14 +65,14 @@ impl Rule for MinMaxOnIndexRule { .map(|(id, col)| (col, id)) .collect::>(); let order = Order { - field_order: vec![FieldOrder { - index: calls.first()?.inputs.first()?.index(), - direct: if kind == AggKind::Min { - Direction::Asc + column_orders: vec![ColumnOrder::new( + calls.first()?.inputs.first()?.index(), + if kind == AggKind::Min { + OrderType::ascending() } else { - Direction::Desc + OrderType::descending() }, - }], + )], }; if let Some(p) = self.try_on_index(logical_agg, logical_scan.clone(), &order, &output_col_map) @@ -103,19 +103,21 @@ impl MinMaxOnIndexRule { let index = logical_scan.indexes().iter().find(|idx| { let s2p_mapping = idx.secondary_to_primary_mapping(); Order { - field_order: idx + column_orders: idx .index_table .pk() .iter() - .map(|idx_item| FieldOrder { - index: *output_col_map - .get( - s2p_mapping - .get(&idx_item.index) - .expect("should be in s2p mapping"), - ) - .unwrap_or(&unmatched_idx), - direct: idx_item.direct, + .map(|idx_item| { + ColumnOrder::new( + *output_col_map + .get( + s2p_mapping + .get(&idx_item.column_index) + .expect("should be in s2p mapping"), + ) + .unwrap_or(&unmatched_idx), + idx_item.order_type, + ) }) .collect(), } @@ -184,15 +186,15 @@ impl MinMaxOnIndexRule { let unmatched_idx = output_col_map.len(); let primary_key = logical_scan.primary_key(); let primary_key_order = Order { - field_order: primary_key + column_orders: primary_key .into_iter() - .map(|op| FieldOrder { - index: *output_col_map.get(&op.column_idx).unwrap_or(&unmatched_idx), - direct: if op.order_type == OrderType::Ascending { - Direction::Asc - } else { - Direction::Desc - }, + .map(|o| { + ColumnOrder::new( + *output_col_map + .get(&o.column_index) + .unwrap_or(&unmatched_idx), + o.order_type, + ) }) .collect::>(), }; diff --git a/src/frontend/src/optimizer/rule/over_agg_to_topn_rule.rs b/src/frontend/src/optimizer/rule/over_agg_to_topn_rule.rs index 31675c9db459..f04091dc8b51 100644 --- a/src/frontend/src/optimizer/rule/over_agg_to_topn_rule.rs +++ b/src/frontend/src/optimizer/rule/over_agg_to_topn_rule.rs @@ -20,7 +20,7 @@ use crate::expr::{ExprImpl, ExprType, WindowFunctionType}; use crate::optimizer::plan_node::{ LogicalFilter, LogicalTopN, PlanTreeNodeUnary, PlanWindowFunction, }; -use crate::optimizer::property::{FieldOrder, Order}; +use crate::optimizer::property::Order; use crate::planner::LIMIT_ALL_COUNT; use crate::PlanRef; @@ -93,13 +93,7 @@ impl Rule for OverAggToTopNRule { offset, with_ties, Order { - field_order: order_by - .iter() - .map(|f| FieldOrder { - index: f.input.index, - direct: f.direction, - }) - .collect(), + column_orders: order_by.to_vec(), }, partition_by.iter().map(|i| i.index).collect(), ) diff --git a/src/frontend/src/optimizer/rule/top_n_on_index_rule.rs b/src/frontend/src/optimizer/rule/top_n_on_index_rule.rs index f8aa3ef5290d..d47e6cbc2897 100644 --- a/src/frontend/src/optimizer/rule/top_n_on_index_rule.rs +++ b/src/frontend/src/optimizer/rule/top_n_on_index_rule.rs @@ -19,11 +19,11 @@ use std::collections::BTreeMap; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::ColumnOrder; use super::{BoxedRule, Rule}; use crate::optimizer::plan_node::{LogicalLimit, LogicalScan, LogicalTopN, PlanTreeNodeUnary}; -use crate::optimizer::property::{Direction, FieldOrder, Order}; +use crate::optimizer::property::Order; use crate::optimizer::PlanRef; pub struct TopNOnIndexRule {} @@ -36,7 +36,7 @@ impl Rule for TopNOnIndexRule { return None; } let order = logical_top_n.topn_order(); - if order.field_order.is_empty() { + if order.column_orders.is_empty() { return None; } let output_col_map = logical_scan @@ -71,19 +71,21 @@ impl TopNOnIndexRule { let index = logical_scan.indexes().iter().find(|idx| { let s2p_mapping = idx.secondary_to_primary_mapping(); Order { - field_order: idx + column_orders: idx .index_table .pk() .iter() - .map(|idx_item| FieldOrder { - index: *output_col_map - .get( - s2p_mapping - .get(&idx_item.index) - .expect("should be in s2p mapping"), - ) - .unwrap_or(&unmatched_idx), - direct: idx_item.direct, + .map(|idx_item| { + ColumnOrder::new( + *output_col_map + .get( + s2p_mapping + .get(&idx_item.column_index) + .expect("should be in s2p mapping"), + ) + .unwrap_or(&unmatched_idx), + idx_item.order_type, + ) }) .collect(), } @@ -128,15 +130,15 @@ impl TopNOnIndexRule { let unmatched_idx = output_col_map.len(); let primary_key = logical_scan.primary_key(); let primary_key_order = Order { - field_order: primary_key + column_orders: primary_key .into_iter() - .map(|op| FieldOrder { - index: *output_col_map.get(&op.column_idx).unwrap_or(&unmatched_idx), - direct: if op.order_type == OrderType::Ascending { - Direction::Asc - } else { - Direction::Desc - }, + .map(|o| { + ColumnOrder::new( + *output_col_map + .get(&o.column_index) + .unwrap_or(&unmatched_idx), + o.order_type, + ) }) .collect::>(), }; diff --git a/src/frontend/src/planner/query.rs b/src/frontend/src/planner/query.rs index 7a24f3a90e62..782fa42127ee 100644 --- a/src/frontend/src/planner/query.rs +++ b/src/frontend/src/planner/query.rs @@ -38,11 +38,13 @@ impl Planner { let extra_order_exprs_len = extra_order_exprs.len(); let mut plan = self.plan_set_expr(body, extra_order_exprs, &order)?; - let order = Order { field_order: order }; + let order = Order { + column_orders: order, + }; if limit.is_some() || offset.is_some() { let limit = limit.unwrap_or(LIMIT_ALL_COUNT); let offset = offset.unwrap_or_default(); - plan = if order.field_order.is_empty() { + plan = if order.column_orders.is_empty() { // Should be rejected by parser. assert!(!with_ties); // Create a logical limit if with limit/offset but without order-by diff --git a/src/frontend/src/planner/select.rs b/src/frontend/src/planner/select.rs index 390afcc98747..235cdf0a19c8 100644 --- a/src/frontend/src/planner/select.rs +++ b/src/frontend/src/planner/select.rs @@ -19,6 +19,7 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_expr::ExprError; use risingwave_pb::plan_common::JoinType; @@ -33,7 +34,7 @@ use crate::optimizer::plan_node::{ LogicalAgg, LogicalApply, LogicalOverAgg, LogicalProject, LogicalProjectSet, LogicalTopN, LogicalValues, PlanAggCall, PlanRef, }; -use crate::optimizer::property::{FieldOrder, Order}; +use crate::optimizer::property::Order; use crate::planner::Planner; use crate::utils::Condition; @@ -50,7 +51,7 @@ impl Planner { .. }: BoundSelect, extra_order_exprs: Vec, - order: &[FieldOrder], + order: &[ColumnOrder], ) -> Result { // Append expressions in ORDER BY. if distinct.is_distinct() && !extra_order_exprs.is_empty() { @@ -65,9 +66,7 @@ impl Planner { let mut distinct_on_exprs: HashMap = 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(|FieldOrder { index, .. }| &select_items[*index]); + 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() { match distinct_on_exprs.get_mut(order_expr) { Some(has_been_covered) => { diff --git a/src/frontend/src/planner/set_expr.rs b/src/frontend/src/planner/set_expr.rs index cb32b5930b70..046e14443c22 100644 --- a/src/frontend/src/planner/set_expr.rs +++ b/src/frontend/src/planner/set_expr.rs @@ -13,11 +13,11 @@ // limitations under the License. use risingwave_common::error::Result; +use risingwave_common::util::sort_util::ColumnOrder; use crate::binder::BoundSetExpr; use crate::expr::ExprImpl; use crate::optimizer::plan_node::PlanRef; -use crate::optimizer::property::FieldOrder; use crate::planner::Planner; impl Planner { @@ -25,7 +25,7 @@ impl Planner { &mut self, set_expr: BoundSetExpr, extra_order_exprs: Vec, - order: &[FieldOrder], + order: &[ColumnOrder], ) -> Result { match set_expr { BoundSetExpr::Select(s) => self.plan_select(*s, extra_order_exprs, order), diff --git a/src/frontend/src/utils/column_index_mapping.rs b/src/frontend/src/utils/column_index_mapping.rs index a5da5dd74b33..6b90326ff0af 100644 --- a/src/frontend/src/utils/column_index_mapping.rs +++ b/src/frontend/src/utils/column_index_mapping.rs @@ -16,31 +16,30 @@ use std::vec; use fixedbitset::FixedBitSet; pub use risingwave_common::util::column_index_mapping::ColIndexMapping; +use risingwave_common::util::sort_util::ColumnOrder; use crate::expr::{Expr, ExprImpl, ExprRewriter, InputRef}; use crate::optimizer::property::{ - Distribution, FieldOrder, FunctionalDependency, FunctionalDependencySet, Order, RequiredDist, + Distribution, FunctionalDependency, FunctionalDependencySet, Order, RequiredDist, }; /// Extension trait for [`ColIndexMapping`] to rewrite frontend structures. #[easy_ext::ext(ColIndexMappingRewriteExt)] impl ColIndexMapping { - /// Rewrite the provided order's field index. It will try its best to give the most accurate + /// Rewrite the provided order's column index. It will try its best to give the most accurate /// order. Order(0,1,2) with mapping(0->1,1->0,2->2) will be rewritten to Order(1,0,2) /// Order(0,1,2) with mapping(0->1,2->0) will be rewritten to Order(1) pub fn rewrite_provided_order(&self, order: &Order) -> Order { - let mut mapped_field = vec![]; - for field in &order.field_order { - match self.try_map(field.index) { - Some(mapped_index) => mapped_field.push(FieldOrder { - index: mapped_index, - direct: field.direct, - }), + let mut mapped_column_orders = vec![]; + for column_order in &order.column_orders { + match self.try_map(column_order.column_index) { + Some(mapped_index) => mapped_column_orders + .push(ColumnOrder::new(mapped_index, column_order.order_type)), None => break, } } Order { - field_order: mapped_field, + column_orders: mapped_column_orders, } } @@ -50,17 +49,15 @@ impl ColIndexMapping { /// Order(0,1,2) with mapping(0->1,2->0) will return None pub fn rewrite_required_order(&self, order: &Order) -> Option { order - .field_order + .column_orders .iter() - .map(|field| { - self.try_map(field.index).map(|mapped_index| FieldOrder { - index: mapped_index, - direct: field.direct, - }) + .map(|o| { + self.try_map(o.column_index) + .map(|mapped_index| ColumnOrder::new(mapped_index, o.order_type)) }) .collect::>>() - .map(|mapped_field| Order { - field_order: mapped_field, + .map(|mapped_column_orders| Order { + column_orders: mapped_column_orders, }) } diff --git a/src/storage/hummock_sdk/src/filter_key_extractor.rs b/src/storage/hummock_sdk/src/filter_key_extractor.rs index 1d9817646a58..359374620a62 100644 --- a/src/storage/hummock_sdk/src/filter_key_extractor.rs +++ b/src/storage/hummock_sdk/src/filter_key_extractor.rs @@ -173,9 +173,7 @@ impl SchemaFilterKeyExtractor { let order_types: Vec = table_catalog .pk .iter() - .map(|col_order| { - OrderType::from_protobuf(&col_order.get_order_type().unwrap().direction()) - }) + .map(|col_order| OrderType::from_protobuf(col_order.get_order_type().unwrap())) .collect(); Self { @@ -478,7 +476,7 @@ mod tests { let prost_table = build_table_with_prefix_column_num(1); let schema_filter_key_extractor = SchemaFilterKeyExtractor::new(&prost_table); - let order_types: Vec = vec![OrderType::Ascending, OrderType::Ascending]; + let order_types: Vec = vec![OrderType::ascending(), OrderType::ascending()]; let schema = vec![DataType::Int64, DataType::Varchar]; let serializer = OrderedRowSerde::new(schema, order_types); let row = OwnedRow::new(vec![ @@ -513,7 +511,7 @@ mod tests { 1, Arc::new(FilterKeyExtractorImpl::Schema(schema_filter_key_extractor)), ); - let order_types: Vec = vec![OrderType::Ascending, OrderType::Ascending]; + let order_types: Vec = vec![OrderType::ascending(), OrderType::ascending()]; let schema = vec![DataType::Int64, DataType::Varchar]; let serializer = OrderedRowSerde::new(schema, order_types); let row = OwnedRow::new(vec![ @@ -536,7 +534,7 @@ mod tests { let output_key = multi_filter_key_extractor.extract(&full_key); let data_types = vec![DataType::Int64]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let deserializer = OrderedRowSerde::new(data_types, order_types); let pk_prefix_len = deserializer.deserialize_prefix_len(&row_bytes, 1).unwrap(); @@ -551,7 +549,7 @@ mod tests { 2, Arc::new(FilterKeyExtractorImpl::Schema(schema_filter_key_extractor)), ); - let order_types: Vec = vec![OrderType::Ascending, OrderType::Ascending]; + let order_types: Vec = vec![OrderType::ascending(), OrderType::ascending()]; let schema = vec![DataType::Int64, DataType::Varchar]; let serializer = OrderedRowSerde::new(schema, order_types); let row = OwnedRow::new(vec![ @@ -574,7 +572,7 @@ mod tests { let output_key = multi_filter_key_extractor.extract(&full_key); let data_types = vec![DataType::Int64, DataType::Varchar]; - let order_types = vec![OrderType::Ascending, OrderType::Ascending]; + let order_types = vec![OrderType::ascending(), OrderType::ascending()]; let deserializer = OrderedRowSerde::new(data_types, order_types); let pk_prefix_len = deserializer.deserialize_prefix_len(&row_bytes, 1).unwrap(); diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 3bd4c7f44928..d61b831f5709 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -158,9 +158,7 @@ where let order_types: Vec = table_catalog .pk .iter() - .map(|col_order| { - OrderType::from_protobuf(&col_order.get_order_type().unwrap().direction()) - }) + .map(|col_order| OrderType::from_protobuf(col_order.get_order_type().unwrap())) .collect(); let dist_key_indices: Vec = table_catalog .distribution_key diff --git a/src/stream/src/common/table/test_state_table.rs b/src/stream/src/common/table/test_state_table.rs index 832f3f573789..40505c9f7c64 100644 --- a/src/stream/src/common/table/test_state_table.rs +++ b/src/stream/src/common/table/test_state_table.rs @@ -40,7 +40,7 @@ async fn test_state_table_update_insert() { ColumnDesc::unnamed(ColumnId::from(2), DataType::Int32), ColumnDesc::unnamed(ColumnId::from(4), DataType::Int32), ]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let pk_index = vec![0_usize]; let read_prefix_len_hint = 1; let table = gen_prost_table( @@ -211,7 +211,7 @@ async fn test_state_table_iter_with_prefix() { let test_env = prepare_hummock_test_env().await; // let pk_columns = vec![0, 1]; leave a message to indicate pk columns - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![ColumnId::from(0), ColumnId::from(1), ColumnId::from(2)]; let column_descs = vec![ @@ -339,7 +339,7 @@ async fn test_state_table_iter_with_pk_range() { let test_env = prepare_hummock_test_env().await; // let pk_columns = vec![0, 1]; leave a message to indicate pk columns - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![ColumnId::from(0), ColumnId::from(1), ColumnId::from(2)]; let column_descs = vec![ @@ -480,7 +480,7 @@ async fn test_mem_table_assertion() { ColumnDesc::unnamed(ColumnId::from(1), DataType::Int32), ColumnDesc::unnamed(ColumnId::from(2), DataType::Int32), ]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let pk_index = vec![0_usize]; let read_prefix_len_hint = 1; let table = gen_prost_table( @@ -515,7 +515,7 @@ async fn test_state_table_iter_with_value_indices() { const TEST_TABLE_ID: TableId = TableId { table_id: 233 }; let test_env = prepare_hummock_test_env().await; - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![ColumnId::from(0), ColumnId::from(1), ColumnId::from(2)]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), @@ -676,7 +676,7 @@ async fn test_state_table_iter_with_shuffle_value_indices() { const TEST_TABLE_ID: TableId = TableId { table_id: 233 }; let test_env = prepare_hummock_test_env().await; - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![ColumnId::from(0), ColumnId::from(1), ColumnId::from(2)]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), @@ -918,7 +918,7 @@ async fn test_state_table_write_chunk() { DataType::Boolean, DataType::Float32, ]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let pk_index = vec![0_usize]; let read_prefix_len_hint = 0; let table = gen_prost_table( @@ -1047,7 +1047,7 @@ async fn test_state_table_write_chunk_visibility() { DataType::Boolean, DataType::Float32, ]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let pk_index = vec![0_usize]; let read_prefix_len_hint = 0; let table = gen_prost_table( @@ -1173,7 +1173,7 @@ async fn test_state_table_write_chunk_value_indices() { DataType::Boolean, DataType::Float32, ]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let pk_index = vec![0_usize]; let read_prefix_len_hint = 0; let table = gen_prost_table_with_value_indices( @@ -1276,7 +1276,7 @@ async fn test_state_table_may_exist() { let test_env = prepare_hummock_test_env().await; // let pk_columns = vec![0, 1]; leave a message to indicate pk columns - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![ColumnId::from(0), ColumnId::from(1), ColumnId::from(2)]; let column_descs = vec![ diff --git a/src/stream/src/common/table/test_storage_table.rs b/src/stream/src/common/table/test_storage_table.rs index 2e5e482cac67..77cb3708489f 100644 --- a/src/stream/src/common/table/test_storage_table.rs +++ b/src/stream/src/common/table/test_storage_table.rs @@ -50,7 +50,7 @@ async fn test_storage_table_value_indices() { ColumnDesc::unnamed(column_ids[4], DataType::Varchar), ]; let pk_indices = vec![0_usize, 2_usize]; - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let value_indices = vec![1, 3, 4]; let read_prefix_len_hint = 2; let table = gen_prost_table_with_value_indices( @@ -179,7 +179,7 @@ async fn test_shuffled_column_id_for_storage_table_get_row() { ColumnDesc::unnamed(column_ids[2], DataType::Int32), ]; - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let pk_indices = vec![0_usize, 1_usize]; let read_prefix_len_hint = 2; let table = gen_prost_table( @@ -281,7 +281,7 @@ async fn test_row_based_storage_table_point_get_in_batch_mode() { ColumnDesc::unnamed(column_ids[2], DataType::Int32), ]; let pk_indices = vec![0_usize, 1_usize]; - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let value_indices: Vec = vec![0, 1, 2]; let read_prefix_len_hint = 0; let table = gen_prost_table_with_value_indices( @@ -376,7 +376,7 @@ async fn test_batch_scan_with_value_indices() { const TEST_TABLE_ID: TableId = TableId { table_id: 233 }; let test_env = prepare_hummock_test_env().await; - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![ ColumnId::from(0), ColumnId::from(1), diff --git a/src/stream/src/common/table/test_utils.rs b/src/stream/src/common/table/test_utils.rs index 426a254e639e..1d317cf80933 100644 --- a/src/stream/src/common/table/test_utils.rs +++ b/src/stream/src/common/table/test_utils.rs @@ -17,7 +17,7 @@ use risingwave_common::catalog::{ColumnDesc, TableId}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::Table as ProstTable; -use risingwave_pb::common::{PbColumnOrder, PbOrderType}; +use risingwave_pb::common::PbColumnOrder; use risingwave_pb::plan_common::ColumnCatalog; pub(crate) fn gen_prost_table( @@ -51,9 +51,7 @@ pub(crate) fn gen_prost_table_with_value_indices( .zip_eq_fast(order_types.iter()) .map(|(idx, order)| PbColumnOrder { column_index: *idx as _, - order_type: Some(PbOrderType { - direction: order.to_protobuf() as _, - }), + order_type: Some(order.to_protobuf()), }) .collect(); let prost_columns = column_descs diff --git a/src/stream/src/executor/aggregation/agg_call.rs b/src/stream/src/executor/aggregation/agg_call.rs index 3221ea90cc8b..bfe86ca8d4c5 100644 --- a/src/stream/src/executor/aggregation/agg_call.rs +++ b/src/stream/src/executor/aggregation/agg_call.rs @@ -15,7 +15,7 @@ use std::slice; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_expr::expr::{AggKind, ExpressionRef}; /// An aggregation function may accept 0, 1 or 2 arguments. @@ -62,7 +62,7 @@ pub struct AggCall { pub return_type: DataType, /// Order requirements specified in order by clause of agg call - pub order_pairs: Vec, + pub column_orders: Vec, /// Whether the stream is append-only. /// Specific streaming aggregator may optimize its implementation diff --git a/src/stream/src/executor/aggregation/distinct.rs b/src/stream/src/executor/aggregation/distinct.rs index 924f6ce0f225..285d979d247e 100644 --- a/src/stream/src/executor/aggregation/distinct.rs +++ b/src/stream/src/executor/aggregation/distinct.rs @@ -275,7 +275,7 @@ mod tests { return_type: DataType::Int64, distinct, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, } @@ -310,12 +310,12 @@ mod tests { // group key columns for data_type in group_key_types { add_column_desc(data_type.clone()); - order_types.push(OrderType::Ascending); + order_types.push(OrderType::ascending()); } // distinct key column add_column_desc(indices_and_calls[0].1.args.arg_types()[0].clone()); - order_types.push(OrderType::Ascending); + order_types.push(OrderType::ascending()); // count columns for (_, _) in indices_and_calls { diff --git a/src/stream/src/executor/aggregation/minput.rs b/src/stream/src/executor/aggregation/minput.rs index a390ecb3b1f0..588a7f3d8009 100644 --- a/src/stream/src/executor/aggregation/minput.rs +++ b/src/stream/src/executor/aggregation/minput.rs @@ -81,22 +81,22 @@ impl MaterializedInputState { // `min`/`max` need not to order by any other columns, but have to // order by the agg value implicitly. let order_type = if agg_call.kind == AggKind::Min { - OrderType::Ascending + OrderType::ascending() } else { - OrderType::Descending + OrderType::descending() }; (vec![arg_col_indices[0]], vec![order_type]) } else { agg_call - .order_pairs + .column_orders .iter() - .map(|p| (p.column_idx, p.order_type)) + .map(|p| (p.column_index, p.order_type)) .unzip() }; let pk_len = pk_indices.len(); order_col_indices.extend(pk_indices.iter()); - order_types.extend(itertools::repeat_n(OrderType::Ascending, pk_len)); + order_types.extend(itertools::repeat_n(OrderType::ascending(), pk_len)); // map argument columns to state table column indices let state_table_arg_col_indices = arg_col_indices @@ -290,7 +290,7 @@ mod tests { use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; - use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::expr::AggKind; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::StateStore; @@ -350,7 +350,7 @@ mod tests { kind, args: AggArgs::Unary(arg_type.clone(), arg_idx), return_type: arg_type, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -376,8 +376,8 @@ mod tests { &input_schema, vec![2, 3], vec![ - OrderType::Ascending, // for AggKind::Min - OrderType::Ascending, + OrderType::ascending(), // for AggKind::Min + OrderType::ascending(), ], ) .await; @@ -485,8 +485,8 @@ mod tests { &input_schema, vec![2, 3], vec![ - OrderType::Descending, // for AggKind::Max - OrderType::Ascending, + OrderType::descending(), // for AggKind::Max + OrderType::ascending(), ], ) .await; @@ -595,8 +595,8 @@ mod tests { &input_schema, vec![0, 3], vec![ - OrderType::Ascending, // for AggKind::Min - OrderType::Ascending, + OrderType::ascending(), // for AggKind::Min + OrderType::ascending(), ], ) .await; @@ -604,8 +604,8 @@ mod tests { &input_schema, vec![1, 3], vec![ - OrderType::Descending, // for AggKind::Max - OrderType::Ascending, + OrderType::descending(), // for AggKind::Max + OrderType::ascending(), ], ) .await; @@ -704,9 +704,9 @@ mod tests { &input_schema, vec![2, 1, 3], vec![ - OrderType::Ascending, // c ASC - OrderType::Descending, // b DESC for AggKind::Max - OrderType::Ascending, // _row_id ASC + OrderType::ascending(), // c ASC + OrderType::descending(), // b DESC for AggKind::Max + OrderType::ascending(), // _row_id ASC ], ) .await; @@ -811,8 +811,8 @@ mod tests { &input_schema, vec![0, 1], vec![ - OrderType::Ascending, // for AggKind::Min - OrderType::Ascending, + OrderType::ascending(), // for AggKind::Min + OrderType::ascending(), ], ) .await; @@ -925,8 +925,8 @@ mod tests { &input_schema, vec![0, 1], vec![ - OrderType::Ascending, // for AggKind::Min - OrderType::Ascending, + OrderType::ascending(), // for AggKind::Min + OrderType::ascending(), ], ) .await; @@ -1044,9 +1044,9 @@ mod tests { kind: AggKind::StringAgg, args: AggArgs::Binary([DataType::Varchar, DataType::Varchar], [0, 1]), return_type: DataType::Varchar, - order_pairs: vec![ - OrderPair::new(2, OrderType::Ascending), // b ASC - OrderPair::new(0, OrderType::Descending), // a DESC + column_orders: vec![ + ColumnOrder::new(2, OrderType::ascending()), // b ASC + ColumnOrder::new(0, OrderType::descending()), // a DESC ], append_only: false, filter: None, @@ -1058,9 +1058,9 @@ mod tests { &input_schema, vec![2, 0, 4, 1], vec![ - OrderType::Ascending, // b ASC - OrderType::Descending, // a DESC - OrderType::Ascending, // _row_id ASC + OrderType::ascending(), // b ASC + OrderType::descending(), // a DESC + OrderType::ascending(), // _row_id ASC ], ) .await; @@ -1146,9 +1146,9 @@ mod tests { kind: AggKind::ArrayAgg, args: AggArgs::Unary(DataType::Int32, 1), // array_agg(b) return_type: DataType::Int32, - order_pairs: vec![ - OrderPair::new(2, OrderType::Ascending), // c ASC - OrderPair::new(0, OrderType::Descending), // a DESC + column_orders: vec![ + ColumnOrder::new(2, OrderType::ascending()), // c ASC + ColumnOrder::new(0, OrderType::descending()), // a DESC ], append_only: false, filter: None, @@ -1160,9 +1160,9 @@ mod tests { &input_schema, vec![2, 0, 3, 1], vec![ - OrderType::Ascending, // c ASC - OrderType::Descending, // a DESC - OrderType::Ascending, // _row_id ASC + OrderType::ascending(), // c ASC + OrderType::descending(), // a DESC + OrderType::ascending(), // _row_id ASC ], ) .await; diff --git a/src/stream/src/executor/aggregation/value.rs b/src/stream/src/executor/aggregation/value.rs index b0b64a1a5339..d2afb071d27d 100644 --- a/src/stream/src/executor/aggregation/value.rs +++ b/src/stream/src/executor/aggregation/value.rs @@ -90,7 +90,7 @@ mod tests { kind: risingwave_expr::expr::AggKind::Count, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -133,7 +133,7 @@ mod tests { kind: risingwave_expr::expr::AggKind::Max, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: true, filter: None, distinct: false, diff --git a/src/stream/src/executor/backfill.rs b/src/stream/src/executor/backfill.rs index ef8a0dfc00cf..338963018cba 100644 --- a/src/stream/src/executor/backfill.rs +++ b/src/stream/src/executor/backfill.rs @@ -25,7 +25,7 @@ use risingwave_common::buffer::BitmapBuilder; use risingwave_common::catalog::Schema; use risingwave_common::row::{self, OwnedRow, Row, RowExt}; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::sort_util::{Direction, OrderType}; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -372,9 +372,11 @@ where .project(table_pk_indices) .iter() .zip_eq_fast(pk_order.iter()) - .cmp_by(current_pos.iter(), |(x, order), y| match order { - OrderType::Ascending => x.cmp(&y), - OrderType::Descending => y.cmp(&x), + .cmp_by(current_pos.iter(), |(x, order), y| { + match order.direction() { + Direction::Ascending => x.cmp(&y), + Direction::Descending => y.cmp(&x), + } }) { Ordering::Less | Ordering::Equal => true, Ordering::Greater => false, diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs index 98768c1aa2b4..177f931ef8c3 100644 --- a/src/stream/src/executor/dynamic_filter.rs +++ b/src/stream/src/executor/dynamic_filter.rs @@ -501,7 +501,7 @@ mod tests { mem_state.clone(), TableId::new(0), vec![column_descs.clone()], - vec![OrderType::Ascending], + vec![OrderType::ascending()], vec![0], ) .await; @@ -509,7 +509,7 @@ mod tests { mem_state, TableId::new(1), vec![column_descs], - vec![OrderType::Ascending], + vec![OrderType::ascending()], vec![0], ) .await; diff --git a/src/stream/src/executor/global_simple_agg.rs b/src/stream/src/executor/global_simple_agg.rs index ed9c82e67c3e..827aedc38597 100644 --- a/src/stream/src/executor/global_simple_agg.rs +++ b/src/stream/src/executor/global_simple_agg.rs @@ -409,7 +409,7 @@ mod tests { kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -418,7 +418,7 @@ mod tests { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -427,7 +427,7 @@ mod tests { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 1), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -436,7 +436,7 @@ mod tests { kind: AggKind::Min, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index 60554ad077b0..b6c41e5846a3 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -757,7 +757,7 @@ mod tests { kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -766,7 +766,7 @@ mod tests { kind: AggKind::Count, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -775,7 +775,7 @@ mod tests { kind: AggKind::Count, args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -862,7 +862,7 @@ mod tests { kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -871,7 +871,7 @@ mod tests { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 1), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -881,7 +881,7 @@ mod tests { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 2), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -969,7 +969,7 @@ mod tests { kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -978,7 +978,7 @@ mod tests { kind: AggKind::Min, args: AggArgs::Unary(DataType::Int64, 1), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -1071,7 +1071,7 @@ mod tests { kind: AggKind::Count, // as row count, index: 0 args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -1080,7 +1080,7 @@ mod tests { kind: AggKind::Min, args: AggArgs::Unary(DataType::Int64, 1), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index 470026ee75fd..5c5674321733 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -1101,7 +1101,7 @@ mod tests { let (state_l, degree_state_l) = create_in_memory_state_table( mem_state.clone(), &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &[0, 1], 0, join_key_indices.len(), @@ -1111,7 +1111,7 @@ mod tests { let (state_r, degree_state_r) = create_in_memory_state_table( mem_state, &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &[0, 1], 2, join_key_indices.len(), @@ -1171,9 +1171,9 @@ mod tests { mem_state.clone(), &[DataType::Int64, DataType::Int64, DataType::Int64], &[ - OrderType::Ascending, - OrderType::Ascending, - OrderType::Ascending, + OrderType::ascending(), + OrderType::ascending(), + OrderType::ascending(), ], &[0, 1, 0], 0, @@ -1185,9 +1185,9 @@ mod tests { mem_state, &[DataType::Int64, DataType::Int64, DataType::Int64], &[ - OrderType::Ascending, - OrderType::Ascending, - OrderType::Ascending, + OrderType::ascending(), + OrderType::ascending(), + OrderType::ascending(), ], &[0, 1, 1], 0, diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index 4821edecf93a..73ddc2b8e40c 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -58,7 +58,7 @@ async fn test_merger_sum_aggr() { kind: AggKind::Count, args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -67,7 +67,7 @@ async fn test_merger_sum_aggr() { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -154,7 +154,7 @@ async fn test_merger_sum_aggr() { kind: AggKind::Sum0, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -163,7 +163,7 @@ async fn test_merger_sum_aggr() { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 1), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, @@ -172,7 +172,7 @@ async fn test_merger_sum_aggr() { kind: AggKind::Count, // as row count, index: 2 args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only, filter: None, distinct: false, diff --git a/src/stream/src/executor/local_simple_agg.rs b/src/stream/src/executor/local_simple_agg.rs index b0c5889fd8d5..457a9a351272 100644 --- a/src/stream/src/executor/local_simple_agg.rs +++ b/src/stream/src/executor/local_simple_agg.rs @@ -203,7 +203,7 @@ mod tests { kind: AggKind::Count, args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -262,7 +262,7 @@ mod tests { kind: AggKind::Count, args: AggArgs::None, return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -271,7 +271,7 @@ mod tests { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 0), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, @@ -280,7 +280,7 @@ mod tests { kind: AggKind::Sum, args: AggArgs::Unary(DataType::Int64, 1), return_type: DataType::Int64, - order_pairs: vec![], + column_orders: vec![], append_only: false, filter: None, distinct: false, diff --git a/src/stream/src/executor/lookup/impl_.rs b/src/stream/src/executor/lookup/impl_.rs index d80ef825f016..12b7610f1635 100644 --- a/src/stream/src/executor/lookup/impl_.rs +++ b/src/stream/src/executor/lookup/impl_.rs @@ -20,7 +20,7 @@ use risingwave_common::catalog::{ColumnDesc, Schema}; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -54,7 +54,7 @@ pub struct LookupExecutorParams { /// should contain all 3 columns. pub arrangement_col_descs: Vec, - /// Should only contain [`OrderPair`] for arrange in the arrangement. + /// Should only contain [`ColumnOrder`] for arrange in the arrangement. /// /// Still using the above `a, b, _row_id` example. If we create an arrangement with join key /// `a`, there will be 3 elements in `arrangement_col_descs`, and only 1 element in @@ -65,7 +65,7 @@ pub struct LookupExecutorParams { /// /// For the MV pk, they will only be contained in `arrangement_col_descs`, without being part /// of this `arrangement_order_rules`. - pub arrangement_order_rules: Vec, + pub arrangement_order_rules: Vec, /// Primary key indices of the lookup result (after reordering). /// @@ -161,7 +161,7 @@ impl LookupExecutor { arrange_join_key_indices.sort_unstable(); let mut arrangement_order_types_indices = arrangement_order_rules .iter() - .map(|x| x.column_idx) + .map(|x| x.column_index) .collect_vec(); arrangement_order_types_indices.sort_unstable(); assert_eq!( @@ -182,7 +182,7 @@ impl LookupExecutor { // resolve mapping from join keys in stream row -> joins keys for arrangement. let key_indices_mapping = arrangement_order_rules .iter() - .map(|x| x.column_idx) // the required column idx in this position + .map(|x| x.column_index) // the required column idx in this position .filter_map(|x| arrange_join_key_indices.iter().position(|y| *y == x)) // the position of the item in join keys .map(|x| stream_join_key_indices[x]) // the actual column idx in stream .collect_vec(); diff --git a/src/stream/src/executor/lookup/sides.rs b/src/stream/src/executor/lookup/sides.rs index e4ab39327c53..f050205f3900 100644 --- a/src/stream/src/executor/lookup/sides.rs +++ b/src/stream/src/executor/lookup/sides.rs @@ -19,7 +19,7 @@ use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::ColumnDesc; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::StateStore; @@ -61,7 +61,7 @@ pub(crate) struct ArrangeJoinSide { /// Order rules of the arrangement (only join key is needed, pk should not be included, used /// for lookup) - pub order_rules: Vec, + pub order_rules: Vec, /// Key indices for the join /// diff --git a/src/stream/src/executor/lookup/tests.rs b/src/stream/src/executor/lookup/tests.rs index 22fed3d52880..66898dbd0d48 100644 --- a/src/stream/src/executor/lookup/tests.rs +++ b/src/stream/src/executor/lookup/tests.rs @@ -22,7 +22,7 @@ use risingwave_common::array::stream_chunk::StreamChunkTestExt; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::{ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, TableId}; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -52,15 +52,15 @@ fn arrangement_col_descs() -> Vec { ] } -fn arrangement_col_arrange_rules() -> Vec { +fn arrangement_col_arrange_rules() -> Vec { vec![ - OrderPair::new(1, OrderType::Ascending), - OrderPair::new(0, OrderType::Ascending), + ColumnOrder::new(1, OrderType::ascending()), + ColumnOrder::new(0, OrderType::ascending()), ] } -fn arrangement_col_arrange_rules_join_key() -> Vec { - vec![OrderPair::new(1, OrderType::Ascending)] +fn arrangement_col_arrange_rules_join_key() -> Vec { + vec![ColumnOrder::new(1, OrderType::ascending())] } /// Create a test arrangement. diff --git a/src/stream/src/executor/managed_state/join/mod.rs b/src/stream/src/executor/managed_state/join/mod.rs index 33456ff859f9..fc7fae8789c6 100644 --- a/src/stream/src/executor/managed_state/join/mod.rs +++ b/src/stream/src/executor/managed_state/join/mod.rs @@ -270,7 +270,7 @@ impl JoinHashMap { .collect(); let pk_serializer = OrderedRowSerde::new( pk_data_types, - vec![OrderType::Ascending; state_pk_indices.len()], + vec![OrderType::ascending(); state_pk_indices.len()], ); let state = TableInner { diff --git a/src/stream/src/executor/managed_state/top_n/top_n_state.rs b/src/stream/src/executor/managed_state/top_n/top_n_state.rs index 41a991d1b050..40441f77bb15 100644 --- a/src/stream/src/executor/managed_state/top_n/top_n_state.rs +++ b/src/stream/src/executor/managed_state/top_n/top_n_state.rs @@ -251,7 +251,7 @@ impl ManagedTopNState { mod tests { use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::DataType; - use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; // use std::collections::BTreeMap; use super::*; @@ -264,11 +264,11 @@ mod tests { let data_types = vec![DataType::Varchar, DataType::Int64]; let schema = Schema::new(data_types.into_iter().map(Field::unnamed).collect()); let storage_key = vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Ascending), + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::ascending()), ]; let pk = vec![0, 1]; - let order_by = vec![OrderPair::new(0, OrderType::Ascending)]; + let order_by = vec![ColumnOrder::new(0, OrderType::ascending())]; create_cache_key_serde(&storage_key, &pk, &schema, &order_by, &[]) } @@ -278,7 +278,7 @@ mod tests { let state_table = { let mut tb = create_in_memory_state_table( &[DataType::Varchar, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &[0, 1], ) .await; @@ -357,7 +357,7 @@ mod tests { let state_table = { let mut tb = create_in_memory_state_table( &[DataType::Varchar, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &[0, 1], ) .await; diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index 675d35406e89..5ae1d4e5cad6 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -29,7 +29,7 @@ use risingwave_common::types::DataType; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::iter_util::{ZipEqDebug, ZipEqFast}; use risingwave_common::util::ordered::OrderedRowSerde; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_common::util::value_encoding::{BasicSerde, ValueRowSerde}; use risingwave_pb::catalog::Table; use risingwave_storage::mem_table::KeyOp; @@ -69,7 +69,7 @@ impl MaterializeExecutor { pub async fn new( input: BoxedExecutor, store: S, - key: Vec, + key: Vec, executor_id: u64, actor_context: ActorContextRef, vnodes: Option>, @@ -77,7 +77,7 @@ impl MaterializeExecutor { watermark_epoch: AtomicU64Ref, conflict_behavior: ConflictBehavior, ) -> Self { - let arrange_columns: Vec = key.iter().map(|k| k.column_idx).collect(); + let arrange_columns: Vec = key.iter().map(|k| k.column_index).collect(); let schema = input.schema().clone(); @@ -188,13 +188,13 @@ impl MaterializeExecutor { input: BoxedExecutor, store: S, table_id: TableId, - keys: Vec, + keys: Vec, column_ids: Vec, executor_id: u64, watermark_epoch: AtomicU64Ref, conflict_behavior: ConflictBehavior, ) -> Self { - let arrange_columns: Vec = keys.iter().map(|k| k.column_idx).collect(); + let arrange_columns: Vec = keys.iter().map(|k| k.column_index).collect(); let arrange_order_types = keys.iter().map(|k| k.order_type).collect(); let schema = input.schema().clone(); let columns = column_ids @@ -583,7 +583,7 @@ mod tests { use risingwave_common::catalog::{ColumnDesc, ConflictBehavior, Field, Schema, TableId}; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; - use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -629,7 +629,7 @@ mod tests { ], ); - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), ColumnDesc::unnamed(column_ids[1], DataType::Int32), @@ -649,7 +649,7 @@ mod tests { Box::new(source), memory_state_store, table_id, - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], column_ids, 1, Arc::new(AtomicU64::new(0)), @@ -746,7 +746,7 @@ mod tests { ], ); - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), ColumnDesc::unnamed(column_ids[1], DataType::Int32), @@ -766,7 +766,7 @@ mod tests { Box::new(source), memory_state_store, table_id, - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], column_ids, 1, Arc::new(AtomicU64::new(0)), @@ -879,7 +879,7 @@ mod tests { ], ); - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), ColumnDesc::unnamed(column_ids[1], DataType::Int32), @@ -899,7 +899,7 @@ mod tests { Box::new(source), memory_state_store, table_id, - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], column_ids, 1, Arc::new(AtomicU64::new(0)), @@ -1062,7 +1062,7 @@ mod tests { ], ); - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), ColumnDesc::unnamed(column_ids[1], DataType::Int32), @@ -1082,7 +1082,7 @@ mod tests { Box::new(source), memory_state_store, table_id, - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], column_ids, 1, Arc::new(AtomicU64::new(0)), @@ -1195,7 +1195,7 @@ mod tests { ], ); - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), ColumnDesc::unnamed(column_ids[1], DataType::Int32), @@ -1215,7 +1215,7 @@ mod tests { Box::new(source), memory_state_store, table_id, - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], column_ids, 1, Arc::new(AtomicU64::new(0)), diff --git a/src/stream/src/executor/mview/test_utils.rs b/src/stream/src/executor/mview/test_utils.rs index df060d1e43a9..72dd393cb25c 100644 --- a/src/stream/src/executor/mview/test_utils.rs +++ b/src/stream/src/executor/mview/test_utils.rs @@ -25,7 +25,7 @@ use crate::common::table::state_table::StateTable; pub async fn gen_basic_table(row_count: usize) -> StorageTable { let state_store = MemoryStateStore::new(); - let order_types = vec![OrderType::Ascending, OrderType::Descending]; + let order_types = vec![OrderType::ascending(), OrderType::descending()]; let column_ids = vec![0.into(), 1.into(), 2.into()]; let column_descs = vec![ ColumnDesc::unnamed(column_ids[0], DataType::Int32), @@ -45,7 +45,7 @@ pub async fn gen_basic_table(row_count: usize) -> StorageTable state_store.clone(), TableId::from(0x42), column_descs.clone(), - vec![OrderType::Ascending], + vec![OrderType::ascending()], vec![0], vec![0, 1, 2], ); diff --git a/src/stream/src/executor/sort.rs b/src/stream/src/executor/sort.rs index 61dcf9d01b9d..67926152e577 100644 --- a/src/stream/src/executor/sort.rs +++ b/src/stream/src/executor/sort.rs @@ -504,7 +504,7 @@ mod tests { ColumnDesc::unnamed(ColumnId::new(0), DataType::Int64), ColumnDesc::unnamed(ColumnId::new(1), DataType::Int64), ]; - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let pk_indices = create_pk_indices(); StateTable::new_without_distribution( memory_state_store, diff --git a/src/stream/src/executor/sort_buffer.rs b/src/stream/src/executor/sort_buffer.rs index 8d65eba6caf0..f5769660247c 100644 --- a/src/stream/src/executor/sort_buffer.rs +++ b/src/stream/src/executor/sort_buffer.rs @@ -287,7 +287,7 @@ mod tests { let row_pretty = |s: &str| OwnedRow::from_pretty_with_tys(&tys, s); - let order_types = vec![OrderType::Ascending]; + let order_types = vec![OrderType::ascending()]; let mut state_table = StateTable::new_without_distribution( state_store.clone(), table_id, diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index a500354e98dc..0558b70e6f5e 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -496,7 +496,7 @@ mod tests { use risingwave_common::catalog::{ColumnId, ConflictBehavior, Field, Schema, TableId}; use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::types::DataType; - use risingwave_common::util::sort_util::{OrderPair, OrderType}; + use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_connector::source::datagen::DatagenSplit; use risingwave_pb::catalog::StreamSourceInfo; use risingwave_pb::plan_common::RowFormatType as ProstRowFormatType; @@ -663,7 +663,7 @@ mod tests { Box::new(executor), mem_state_store.clone(), TableId::from(0x2333), - vec![OrderPair::new(0, OrderType::Ascending)], + vec![ColumnOrder::new(0, OrderType::ascending())], column_ids, 2, Arc::new(AtomicU64::new(0)), diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index 2a4b32596434..8d86ead85473 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -254,17 +254,17 @@ pub mod agg_executor { }; for idx in group_key_indices { - add_column(*idx, input_fields[*idx].data_type(), OrderType::Ascending); + add_column(*idx, input_fields[*idx].data_type(), OrderType::ascending()); } add_column(agg_call.args.val_indices()[0], agg_call.args.arg_types()[0].clone(), if agg_call.kind == AggKind::Max { - OrderType::Descending + OrderType::descending() } else { - OrderType::Ascending + OrderType::ascending() }); for idx in pk_indices { - add_column(*idx, input_fields[*idx].data_type(), OrderType::Ascending); + add_column(*idx, input_fields[*idx].data_type(), OrderType::ascending()); } let state_table = StateTable::new_without_distribution( @@ -316,7 +316,7 @@ pub mod agg_executor { group_key_indices.iter().for_each(|idx| { add_column_desc(input_fields[*idx].data_type()); - order_types.push(OrderType::Ascending); + order_types.push(OrderType::ascending()); }); agg_calls.iter().for_each(|agg_call| { diff --git a/src/stream/src/executor/top_n/group_top_n.rs b/src/stream/src/executor/top_n/group_top_n.rs index 2aeffa4fff1f..de8a42edb86e 100644 --- a/src/stream/src/executor/top_n/group_top_n.rs +++ b/src/stream/src/executor/top_n/group_top_n.rs @@ -22,7 +22,7 @@ use risingwave_common::hash::HashKey; use risingwave_common::row::RowExt; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_storage::StateStore; use super::top_n_cache::TopNCacheTrait; @@ -44,9 +44,9 @@ impl GroupTopNExecutor, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, group_by: Vec, state_table: StateTable, @@ -98,9 +98,9 @@ impl InnerGroupTopNExecutorNew #[allow(clippy::too_many_arguments)] pub fn new( input_info: ExecutorInfo, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, group_by: Vec, state_table: StateTable, @@ -123,7 +123,7 @@ impl InnerGroupTopNExecutorNew offset: offset_and_limit.0, limit: offset_and_limit.1, managed_state, - storage_key_indices: storage_key.into_iter().map(|op| op.column_idx).collect(), + storage_key_indices: storage_key.into_iter().map(|op| op.column_index).collect(), group_by, caches: GroupTopNCache::new(lru_manager), cache_key_serde, @@ -280,22 +280,22 @@ mod tests { } } - fn storage_key() -> Vec { + fn storage_key() -> Vec { vec![ - OrderPair::new(1, OrderType::Ascending), - OrderPair::new(2, OrderType::Ascending), - OrderPair::new(0, OrderType::Ascending), + ColumnOrder::new(1, OrderType::ascending()), + ColumnOrder::new(2, OrderType::ascending()), + ColumnOrder::new(0, OrderType::ascending()), ] } /// group by 1, order by 2 - fn order_by_1() -> Vec { - vec![OrderPair::new(2, OrderType::Ascending)] + fn order_by_1() -> Vec { + vec![ColumnOrder::new(2, OrderType::ascending())] } /// group by 1,2, order by 0 - fn order_by_2() -> Vec { - vec![OrderPair::new(0, OrderType::Ascending)] + fn order_by_2() -> Vec { + vec![ColumnOrder::new(0, OrderType::ascending())] } fn pk_indices() -> PkIndices { @@ -360,9 +360,9 @@ mod tests { let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64, DataType::Int64], &[ - OrderType::Ascending, - OrderType::Ascending, - OrderType::Ascending, + OrderType::ascending(), + OrderType::ascending(), + OrderType::ascending(), ], &pk_indices(), ) @@ -456,9 +456,9 @@ mod tests { let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64, DataType::Int64], &[ - OrderType::Ascending, - OrderType::Ascending, - OrderType::Ascending, + OrderType::ascending(), + OrderType::ascending(), + OrderType::ascending(), ], &pk_indices(), ) @@ -546,9 +546,9 @@ mod tests { let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64, DataType::Int64], &[ - OrderType::Ascending, - OrderType::Ascending, - OrderType::Ascending, + OrderType::ascending(), + OrderType::ascending(), + OrderType::ascending(), ], &pk_indices(), ) diff --git a/src/stream/src/executor/top_n/group_top_n_appendonly.rs b/src/stream/src/executor/top_n/group_top_n_appendonly.rs index ea2141af2d03..48b31d5d1fc4 100644 --- a/src/stream/src/executor/top_n/group_top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/group_top_n_appendonly.rs @@ -35,7 +35,7 @@ use risingwave_common::hash::HashKey; use risingwave_common::row::{RowDeserializer, RowExt}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_storage::StateStore; use super::group_top_n::GroupTopNCache; @@ -63,9 +63,9 @@ impl pub fn new( input: Box, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, group_by: Vec, state_table: StateTable, @@ -119,9 +119,9 @@ impl #[allow(clippy::too_many_arguments)] pub fn new( input_info: ExecutorInfo, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, group_by: Vec, state_table: StateTable, @@ -144,7 +144,7 @@ impl offset: offset_and_limit.0, limit: offset_and_limit.1, managed_state, - storage_key_indices: storage_key.into_iter().map(|op| op.column_idx).collect(), + storage_key_indices: storage_key.into_iter().map(|op| op.column_index).collect(), group_by, caches: GroupTopNCache::new(lru_manager), cache_key_serde, diff --git a/src/stream/src/executor/top_n/top_n_appendonly.rs b/src/stream/src/executor/top_n/top_n_appendonly.rs index 0397a286036f..626b45a60f9d 100644 --- a/src/stream/src/executor/top_n/top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/top_n_appendonly.rs @@ -16,7 +16,7 @@ use async_trait::async_trait; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::row::{RowDeserializer, RowExt}; use risingwave_common::util::epoch::EpochPair; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_storage::StateStore; use super::top_n_cache::AppendOnlyTopNCacheTrait; @@ -41,9 +41,9 @@ impl AppendOnlyTopNExecutor { pub fn new_without_ties( input: Box, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -69,9 +69,9 @@ impl AppendOnlyTopNExecutor { pub fn new_with_ties( input: Box, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -113,9 +113,9 @@ impl InnerAppendOnlyTopNExecutor, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -137,7 +137,7 @@ impl InnerAppendOnlyTopNExecutor Vec { + fn storage_key() -> Vec { order_by() } - fn order_by() -> Vec { + fn order_by() -> Vec { vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(1, OrderType::Ascending), + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(1, OrderType::ascending()), ] } @@ -284,7 +284,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -366,7 +366,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; diff --git a/src/stream/src/executor/top_n/top_n_plain.rs b/src/stream/src/executor/top_n/top_n_plain.rs index fc552d8c20d4..8356380e55f3 100644 --- a/src/stream/src/executor/top_n/top_n_plain.rs +++ b/src/stream/src/executor/top_n/top_n_plain.rs @@ -16,7 +16,7 @@ use async_trait::async_trait; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::row::RowExt; use risingwave_common::util::epoch::EpochPair; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_storage::StateStore; use super::utils::*; @@ -37,9 +37,9 @@ impl TopNExecutor { pub fn new_without_ties( input: Box, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -65,9 +65,9 @@ impl TopNExecutor { pub fn new_with_ties( input: Box, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -94,9 +94,9 @@ impl TopNExecutor { pub fn new_with_ties_for_test( input: Box, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -143,9 +143,9 @@ impl InnerTopNExecutorNew { #[allow(clippy::too_many_arguments)] pub fn new( input_info: ExecutorInfo, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, state_table: StateTable, ) -> StreamResult { @@ -166,7 +166,7 @@ impl InnerTopNExecutorNew { identity: format!("TopNExecutor {:X}", executor_id), }, managed_state, - storage_key_indices: storage_key.into_iter().map(|op| op.column_idx).collect(), + storage_key_indices: storage_key.into_iter().map(|op| op.column_index).collect(), cache: TopNCache::new(num_offset, num_limit), cache_key_serde, }) @@ -295,14 +295,14 @@ mod tests { } } - fn storage_key() -> Vec { + fn storage_key() -> Vec { let mut v = order_by(); - v.extend([OrderPair::new(1, OrderType::Ascending)]); + v.extend([ColumnOrder::new(1, OrderType::ascending())]); v } - fn order_by() -> Vec { - vec![OrderPair::new(0, OrderType::Ascending)] + fn order_by() -> Vec { + vec![ColumnOrder::new(0, OrderType::ascending())] } fn pk_indices() -> PkIndices { @@ -334,7 +334,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -430,7 +430,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -538,7 +538,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -645,7 +645,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -847,14 +847,14 @@ mod tests { )) } - fn storage_key() -> Vec { + fn storage_key() -> Vec { order_by() } - fn order_by() -> Vec { + fn order_by() -> Vec { vec![ - OrderPair::new(0, OrderType::Ascending), - OrderPair::new(3, OrderType::Ascending), + ColumnOrder::new(0, OrderType::ascending()), + ColumnOrder::new(3, OrderType::ascending()), ] } @@ -872,7 +872,7 @@ mod tests { DataType::Int64, DataType::Int64, ], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -949,7 +949,7 @@ mod tests { DataType::Int64, DataType::Int64, ], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), state_store.clone(), ) @@ -1001,7 +1001,7 @@ mod tests { DataType::Int64, DataType::Int64, ], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), state_store, ) @@ -1113,14 +1113,14 @@ mod tests { )) } - fn storage_key() -> Vec { + fn storage_key() -> Vec { let mut v = order_by(); - v.push(OrderPair::new(1, OrderType::Ascending)); + v.push(ColumnOrder::new(1, OrderType::ascending())); v } - fn order_by() -> Vec { - vec![OrderPair::new(0, OrderType::Ascending)] + fn order_by() -> Vec { + vec![ColumnOrder::new(0, OrderType::ascending())] } fn pk_indices() -> PkIndices { @@ -1132,7 +1132,7 @@ mod tests { let source = create_source(); let state_table = create_in_memory_state_table( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), ) .await; @@ -1277,7 +1277,7 @@ mod tests { let state_store = MemoryStateStore::new(); let state_table = create_in_memory_state_table_from_state_store( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), state_store.clone(), ) @@ -1330,7 +1330,7 @@ mod tests { let state_table = create_in_memory_state_table_from_state_store( &[DataType::Int64, DataType::Int64], - &[OrderType::Ascending, OrderType::Ascending], + &[OrderType::ascending(), OrderType::ascending()], &pk_indices(), state_store, ) diff --git a/src/stream/src/executor/top_n/utils.rs b/src/stream/src/executor/top_n/utils.rs index 0b5676c90c23..e062b72e8f46 100644 --- a/src/stream/src/executor/top_n/utils.rs +++ b/src/stream/src/executor/top_n/utils.rs @@ -26,7 +26,7 @@ use risingwave_common::row::{CompactedRow, Row, RowDeserializer}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::ordered::OrderedRowSerde; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use super::top_n_cache::CacheKey; use crate::executor::error::{StreamExecutorError, StreamExecutorResult}; @@ -191,16 +191,16 @@ pub fn serialize_pk_to_cache_key(pk: impl Row, cache_key_serde: &CacheKeySerde) pub type CacheKeySerde = (OrderedRowSerde, OrderedRowSerde, usize); pub fn create_cache_key_serde( - storage_key: &[OrderPair], + storage_key: &[ColumnOrder], pk_indices: PkIndicesRef<'_>, schema: &Schema, - order_by: &[OrderPair], + order_by: &[ColumnOrder], group_by: &[usize], ) -> CacheKeySerde { { // validate storage_key = group_by + order_by + additional_pk for i in 0..group_by.len() { - assert_eq!(storage_key[i].column_idx, group_by[i]); + assert_eq!(storage_key[i].column_index, group_by[i]); } for i in group_by.len()..(group_by.len() + order_by.len()) { assert_eq!(storage_key[i], order_by[i - group_by.len()]); @@ -208,7 +208,7 @@ pub fn create_cache_key_serde( let pk_indices = pk_indices.iter().copied().collect::>(); for i in (group_by.len() + order_by.len())..storage_key.len() { assert!( - pk_indices.contains(&storage_key[i].column_idx), + pk_indices.contains(&storage_key[i].column_index), "storage_key = {:?}, pk_indices = {:?}", storage_key, pk_indices @@ -219,7 +219,7 @@ pub fn create_cache_key_serde( let (cache_key_data_types, cache_key_order_types): (Vec<_>, Vec<_>) = storage_key [group_by.len()..] .iter() - .map(|o| (schema[o.column_idx].data_type(), o.order_type)) + .map(|o| (schema[o.column_index].data_type(), o.order_type)) .unzip(); let order_by_len = order_by.len(); diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index a9f6f67b42fa..44b2a5c02f22 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -346,7 +346,7 @@ mod tests { let table = create_in_memory_state_table( mem_state, &[DataType::Int16, WATERMARK_TYPE], - &[OrderType::Ascending], + &[OrderType::ascending()], &[0], &[1], 0, diff --git a/src/stream/src/from_proto/agg_common.rs b/src/stream/src/from_proto/agg_common.rs index bf409a699a41..29f4001837b6 100644 --- a/src/stream/src/from_proto/agg_common.rs +++ b/src/stream/src/from_proto/agg_common.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use risingwave_common::bail; use risingwave_common::buffer::Bitmap; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::expr::{build_from_prost, AggKind}; use super::*; @@ -46,16 +46,15 @@ pub fn build_agg_call_from_prost( ), _ => bail!("Too many/few arguments for {:?}", agg_kind), }; - let order_pairs = agg_call_proto + let column_orders = agg_call_proto .get_order_by() .iter() .map(|col_order| { let col_idx = col_order.get_column_index() as usize; - let order_type = - OrderType::from_protobuf(&col_order.get_order_type().unwrap().direction()); + let order_type = OrderType::from_protobuf(col_order.get_order_type().unwrap()); // TODO(yuchao): `nulls first/last` is not supported yet, so it's ignore here, // see also `risingwave_common::util::sort_util::compare_values` - OrderPair::new(col_idx, order_type) + ColumnOrder::new(col_idx, order_type) }) .collect(); let filter = match agg_call_proto.filter { @@ -66,7 +65,7 @@ pub fn build_agg_call_from_prost( kind: agg_kind, args, return_type: DataType::from(agg_call_proto.get_return_type()?), - order_pairs, + column_orders, append_only, filter, distinct: agg_call_proto.distinct, diff --git a/src/stream/src/from_proto/batch_query.rs b/src/stream/src/from_proto/batch_query.rs index 24ee256da0d0..a03825ab461e 100644 --- a/src/stream/src/from_proto/batch_query.rs +++ b/src/stream/src/from_proto/batch_query.rs @@ -44,7 +44,7 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { let order_types = table_desc .pk .iter() - .map(|desc| OrderType::from_protobuf(&desc.get_order_type().unwrap().direction())) + .map(|desc| OrderType::from_protobuf(desc.get_order_type().unwrap())) .collect_vec(); let column_descs = table_desc diff --git a/src/stream/src/from_proto/chain.rs b/src/stream/src/from_proto/chain.rs index 241c93aaedf1..3258353c4823 100644 --- a/src/stream/src/from_proto/chain.rs +++ b/src/stream/src/from_proto/chain.rs @@ -90,9 +90,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { let order_types = table_desc .pk .iter() - .map(|desc| { - OrderType::from_protobuf(&desc.get_order_type().unwrap().direction()) - }) + .map(|desc| OrderType::from_protobuf(desc.get_order_type().unwrap())) .collect_vec(); let column_descs = table_desc diff --git a/src/stream/src/from_proto/group_top_n.rs b/src/stream/src/from_proto/group_top_n.rs index 3284da6f32c9..9ca78c9a0ba7 100644 --- a/src/stream/src/from_proto/group_top_n.rs +++ b/src/stream/src/from_proto/group_top_n.rs @@ -16,7 +16,7 @@ use std::sync::Arc; use risingwave_common::hash::{HashKey, HashKeyDispatcher}; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::stream_plan::GroupTopNNode; use super::*; @@ -47,14 +47,18 @@ impl ExecutorBuilder for GroupTopNExecutorBuilder { let storage_key = table .get_pk() .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); let [input]: [_; 1] = params.input.try_into().unwrap(); let group_key_types = group_by .iter() .map(|i| input.schema()[*i].data_type()) .collect(); - let order_by = node.order_by.iter().map(OrderPair::from_protobuf).collect(); + let order_by = node + .order_by + .iter() + .map(ColumnOrder::from_protobuf) + .collect(); assert_eq!(¶ms.pk_indices, input.pk_indices()); let args = GroupTopNExecutorDispatcherArgs { @@ -77,9 +81,9 @@ impl ExecutorBuilder for GroupTopNExecutorBuilder { struct GroupTopNExecutorDispatcherArgs { input: BoxedExecutor, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, group_by: Vec, state_table: StateTable, diff --git a/src/stream/src/from_proto/group_top_n_appendonly.rs b/src/stream/src/from_proto/group_top_n_appendonly.rs index 1d8d4137cd9a..4312a4484ba0 100644 --- a/src/stream/src/from_proto/group_top_n_appendonly.rs +++ b/src/stream/src/from_proto/group_top_n_appendonly.rs @@ -30,7 +30,7 @@ use std::sync::Arc; use risingwave_common::hash::{HashKey, HashKeyDispatcher}; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::stream_plan::GroupTopNNode; use super::*; @@ -61,14 +61,18 @@ impl ExecutorBuilder for AppendOnlyGroupTopNExecutorBuilder { let storage_key = table .get_pk() .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); let [input]: [_; 1] = params.input.try_into().unwrap(); let group_key_types = group_by .iter() .map(|i| input.schema()[*i].data_type()) .collect(); - let order_by = node.order_by.iter().map(OrderPair::from_protobuf).collect(); + let order_by = node + .order_by + .iter() + .map(ColumnOrder::from_protobuf) + .collect(); assert_eq!(¶ms.pk_indices, input.pk_indices()); let args = AppendOnlyGroupTopNExecutorDispatcherArgs { @@ -91,9 +95,9 @@ impl ExecutorBuilder for AppendOnlyGroupTopNExecutorBuilder { struct AppendOnlyGroupTopNExecutorDispatcherArgs { input: BoxedExecutor, ctx: ActorContextRef, - storage_key: Vec, + storage_key: Vec, offset_and_limit: (usize, usize), - order_by: Vec, + order_by: Vec, executor_id: u64, group_by: Vec, state_table: StateTable, diff --git a/src/stream/src/from_proto/lookup.rs b/src/stream/src/from_proto/lookup.rs index 166c9c1eff9a..7696cea1d0cf 100644 --- a/src/stream/src/from_proto/lookup.rs +++ b/src/stream/src/from_proto/lookup.rs @@ -13,7 +13,7 @@ // limitations under the License. use risingwave_common::catalog::{ColumnDesc, TableId, TableOption}; -use risingwave_common::util::sort_util::{OrderPair, OrderType}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::plan_common::StorageTableDesc; use risingwave_pb::stream_plan::LookupNode; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -42,7 +42,7 @@ impl ExecutorBuilder for LookupExecutorBuilder { .get_arrangement_table_info()? .arrange_key_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); let arrangement_col_descs = lookup @@ -65,7 +65,7 @@ impl ExecutorBuilder for LookupExecutorBuilder { let order_types = table_desc .pk .iter() - .map(|desc| OrderType::from_protobuf(&desc.get_order_type().unwrap().direction())) + .map(|desc| OrderType::from_protobuf(desc.get_order_type().unwrap())) .collect_vec(); let column_descs = table_desc diff --git a/src/stream/src/from_proto/mview.rs b/src/stream/src/from_proto/mview.rs index 00ba4c4bd5ce..d61bc7ccfe30 100644 --- a/src/stream/src/from_proto/mview.rs +++ b/src/stream/src/from_proto/mview.rs @@ -15,7 +15,7 @@ use std::sync::Arc; use risingwave_common::catalog::ConflictBehavior; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAwareSerde; use risingwave_common::util::value_encoding::BasicSerde; use risingwave_pb::stream_plan::{ArrangeNode, MaterializeNode}; @@ -40,7 +40,7 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { let order_key = node .column_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); let table = node.get_table()?; @@ -104,7 +104,7 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { .get_table_info()? .arrange_key_orders .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) .collect(); let table = node.get_table()?; diff --git a/src/stream/src/from_proto/top_n.rs b/src/stream/src/from_proto/top_n.rs index 757fa2b7b649..df1261112700 100644 --- a/src/stream/src/from_proto/top_n.rs +++ b/src/stream/src/from_proto/top_n.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::stream_plan::TopNNode; use super::*; @@ -41,9 +41,13 @@ impl ExecutorBuilder for TopNExecutorNewBuilder { let storage_key = table .get_pk() .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) + .collect(); + let order_by = node + .order_by + .iter() + .map(ColumnOrder::from_protobuf) .collect(); - let order_by = node.order_by.iter().map(OrderPair::from_protobuf).collect(); assert_eq!(¶ms.pk_indices, input.pk_indices()); if node.with_ties { diff --git a/src/stream/src/from_proto/top_n_appendonly.rs b/src/stream/src/from_proto/top_n_appendonly.rs index f4d06b8b9760..933cf0e5d507 100644 --- a/src/stream/src/from_proto/top_n_appendonly.rs +++ b/src/stream/src/from_proto/top_n_appendonly.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use risingwave_common::util::sort_util::OrderPair; +use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::stream_plan::TopNNode; use super::*; @@ -41,9 +41,13 @@ impl ExecutorBuilder for AppendOnlyTopNExecutorBuilder { let storage_key = table .get_pk() .iter() - .map(OrderPair::from_protobuf) + .map(ColumnOrder::from_protobuf) + .collect(); + let order_by = node + .order_by + .iter() + .map(ColumnOrder::from_protobuf) .collect(); - let order_by = node.order_by.iter().map(OrderPair::from_protobuf).collect(); assert_eq!(¶ms.pk_indices, input.pk_indices()); if node.with_ties {