diff --git a/src/batch/benches/filter.rs b/src/batch/benches/filter.rs index 8cc78dcfbe41..afe3e6a8ad5a 100644 --- a/src/batch/benches/filter.rs +++ b/src/batch/benches/filter.rs @@ -21,7 +21,7 @@ use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::value_encoding::serialize_datum; use risingwave_expr::expr::build_from_prost; use risingwave_pb::data::data_type::TypeName; -use risingwave_pb::data::Datum as ProstDatum; +use risingwave_pb::data::PbDatum; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::{ ConstantValue as TConstValue, Equal, InputRef, Modulus, @@ -53,7 +53,7 @@ fn create_filter_executor(chunk_size: usize, chunk_num: usize) -> BoxedExecutor type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some(ScalarImpl::Int64(2)).as_ref()), })), }; @@ -76,7 +76,7 @@ fn create_filter_executor(chunk_size: usize, chunk_num: usize) -> BoxedExecutor type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some(ScalarImpl::Int64(0)).as_ref()), })), }; diff --git a/src/batch/benches/hash_join.rs b/src/batch/benches/hash_join.rs index fa853bc515db..77f92c23dc18 100644 --- a/src/batch/benches/hash_join.rs +++ b/src/batch/benches/hash_join.rs @@ -24,7 +24,7 @@ use risingwave_common::util::value_encoding::serialize_datum; use risingwave_common::{enable_jemalloc_on_linux, hash}; use risingwave_expr::expr::build_from_prost; use risingwave_pb::data::data_type::TypeName; -use risingwave_pb::data::Datum as ProstDatum; +use risingwave_pb::data::PbDatum; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::{ ConstantValue as TConstValue, GreaterThan, InputRef, Modulus, @@ -58,7 +58,7 @@ fn create_hash_join_executor( type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some(ScalarImpl::Int64(123)).as_ref()), })), }; @@ -88,7 +88,7 @@ fn create_hash_join_executor( type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some(ScalarImpl::Int64(456)).as_ref()), })), }; @@ -141,7 +141,7 @@ fn create_hash_join_executor( type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some(ScalarImpl::Int64(100)).as_ref()), })), }; diff --git a/src/batch/benches/nested_loop_join.rs b/src/batch/benches/nested_loop_join.rs index b8d39508b242..50486193c2f3 100644 --- a/src/batch/benches/nested_loop_join.rs +++ b/src/batch/benches/nested_loop_join.rs @@ -20,7 +20,7 @@ use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::value_encoding::serialize_datum; use risingwave_expr::expr::build_from_prost; use risingwave_pb::data::data_type::TypeName; -use risingwave_pb::data::Datum as ProstDatum; +use risingwave_pb::data::PbDatum; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::{ ConstantValue as TConstValue, Equal, InputRef, Modulus, @@ -68,7 +68,7 @@ fn create_nested_loop_join_executor( type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some(ScalarImpl::Int64(2)).as_ref()), })), }; @@ -79,7 +79,7 @@ fn create_nested_loop_join_executor( type_name: TypeName::Int64 as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some(ScalarImpl::Int64(3)).as_ref()), })), }; diff --git a/src/batch/src/executor/filter.rs b/src/batch/src/executor/filter.rs index 0a4ee63e306f..c7c04f537423 100644 --- a/src/batch/src/executor/filter.rs +++ b/src/batch/src/executor/filter.rs @@ -131,7 +131,7 @@ mod tests { use risingwave_common::util::value_encoding::serialize_datum; use risingwave_expr::expr::build_from_prost; use risingwave_pb::data::data_type::TypeName; - use risingwave_pb::data::Datum as ProstDatum; + use risingwave_pb::data::PbDatum; use risingwave_pb::expr::expr_node::Type::InputRef; use risingwave_pb::expr::expr_node::{RexNode, Type}; use risingwave_pb::expr::{ExprNode, FunctionCall}; @@ -242,7 +242,7 @@ mod tests { }], ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum( Some(ScalarImpl::List(ListValue::new(vec![Some( 2.to_scalar_value(), diff --git a/src/batch/src/executor/generic_exchange.rs b/src/batch/src/executor/generic_exchange.rs index b3f1565352ad..38adff661ad3 100644 --- a/src/batch/src/executor/generic_exchange.rs +++ b/src/batch/src/executor/generic_exchange.rs @@ -21,7 +21,7 @@ use risingwave_common::error::{Result, RwError}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::select_all; use risingwave_pb::batch_plan::plan_node::NodeBody; -use risingwave_pb::batch_plan::ExchangeSource as ProstExchangeSource; +use risingwave_pb::batch_plan::PbExchangeSource; use risingwave_pb::plan_common::Field as NodeField; use risingwave_rpc_client::ComputeClientPoolRef; @@ -36,7 +36,7 @@ use super::BatchTaskMetricsWithTaskLabels; use crate::executor::{BoxedDataChunkStream, BoxedExecutor, BoxedExecutorBuilder, Executor}; pub struct GenericExchangeExecutor { - proto_sources: Vec, + proto_sources: Vec, /// Mock-able CreateSource. source_creators: Vec, context: C, @@ -56,7 +56,7 @@ pub trait CreateSource: Send { async fn create_source( &self, context: impl BatchTaskContext, - prost_source: &ProstExchangeSource, + prost_source: &PbExchangeSource, ) -> Result; } @@ -76,7 +76,7 @@ impl CreateSource for DefaultCreateSource { async fn create_source( &self, context: impl BatchTaskContext, - prost_source: &ProstExchangeSource, + prost_source: &PbExchangeSource, ) -> Result { let peer_addr = prost_source.get_host()?.into(); let task_output_id = prost_source.get_task_output_id()?; @@ -127,7 +127,7 @@ impl BoxedExecutorBuilder for GenericExchangeExecutorBuilder { )?; ensure!(!node.get_sources().is_empty()); - let proto_sources: Vec = node.get_sources().to_vec(); + let proto_sources: Vec = node.get_sources().to_vec(); let source_creators = vec![DefaultCreateSource::new(source.context().client_pool()); proto_sources.len()]; @@ -189,7 +189,7 @@ impl GenericExchangeExec #[try_stream(boxed, ok = DataChunk, error = RwError)] async fn data_chunk_stream( - prost_source: ProstExchangeSource, + prost_source: PbExchangeSource, source_creator: CS, context: C, metrics: Option, @@ -264,7 +264,7 @@ mod tests { let chunks = vec![Some(chunk); 100]; let fake_exchange_source = FakeExchangeSource::new(chunks); let fake_create_source = FakeCreateSource::new(fake_exchange_source); - proto_sources.push(ProstExchangeSource::default()); + proto_sources.push(PbExchangeSource::default()); source_creators.push(fake_create_source); } diff --git a/src/batch/src/executor/hash_agg.rs b/src/batch/src/executor/hash_agg.rs index 42178ca5c7ac..04b70d131514 100644 --- a/src/batch/src/executor/hash_agg.rs +++ b/src/batch/src/executor/hash_agg.rs @@ -270,7 +270,7 @@ mod tests { use risingwave_common::catalog::{Field, Schema}; use risingwave_common::test_prelude::DataChunkTestExt; use risingwave_pb::data::data_type::TypeName; - use risingwave_pb::data::DataType as ProstDataType; + use risingwave_pb::data::PbDataType; use risingwave_pb::expr::agg_call::Type; use risingwave_pb::expr::{AggCall, InputRef}; @@ -309,12 +309,12 @@ mod tests { r#type: Type::Sum as i32, args: vec![InputRef { index: 2, - r#type: Some(ProstDataType { + r#type: Some(PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }), }], - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int64 as i32, ..Default::default() }), @@ -382,7 +382,7 @@ mod tests { let agg_call = AggCall { r#type: Type::Count as i32, args: vec![], - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int64 as i32, ..Default::default() }), diff --git a/src/batch/src/executor/join/local_lookup_join.rs b/src/batch/src/executor/join/local_lookup_join.rs index 82a11c8a82c4..cbcd1dea651f 100644 --- a/src/batch/src/executor/join/local_lookup_join.rs +++ b/src/batch/src/executor/join/local_lookup_join.rs @@ -32,8 +32,8 @@ use risingwave_pb::batch_plan::exchange_info::DistributionMode; use risingwave_pb::batch_plan::exchange_source::LocalExecutePlan::Plan; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ - ExchangeInfo, ExchangeNode, ExchangeSource as ProstExchangeSource, LocalExecutePlan, - PlanFragment, PlanNode, RowSeqScanNode, TaskId as ProstTaskId, TaskOutputId, + ExchangeInfo, ExchangeNode, LocalExecutePlan, PbExchangeSource, PbTaskId, PlanFragment, + PlanNode, RowSeqScanNode, TaskOutputId, }; use risingwave_pb::common::{BatchQueryEpoch, WorkerNode}; use risingwave_pb::plan_common::StorageTableDesc; @@ -113,8 +113,8 @@ impl InnerSideExecutorBuilder { Ok(row_seq_scan_node) } - /// Creates the `ProstExchangeSource` using the given `id`. - fn build_prost_exchange_source(&self, id: &ParallelUnitId) -> Result { + /// Creates the `PbExchangeSource` using the given `id`. + fn build_prost_exchange_source(&self, id: &ParallelUnitId) -> Result { let worker = self.pu_to_worker_mapping.get(id).ok_or_else(|| { internal_error("No worker node found for the given parallel unit id.") })?; @@ -134,9 +134,9 @@ impl InnerSideExecutorBuilder { epoch: Some(self.epoch.clone()), }; - let prost_exchange_source = ProstExchangeSource { + let prost_exchange_source = PbExchangeSource { task_output_id: Some(TaskOutputId { - task_id: Some(ProstTaskId { + task_id: Some(PbTaskId { // FIXME: We should replace this random generated uuid to current query_id for // better dashboard. However, due to the lack of info of // stage_id and task_id, we can not do it now. Now just make sure it will not diff --git a/src/batch/src/executor/join/mod.rs b/src/batch/src/executor/join/mod.rs index 2c25ccbd6769..4e077e3e92d1 100644 --- a/src/batch/src/executor/join/mod.rs +++ b/src/batch/src/executor/join/mod.rs @@ -31,7 +31,7 @@ use risingwave_common::error::Result; use risingwave_common::row::Row; use risingwave_common::types::{DataType, DatumRef}; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_pb::plan_common::JoinType as JoinTypeProst; +use risingwave_pb::plan_common::JoinType as JoinTypePb; use crate::error::BatchError; @@ -61,17 +61,17 @@ impl JoinType { ) } - pub fn from_prost(prost: JoinTypeProst) -> Self { + pub fn from_prost(prost: JoinTypePb) -> Self { match prost { - JoinTypeProst::Inner => JoinType::Inner, - JoinTypeProst::LeftOuter => JoinType::LeftOuter, - JoinTypeProst::LeftSemi => JoinType::LeftSemi, - JoinTypeProst::LeftAnti => JoinType::LeftAnti, - JoinTypeProst::RightOuter => JoinType::RightOuter, - JoinTypeProst::RightSemi => JoinType::RightSemi, - JoinTypeProst::RightAnti => JoinType::RightAnti, - JoinTypeProst::FullOuter => JoinType::FullOuter, - JoinTypeProst::Unspecified => unreachable!(), + JoinTypePb::Inner => JoinType::Inner, + JoinTypePb::LeftOuter => JoinType::LeftOuter, + JoinTypePb::LeftSemi => JoinType::LeftSemi, + JoinTypePb::LeftAnti => JoinType::LeftAnti, + JoinTypePb::RightOuter => JoinType::RightOuter, + JoinTypePb::RightSemi => JoinType::RightSemi, + JoinTypePb::RightAnti => JoinType::RightAnti, + JoinTypePb::FullOuter => JoinType::FullOuter, + JoinTypePb::Unspecified => unreachable!(), } } diff --git a/src/batch/src/executor/merge_sort_exchange.rs b/src/batch/src/executor/merge_sort_exchange.rs index 3108ed5a8f0d..d098cdc279f3 100644 --- a/src/batch/src/executor/merge_sort_exchange.rs +++ b/src/batch/src/executor/merge_sort_exchange.rs @@ -22,7 +22,7 @@ use risingwave_common::error::{Result, RwError}; use risingwave_common::types::ToOwnedDatum; use risingwave_common::util::sort_util::{ColumnOrder, HeapElem}; use risingwave_pb::batch_plan::plan_node::NodeBody; -use risingwave_pb::batch_plan::ExchangeSource as ProstExchangeSource; +use risingwave_pb::batch_plan::PbExchangeSource; use crate::exchange_source::ExchangeSourceImpl; use crate::executor::{ @@ -41,7 +41,7 @@ pub struct MergeSortExchangeExecutorImpl { source_inputs: Vec>, column_orders: Arc>, min_heap: BinaryHeap, - proto_sources: Vec, + proto_sources: Vec, sources: Vec, // impl /// Mock-able CreateSource. source_creators: Vec, @@ -199,7 +199,7 @@ impl BoxedExecutorBuilder for MergeSortExchangeExecutorBuilder { 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(); + let proto_sources: Vec = exchange_node.get_sources().to_vec(); let source_creators = vec![DefaultCreateSource::new(source.context().client_pool()); proto_sources.len()]; ensure!(!exchange_node.get_sources().is_empty()); @@ -253,11 +253,11 @@ mod tests { let fake_exchange_source = FakeExchangeSource::new(vec![Some(chunk)]); let fake_create_source = FakeCreateSource::new(fake_exchange_source); - let mut proto_sources: Vec = vec![]; + let mut proto_sources: Vec = vec![]; let mut source_creators = vec![]; let num_sources = 2; for _ in 0..num_sources { - proto_sources.push(ProstExchangeSource::default()); + proto_sources.push(PbExchangeSource::default()); source_creators.push(fake_create_source.clone()); } let column_orders = Arc::new(vec![ColumnOrder { diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index ff85c382c6f5..a4eab294934d 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -29,7 +29,7 @@ use risingwave_common::util::select_all; 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}; +use risingwave_pb::batch_plan::{scan_range, PbScanRange}; use risingwave_pb::common::BatchQueryEpoch; use risingwave_pb::plan_common::StorageTableDesc; use risingwave_storage::store::PrefetchOptions; @@ -75,7 +75,7 @@ impl ScanRange { /// Create a scan range from the prost representation. pub fn new( - scan_range: ProstScanRange, + scan_range: PbScanRange, mut pk_types: impl Iterator, ) -> Result { let pk_prefix = OwnedRow::new( diff --git a/src/batch/src/executor/sort_agg.rs b/src/batch/src/executor/sort_agg.rs index b967ffc82a50..322335fa91e7 100644 --- a/src/batch/src/executor/sort_agg.rs +++ b/src/batch/src/executor/sort_agg.rs @@ -289,11 +289,11 @@ mod tests { use risingwave_common::types::DataType; use risingwave_expr::expr::build_from_prost; use risingwave_pb::data::data_type::TypeName; - use risingwave_pb::data::DataType as ProstDataType; + use risingwave_pb::data::PbDataType; use risingwave_pb::expr::agg_call::Type; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::InputRef; - use risingwave_pb::expr::{AggCall, ExprNode, InputRef as ProstInputRef}; + use risingwave_pb::expr::{AggCall, ExprNode, PbInputRef}; use super::*; use crate::executor::test_utils::MockExecutor; @@ -442,7 +442,7 @@ mod tests { .map(|idx| { build_from_prost(&ExprNode { expr_type: InputRef as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }), @@ -550,14 +550,14 @@ mod tests { let prost = AggCall { r#type: Type::Sum as i32, - args: vec![ProstInputRef { + args: vec![PbInputRef { index: 0, - r#type: Some(ProstDataType { + r#type: Some(PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }), }], - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int64 as i32, ..Default::default() }), @@ -635,14 +635,14 @@ mod tests { let prost = AggCall { r#type: Type::Sum as i32, - args: vec![ProstInputRef { + args: vec![PbInputRef { index: 0, - r#type: Some(ProstDataType { + r#type: Some(PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }), }], - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int64 as i32, ..Default::default() }), @@ -656,7 +656,7 @@ mod tests { .map(|idx| { build_from_prost(&ExprNode { expr_type: InputRef as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }), @@ -759,14 +759,14 @@ mod tests { let prost = AggCall { r#type: Type::Sum as i32, - args: vec![ProstInputRef { + args: vec![PbInputRef { index: 0, - r#type: Some(ProstDataType { + r#type: Some(PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }), }], - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int64 as i32, ..Default::default() }), @@ -780,7 +780,7 @@ mod tests { .map(|idx| { build_from_prost(&ExprNode { expr_type: InputRef as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }), diff --git a/src/batch/src/executor/test_utils.rs b/src/batch/src/executor/test_utils.rs index cad53e7d70fa..1ddc0dd03699 100644 --- a/src/batch/src/executor/test_utils.rs +++ b/src/batch/src/executor/test_utils.rs @@ -27,7 +27,7 @@ use risingwave_common::row::Row; use risingwave_common::types::{DataType, Datum, ToOwnedDatum}; use risingwave_common::util::iter_util::{ZipEqDebug, ZipEqFast}; use risingwave_expr::expr::BoxedExpression; -use risingwave_pb::batch_plan::ExchangeSource as ProstExchangeSource; +use risingwave_pb::batch_plan::PbExchangeSource; use crate::exchange_source::{ExchangeSource, ExchangeSourceImpl}; use crate::executor::{ @@ -288,7 +288,7 @@ impl CreateSource for FakeCreateSource { async fn create_source( &self, _: impl BatchTaskContext, - _: &ProstExchangeSource, + _: &PbExchangeSource, ) -> Result { Ok(ExchangeSourceImpl::Fake(self.fake_exchange_source.clone())) } diff --git a/src/batch/src/task/data_chunk_in_channel.rs b/src/batch/src/task/data_chunk_in_channel.rs index 984c451e8119..afb74a6e4720 100644 --- a/src/batch/src/task/data_chunk_in_channel.rs +++ b/src/batch/src/task/data_chunk_in_channel.rs @@ -13,7 +13,7 @@ // limitations under the License. use risingwave_common::array::DataChunk; -use risingwave_pb::data::DataChunk as ProstDataChunk; +use risingwave_pb::data::PbDataChunk; use tokio::sync::OnceCell; #[derive(Debug, Clone)] @@ -21,7 +21,7 @@ pub(super) struct DataChunkInChannel { data_chunk: DataChunk, /// If the data chunk is only needed to transfer locally, /// this field should not be initialized. - prost_data_chunk: OnceCell, + prost_data_chunk: OnceCell, } impl DataChunkInChannel { @@ -32,7 +32,7 @@ impl DataChunkInChannel { } } - pub async fn to_protobuf(&self) -> ProstDataChunk { + pub async fn to_protobuf(&self) -> PbDataChunk { let prost_data_chunk = self .prost_data_chunk .get_or_init(|| async { diff --git a/src/batch/src/task/task_execution.rs b/src/batch/src/task/task_execution.rs index 4f37fc4fd291..62502ad471e0 100644 --- a/src/batch/src/task/task_execution.rs +++ b/src/batch/src/task/task_execution.rs @@ -22,9 +22,7 @@ use minitrace::prelude::*; use parking_lot::Mutex; use risingwave_common::array::DataChunk; use risingwave_common::error::{ErrorCode, Result, RwError}; -use risingwave_pb::batch_plan::{ - PlanFragment, TaskId as ProstTaskId, TaskOutputId as ProstOutputId, -}; +use risingwave_pb::batch_plan::{PbTaskId, PbTaskOutputId, PlanFragment}; use risingwave_pb::common::BatchQueryEpoch; use risingwave_pb::task_service::task_info_response::TaskStatus; use risingwave_pb::task_service::{GetDataResponse, TaskInfoResponse}; @@ -139,8 +137,8 @@ impl Debug for TaskOutputId { } } -impl From<&ProstTaskId> for TaskId { - fn from(prost: &ProstTaskId) -> Self { +impl From<&PbTaskId> for TaskId { + fn from(prost: &PbTaskId) -> Self { TaskId { task_id: prost.task_id, stage_id: prost.stage_id, @@ -150,8 +148,8 @@ impl From<&ProstTaskId> for TaskId { } impl TaskId { - pub fn to_prost(&self) -> ProstTaskId { - ProstTaskId { + pub fn to_prost(&self) -> PbTaskId { + PbTaskId { task_id: self.task_id, stage_id: self.stage_id, query_id: self.query_id.clone(), @@ -159,10 +157,10 @@ impl TaskId { } } -impl TryFrom<&ProstOutputId> for TaskOutputId { +impl TryFrom<&PbTaskOutputId> for TaskOutputId { type Error = RwError; - fn try_from(prost: &ProstOutputId) -> Result { + fn try_from(prost: &PbTaskOutputId) -> Result { Ok(TaskOutputId { task_id: TaskId::from(prost.get_task_id()?), output_id: prost.get_output_id(), @@ -171,8 +169,8 @@ impl TryFrom<&ProstOutputId> for TaskOutputId { } impl TaskOutputId { - pub fn to_prost(&self) -> ProstOutputId { - ProstOutputId { + pub fn to_prost(&self) -> PbTaskOutputId { + PbTaskOutputId { task_id: Some(self.task_id.to_prost()), output_id: self.output_id, } @@ -299,7 +297,7 @@ pub struct BatchTaskExecution { impl BatchTaskExecution { pub fn new( - prost_tid: &ProstTaskId, + prost_tid: &PbTaskId, plan: PlanFragment, context: C, epoch: BatchQueryEpoch, @@ -572,7 +570,7 @@ impl BatchTaskExecution { }; } - pub fn get_task_output(&self, output_id: &ProstOutputId) -> Result { + pub fn get_task_output(&self, output_id: &PbTaskOutputId) -> Result { let task_id = TaskId::from(output_id.get_task_id()?); let receiver = self.receivers.lock()[output_id.get_output_id() as usize] .take() diff --git a/src/batch/src/task/task_manager.rs b/src/batch/src/task/task_manager.rs index b9c89c794bff..7f50b43cd581 100644 --- a/src/batch/src/task/task_manager.rs +++ b/src/batch/src/task/task_manager.rs @@ -21,9 +21,7 @@ use parking_lot::Mutex; use risingwave_common::config::BatchConfig; use risingwave_common::error::ErrorCode::{self, TaskNotFound}; use risingwave_common::error::Result; -use risingwave_pb::batch_plan::{ - PlanFragment, TaskId as ProstTaskId, TaskOutputId as ProstTaskOutputId, -}; +use risingwave_pb::batch_plan::{PbTaskId, PbTaskOutputId, PlanFragment}; use risingwave_pb::common::BatchQueryEpoch; use risingwave_pb::task_service::{GetDataResponse, TaskInfoResponse}; use tokio::runtime::Runtime; @@ -84,7 +82,7 @@ impl BatchManager { pub async fn fire_task( &self, - tid: &ProstTaskId, + tid: &PbTaskId, plan: PlanFragment, epoch: BatchQueryEpoch, context: ComputeNodeContext, @@ -116,7 +114,7 @@ impl BatchManager { &self, tx: Sender>, peer_addr: SocketAddr, - pb_task_output_id: &ProstTaskOutputId, + pb_task_output_id: &PbTaskOutputId, ) -> Result<()> { let task_id = TaskOutputId::try_from(pb_task_output_id)?; tracing::trace!(target: "events::compute::exchange", peer_addr = %peer_addr, from = ?task_id, "serve exchange RPC"); @@ -138,7 +136,7 @@ impl BatchManager { Ok(()) } - pub fn take_output(&self, output_id: &ProstTaskOutputId) -> Result { + pub fn take_output(&self, output_id: &PbTaskOutputId) -> Result { let task_id = TaskId::from(output_id.get_task_id()?); self.tasks .lock() @@ -147,7 +145,7 @@ impl BatchManager { .get_task_output(output_id) } - pub fn abort_task(&self, sid: &ProstTaskId, msg: String) { + pub fn abort_task(&self, sid: &PbTaskId, msg: String) { let sid = TaskId::from(sid); match self.tasks.lock().remove(&sid) { Some(task) => { @@ -260,8 +258,8 @@ mod tests { use risingwave_pb::batch_plan::exchange_info::DistributionMode; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ - ExchangeInfo, PlanFragment, PlanNode, TableFunctionNode, TaskId as ProstTaskId, - TaskOutputId as ProstTaskOutputId, ValuesNode, + ExchangeInfo, PbTaskId, PbTaskOutputId, PlanFragment, PlanNode, TableFunctionNode, + ValuesNode, }; use risingwave_pb::expr::table_function::Type; use risingwave_pb::expr::TableFunction; @@ -285,7 +283,7 @@ mod tests { Code::Internal ); - let output_id = ProstTaskOutputId { + let output_id = PbTaskOutputId { task_id: Some(risingwave_pb::batch_plan::TaskId { stage_id: 0, task_id: 0, @@ -317,7 +315,7 @@ mod tests { }), }; let context = ComputeNodeContext::for_test(); - let task_id = ProstTaskId { + let task_id = PbTaskId { query_id: "".to_string(), stage_id: 0, task_id: 0, @@ -373,7 +371,7 @@ mod tests { }), }; let context = ComputeNodeContext::for_test(); - let task_id = ProstTaskId { + let task_id = PbTaskId { query_id: "".to_string(), stage_id: 0, task_id: 0, diff --git a/src/common/src/array/bool_array.rs b/src/common/src/array/bool_array.rs index 6e392b34ce04..13c651546b86 100644 --- a/src/common/src/array/bool_array.rs +++ b/src/common/src/array/bool_array.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_pb::data::{Array as ProstArray, ArrayType}; +use risingwave_pb::data::{ArrayType, PbArray}; use super::{Array, ArrayBuilder, ArrayMeta}; use crate::array::ArrayBuilderImpl; @@ -79,11 +79,11 @@ impl Array for BoolArray { self.data.len() } - fn to_protobuf(&self) -> ProstArray { + fn to_protobuf(&self) -> PbArray { let value = self.data.to_protobuf(); let null_bitmap = self.null_bitmap().to_protobuf(); - ProstArray { + PbArray { null_bitmap: Some(null_bitmap), values: vec![value], array_type: ArrayType::Bool as i32, diff --git a/src/common/src/array/bytes_array.rs b/src/common/src/array/bytes_array.rs index ea27ee37131d..f31c641b4b97 100644 --- a/src/common/src/array/bytes_array.rs +++ b/src/common/src/array/bytes_array.rs @@ -17,7 +17,7 @@ use std::mem::size_of; use risingwave_pb::common::buffer::CompressionType; use risingwave_pb::common::Buffer; -use risingwave_pb::data::{Array as ProstArray, ArrayType}; +use risingwave_pb::data::{ArrayType, PbArray}; use super::{Array, ArrayBuilder, ArrayMeta}; use crate::array::ArrayBuilderImpl; @@ -47,7 +47,7 @@ impl Array for BytesArray { self.offset.len() - 1 } - fn to_protobuf(&self) -> ProstArray { + fn to_protobuf(&self) -> PbArray { let offset_buffer = self .offset .iter() @@ -82,7 +82,7 @@ impl Array for BytesArray { }, ]; let null_bitmap = self.null_bitmap().to_protobuf(); - ProstArray { + PbArray { null_bitmap: Some(null_bitmap), values, array_type: ArrayType::Bytea as i32, diff --git a/src/common/src/array/column.rs b/src/common/src/array/column.rs index edc0952a9d9b..d235acb7874e 100644 --- a/src/common/src/array/column.rs +++ b/src/common/src/array/column.rs @@ -15,7 +15,7 @@ use std::sync::Arc; use futures_async_stream::try_stream; -use risingwave_pb::data::Column as ProstColumn; +use risingwave_pb::data::PbColumn; use super::{Array, ArrayError, ArrayResult, I64Array}; use crate::array::{ArrayImpl, ArrayRef}; @@ -32,12 +32,12 @@ impl Column { Column { array } } - pub fn to_protobuf(&self) -> ProstColumn { + pub fn to_protobuf(&self) -> PbColumn { let array = self.array.to_protobuf(); - ProstColumn { array: Some(array) } + PbColumn { array: Some(array) } } - pub fn from_protobuf(col: &ProstColumn, cardinality: usize) -> ArrayResult { + pub fn from_protobuf(col: &PbColumn, cardinality: usize) -> ArrayResult { Ok(Column { array: Arc::new(ArrayImpl::from_protobuf(col.get_array()?, cardinality)?), }) diff --git a/src/common/src/array/column_proto_readers.rs b/src/common/src/array/column_proto_readers.rs index 65345dce6efe..195f0428bdc9 100644 --- a/src/common/src/array/column_proto_readers.rs +++ b/src/common/src/array/column_proto_readers.rs @@ -17,7 +17,7 @@ use std::io::{Cursor, Read}; use anyhow::anyhow; use byteorder::{BigEndian, ReadBytesExt}; use paste::paste; -use risingwave_pb::data::Array as ProstArray; +use risingwave_pb::data::PbArray; use crate::array::value_reader::{PrimitiveValueReader, VarSizedValueReader}; use crate::array::{ @@ -33,7 +33,7 @@ use crate::types::{NaiveDateTimeWrapper, NaiveDateWrapper, NaiveTimeWrapper}; // https://arrow.apache.org/docs/format/Flight.html pub fn read_numeric_array>( - array: &ProstArray, + array: &PbArray, cardinality: usize, ) -> ArrayResult { ensure!( @@ -60,7 +60,7 @@ pub fn read_numeric_array> Ok(arr.into()) } -pub fn read_bool_array(array: &ProstArray, cardinality: usize) -> ArrayResult { +pub fn read_bool_array(array: &PbArray, cardinality: usize) -> ArrayResult { ensure!( array.get_values().len() == 1, "Must have only 1 buffer in a bool array" @@ -116,7 +116,7 @@ macro_rules! read_one_value_array { ($({ $type:ident, $builder:ty }),*) => { paste! { $( - pub fn [](array: &ProstArray, cardinality: usize) -> ArrayResult { + pub fn [](array: &PbArray, cardinality: usize) -> ArrayResult { ensure!( array.get_values().len() == 1, "Must have only 1 buffer in a {} array", stringify!($type) @@ -159,7 +159,7 @@ fn read_offset(offset_cursor: &mut Cursor<&[u8]>) -> ArrayResult { } pub fn read_string_array>( - array: &ProstArray, + array: &PbArray, cardinality: usize, ) -> ArrayResult { ensure!( diff --git a/src/common/src/array/data_chunk.rs b/src/common/src/array/data_chunk.rs index f597e9a40590..4b23db688480 100644 --- a/src/common/src/array/data_chunk.rs +++ b/src/common/src/array/data_chunk.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use bytes::{Bytes, BytesMut}; use itertools::Itertools; -use risingwave_pb::data::DataChunk as ProstDataChunk; +use risingwave_pb::data::PbDataChunk; use super::{ArrayResult, Vis}; use crate::array::column::Column; @@ -152,12 +152,12 @@ impl DataChunk { &self.columns } - pub fn to_protobuf(&self) -> ProstDataChunk { + pub fn to_protobuf(&self) -> PbDataChunk { assert!( matches!(self.vis2, Vis::Compact(_)), "must be compacted before transfer" ); - let mut proto = ProstDataChunk { + let mut proto = PbDataChunk { cardinality: self.cardinality() as u32, columns: Default::default(), }; @@ -189,7 +189,7 @@ impl DataChunk { } } - pub fn from_protobuf(proto: &ProstDataChunk) -> ArrayResult { + pub fn from_protobuf(proto: &PbDataChunk) -> ArrayResult { let mut columns = vec![]; for any_col in proto.get_columns() { let cardinality = proto.get_cardinality() as usize; diff --git a/src/common/src/array/error.rs b/src/common/src/array/error.rs index ed6decb05068..a18041defe54 100644 --- a/src/common/src/array/error.rs +++ b/src/common/src/array/error.rs @@ -13,15 +13,15 @@ // limitations under the License. pub use anyhow::anyhow; -use risingwave_pb::ProstFieldNotFound; +use risingwave_pb::PbFieldNotFound; use thiserror::Error; use crate::error::{ErrorCode, RwError}; #[derive(Error, Debug)] pub enum ArrayError { - #[error("Prost decode error: {0}")] - ProstDecode(#[from] prost::DecodeError), + #[error("Pb decode error: {0}")] + PbDecode(#[from] prost::DecodeError), #[error("I/O error: {0}")] Io(#[from] std::io::Error), @@ -36,8 +36,8 @@ impl From for RwError { } } -impl From for ArrayError { - fn from(err: ProstFieldNotFound) -> Self { +impl From for ArrayError { + fn from(err: PbFieldNotFound) -> Self { anyhow!("Failed to decode prost: field not found `{}`", err.0).into() } } diff --git a/src/common/src/array/jsonb_array.rs b/src/common/src/array/jsonb_array.rs index ad393bc59d0a..e882c49bc3cc 100644 --- a/src/common/src/array/jsonb_array.rs +++ b/src/common/src/array/jsonb_array.rs @@ -353,7 +353,7 @@ impl Array for JsonbArray { self.data.len() } - fn to_protobuf(&self) -> super::ProstArray { + fn to_protobuf(&self) -> super::PbArray { // The memory layout contains `serde_json::Value` trees, but in protobuf we transmit this as // variable length bytes in value encoding. That is, one buffer of length n+1 containing // start and end offsets into the 2nd buffer containing all value bytes concatenated. @@ -389,10 +389,10 @@ impl Array for JsonbArray { ]; let null_bitmap = self.null_bitmap().to_protobuf(); - super::ProstArray { + super::PbArray { null_bitmap: Some(null_bitmap), values, - array_type: super::ProstArrayType::Jsonb as i32, + array_type: super::PbArrayType::Jsonb as i32, struct_array_data: None, list_array_data: None, } diff --git a/src/common/src/array/list_array.rs b/src/common/src/array/list_array.rs index 1c060b92c8c5..3d5b1e0c3e06 100644 --- a/src/common/src/array/list_array.rs +++ b/src/common/src/array/list_array.rs @@ -20,7 +20,7 @@ use std::hash::Hash; use bytes::{Buf, BufMut}; use itertools::EitherOrBoth::{Both, Left, Right}; use itertools::Itertools; -use risingwave_pb::data::{Array as ProstArray, ArrayType as ProstArrayType, ListArrayData}; +use risingwave_pb::data::{ListArrayData, PbArray, PbArrayType}; use serde::{Deserializer, Serializer}; use super::{Array, ArrayBuilder, ArrayBuilderImpl, ArrayImpl, ArrayMeta, ArrayResult, RowRef}; @@ -174,10 +174,10 @@ impl Array for ListArray { self.bitmap.len() } - fn to_protobuf(&self) -> ProstArray { + fn to_protobuf(&self) -> PbArray { let value = self.value.to_protobuf(); - ProstArray { - array_type: ProstArrayType::List as i32, + PbArray { + array_type: PbArrayType::List as i32, struct_array_data: None, list_array_data: Some(Box::new(ListArrayData { offsets: self.offsets.clone(), @@ -219,7 +219,7 @@ impl Array for ListArray { } impl ListArray { - pub fn from_protobuf(array: &ProstArray) -> ArrayResult { + pub fn from_protobuf(array: &PbArray) -> ArrayResult { ensure!( array.values.is_empty(), "Must have no buffer in a list array" diff --git a/src/common/src/array/mod.rs b/src/common/src/array/mod.rs index 7e955215bccc..881f1b041b4c 100644 --- a/src/common/src/array/mod.rs +++ b/src/common/src/array/mod.rs @@ -58,7 +58,7 @@ pub use jsonb_array::{JsonbArray, JsonbArrayBuilder, JsonbRef, JsonbVal}; pub use list_array::{ListArray, ListArrayBuilder, ListRef, ListValue}; use paste::paste; pub use primitive_array::{PrimitiveArray, PrimitiveArrayBuilder, PrimitiveArrayItemType}; -use risingwave_pb::data::{Array as ProstArray, ArrayType as ProstArrayType}; +use risingwave_pb::data::{PbArray, PbArrayType}; pub use stream_chunk::{Op, StreamChunk, StreamChunkTestExt}; pub use struct_array::{StructArray, StructArrayBuilder, StructRef, StructValue}; pub use utf8_array::*; @@ -221,7 +221,7 @@ pub trait Array: std::fmt::Debug + Send + Sync + Sized + 'static + Into ProstArray; + fn to_protobuf(&self) -> PbArray; /// Get the null `Bitmap` from `Array`. fn null_bitmap(&self) -> &Bitmap; @@ -575,7 +575,7 @@ macro_rules! impl_array { } } - pub fn to_protobuf(&self) -> ProstArray { + pub fn to_protobuf(&self) -> PbArray { match self { $( Self::$variant_name(inner) => inner.to_protobuf(), )* } @@ -662,40 +662,40 @@ impl ArrayImpl { (0..self.len()).map(|i| self.value_at(i)) } - pub fn from_protobuf(array: &ProstArray, cardinality: usize) -> ArrayResult { + pub fn from_protobuf(array: &PbArray, cardinality: usize) -> ArrayResult { use self::column_proto_readers::*; use crate::array::value_reader::*; let array = match array.array_type() { - ProstArrayType::Int16 => read_numeric_array::(array, cardinality)?, - ProstArrayType::Int32 => read_numeric_array::(array, cardinality)?, - ProstArrayType::Int64 => read_numeric_array::(array, cardinality)?, - ProstArrayType::Serial => { + PbArrayType::Int16 => read_numeric_array::(array, cardinality)?, + PbArrayType::Int32 => read_numeric_array::(array, cardinality)?, + PbArrayType::Int64 => read_numeric_array::(array, cardinality)?, + PbArrayType::Serial => { read_numeric_array::(array, cardinality)? } - ProstArrayType::Float32 => { + PbArrayType::Float32 => { read_numeric_array::(array, cardinality)? } - ProstArrayType::Float64 => { + PbArrayType::Float64 => { read_numeric_array::(array, cardinality)? } - ProstArrayType::Bool => read_bool_array(array, cardinality)?, - ProstArrayType::Utf8 => { + PbArrayType::Bool => read_bool_array(array, cardinality)?, + PbArrayType::Utf8 => { read_string_array::(array, cardinality)? } - ProstArrayType::Decimal => { + PbArrayType::Decimal => { read_numeric_array::(array, cardinality)? } - ProstArrayType::Date => read_naive_date_array(array, cardinality)?, - ProstArrayType::Time => read_naive_time_array(array, cardinality)?, - ProstArrayType::Timestamp => read_naive_date_time_array(array, cardinality)?, - ProstArrayType::Interval => read_interval_unit_array(array, cardinality)?, - ProstArrayType::Jsonb => { + PbArrayType::Date => read_naive_date_array(array, cardinality)?, + PbArrayType::Time => read_naive_time_array(array, cardinality)?, + PbArrayType::Timestamp => read_naive_date_time_array(array, cardinality)?, + PbArrayType::Interval => read_interval_unit_array(array, cardinality)?, + PbArrayType::Jsonb => { read_string_array::(array, cardinality)? } - ProstArrayType::Struct => StructArray::from_protobuf(array)?, - ProstArrayType::List => ListArray::from_protobuf(array)?, - ProstArrayType::Unspecified => unreachable!(), - ProstArrayType::Bytea => { + PbArrayType::Struct => StructArray::from_protobuf(array)?, + PbArrayType::List => ListArray::from_protobuf(array)?, + PbArrayType::Unspecified => unreachable!(), + PbArrayType::Bytea => { read_string_array::(array, cardinality)? } }; diff --git a/src/common/src/array/primitive_array.rs b/src/common/src/array/primitive_array.rs index 00601851dc73..b9c0d878283f 100644 --- a/src/common/src/array/primitive_array.rs +++ b/src/common/src/array/primitive_array.rs @@ -18,7 +18,7 @@ use std::mem::size_of; use risingwave_pb::common::buffer::CompressionType; use risingwave_pb::common::Buffer; -use risingwave_pb::data::{Array as ProstArray, ArrayType}; +use risingwave_pb::data::{ArrayType, PbArray}; use super::{Array, ArrayBuilder, ArrayResult}; use crate::array::serial_array::Serial; @@ -188,7 +188,7 @@ impl Array for PrimitiveArray { self.data.len() } - fn to_protobuf(&self) -> ProstArray { + fn to_protobuf(&self) -> PbArray { let mut output_buffer = Vec::::with_capacity(self.len() * size_of::()); for v in self.iter() { @@ -200,7 +200,7 @@ impl Array for PrimitiveArray { body: output_buffer, }; let null_bitmap = self.null_bitmap().to_protobuf(); - ProstArray { + PbArray { null_bitmap: Some(null_bitmap), values: vec![buffer], array_type: T::array_type() as i32, diff --git a/src/common/src/array/stream_chunk.rs b/src/common/src/array/stream_chunk.rs index 46fbb511e060..880a9dac349b 100644 --- a/src/common/src/array/stream_chunk.rs +++ b/src/common/src/array/stream_chunk.rs @@ -15,7 +15,7 @@ use std::fmt; use itertools::Itertools; -use risingwave_pb::data::{Op as ProstOp, StreamChunk as ProstStreamChunk}; +use risingwave_pb::data::{PbOp, PbStreamChunk}; use super::{ArrayResult, DataChunkTestExt}; use crate::array::column::Column; @@ -41,22 +41,22 @@ pub enum Op { } impl Op { - pub fn to_protobuf(self) -> ProstOp { + pub fn to_protobuf(self) -> PbOp { match self { - Op::Insert => ProstOp::Insert, - Op::Delete => ProstOp::Delete, - Op::UpdateInsert => ProstOp::UpdateInsert, - Op::UpdateDelete => ProstOp::UpdateDelete, + Op::Insert => PbOp::Insert, + Op::Delete => PbOp::Delete, + Op::UpdateInsert => PbOp::UpdateInsert, + Op::UpdateDelete => PbOp::UpdateDelete, } } pub fn from_protobuf(prost: &i32) -> ArrayResult { - let op = match ProstOp::from_i32(*prost) { - Some(ProstOp::Insert) => Op::Insert, - Some(ProstOp::Delete) => Op::Delete, - Some(ProstOp::UpdateInsert) => Op::UpdateInsert, - Some(ProstOp::UpdateDelete) => Op::UpdateDelete, - Some(ProstOp::Unspecified) => unreachable!(), + let op = match PbOp::from_i32(*prost) { + Some(PbOp::Insert) => Op::Insert, + Some(PbOp::Delete) => Op::Delete, + Some(PbOp::UpdateInsert) => Op::UpdateInsert, + Some(PbOp::UpdateDelete) => Op::UpdateDelete, + Some(PbOp::Unspecified) => unreachable!(), None => bail!("No such op type"), }; Ok(op) @@ -189,15 +189,15 @@ impl StreamChunk { (self.ops, columns, visibility) } - pub fn to_protobuf(&self) -> ProstStreamChunk { - ProstStreamChunk { + pub fn to_protobuf(&self) -> PbStreamChunk { + PbStreamChunk { cardinality: self.cardinality() as u32, ops: self.ops.iter().map(|op| op.to_protobuf() as i32).collect(), columns: self.columns().iter().map(|col| col.to_protobuf()).collect(), } } - pub fn from_protobuf(prost: &ProstStreamChunk) -> ArrayResult { + pub fn from_protobuf(prost: &PbStreamChunk) -> ArrayResult { let cardinality = prost.get_cardinality() as usize; let mut ops = Vec::with_capacity(cardinality); for op in prost.get_ops() { diff --git a/src/common/src/array/struct_array.rs b/src/common/src/array/struct_array.rs index e1d52cb1a723..2e688396623b 100644 --- a/src/common/src/array/struct_array.rs +++ b/src/common/src/array/struct_array.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use bytes::{Buf, BufMut}; use itertools::Itertools; -use risingwave_pb::data::{Array as ProstArray, ArrayType as ProstArrayType, StructArrayData}; +use risingwave_pb::data::{PbArray, PbArrayType, StructArrayData}; use super::{Array, ArrayBuilder, ArrayBuilderImpl, ArrayImpl, ArrayMeta, ArrayResult}; use crate::array::ArrayRef; @@ -173,11 +173,11 @@ impl Array for StructArray { self.len } - fn to_protobuf(&self) -> ProstArray { + fn to_protobuf(&self) -> PbArray { let children_array = self.children.iter().map(|a| a.to_protobuf()).collect(); let children_type = self.children_type.iter().map(|t| t.to_protobuf()).collect(); - ProstArray { - array_type: ProstArrayType::Struct as i32, + PbArray { + array_type: PbArrayType::Struct as i32, struct_array_data: Some(StructArrayData { children_array, children_type, @@ -220,7 +220,7 @@ impl Array for StructArray { } impl StructArray { - pub fn from_protobuf(array: &ProstArray) -> ArrayResult { + pub fn from_protobuf(array: &PbArray) -> ArrayResult { ensure!( array.values.is_empty(), "Must have no buffer in a struct array" diff --git a/src/common/src/array/utf8_array.rs b/src/common/src/array/utf8_array.rs index b2d080e6cacf..e929d7a145c9 100644 --- a/src/common/src/array/utf8_array.rs +++ b/src/common/src/array/utf8_array.rs @@ -14,7 +14,7 @@ use std::fmt::{Display, Write}; -use risingwave_pb::data::{Array as ProstArray, ArrayType}; +use risingwave_pb::data::{ArrayType, PbArray}; use super::bytes_array::{BytesWriter, PartialBytesWriter}; use super::{Array, ArrayBuilder, ArrayMeta, BytesArray, BytesArrayBuilder}; @@ -43,8 +43,8 @@ impl Array for Utf8Array { } #[inline] - fn to_protobuf(&self) -> ProstArray { - ProstArray { + fn to_protobuf(&self) -> PbArray { + PbArray { array_type: ArrayType::Utf8 as i32, ..self.bytes.to_protobuf() } diff --git a/src/common/src/buffer/bitmap.rs b/src/common/src/buffer/bitmap.rs index d149dc6096c2..d75a150e2c31 100644 --- a/src/common/src/buffer/bitmap.rs +++ b/src/common/src/buffer/bitmap.rs @@ -40,7 +40,7 @@ use std::iter::{self, TrustedLen}; use std::ops::{BitAnd, BitAndAssign, BitOr, BitOrAssign, BitXor, Not, RangeInclusive}; use risingwave_pb::common::buffer::CompressionType; -use risingwave_pb::common::Buffer as ProstBuffer; +use risingwave_pb::common::PbBuffer; #[derive(Default, Debug)] pub struct BitmapBuilder { @@ -525,21 +525,21 @@ impl FromIterator> for Bitmap { } impl Bitmap { - pub fn to_protobuf(&self) -> ProstBuffer { + pub fn to_protobuf(&self) -> PbBuffer { let mut body = Vec::with_capacity((self.num_bits + 7) % 8 + 1); body.push((self.num_bits % 8) as u8); body.extend_from_slice(unsafe { std::slice::from_raw_parts(self.bits.as_ptr() as *const u8, (self.num_bits + 7) / 8) }); - ProstBuffer { + PbBuffer { body, compression: CompressionType::None as i32, } } } -impl From<&ProstBuffer> for Bitmap { - fn from(buf: &ProstBuffer) -> Self { +impl From<&PbBuffer> for Bitmap { + fn from(buf: &PbBuffer) -> Self { let last_byte_num_bits = buf.body[0]; let num_bits = ((buf.body.len() - 1) * 8) - ((8 - last_byte_num_bits) % 8) as usize; @@ -762,7 +762,7 @@ mod tests { #[test] fn test_bitmap_from_protobuf() { let bitmap_bytes = vec![3u8 /* len % BITS */, 0b0101_0010, 0b110]; - let buf = ProstBuffer { + let buf = PbBuffer { body: bitmap_bytes, compression: CompressionType::None as _, }; diff --git a/src/common/src/catalog/schema.rs b/src/common/src/catalog/schema.rs index 7df02894514d..ece895f3ccaf 100644 --- a/src/common/src/catalog/schema.rs +++ b/src/common/src/catalog/schema.rs @@ -15,7 +15,7 @@ use std::ops::Index; use itertools::Itertools; -use risingwave_pb::plan_common::{ColumnDesc as ProstColumnDesc, Field as ProstField}; +use risingwave_pb::plan_common::{PbColumnDesc, PbField}; use super::ColumnDesc; use crate::array::ArrayBuilderImpl; @@ -40,8 +40,8 @@ impl std::fmt::Debug for Field { } impl Field { - pub fn to_prost(&self) -> ProstField { - ProstField { + pub fn to_prost(&self) -> PbField { + PbField { data_type: Some(self.data_type.to_protobuf()), name: self.name.to_string(), } @@ -74,8 +74,8 @@ impl From for Field { } } -impl From<&ProstColumnDesc> for Field { - fn from(pb_column_desc: &ProstColumnDesc) -> Self { +impl From<&PbColumnDesc> for Field { + fn from(pb_column_desc: &PbColumnDesc) -> Self { Self { data_type: pb_column_desc.column_type.as_ref().unwrap().into(), name: pb_column_desc.name.clone(), @@ -162,7 +162,7 @@ impl Schema { .collect() } - pub fn to_prost(&self) -> Vec { + pub fn to_prost(&self) -> Vec { self.fields .clone() .into_iter() @@ -224,8 +224,8 @@ impl Field { } } -impl From<&ProstField> for Field { - fn from(prost_field: &ProstField) -> Self { +impl From<&PbField> for Field { + fn from(prost_field: &PbField) -> Self { Self { data_type: DataType::from(prost_field.get_data_type().expect("data type not found")), name: prost_field.get_name().clone(), diff --git a/src/common/src/error.rs b/src/common/src/error.rs index 6d1de86fdd4e..63f9c2e1d9a3 100644 --- a/src/common/src/error.rs +++ b/src/common/src/error.rs @@ -20,7 +20,7 @@ use std::io::Error as IoError; use std::time::{Duration, SystemTime}; use memcomparable::Error as MemComparableError; -use risingwave_pb::ProstFieldNotFound; +use risingwave_pb::PbFieldNotFound; use thiserror::Error; use tokio::task::JoinError; use tonic::Code; @@ -250,8 +250,8 @@ impl PartialEq for ErrorCode { } } -impl From for RwError { - fn from(err: ProstFieldNotFound) -> Self { +impl From for RwError { + fn from(err: PbFieldNotFound) -> Self { ErrorCode::InternalError(format!( "Failed to decode prost: field not found `{}`", err.0 diff --git a/src/common/src/system_param/reader.rs b/src/common/src/system_param/reader.rs index 723fbd61b28d..bf6de6b672c8 100644 --- a/src/common/src/system_param/reader.rs +++ b/src/common/src/system_param/reader.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_pb::meta::SystemParams as ProstSystemParams; +use risingwave_pb::meta::PbSystemParams; use tracing::warn; use super::system_params_to_kv; @@ -23,11 +23,11 @@ use super::system_params_to_kv; /// compatibility. #[derive(Clone, Debug, PartialEq)] pub struct SystemParamsReader { - prost: ProstSystemParams, + prost: PbSystemParams, } -impl From for SystemParamsReader { - fn from(prost: ProstSystemParams) -> Self { +impl From for SystemParamsReader { + fn from(prost: PbSystemParams) -> Self { Self { prost } } } diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index 55f324ba5ee6..c653e297a517 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -20,7 +20,7 @@ use bytes::{Buf, BufMut, Bytes}; use num_traits::Float; use parse_display::{Display, FromStr}; use postgres_types::FromSql; -use risingwave_pb::data::DataType as ProstDataType; +use risingwave_pb::data::PbDataType; use serde::{Deserialize, Serialize}; use crate::array::{ArrayError, ArrayResult, NULL_VAL_FOR_HASH}; @@ -207,8 +207,8 @@ pub fn unnested_list_type(datatype: DataType) -> DataType { } } -impl From<&ProstDataType> for DataType { - fn from(proto: &ProstDataType) -> DataType { +impl From<&PbDataType> for DataType { + fn from(proto: &PbDataType) -> DataType { match proto.get_type_name().expect("missing type field") { TypeName::Int16 => DataType::Int16, TypeName::Int32 => DataType::Int32, @@ -296,8 +296,8 @@ impl DataType { } } - pub fn to_protobuf(&self) -> ProstDataType { - let mut pb = ProstDataType { + pub fn to_protobuf(&self) -> PbDataType { + let mut pb = PbDataType { type_name: self.prost_type_name() as i32, is_nullable: true, ..Default::default() @@ -397,7 +397,7 @@ impl DataType { } } -impl From for ProstDataType { +impl From for PbDataType { fn from(data_type: DataType) -> Self { data_type.to_protobuf() } diff --git a/src/common/src/util/addr.rs b/src/common/src/util/addr.rs index 6cbd44840510..231a37becae0 100644 --- a/src/common/src/util/addr.rs +++ b/src/common/src/util/addr.rs @@ -16,7 +16,7 @@ use std::net::SocketAddr; use std::str::FromStr; use anyhow::anyhow; -use risingwave_pb::common::HostAddress as ProstHostAddress; +use risingwave_pb::common::PbHostAddress; /// General host address and port. #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -71,8 +71,8 @@ impl FromStr for HostAddr { } } -impl From<&ProstHostAddress> for HostAddr { - fn from(addr: &ProstHostAddress) -> Self { +impl From<&PbHostAddress> for HostAddr { + fn from(addr: &PbHostAddress) -> Self { HostAddr { host: addr.get_host().to_string(), port: addr.get_port() as u16, @@ -81,8 +81,8 @@ impl From<&ProstHostAddress> for HostAddr { } impl HostAddr { - pub fn to_protobuf(&self) -> ProstHostAddress { - ProstHostAddress { + pub fn to_protobuf(&self) -> PbHostAddress { + PbHostAddress { host: self.host.clone(), port: self.port as i32, } diff --git a/src/common/src/util/column_index_mapping.rs b/src/common/src/util/column_index_mapping.rs index ffe67fbe32e3..b5efce2ee049 100644 --- a/src/common/src/util/column_index_mapping.rs +++ b/src/common/src/util/column_index_mapping.rs @@ -17,7 +17,7 @@ use std::fmt::Debug; use std::vec; use itertools::Itertools; -use risingwave_pb::catalog::ColIndexMapping as ProstColIndexMapping; +use risingwave_pb::catalog::PbColIndexMapping; use risingwave_pb::stream_plan::DispatchStrategy; /// `ColIndexMapping` is a partial mapping from usize to usize. @@ -282,8 +282,8 @@ impl ColIndexMapping { } impl ColIndexMapping { - pub fn to_protobuf(&self) -> ProstColIndexMapping { - ProstColIndexMapping { + pub fn to_protobuf(&self) -> PbColIndexMapping { + PbColIndexMapping { target_size: self.target_size as u64, map: self .map @@ -293,7 +293,7 @@ impl ColIndexMapping { } } - pub fn from_protobuf(prost: &ProstColIndexMapping) -> ColIndexMapping { + pub fn from_protobuf(prost: &PbColIndexMapping) -> ColIndexMapping { ColIndexMapping { target_size: prost.target_size as usize, map: prost.map.iter().map(|&x| x.try_into().ok()).collect(), diff --git a/src/common/src/util/scan_range.rs b/src/common/src/util/scan_range.rs index 66c33e656f8e..8fa553a5669b 100644 --- a/src/common/src/util/scan_range.rs +++ b/src/common/src/util/scan_range.rs @@ -15,8 +15,8 @@ use std::ops::{Bound, RangeBounds}; use paste::paste; -use risingwave_pb::batch_plan::scan_range::Bound as BoundProst; -use risingwave_pb::batch_plan::ScanRange as ScanRangeProst; +use risingwave_pb::batch_plan::scan_range::Bound as BoundPb; +use risingwave_pb::batch_plan::ScanRange as ScanRangePb; use super::value_encoding::serialize_datum; use crate::catalog::get_dist_key_in_pk_indices; @@ -26,20 +26,20 @@ use crate::types::{Datum, ScalarImpl}; use crate::util::hash_util::Crc32FastBuilder; use crate::util::value_encoding::serialize_datum_into; -/// See also [`ScanRangeProst`] +/// See also [`ScanRangePb`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct ScanRange { pub eq_conds: Vec, pub range: (Bound, Bound), } -fn bound_to_proto(bound: &Bound) -> Option { +fn bound_to_proto(bound: &Bound) -> Option { match bound { - Bound::Included(literal) => Some(BoundProst { + Bound::Included(literal) => Some(BoundPb { value: serialize_datum(Some(literal)), inclusive: true, }), - Bound::Excluded(literal) => Some(BoundProst { + Bound::Excluded(literal) => Some(BoundPb { value: serialize_datum(Some(literal)), inclusive: false, }), @@ -48,8 +48,8 @@ fn bound_to_proto(bound: &Bound) -> Option { } impl ScanRange { - pub fn to_protobuf(&self) -> ScanRangeProst { - ScanRangeProst { + pub fn to_protobuf(&self) -> ScanRangePb { + ScanRangePb { eq_conds: self .eq_conds .iter() diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index 68da7aa01c14..9e8d0d0a9317 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -41,7 +41,7 @@ use risingwave_common::util::iter_util::ZipEqFast; 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; +use risingwave_pb::plan_common::PbRowFormatType; use risingwave_source::connector_test_utils::create_source_desc_builder; use risingwave_source::dml_manager::DmlManager; use risingwave_storage::memory::MemoryStateStore; @@ -111,7 +111,7 @@ async fn test_table_materialize() -> StreamResult<()> { ], }; let source_info = StreamSourceInfo { - row_format: ProstRowFormatType::Json as i32, + row_format: PbRowFormatType::Json as i32, ..Default::default() }; let properties = convert_args!(hashmap!( diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index 0fa3ea870a3c..87ad834a4f3e 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -341,7 +341,7 @@ mod test { use std::path::PathBuf; - use risingwave_pb::data::data_type::TypeName as ProstTypeName; + use risingwave_pb::data::data_type::PbTypeName; use super::*; @@ -413,35 +413,29 @@ mod test { assert_eq!(columns[2].name, "timestamp".to_string()); let data_type = columns[3].column_type.as_ref().unwrap(); - assert_eq!(data_type.get_type_name().unwrap(), ProstTypeName::List); + assert_eq!(data_type.get_type_name().unwrap(), PbTypeName::List); let inner_field_type = data_type.field_type.clone(); assert_eq!( inner_field_type[0].get_type_name().unwrap(), - ProstTypeName::Struct + PbTypeName::Struct ); let struct_inner = inner_field_type[0].field_type.clone(); - assert_eq!( - struct_inner[0].get_type_name().unwrap(), - ProstTypeName::Int32 - ); - assert_eq!( - struct_inner[1].get_type_name().unwrap(), - ProstTypeName::Int32 - ); + assert_eq!(struct_inner[0].get_type_name().unwrap(), PbTypeName::Int32); + assert_eq!(struct_inner[1].get_type_name().unwrap(), PbTypeName::Int32); assert_eq!( struct_inner[2].get_type_name().unwrap(), - ProstTypeName::Varchar + PbTypeName::Varchar ); assert_eq!(columns[4].name, "contacts".to_string()); let inner_field_type = columns[4].column_type.as_ref().unwrap().field_type.clone(); assert_eq!( inner_field_type[0].get_type_name().unwrap(), - ProstTypeName::List + PbTypeName::List ); assert_eq!( inner_field_type[1].get_type_name().unwrap(), - ProstTypeName::List + PbTypeName::List ); Ok(()) } diff --git a/src/connector/src/sink/catalog/desc.rs b/src/connector/src/sink/catalog/desc.rs index a1b7898df666..6f0cd2da6c35 100644 --- a/src/connector/src/sink/catalog/desc.rs +++ b/src/connector/src/sink/catalog/desc.rs @@ -17,8 +17,8 @@ use std::collections::BTreeMap; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, DatabaseId, SchemaId, TableId, UserId}; use risingwave_common::util::sort_util::ColumnOrder; -use risingwave_pb::plan_common::ColumnDesc as ProstColumnDesc; -use risingwave_pb::stream_plan::SinkDesc as ProstSinkDesc; +use risingwave_pb::plan_common::PbColumnDesc; +use risingwave_pb::stream_plan::PbSinkDesc; use super::{SinkCatalog, SinkId, SinkType}; @@ -80,15 +80,15 @@ impl SinkDesc { } } - pub fn to_proto(&self) -> ProstSinkDesc { - ProstSinkDesc { + pub fn to_proto(&self) -> PbSinkDesc { + PbSinkDesc { id: self.id.sink_id, name: self.name.clone(), definition: self.definition.clone(), columns: self .columns .iter() - .map(|column| Into::::into(&column.column_desc)) + .map(|column| Into::::into(&column.column_desc)) .collect_vec(), plan_pk: self.plan_pk.iter().map(|k| k.to_protobuf()).collect_vec(), downstream_pk: self.downstream_pk.iter().map(|idx| *idx as _).collect_vec(), diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index d221f13b548a..e834a8747178 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -21,7 +21,7 @@ use risingwave_common::catalog::{ ColumnCatalog, DatabaseId, Field, Schema, SchemaId, TableId, UserId, }; use risingwave_common::util::sort_util::ColumnOrder; -use risingwave_pb::catalog::{Sink as ProstSink, SinkType as ProstSinkType}; +use risingwave_pb::catalog::{PbSink, PbSinkType}; #[derive(Clone, Copy, Debug, Default, Hash, PartialOrd, PartialEq, Eq)] pub struct SinkId { @@ -77,20 +77,20 @@ impl SinkType { self == &Self::Upsert } - pub fn to_proto(self) -> ProstSinkType { + pub fn to_proto(self) -> PbSinkType { match self { - SinkType::AppendOnly => ProstSinkType::AppendOnly, - SinkType::ForceAppendOnly => ProstSinkType::ForceAppendOnly, - SinkType::Upsert => ProstSinkType::Upsert, + SinkType::AppendOnly => PbSinkType::AppendOnly, + SinkType::ForceAppendOnly => PbSinkType::ForceAppendOnly, + SinkType::Upsert => PbSinkType::Upsert, } } - pub fn from_proto(pb: ProstSinkType) -> Self { + pub fn from_proto(pb: PbSinkType) -> Self { match pb { - ProstSinkType::AppendOnly => SinkType::AppendOnly, - ProstSinkType::ForceAppendOnly => SinkType::ForceAppendOnly, - ProstSinkType::Upsert => SinkType::Upsert, - ProstSinkType::Unspecified => unreachable!(), + PbSinkType::AppendOnly => SinkType::AppendOnly, + PbSinkType::ForceAppendOnly => SinkType::ForceAppendOnly, + PbSinkType::Upsert => SinkType::Upsert, + PbSinkType::Unspecified => unreachable!(), } } } @@ -141,8 +141,8 @@ pub struct SinkCatalog { } impl SinkCatalog { - pub fn to_proto(&self) -> ProstSink { - ProstSink { + pub fn to_proto(&self) -> PbSink { + PbSink { id: self.id.into(), schema_id: self.schema_id.schema_id, database_id: self.database_id.database_id, @@ -185,8 +185,8 @@ impl SinkCatalog { } } -impl From for SinkCatalog { - fn from(pb: ProstSink) -> Self { +impl From for SinkCatalog { + fn from(pb: PbSink) -> Self { let sink_type = pb.get_sink_type().unwrap(); SinkCatalog { id: pb.id.into(), @@ -218,8 +218,8 @@ impl From for SinkCatalog { } } -impl From<&ProstSink> for SinkCatalog { - fn from(pb: &ProstSink) -> Self { +impl From<&PbSink> for SinkCatalog { + fn from(pb: &PbSink) -> Self { pb.clone().into() } } diff --git a/src/expr/src/error.rs b/src/expr/src/error.rs index d5eeddb11c64..1d03962c6677 100644 --- a/src/expr/src/error.rs +++ b/src/expr/src/error.rs @@ -17,7 +17,7 @@ use std::borrow::Cow; use risingwave_common::array::ArrayError; use risingwave_common::error::{ErrorCode, RwError}; use risingwave_common::types::DataType; -use risingwave_pb::ProstFieldNotFound; +use risingwave_pb::PbFieldNotFound; use thiserror::Error; /// A specialized Result type for expression operations. @@ -89,8 +89,8 @@ impl From for ExprError { } } -impl From for ExprError { - fn from(err: ProstFieldNotFound) -> Self { +impl From for ExprError { + fn from(err: PbFieldNotFound) -> Self { Self::Internal(anyhow::anyhow!( "Failed to decode prost: field not found `{}`", err.0 diff --git a/src/expr/src/expr/build_expr_from_prost.rs b/src/expr/src/expr/build_expr_from_prost.rs index eadc170431e1..50bcef7c3091 100644 --- a/src/expr/src/expr/build_expr_from_prost.rs +++ b/src/expr/src/expr/build_expr_from_prost.rs @@ -455,7 +455,7 @@ mod tests { use risingwave_common::types::Scalar; use risingwave_common::util::value_encoding::serialize_datum; use risingwave_pb::data::data_type::TypeName; - use risingwave_pb::data::{DataType as ProstDataType, Datum as ProstDatum}; + use risingwave_pb::data::{PbDataType, PbDatum}; use risingwave_pb::expr::expr_node::{RexNode, Type}; use risingwave_pb::expr::{ExprNode, FunctionCall}; @@ -467,21 +467,21 @@ mod tests { children: vec![ ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Varchar as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some("foo".into()).as_ref()), })), }, ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Varchar as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some("bar".into()).as_ref()), })), }, @@ -491,9 +491,9 @@ mod tests { children: vec![ ExprNode { expr_type: Type::Array as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::List as i32, - field_type: vec![ProstDataType { + field_type: vec![PbDataType { type_name: TypeName::Varchar as i32, ..Default::default() }], @@ -503,11 +503,11 @@ mod tests { }, ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some(1_i32.to_scalar_value()).as_ref()), })), }, @@ -515,7 +515,7 @@ mod tests { }; let access = ExprNode { expr_type: Type::ArrayAccess as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Varchar as i32, ..Default::default() }), @@ -532,18 +532,18 @@ mod tests { fn test_build_extract_expr() { let left = ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Varchar as i32, precision: 11, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some("DAY".into()).as_ref()), })), }; let right_date = ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Date as i32, ..Default::default() }), @@ -551,7 +551,7 @@ mod tests { }; let right_time = ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Timestamp as i32, ..Default::default() }), @@ -560,7 +560,7 @@ mod tests { let expr = ExprNode { expr_type: Type::Extract as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int64 as i32, ..Default::default() }), @@ -571,7 +571,7 @@ mod tests { assert!(build_binary_expr_prost(&expr).is_ok()); let expr = ExprNode { expr_type: Type::Extract as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int64 as i32, ..Default::default() }), diff --git a/src/expr/src/expr/expr_array_concat.rs b/src/expr/src/expr/expr_array_concat.rs index f904afc7ebb4..08e58d17189e 100644 --- a/src/expr/src/expr/expr_array_concat.rs +++ b/src/expr/src/expr/expr_array_concat.rs @@ -386,8 +386,8 @@ mod tests { use itertools::Itertools; use risingwave_common::array::DataChunk; use risingwave_common::types::ScalarImpl; - use risingwave_pb::data::Datum as ProstDatum; - use risingwave_pb::expr::expr_node::{RexNode, Type as ProstType}; + use risingwave_pb::data::PbDatum; + use risingwave_pb::expr::expr_node::{PbType, RexNode}; use risingwave_pb::expr::{ExprNode, FunctionCall}; use super::*; @@ -395,9 +395,9 @@ mod tests { fn make_i64_expr_node(value: i64) -> ExprNode { ExprNode { - expr_type: ProstType::ConstantValue as i32, + expr_type: PbType::ConstantValue as i32, return_type: Some(DataType::Int64.to_protobuf()), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: value.to_be_bytes().to_vec(), })), } @@ -405,7 +405,7 @@ mod tests { fn make_i64_array_expr_node(values: Vec) -> ExprNode { ExprNode { - expr_type: ProstType::Array as i32, + expr_type: PbType::Array as i32, return_type: Some( DataType::List { datatype: Box::new(DataType::Int64), @@ -420,7 +420,7 @@ mod tests { fn make_i64_array_array_expr_node(values: Vec>) -> ExprNode { ExprNode { - expr_type: ProstType::Array as i32, + expr_type: PbType::Array as i32, return_type: Some( DataType::List { datatype: Box::new(DataType::List { @@ -441,7 +441,7 @@ mod tests { let left = make_i64_array_expr_node(vec![42]); let right = make_i64_array_expr_node(vec![43]); let expr = ExprNode { - expr_type: ProstType::ArrayCat as i32, + expr_type: PbType::ArrayCat as i32, return_type: Some( DataType::List { datatype: Box::new(DataType::Int64), @@ -459,7 +459,7 @@ mod tests { let left = make_i64_array_array_expr_node(vec![vec![42]]); let right = make_i64_array_array_expr_node(vec![vec![43]]); let expr = ExprNode { - expr_type: ProstType::ArrayCat as i32, + expr_type: PbType::ArrayCat as i32, return_type: Some( DataType::List { datatype: Box::new(DataType::Int64), @@ -477,7 +477,7 @@ mod tests { let left = make_i64_array_expr_node(vec![42]); let right = make_i64_expr_node(43); let expr = ExprNode { - expr_type: ProstType::ArrayAppend as i32, + expr_type: PbType::ArrayAppend as i32, return_type: Some( DataType::List { datatype: Box::new(DataType::Int64), @@ -495,7 +495,7 @@ mod tests { let left = make_i64_array_array_expr_node(vec![vec![42]]); let right = make_i64_array_expr_node(vec![43]); let expr = ExprNode { - expr_type: ProstType::ArrayAppend as i32, + expr_type: PbType::ArrayAppend as i32, return_type: Some( DataType::List { datatype: Box::new(DataType::Int64), @@ -513,7 +513,7 @@ mod tests { let left = make_i64_expr_node(43); let right = make_i64_array_expr_node(vec![42]); let expr = ExprNode { - expr_type: ProstType::ArrayPrepend as i32, + expr_type: PbType::ArrayPrepend as i32, return_type: Some( DataType::List { datatype: Box::new(DataType::Int64), @@ -531,7 +531,7 @@ mod tests { let left = make_i64_array_expr_node(vec![43]); let right = make_i64_array_array_expr_node(vec![vec![42]]); let expr = ExprNode { - expr_type: ProstType::ArrayPrepend as i32, + expr_type: PbType::ArrayPrepend as i32, return_type: Some( DataType::List { datatype: Box::new(DataType::Int64), diff --git a/src/expr/src/expr/expr_array_distinct.rs b/src/expr/src/expr/expr_array_distinct.rs index 010f184023e8..d9f0c3ad3427 100644 --- a/src/expr/src/expr/expr_array_distinct.rs +++ b/src/expr/src/expr/expr_array_distinct.rs @@ -131,8 +131,8 @@ mod tests { use itertools::Itertools; use risingwave_common::array::DataChunk; use risingwave_common::types::ScalarImpl; - use risingwave_pb::data::Datum as ProstDatum; - use risingwave_pb::expr::expr_node::{RexNode, Type as ProstType}; + use risingwave_pb::data::PbDatum; + use risingwave_pb::expr::expr_node::{PbType, RexNode}; use risingwave_pb::expr::{ExprNode, FunctionCall}; use super::*; @@ -140,9 +140,9 @@ mod tests { fn make_i64_expr_node(value: i64) -> ExprNode { ExprNode { - expr_type: ProstType::ConstantValue as i32, + expr_type: PbType::ConstantValue as i32, return_type: Some(DataType::Int64.to_protobuf()), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: value.to_be_bytes().to_vec(), })), } @@ -150,7 +150,7 @@ mod tests { fn make_i64_array_expr_node(values: Vec) -> ExprNode { ExprNode { - expr_type: ProstType::Array as i32, + expr_type: PbType::Array as i32, return_type: Some( DataType::List { datatype: Box::new(DataType::Int64), @@ -165,7 +165,7 @@ mod tests { fn make_i64_array_array_expr_node(values: Vec>) -> ExprNode { ExprNode { - expr_type: ProstType::Array as i32, + expr_type: PbType::Array as i32, return_type: Some( DataType::List { datatype: Box::new(DataType::List { @@ -185,7 +185,7 @@ mod tests { { let array = make_i64_array_expr_node(vec![12]); let expr = ExprNode { - expr_type: ProstType::ArrayDistinct as i32, + expr_type: PbType::ArrayDistinct as i32, return_type: Some( DataType::List { datatype: Box::new(DataType::Int64), @@ -202,7 +202,7 @@ mod tests { { let array = make_i64_array_array_expr_node(vec![vec![42], vec![42]]); let expr = ExprNode { - expr_type: ProstType::ArrayDistinct as i32, + expr_type: PbType::ArrayDistinct as i32, return_type: Some( DataType::List { datatype: Box::new(DataType::Int64), diff --git a/src/expr/src/expr/expr_case.rs b/src/expr/src/expr/expr_case.rs index 8754db3d71f1..96876b33b67b 100644 --- a/src/expr/src/expr/expr_case.rs +++ b/src/expr/src/expr/expr_case.rs @@ -164,7 +164,7 @@ mod tests { use risingwave_common::test_prelude::DataChunkTestExt; use risingwave_common::types::Scalar; use risingwave_pb::data::data_type::TypeName; - use risingwave_pb::data::DataType as ProstDataType; + use risingwave_pb::data::PbDataType; use risingwave_pb::expr::expr_node::Type; use risingwave_pb::expr::FunctionCall; @@ -178,7 +178,7 @@ mod tests { children: vec![ ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Boolean as i32, ..Default::default() }), @@ -186,7 +186,7 @@ mod tests { }, ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }), @@ -196,7 +196,7 @@ mod tests { }; let p = ExprNode { expr_type: Type::Case as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }), diff --git a/src/expr/src/expr/expr_coalesce.rs b/src/expr/src/expr/expr_coalesce.rs index 4eb3355daacb..94c4b3523141 100644 --- a/src/expr/src/expr/expr_coalesce.rs +++ b/src/expr/src/expr/expr_coalesce.rs @@ -119,7 +119,7 @@ mod tests { use risingwave_common::test_prelude::DataChunkTestExt; use risingwave_common::types::{Scalar, ScalarImpl}; use risingwave_pb::data::data_type::TypeName; - use risingwave_pb::data::DataType as ProstDataType; + use risingwave_pb::data::PbDataType; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::Coalesce; use risingwave_pb::expr::{ExprNode, FunctionCall}; @@ -131,7 +131,7 @@ mod tests { pub fn make_coalesce_function(children: Vec, ret: TypeName) -> ExprNode { ExprNode { expr_type: Coalesce as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: ret as i32, ..Default::default() }), diff --git a/src/expr/src/expr/expr_concat_ws.rs b/src/expr/src/expr/expr_concat_ws.rs index 0a5780bc3db1..c1fa8c98bf1f 100644 --- a/src/expr/src/expr/expr_concat_ws.rs +++ b/src/expr/src/expr/expr_concat_ws.rs @@ -162,7 +162,7 @@ mod tests { use risingwave_common::row::OwnedRow; use risingwave_common::types::Datum; use risingwave_pb::data::data_type::TypeName; - use risingwave_pb::data::DataType as ProstDataType; + use risingwave_pb::data::PbDataType; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::ConcatWs; use risingwave_pb::expr::{ExprNode, FunctionCall}; @@ -174,7 +174,7 @@ mod tests { pub fn make_concat_ws_function(children: Vec, ret: TypeName) -> ExprNode { ExprNode { expr_type: ConcatWs as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: ret as i32, ..Default::default() }), diff --git a/src/expr/src/expr/expr_in.rs b/src/expr/src/expr/expr_in.rs index 6f8abb1f4cc1..21c93e9538a2 100644 --- a/src/expr/src/expr/expr_in.rs +++ b/src/expr/src/expr/expr_in.rs @@ -132,7 +132,7 @@ mod tests { use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::value_encoding::serialize_datum; use risingwave_pb::data::data_type::TypeName; - use risingwave_pb::data::{DataType as ProstDataType, Datum as ProstDatum}; + use risingwave_pb::data::{PbDataType, PbDatum}; use risingwave_pb::expr::expr_node::{RexNode, Type}; use risingwave_pb::expr::{ExprNode, FunctionCall}; @@ -143,7 +143,7 @@ mod tests { fn test_in_expr() { let input_ref_expr_node = ExprNode { expr_type: Type::InputRef as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Varchar as i32, ..Default::default() }), @@ -152,21 +152,21 @@ mod tests { let constant_values = vec![ ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Varchar as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some("ABC".into()).as_ref()), })), }, ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Varchar as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some("def".into()).as_ref()), })), }, @@ -178,7 +178,7 @@ mod tests { }; let p = ExprNode { expr_type: Type::In as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Boolean as i32, ..Default::default() }), diff --git a/src/expr/src/expr/expr_literal.rs b/src/expr/src/expr/expr_literal.rs index 376ed2dda42e..55da473a7309 100644 --- a/src/expr/src/expr/expr_literal.rs +++ b/src/expr/src/expr/expr_literal.rs @@ -125,7 +125,7 @@ mod tests { use risingwave_common::types::{Decimal, IntervalUnit, IntoOrdered}; use risingwave_common::util::value_encoding::serialize_datum; use risingwave_pb::data::data_type::{IntervalType, TypeName}; - use risingwave_pb::data::{DataType as ProstDataType, Datum as ProstDatum}; + use risingwave_pb::data::{PbDataType, PbDatum}; use risingwave_pb::expr::expr_node::RexNode::Constant; use risingwave_pb::expr::expr_node::Type; use risingwave_pb::expr::ExprNode; @@ -142,25 +142,25 @@ mod tests { let body = serialize_datum(Some(value.clone().to_scalar_value()).as_ref()); let expr = ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Struct as i32, field_type: vec![ - ProstDataType { + PbDataType { type_name: TypeName::Varchar as i32, ..Default::default() }, - ProstDataType { + PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }, - ProstDataType { + PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }, ], ..Default::default() }), - rex_node: Some(Constant(ProstDatum { body })), + rex_node: Some(Constant(PbDatum { body })), }; let expr = LiteralExpression::try_from(&expr).unwrap(); assert_eq!(value.to_scalar_value(), expr.literal().unwrap()); @@ -236,12 +236,12 @@ mod tests { fn make_expression(bytes: Option>, data_type: TypeName) -> ExprNode { ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: data_type as i32, interval_type: IntervalType::Month as i32, ..Default::default() }), - rex_node: bytes.map(|bs| RexNode::Constant(ProstDatum { body: bs })), + rex_node: bytes.map(|bs| RexNode::Constant(PbDatum { body: bs })), } } diff --git a/src/expr/src/expr/expr_unary.rs b/src/expr/src/expr/expr_unary.rs index b8c3bea2bac2..bacf418b5fb4 100644 --- a/src/expr/src/expr/expr_unary.rs +++ b/src/expr/src/expr/expr_unary.rs @@ -17,7 +17,7 @@ use risingwave_common::array::*; use risingwave_common::buffer::Bitmap; use risingwave_common::types::*; -use risingwave_pb::expr::expr_node::Type as ProstType; +use risingwave_pb::expr::expr_node::PbType; use super::expr_is_null::{IsNotNullExpression, IsNullExpression}; use super::template::{UnaryBytesExpression, UnaryExpression}; @@ -125,7 +125,7 @@ macro_rules! gen_round_expr { /// Create a new unary expression. pub fn new_unary_expr( - expr_type: ProstType, + expr_type: PbType, return_type: DataType, child_expr: BoxedExpression, ) -> Result { @@ -133,7 +133,7 @@ pub fn new_unary_expr( let expr: BoxedExpression = match (expr_type, return_type.clone(), child_expr.return_type()) { ( - ProstType::Cast, + PbType::Cast, DataType::List { datatype: target_elem_type, }, @@ -143,7 +143,7 @@ pub fn new_unary_expr( return_type, move |input| str_to_list(input, &target_elem_type), )), - (ProstType::Cast, DataType::Struct(rty), DataType::Struct(lty)) => { + (PbType::Cast, DataType::Struct(rty), DataType::Struct(lty)) => { Box::new(UnaryExpression::::new( child_expr, return_type, @@ -151,7 +151,7 @@ pub fn new_unary_expr( )) } ( - ProstType::Cast, + PbType::Cast, DataType::List { datatype: target_elem_type, }, @@ -163,7 +163,7 @@ pub fn new_unary_expr( return_type, move |input| list_cast(input, &source_elem_type, &target_elem_type), )), - (ProstType::Cast, _, _) => { + (PbType::Cast, _, _) => { macro_rules! gen_cast_impl { ($( { $input:ident, $cast:ident, $func:expr, $infallible:ident } ),*) => { match (child_expr.return_type(), return_type.clone()) { @@ -200,86 +200,86 @@ pub fn new_unary_expr( for_all_cast_variants! { gen_cast_impl } } - (ProstType::BoolOut, _, DataType::Boolean) => Box::new( + (PbType::BoolOut, _, DataType::Boolean) => Box::new( UnaryBytesExpression::::new(child_expr, return_type, bool_out), ), - (ProstType::Not, _, _) => Box::new(BooleanUnaryExpression::new( + (PbType::Not, _, _) => Box::new(BooleanUnaryExpression::new( child_expr, |a| BoolArray::new(!a.data() & a.null_bitmap(), a.null_bitmap().clone()), conjunction::not, )), - (ProstType::IsTrue, _, _) => Box::new(BooleanUnaryExpression::new( + (PbType::IsTrue, _, _) => Box::new(BooleanUnaryExpression::new( child_expr, |a| BoolArray::new(a.to_bitmap(), Bitmap::ones(a.len())), is_true, )), - (ProstType::IsNotTrue, _, _) => Box::new(BooleanUnaryExpression::new( + (PbType::IsNotTrue, _, _) => Box::new(BooleanUnaryExpression::new( child_expr, |a| BoolArray::new(!a.to_bitmap(), Bitmap::ones(a.len())), is_not_true, )), - (ProstType::IsFalse, _, _) => Box::new(BooleanUnaryExpression::new( + (PbType::IsFalse, _, _) => Box::new(BooleanUnaryExpression::new( child_expr, |a| BoolArray::new(!a.data() & a.null_bitmap(), Bitmap::ones(a.len())), is_false, )), - (ProstType::IsNotFalse, _, _) => Box::new(BooleanUnaryExpression::new( + (PbType::IsNotFalse, _, _) => Box::new(BooleanUnaryExpression::new( child_expr, |a| BoolArray::new(a.data() | !a.null_bitmap(), Bitmap::ones(a.len())), is_not_false, )), - (ProstType::IsNull, _, _) => Box::new(IsNullExpression::new(child_expr)), - (ProstType::IsNotNull, _, _) => Box::new(IsNotNullExpression::new(child_expr)), - (ProstType::Upper, _, _) => Box::new(UnaryBytesExpression::::new( + (PbType::IsNull, _, _) => Box::new(IsNullExpression::new(child_expr)), + (PbType::IsNotNull, _, _) => Box::new(IsNotNullExpression::new(child_expr)), + (PbType::Upper, _, _) => Box::new(UnaryBytesExpression::::new( child_expr, return_type, upper, )), - (ProstType::Lower, _, _) => Box::new(UnaryBytesExpression::::new( + (PbType::Lower, _, _) => Box::new(UnaryBytesExpression::::new( child_expr, return_type, lower, )), - (ProstType::Md5, _, _) => Box::new(UnaryBytesExpression::::new( + (PbType::Md5, _, _) => Box::new(UnaryBytesExpression::::new( child_expr, return_type, md5, )), - (ProstType::Ascii, _, _) => Box::new(UnaryExpression::::new( + (PbType::Ascii, _, _) => Box::new(UnaryExpression::::new( child_expr, return_type, ascii, )), - (ProstType::CharLength, _, _) => Box::new(UnaryExpression::::new( + (PbType::CharLength, _, _) => Box::new(UnaryExpression::::new( child_expr, return_type, length_default, )), - (ProstType::OctetLength, _, _) => Box::new(UnaryExpression::::new( + (PbType::OctetLength, _, _) => Box::new(UnaryExpression::::new( child_expr, return_type, octet_length, )), - (ProstType::BitLength, _, _) => Box::new(UnaryExpression::::new( + (PbType::BitLength, _, _) => Box::new(UnaryExpression::::new( child_expr, return_type, bit_length, )), - (ProstType::Neg, _, _) => { + (PbType::Neg, _, _) => { gen_unary_atm_expr! { "Neg", child_expr, return_type, general_neg, { { decimal, decimal, general_neg }, } } } - (ProstType::Abs, _, _) => { + (PbType::Abs, _, _) => { gen_unary_atm_expr! { "Abs", child_expr, return_type, general_abs, { {decimal, decimal, decimal_abs}, } } } - (ProstType::BitwiseNot, _, _) => { + (PbType::BitwiseNot, _, _) => { gen_unary_impl_fast! { [ "BitwiseNot", child_expr, return_type], { int16, int16, general_bitnot:: }, @@ -287,32 +287,32 @@ pub fn new_unary_expr( { int64, int64, general_bitnot:: }, } } - (ProstType::Ceil, _, _) => { + (PbType::Ceil, _, _) => { gen_round_expr! {"Ceil", child_expr, return_type, ceil_f64, ceil_decimal} } - (ProstType::Floor, DataType::Float64, DataType::Float64) => { + (PbType::Floor, DataType::Float64, DataType::Float64) => { gen_round_expr! {"Floor", child_expr, return_type, floor_f64, floor_decimal} } - (ProstType::Round, _, _) => { + (PbType::Round, _, _) => { gen_round_expr! {"Ceil", child_expr, return_type, round_f64, round_decimal} } - (ProstType::Exp, _, _) => Box::new(UnaryExpression::::new( + (PbType::Exp, _, _) => Box::new(UnaryExpression::::new( child_expr, return_type, exp_f64, )), - (ProstType::ToTimestamp, DataType::Timestamptz, DataType::Float64) => { + (PbType::ToTimestamp, DataType::Timestamptz, DataType::Float64) => { Box::new(UnaryExpression::::new( child_expr, return_type, f64_sec_to_timestamptz, )) } - (ProstType::JsonbTypeof, DataType::Varchar, DataType::Jsonb) => { + (PbType::JsonbTypeof, DataType::Varchar, DataType::Jsonb) => { UnaryBytesExpression::::new(child_expr, return_type, jsonb_typeof) .boxed() } - (ProstType::JsonbArrayLength, DataType::Int32, DataType::Jsonb) => { + (PbType::JsonbArrayLength, DataType::Int32, DataType::Jsonb) => { UnaryExpression::::new( child_expr, return_type, diff --git a/src/expr/src/expr/expr_vnode.rs b/src/expr/src/expr/expr_vnode.rs index c31aee110077..6846af61a7b7 100644 --- a/src/expr/src/expr/expr_vnode.rs +++ b/src/expr/src/expr/expr_vnode.rs @@ -93,7 +93,7 @@ mod tests { use risingwave_common::hash::VirtualNode; use risingwave_common::row::Row; use risingwave_pb::data::data_type::TypeName; - use risingwave_pb::data::DataType as ProstDataType; + use risingwave_pb::data::PbDataType; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::Vnode; use risingwave_pb::expr::{ExprNode, FunctionCall}; @@ -105,7 +105,7 @@ mod tests { pub fn make_vnode_function(children: Vec) -> ExprNode { ExprNode { expr_type: Vnode as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int16 as i32, ..Default::default() }), diff --git a/src/expr/src/expr/test_utils.rs b/src/expr/src/expr/test_utils.rs index d32cada054ac..5b80387c359a 100644 --- a/src/expr/src/expr/test_utils.rs +++ b/src/expr/src/expr/test_utils.rs @@ -21,7 +21,7 @@ use risingwave_common::types::{DataType, IntervalUnit, ScalarImpl}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::value_encoding::serialize_datum; use risingwave_pb::data::data_type::TypeName; -use risingwave_pb::data::{DataType as ProstDataType, Datum as ProstDatum}; +use risingwave_pb::data::{PbDataType, PbDatum}; use risingwave_pb::expr::expr_node::Type::{Field, InputRef}; use risingwave_pb::expr::expr_node::{self, RexNode, Type}; use risingwave_pb::expr::{ExprNode, FunctionCall}; @@ -38,7 +38,7 @@ pub fn make_expression(kind: Type, rets: &[TypeName], indices: &[usize]) -> Expr exprs.push(make_input_ref(*idx, *ret)); } let function_call = FunctionCall { children: exprs }; - let return_type = ProstDataType { + let return_type = PbDataType { type_name: TypeName::Timestamp as i32, ..Default::default() }; @@ -52,7 +52,7 @@ pub fn make_expression(kind: Type, rets: &[TypeName], indices: &[usize]) -> Expr pub fn make_input_ref(idx: usize, ret: TypeName) -> ExprNode { ExprNode { expr_type: InputRef as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: ret as i32, ..Default::default() }), @@ -63,11 +63,11 @@ pub fn make_input_ref(idx: usize, ret: TypeName) -> ExprNode { pub fn make_i32_literal(data: i32) -> ExprNode { ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Int32 as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some(ScalarImpl::Int32(data)).as_ref()), })), } @@ -76,11 +76,11 @@ pub fn make_i32_literal(data: i32) -> ExprNode { pub fn make_string_literal(data: &str) -> ExprNode { ExprNode { expr_type: Type::ConstantValue as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: TypeName::Varchar as i32, ..Default::default() }), - rex_node: Some(RexNode::Constant(ProstDatum { + rex_node: Some(RexNode::Constant(PbDatum { body: serialize_datum(Some(ScalarImpl::Utf8(data.into())).as_ref()), })), } @@ -89,7 +89,7 @@ pub fn make_string_literal(data: &str) -> ExprNode { pub fn make_field_function(children: Vec, ret: TypeName) -> ExprNode { ExprNode { expr_type: Field as i32, - return_type: Some(ProstDataType { + return_type: Some(PbDataType { type_name: ret as i32, ..Default::default() }), diff --git a/src/expr/src/table_function/generate_series.rs b/src/expr/src/table_function/generate_series.rs index 96e67ffe7c57..007e0d2fbf2c 100644 --- a/src/expr/src/table_function/generate_series.rs +++ b/src/expr/src/table_function/generate_series.rs @@ -154,7 +154,7 @@ where } pub fn new_generate_series( - prost: &TableFunctionProst, + prost: &TableFunctionPb, chunk_size: usize, ) -> Result { let return_type = DataType::from(prost.get_return_type().unwrap()); diff --git a/src/expr/src/table_function/mod.rs b/src/expr/src/table_function/mod.rs index fa947f80dc2f..14595a09a0e5 100644 --- a/src/expr/src/table_function/mod.rs +++ b/src/expr/src/table_function/mod.rs @@ -19,9 +19,7 @@ use itertools::Itertools; use risingwave_common::array::{ArrayRef, DataChunk}; use risingwave_common::types::DataType; use risingwave_pb::expr::project_set_select_item::SelectItem; -use risingwave_pb::expr::{ - ProjectSetSelectItem as SelectItemProst, TableFunction as TableFunctionProst, -}; +use risingwave_pb::expr::{ProjectSetSelectItem as SelectItemPb, TableFunction as TableFunctionPb}; use super::Result; use crate::expr::{build_from_prost as expr_build_from_prost, BoxedExpression}; @@ -56,10 +54,7 @@ pub trait TableFunction: std::fmt::Debug + Sync + Send { pub type BoxedTableFunction = Box; -pub fn build_from_prost( - prost: &TableFunctionProst, - chunk_size: usize, -) -> Result { +pub fn build_from_prost(prost: &TableFunctionPb, chunk_size: usize) -> Result { use risingwave_pb::expr::table_function::Type::*; match prost.get_function_type().unwrap() { @@ -110,7 +105,7 @@ pub fn repeat_tf(expr: BoxedExpression, n: usize) -> BoxedTableFunction { Mock { expr, n }.boxed() } -/// See also [`SelectItemProst`] +/// See also [`SelectItemPb`] #[derive(Debug)] pub enum ProjectSetSelectItem { TableFunction(BoxedTableFunction), @@ -130,7 +125,7 @@ impl From for ProjectSetSelectItem { } impl ProjectSetSelectItem { - pub fn from_prost(prost: &SelectItemProst, chunk_size: usize) -> Result { + pub fn from_prost(prost: &SelectItemPb, chunk_size: usize) -> Result { match prost.select_item.as_ref().unwrap() { SelectItem::Expr(expr) => expr_build_from_prost(expr).map(Into::into), SelectItem::TableFunction(tf) => build_from_prost(tf, chunk_size).map(Into::into), diff --git a/src/expr/src/table_function/regexp_matches.rs b/src/expr/src/table_function/regexp_matches.rs index 5347e2aa532d..1f1c1744c8d8 100644 --- a/src/expr/src/table_function/regexp_matches.rs +++ b/src/expr/src/table_function/regexp_matches.rs @@ -103,7 +103,7 @@ impl TableFunction for RegexpMatches { } pub fn new_regexp_matches( - prost: &TableFunctionProst, + prost: &TableFunctionPb, chunk_size: usize, ) -> Result { ensure!( diff --git a/src/expr/src/table_function/unnest.rs b/src/expr/src/table_function/unnest.rs index ef3fe02f168a..80266675e2ab 100644 --- a/src/expr/src/table_function/unnest.rs +++ b/src/expr/src/table_function/unnest.rs @@ -78,7 +78,7 @@ impl TableFunction for Unnest { } } -pub fn new_unnest(prost: &TableFunctionProst, chunk_size: usize) -> Result { +pub fn new_unnest(prost: &TableFunctionPb, chunk_size: usize) -> Result { let return_type = DataType::from(prost.get_return_type().unwrap()); let args: Vec<_> = prost.args.iter().map(expr_build_from_prost).try_collect()?; let [list]: [_; 1] = args.try_into().unwrap(); diff --git a/src/expr/src/table_function/user_defined.rs b/src/expr/src/table_function/user_defined.rs index 38a5f7be2d16..3f385b51983c 100644 --- a/src/expr/src/table_function/user_defined.rs +++ b/src/expr/src/table_function/user_defined.rs @@ -64,10 +64,7 @@ impl TableFunction for UserDefinedTableFunction { } #[cfg(not(madsim))] -pub fn new_user_defined( - prost: &TableFunctionProst, - chunk_size: usize, -) -> Result { +pub fn new_user_defined(prost: &TableFunctionPb, chunk_size: usize) -> Result { let Some(udtf) = &prost.udtf else { bail!("expect UDTF"); }; @@ -108,7 +105,7 @@ impl TableFunction for UserDefinedTableFunction { #[cfg(madsim)] pub fn new_user_defined( - _prost: &TableFunctionProst, + _prost: &TableFunctionPb, _chunk_size: usize, ) -> Result { panic!("UDF is not supported in simulation yet"); diff --git a/src/expr/src/vector_op/agg/filter.rs b/src/expr/src/vector_op/agg/filter.rs index 4295d9d3a6c7..71aa96dffdd7 100644 --- a/src/expr/src/vector_op/agg/filter.rs +++ b/src/expr/src/vector_op/agg/filter.rs @@ -114,7 +114,7 @@ mod tests { use std::sync::Arc; use risingwave_common::test_prelude::DataChunkTestExt; - use risingwave_pb::expr::expr_node::Type as ProstType; + use risingwave_pb::expr::expr_node::PbType; use super::*; use crate::expr::{new_binary_expr, Expression, InputRefExpression, LiteralExpression}; @@ -188,7 +188,7 @@ mod tests { // filter (where $1 > 5) let condition = Arc::from( new_binary_expr( - ProstType::GreaterThan, + PbType::GreaterThan, DataType::Boolean, InputRefExpression::new(DataType::Int64, 0).boxed(), LiteralExpression::new(DataType::Int64, Some((5_i64).into())).boxed(), @@ -230,7 +230,7 @@ mod tests { async fn test_selective_agg_null_condition() -> Result<()> { let condition = Arc::from( new_binary_expr( - ProstType::Equal, + PbType::Equal, DataType::Boolean, InputRefExpression::new(DataType::Int64, 0).boxed(), LiteralExpression::new(DataType::Int64, None).boxed(), diff --git a/src/frontend/src/catalog/catalog_service.rs b/src/frontend/src/catalog/catalog_service.rs index 736cb307c0b3..6700f94ff8e7 100644 --- a/src/frontend/src/catalog/catalog_service.rs +++ b/src/frontend/src/catalog/catalog_service.rs @@ -21,9 +21,7 @@ use risingwave_common::error::ErrorCode::InternalError; use risingwave_common::error::{Result, RwError}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::catalog::{ - Database as ProstDatabase, Function as ProstFunction, Index as ProstIndex, - Schema as ProstSchema, Sink as ProstSink, Source as ProstSource, Table as ProstTable, - View as ProstView, + PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, }; use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_rpc_client::MetaClient; @@ -64,40 +62,40 @@ pub trait CatalogWriter: Send + Sync { owner: UserId, ) -> Result<()>; - async fn create_view(&self, view: ProstView) -> Result<()>; + async fn create_view(&self, view: PbView) -> Result<()>; async fn create_materialized_view( &self, - table: ProstTable, + table: PbTable, graph: StreamFragmentGraph, ) -> Result<()>; async fn create_table( &self, - source: Option, - table: ProstTable, + source: Option, + table: PbTable, graph: StreamFragmentGraph, ) -> Result<()>; async fn replace_table( &self, - table: ProstTable, + table: PbTable, graph: StreamFragmentGraph, mapping: ColIndexMapping, ) -> Result<()>; async fn create_index( &self, - index: ProstIndex, - table: ProstTable, + index: PbIndex, + table: PbTable, graph: StreamFragmentGraph, ) -> Result<()>; - async fn create_source(&self, source: ProstSource) -> Result<()>; + async fn create_source(&self, source: PbSource) -> Result<()>; - async fn create_sink(&self, sink: ProstSink, graph: StreamFragmentGraph) -> Result<()>; + async fn create_sink(&self, sink: PbSink, graph: StreamFragmentGraph) -> Result<()>; - async fn create_function(&self, function: ProstFunction) -> Result<()>; + async fn create_function(&self, function: PbFunction) -> Result<()>; async fn drop_table(&self, source_id: Option, table_id: TableId) -> Result<()>; @@ -129,7 +127,7 @@ impl CatalogWriter for CatalogWriterImpl { async fn create_database(&self, db_name: &str, owner: UserId) -> Result<()> { let (_, version) = self .meta_client - .create_database(ProstDatabase { + .create_database(PbDatabase { name: db_name.to_string(), id: 0, owner, @@ -146,7 +144,7 @@ impl CatalogWriter for CatalogWriterImpl { ) -> Result<()> { let (_, version) = self .meta_client - .create_schema(ProstSchema { + .create_schema(PbSchema { id: 0, name: schema_name.to_string(), database_id: db_id, @@ -159,7 +157,7 @@ impl CatalogWriter for CatalogWriterImpl { // TODO: maybe here to pass a materialize plan node async fn create_materialized_view( &self, - table: ProstTable, + table: PbTable, graph: StreamFragmentGraph, ) -> Result<()> { let (_, version) = self @@ -169,15 +167,15 @@ impl CatalogWriter for CatalogWriterImpl { self.wait_version(version).await } - async fn create_view(&self, view: ProstView) -> Result<()> { + async fn create_view(&self, view: PbView) -> Result<()> { let (_, version) = self.meta_client.create_view(view).await?; self.wait_version(version).await } async fn create_index( &self, - index: ProstIndex, - table: ProstTable, + index: PbIndex, + table: PbTable, graph: StreamFragmentGraph, ) -> Result<()> { let (_, version) = self.meta_client.create_index(index, table, graph).await?; @@ -186,8 +184,8 @@ impl CatalogWriter for CatalogWriterImpl { async fn create_table( &self, - source: Option, - table: ProstTable, + source: Option, + table: PbTable, graph: StreamFragmentGraph, ) -> Result<()> { let (_, version) = self.meta_client.create_table(source, table, graph).await?; @@ -196,7 +194,7 @@ impl CatalogWriter for CatalogWriterImpl { async fn replace_table( &self, - table: ProstTable, + table: PbTable, graph: StreamFragmentGraph, mapping: ColIndexMapping, ) -> Result<()> { @@ -207,17 +205,17 @@ impl CatalogWriter for CatalogWriterImpl { self.wait_version(version).await } - async fn create_source(&self, source: ProstSource) -> Result<()> { + async fn create_source(&self, source: PbSource) -> Result<()> { let (_id, version) = self.meta_client.create_source(source).await?; self.wait_version(version).await } - async fn create_sink(&self, sink: ProstSink, graph: StreamFragmentGraph) -> Result<()> { + async fn create_sink(&self, sink: PbSink, graph: StreamFragmentGraph) -> Result<()> { let (_id, version) = self.meta_client.create_sink(sink, graph).await?; self.wait_version(version).await } - async fn create_function(&self, function: ProstFunction) -> Result<()> { + async fn create_function(&self, function: PbFunction) -> Result<()> { let (_, version) = self.meta_client.create_function(function).await?; self.wait_version(version).await } diff --git a/src/frontend/src/catalog/database_catalog.rs b/src/frontend/src/catalog/database_catalog.rs index c63b2aeda666..74b5ee26985d 100644 --- a/src/frontend/src/catalog/database_catalog.rs +++ b/src/frontend/src/catalog/database_catalog.rs @@ -16,7 +16,7 @@ use std::collections::HashMap; use itertools::Itertools; use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_pb::catalog::{Database as ProstDatabase, Schema as ProstSchema}; +use risingwave_pb::catalog::{PbDatabase, PbSchema}; use crate::catalog::schema_catalog::SchemaCatalog; use crate::catalog::{DatabaseId, SchemaId}; @@ -32,7 +32,7 @@ pub struct DatabaseCatalog { } impl DatabaseCatalog { - pub fn create_schema(&mut self, proto: &ProstSchema) { + pub fn create_schema(&mut self, proto: &PbSchema) { let name = proto.name.clone(); let id = proto.id; let schema = proto.into(); @@ -51,11 +51,11 @@ impl DatabaseCatalog { self.schema_by_name.keys().cloned().collect_vec() } - pub fn get_all_schema_info(&self) -> Vec { + pub fn get_all_schema_info(&self) -> Vec { self.schema_by_name .values() .cloned() - .map(|schema| ProstSchema { + .map(|schema| PbSchema { id: schema.id(), database_id: self.id, name: schema.name(), @@ -94,8 +94,8 @@ impl DatabaseCatalog { self.owner } } -impl From<&ProstDatabase> for DatabaseCatalog { - fn from(db: &ProstDatabase) -> Self { +impl From<&PbDatabase> for DatabaseCatalog { + fn from(db: &PbDatabase) -> Self { Self { id: db.id, name: db.name.clone(), diff --git a/src/frontend/src/catalog/function_catalog.rs b/src/frontend/src/catalog/function_catalog.rs index 2d2de2fd7c22..70daeaf065b3 100644 --- a/src/frontend/src/catalog/function_catalog.rs +++ b/src/frontend/src/catalog/function_catalog.rs @@ -14,8 +14,8 @@ use risingwave_common::catalog::FunctionId; use risingwave_common::types::DataType; -use risingwave_pb::catalog::function::Kind as ProstKind; -use risingwave_pb::catalog::Function as ProstFunction; +use risingwave_pb::catalog::function::PbKind; +use risingwave_pb::catalog::PbFunction; #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct FunctionCatalog { @@ -37,8 +37,8 @@ pub enum FunctionKind { Aggregate, } -impl From<&ProstKind> for FunctionKind { - fn from(prost: &ProstKind) -> Self { +impl From<&PbKind> for FunctionKind { + fn from(prost: &PbKind) -> Self { use risingwave_pb::catalog::function::*; match prost { Kind::Scalar(ScalarFunction {}) => Self::Scalar, @@ -48,8 +48,8 @@ impl From<&ProstKind> for FunctionKind { } } -impl From<&ProstFunction> for FunctionCatalog { - fn from(prost: &ProstFunction) -> Self { +impl From<&PbFunction> for FunctionCatalog { + fn from(prost: &PbFunction) -> Self { FunctionCatalog { id: prost.id.into(), name: prost.name.clone(), diff --git a/src/frontend/src/catalog/index_catalog.rs b/src/frontend/src/catalog/index_catalog.rs index f0ac7a0fac77..9cd7c44365eb 100644 --- a/src/frontend/src/catalog/index_catalog.rs +++ b/src/frontend/src/catalog/index_catalog.rs @@ -19,7 +19,7 @@ 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::catalog::PbIndex; use risingwave_pb::expr::expr_node::RexNode; use super::ColumnId; @@ -50,7 +50,7 @@ pub struct IndexCatalog { impl IndexCatalog { pub fn build_from( - index_prost: &ProstIndex, + index_prost: &PbIndex, index_table: &TableCatalog, primary_table: &TableCatalog, ) -> Self { @@ -134,8 +134,8 @@ impl IndexCatalog { &self.primary_to_secondary_mapping } - pub fn to_prost(&self, schema_id: SchemaId, database_id: DatabaseId) -> ProstIndex { - ProstIndex { + pub fn to_prost(&self, schema_id: SchemaId, database_id: DatabaseId) -> PbIndex { + PbIndex { id: self.id.index_id, schema_id, database_id, diff --git a/src/frontend/src/catalog/root_catalog.rs b/src/frontend/src/catalog/root_catalog.rs index 5077b6e4a2b6..9302f23ac3f8 100644 --- a/src/frontend/src/catalog/root_catalog.rs +++ b/src/frontend/src/catalog/root_catalog.rs @@ -21,9 +21,7 @@ use risingwave_common::session_config::{SearchPath, USER_NAME_WILD_CARD}; use risingwave_common::types::DataType; use risingwave_connector::sink::catalog::SinkCatalog; use risingwave_pb::catalog::{ - Database as ProstDatabase, Function as ProstFunction, Index as ProstIndex, - Schema as ProstSchema, Sink as ProstSink, Source as ProstSource, Table as ProstTable, - View as ProstView, + PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, }; use super::function_catalog::FunctionCatalog; @@ -122,7 +120,7 @@ impl Catalog { self.table_by_id.clear(); } - pub fn create_database(&mut self, db: &ProstDatabase) { + pub fn create_database(&mut self, db: &PbDatabase) { let name = db.name.clone(); let id = db.id; @@ -132,7 +130,7 @@ impl Catalog { self.db_name_by_id.try_insert(id, name).unwrap(); } - pub fn create_schema(&mut self, proto: &ProstSchema) { + pub fn create_schema(&mut self, proto: &PbSchema) { self.get_database_mut(proto.database_id) .unwrap() .create_schema(proto); @@ -148,7 +146,7 @@ impl Catalog { } } - pub fn create_table(&mut self, proto: &ProstTable) { + pub fn create_table(&mut self, proto: &PbTable) { self.table_by_id.insert(proto.id.into(), proto.into()); self.get_database_mut(proto.database_id) .unwrap() @@ -157,7 +155,7 @@ impl Catalog { .create_table(proto); } - pub fn create_index(&mut self, proto: &ProstIndex) { + pub fn create_index(&mut self, proto: &PbIndex) { self.get_database_mut(proto.database_id) .unwrap() .get_schema_mut(proto.schema_id) @@ -165,7 +163,7 @@ impl Catalog { .create_index(proto); } - pub fn create_source(&mut self, proto: &ProstSource) { + pub fn create_source(&mut self, proto: &PbSource) { self.get_database_mut(proto.database_id) .unwrap() .get_schema_mut(proto.schema_id) @@ -173,7 +171,7 @@ impl Catalog { .create_source(proto); } - pub fn create_sink(&mut self, proto: &ProstSink) { + pub fn create_sink(&mut self, proto: &PbSink) { self.get_database_mut(proto.database_id) .unwrap() .get_schema_mut(proto.schema_id) @@ -181,7 +179,7 @@ impl Catalog { .create_sink(proto); } - pub fn create_view(&mut self, proto: &ProstView) { + pub fn create_view(&mut self, proto: &PbView) { self.get_database_mut(proto.database_id) .unwrap() .get_schema_mut(proto.schema_id) @@ -189,7 +187,7 @@ impl Catalog { .create_view(proto); } - pub fn create_function(&mut self, proto: &ProstFunction) { + pub fn create_function(&mut self, proto: &PbFunction) { self.get_database_mut(proto.database_id) .unwrap() .get_schema_mut(proto.schema_id) @@ -215,7 +213,7 @@ impl Catalog { .drop_table(tb_id); } - pub fn update_table(&mut self, proto: &ProstTable) { + pub fn update_table(&mut self, proto: &PbTable) { self.table_by_id.insert(proto.id.into(), proto.into()); self.get_database_mut(proto.database_id) .unwrap() @@ -289,7 +287,7 @@ impl Catalog { Ok(self.get_database_by_name(db_name)?.get_all_schema_names()) } - pub fn get_all_schema_info(&self, db_name: &str) -> CatalogResult> { + pub fn get_all_schema_info(&self, db_name: &str) -> CatalogResult> { Ok(self.get_database_by_name(db_name)?.get_all_schema_info()) } diff --git a/src/frontend/src/catalog/schema_catalog.rs b/src/frontend/src/catalog/schema_catalog.rs index 1e1862965b84..40cdc7125f59 100644 --- a/src/frontend/src/catalog/schema_catalog.rs +++ b/src/frontend/src/catalog/schema_catalog.rs @@ -19,10 +19,7 @@ use std::sync::Arc; use risingwave_common::catalog::{valid_table_name, FunctionId, IndexId, TableId}; use risingwave_common::types::DataType; use risingwave_connector::sink::catalog::SinkCatalog; -use risingwave_pb::catalog::{ - Function as ProstFunction, Index as ProstIndex, Schema as ProstSchema, Sink as ProstSink, - Source as ProstSource, Table as ProstTable, View as ProstView, -}; +use risingwave_pb::catalog::{PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView}; use super::source_catalog::SourceCatalog; use super::ViewId; @@ -60,7 +57,7 @@ pub struct SchemaCatalog { } impl SchemaCatalog { - pub fn create_table(&mut self, prost: &ProstTable) { + pub fn create_table(&mut self, prost: &PbTable) { let name = prost.name.clone(); let id = prost.id.into(); let table: TableCatalog = prost.into(); @@ -78,7 +75,7 @@ impl SchemaCatalog { .unwrap(); } - pub fn update_table(&mut self, prost: &ProstTable) { + pub fn update_table(&mut self, prost: &PbTable) { let name = prost.name.clone(); let id = prost.id.into(); let table: TableCatalog = prost.into(); @@ -94,7 +91,7 @@ impl SchemaCatalog { self.indexes_by_table_id.remove(&table_ref.id); } - pub fn create_index(&mut self, prost: &ProstIndex) { + pub fn create_index(&mut self, prost: &PbIndex) { let name = prost.name.clone(); let id = prost.id.into(); @@ -135,7 +132,7 @@ impl SchemaCatalog { }; } - pub fn create_source(&mut self, prost: &ProstSource) { + pub fn create_source(&mut self, prost: &PbSource) { let name = prost.name.clone(); let id = prost.id; let source = SourceCatalog::from(prost); @@ -152,7 +149,7 @@ impl SchemaCatalog { self.source_by_name.remove(&source_ref.name).unwrap(); } - pub fn create_sink(&mut self, prost: &ProstSink) { + pub fn create_sink(&mut self, prost: &PbSink) { let name = prost.name.clone(); let id = prost.id; let sink = SinkCatalog::from(prost); @@ -169,7 +166,7 @@ impl SchemaCatalog { self.sink_by_name.remove(&sink_ref.name).unwrap(); } - pub fn create_view(&mut self, prost: &ProstView) { + pub fn create_view(&mut self, prost: &PbView) { let name = prost.name.clone(); let id = prost.id; let view = ViewCatalog::from(prost); @@ -186,7 +183,7 @@ impl SchemaCatalog { self.view_by_name.remove(&view_ref.name).unwrap(); } - pub fn create_function(&mut self, prost: &ProstFunction) { + pub fn create_function(&mut self, prost: &PbFunction) { let name = prost.name.clone(); let id = prost.id; let function = FunctionCatalog::from(prost); @@ -331,8 +328,8 @@ impl SchemaCatalog { } } -impl From<&ProstSchema> for SchemaCatalog { - fn from(schema: &ProstSchema) -> Self { +impl From<&PbSchema> for SchemaCatalog { + fn from(schema: &PbSchema) -> Self { Self { id: schema.id, owner: schema.owner, diff --git a/src/frontend/src/catalog/source_catalog.rs b/src/frontend/src/catalog/source_catalog.rs index a25b89413faa..687604e5b5ba 100644 --- a/src/frontend/src/catalog/source_catalog.rs +++ b/src/frontend/src/catalog/source_catalog.rs @@ -15,14 +15,14 @@ use std::collections::BTreeMap; use risingwave_common::catalog::ColumnCatalog; -use risingwave_pb::catalog::{Source as ProstSource, StreamSourceInfo, WatermarkDesc}; +use risingwave_pb::catalog::{PbSource, StreamSourceInfo, WatermarkDesc}; use super::{ColumnId, RelationCatalog, SourceId}; use crate::user::UserId; use crate::WithOptions; /// This struct `SourceCatalog` is used in frontend. -/// Compared with `ProstSource`, it only maintains information used during optimization. +/// Compared with `PbSource`, it only maintains information used during optimization. #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct SourceCatalog { pub id: SourceId, @@ -37,8 +37,8 @@ pub struct SourceCatalog { pub watermark_descs: Vec, } -impl From<&ProstSource> for SourceCatalog { - fn from(prost: &ProstSource) -> Self { +impl From<&PbSource> for SourceCatalog { + fn from(prost: &PbSource) -> Self { let id = prost.id; let name = prost.name.clone(); let prost_columns = prost.columns.clone(); diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 779409d4a125..36a12611ce95 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -20,10 +20,8 @@ use risingwave_common::catalog::{ColumnCatalog, TableDesc, TableId, TableVersion 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 risingwave_pb::catalog::table::{OptionalAssociatedSourceId, PbTableType, PbTableVersion}; +use risingwave_pb::catalog::PbTable; use super::{ColumnId, ConflictBehaviorType, DatabaseId, FragmentId, RelationCatalog, SchemaId}; use crate::user::UserId; @@ -150,27 +148,27 @@ impl Default for TableType { } impl TableType { - fn from_prost(prost: ProstTableType) -> Self { + fn from_prost(prost: PbTableType) -> Self { match prost { - ProstTableType::Table => Self::Table, - ProstTableType::MaterializedView => Self::MaterializedView, - ProstTableType::Index => Self::Index, - ProstTableType::Internal => Self::Internal, - ProstTableType::Unspecified => unreachable!(), + PbTableType::Table => Self::Table, + PbTableType::MaterializedView => Self::MaterializedView, + PbTableType::Index => Self::Index, + PbTableType::Internal => Self::Internal, + PbTableType::Unspecified => unreachable!(), } } - fn to_prost(self) -> ProstTableType { + fn to_prost(self) -> PbTableType { match self { - Self::Table => ProstTableType::Table, - Self::MaterializedView => ProstTableType::MaterializedView, - Self::Index => ProstTableType::Index, - Self::Internal => ProstTableType::Internal, + Self::Table => PbTableType::Table, + Self::MaterializedView => PbTableType::MaterializedView, + Self::Index => PbTableType::Index, + Self::Internal => PbTableType::Internal, } } } -/// The version of a table, used by schema change. See [`ProstTableVersion`]. +/// The version of a table, used by schema change. See [`PbTableVersion`]. #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct TableVersion { pub version_id: TableVersionId, @@ -189,15 +187,15 @@ impl TableVersion { } } - pub fn from_prost(prost: ProstTableVersion) -> Self { + pub fn from_prost(prost: PbTableVersion) -> Self { Self { version_id: prost.version, next_column_id: ColumnId::from(prost.next_column_id), } } - pub fn to_prost(&self) -> ProstTableVersion { - ProstTableVersion { + pub fn to_prost(&self) -> PbTableVersion { + PbTableVersion { version: self.version_id, next_column_id: self.next_column_id.into(), } @@ -315,7 +313,7 @@ impl TableCatalog { self.distribution_key.as_ref() } - pub fn to_internal_table_prost(&self) -> ProstTable { + pub fn to_internal_table_prost(&self) -> PbTable { use risingwave_common::catalog::{DatabaseId, SchemaId}; self.to_prost( SchemaId::placeholder().schema_id, @@ -338,8 +336,8 @@ impl TableCatalog { self.version().map(|v| v.version_id) } - pub fn to_prost(&self, schema_id: SchemaId, database_id: DatabaseId) -> ProstTable { - ProstTable { + pub fn to_prost(&self, schema_id: SchemaId, database_id: DatabaseId) -> PbTable { + PbTable { id: self.id.table_id, schema_id, database_id, @@ -374,8 +372,8 @@ impl TableCatalog { } } -impl From for TableCatalog { - fn from(tb: ProstTable) -> Self { +impl From for TableCatalog { + fn from(tb: PbTable) -> Self { let id = tb.id; let table_type = tb.get_table_type().unwrap(); let associated_source_id = tb.optional_associated_source_id.map(|id| match id { @@ -433,8 +431,8 @@ impl From for TableCatalog { } } -impl From<&ProstTable> for TableCatalog { - fn from(tb: &ProstTable) -> Self { +impl From<&PbTable> for TableCatalog { + fn from(tb: &PbTable) -> Self { tb.clone().into() } } @@ -456,10 +454,8 @@ mod tests { 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, - }; + use risingwave_pb::catalog::PbTable; + use risingwave_pb::plan_common::{PbColumnCatalog, PbColumnDesc}; use super::*; use crate::catalog::table_catalog::{TableCatalog, TableType}; @@ -467,33 +463,25 @@ mod tests { #[test] fn test_into_table_catalog() { - let table: TableCatalog = ProstTable { + let table: TableCatalog = PbTable { id: 0, schema_id: 0, database_id: 0, name: "test".to_string(), - table_type: ProstTableType::Table as i32, + table_type: PbTableType::Table as i32, columns: vec![ - ProstColumnCatalog { + PbColumnCatalog { column_desc: Some((&row_id_column_desc()).into()), is_hidden: true, }, - ProstColumnCatalog { - column_desc: Some(ProstColumnDesc::new_struct( + PbColumnCatalog { + column_desc: Some(PbColumnDesc::new_struct( "country", 1, ".test.Country", vec![ - ProstColumnDesc::new_atomic( - DataType::Varchar.to_protobuf(), - "address", - 2, - ), - ProstColumnDesc::new_atomic( - DataType::Varchar.to_protobuf(), - "zipcode", - 3, - ), + PbColumnDesc::new_atomic(DataType::Varchar.to_protobuf(), "address", 2), + PbColumnDesc::new_atomic(DataType::Varchar.to_protobuf(), "zipcode", 3), ], )), is_hidden: false, @@ -517,7 +505,7 @@ mod tests { read_prefix_len_hint: 0, vnode_col_index: None, row_id_index: None, - version: Some(ProstTableVersion { + version: Some(PbTableVersion { version: 0, next_column_id: 2, }), diff --git a/src/frontend/src/catalog/view_catalog.rs b/src/frontend/src/catalog/view_catalog.rs index 71401657592c..24334e237cc2 100644 --- a/src/frontend/src/catalog/view_catalog.rs +++ b/src/frontend/src/catalog/view_catalog.rs @@ -13,7 +13,7 @@ // limitations under the License. use risingwave_common::catalog::Field; -use risingwave_pb::catalog::View as ProstView; +use risingwave_pb::catalog::PbView; use super::{RelationCatalog, ViewId}; use crate::user::UserId; @@ -30,8 +30,8 @@ pub struct ViewCatalog { pub columns: Vec, } -impl From<&ProstView> for ViewCatalog { - fn from(view: &ProstView) -> Self { +impl From<&PbView> for ViewCatalog { + fn from(view: &PbView) -> Self { ViewCatalog { id: view.id, name: view.name.clone(), diff --git a/src/frontend/src/expr/input_ref.rs b/src/frontend/src/expr/input_ref.rs index 7a7d361303ce..58732392b6f3 100644 --- a/src/frontend/src/expr/input_ref.rs +++ b/src/frontend/src/expr/input_ref.rs @@ -17,7 +17,7 @@ use std::fmt; use itertools::Itertools; use risingwave_common::catalog::Schema; use risingwave_common::types::DataType; -use risingwave_pb::expr::InputRef as ProstInputRef; +use risingwave_pb::expr::PbInputRef; use super::Expr; use crate::expr::ExprType; @@ -108,8 +108,8 @@ impl InputRef { } /// Convert to protobuf. - pub fn to_proto(&self) -> ProstInputRef { - ProstInputRef { + pub fn to_proto(&self) -> PbInputRef { + PbInputRef { index: self.index as _, r#type: Some(self.data_type.to_protobuf()), } diff --git a/src/frontend/src/expr/literal.rs b/src/frontend/src/expr/literal.rs index 716b1d096fdc..2cc42bea857a 100644 --- a/src/frontend/src/expr/literal.rs +++ b/src/frontend/src/expr/literal.rs @@ -16,7 +16,7 @@ use risingwave_common::array::list_array::display_for_explain; use risingwave_common::types::to_text::ToText; use risingwave_common::types::{literal_type_match, DataType, Datum}; use risingwave_common::util::value_encoding::{deserialize_datum, serialize_datum}; -use risingwave_pb::data::Datum as ProstDatum; +use risingwave_pb::data::PbDatum; use risingwave_pb::expr::expr_node::RexNode; use super::Expr; @@ -110,7 +110,7 @@ impl Expr for Literal { /// Convert a literal value (datum) into protobuf. fn literal_to_value_encoding(d: &Datum) -> RexNode { let body = serialize_datum(d.as_ref()); - RexNode::Constant(ProstDatum { body }) + RexNode::Constant(PbDatum { body }) } /// Convert protobuf into a literal value (datum). diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index b3fe68404c36..8245f5786bfb 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -20,7 +20,7 @@ use risingwave_common::error::ErrorCode; use risingwave_common::types::{unnested_list_type, DataType, ScalarImpl}; use risingwave_pb::expr::table_function::Type; use risingwave_pb::expr::{ - TableFunction as TableFunctionProst, UserDefinedTableFunction as UserDefinedTableFunctionProst, + TableFunction as TableFunctionPb, UserDefinedTableFunction as UserDefinedTableFunctionPb, }; use super::{Expr, ExprImpl, ExprRewriter, RwResult}; @@ -229,15 +229,15 @@ impl TableFunction { } } - pub fn to_protobuf(&self) -> TableFunctionProst { - TableFunctionProst { + pub fn to_protobuf(&self) -> TableFunctionPb { + TableFunctionPb { function_type: self.function_type.to_protobuf() as i32, args: self.args.iter().map(|c| c.to_expr_proto()).collect_vec(), return_type: Some(self.return_type.to_protobuf()), udtf: self .udtf_catalog .as_ref() - .map(|c| UserDefinedTableFunctionProst { + .map(|c| UserDefinedTableFunctionPb { arg_types: c.arg_types.iter().map(|t| t.to_protobuf()).collect(), language: c.language.clone(), link: c.link.clone(), diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index a2db235c6d40..df9af08fb413 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -21,7 +21,7 @@ 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::{ColumnOrder, OrderType}; -use risingwave_pb::catalog::{Index as ProstIndex, Table as ProstTable}; +use risingwave_pb::catalog::{PbIndex, PbTable}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_pb::user::grant_privilege::{Action, Object}; use risingwave_sqlparser::ast::{Ident, ObjectName, OrderByExpr}; @@ -47,7 +47,7 @@ pub(crate) fn gen_create_index_plan( columns: Vec, include: Vec, distributed_by: Vec, -) -> Result<(PlanRef, ProstTable, ProstIndex)> { +) -> Result<(PlanRef, PbTable, PbIndex)> { let columns = check_columns(columns)?; let db_name = session.database(); let (schema_name, table_name) = Binder::resolve_schema_qualified_name(db_name, table_name)?; @@ -191,7 +191,7 @@ pub(crate) fn gen_create_index_plan( index_table_prost.owner = session.user_id(); - let index_prost = ProstIndex { + let index_prost = PbIndex { id: IndexId::placeholder().index_id, schema_id: index_schema_id, database_id: index_database_id, diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index d27d4a05f72a..32a010c45e39 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -14,7 +14,7 @@ use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::error::{ErrorCode, Result}; -use risingwave_pb::catalog::Table as ProstTable; +use risingwave_pb::catalog::PbTable; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_pb::user::grant_privilege::Action; use risingwave_sqlparser::ast::{Ident, ObjectName, Query}; @@ -77,7 +77,7 @@ pub fn gen_create_mv_plan( query: Query, name: ObjectName, columns: Vec, -) -> Result<(PlanRef, ProstTable)> { +) -> Result<(PlanRef, PbTable)> { let db_name = session.database(); let (schema_name, table_name) = Binder::resolve_schema_qualified_name(db_name, name)?; diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 019c219bb477..71ca9267b9a1 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -34,7 +34,7 @@ use risingwave_connector::source::{ GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, NEXMARK_CONNECTOR, PULSAR_CONNECTOR, }; -use risingwave_pb::catalog::{Source as ProstSource, StreamSourceInfo, WatermarkDesc}; +use risingwave_pb::catalog::{PbSource, StreamSourceInfo, WatermarkDesc}; use risingwave_pb::plan_common::RowFormatType; use risingwave_sqlparser::ast::{ AvroSchema, CreateSourceStatement, DebeziumAvroSchema, ProtobufSchema, SourceSchema, @@ -634,7 +634,7 @@ pub async fn handle_create_source( let columns = columns.into_iter().map(|c| c.to_protobuf()).collect_vec(); - let source = ProstSource { + let source = PbSource { id: TableId::placeholder().table_id, schema_id, database_id, diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 571685f9a5e5..96318e900bec 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -23,9 +23,7 @@ use risingwave_common::catalog::{ USER_COLUMN_ID_OFFSET, }; use risingwave_common::error::{ErrorCode, Result}; -use risingwave_pb::catalog::{ - Source as ProstSource, StreamSourceInfo, Table as ProstTable, WatermarkDesc, -}; +use risingwave_pb::catalog::{PbSource, PbTable, StreamSourceInfo, WatermarkDesc}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ ColumnDef, ColumnOption, DataType as AstDataType, ObjectName, SourceSchema, SourceWatermark, @@ -290,7 +288,7 @@ pub(crate) async fn gen_create_table_plan_with_source( source_watermarks: Vec, mut col_id_gen: ColumnIdGenerator, append_only: bool, -) -> Result<(PlanRef, Option, ProstTable)> { +) -> Result<(PlanRef, Option, PbTable)> { let (column_descs, pk_column_id_from_columns) = bind_sql_columns(columns, &mut col_id_gen)?; let mut properties = context.with_options().inner().clone().into_iter().collect(); @@ -343,7 +341,7 @@ pub(crate) fn gen_create_table_plan( mut col_id_gen: ColumnIdGenerator, source_watermarks: Vec, append_only: bool, -) -> Result<(PlanRef, Option, ProstTable)> { +) -> Result<(PlanRef, Option, PbTable)> { let definition = context.normalized_sql().to_owned(); let (column_descs, pk_column_id_from_columns) = bind_sql_columns(columns, &mut col_id_gen)?; @@ -374,7 +372,7 @@ pub(crate) fn gen_create_table_plan_without_bind( source_watermarks: Vec, append_only: bool, version: Option, -) -> Result<(PlanRef, Option, ProstTable)> { +) -> Result<(PlanRef, Option, PbTable)> { let (columns, pk_column_ids, row_id_index) = bind_sql_table_constraints(column_descs, pk_column_id_from_columns, constraints)?; @@ -414,13 +412,13 @@ fn gen_table_plan_inner( append_only: bool, version: Option, /* TODO: this should always be `Some` if we support `ALTER * TABLE` for `CREATE TABLE AS`. */ -) -> Result<(PlanRef, Option, ProstTable)> { +) -> Result<(PlanRef, Option, PbTable)> { let session = context.session_ctx(); let db_name = session.database(); let (schema_name, name) = Binder::resolve_schema_qualified_name(db_name, table_name)?; let (database_id, schema_id) = session.get_database_and_schema_id_for_create(schema_name)?; - let source = source_info.map(|source_info| ProstSource { + let source = source_info.map(|source_info| PbSource { id: TableId::placeholder().table_id, schema_id, database_id, diff --git a/src/frontend/src/handler/create_view.rs b/src/frontend/src/handler/create_view.rs index cf6a07364d81..d0850bb6e539 100644 --- a/src/frontend/src/handler/create_view.rs +++ b/src/frontend/src/handler/create_view.rs @@ -19,7 +19,7 @@ use std::collections::HashSet; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::error::Result; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_pb::catalog::View as ProstView; +use risingwave_pb::catalog::PbView; use risingwave_sqlparser::ast::{Ident, ObjectName, Query, Statement}; use super::RwPgResponse; @@ -80,7 +80,7 @@ pub async fn handle_create_view( .collect() }; - let view = ProstView { + let view = PbView { id: 0, schema_id, database_id, diff --git a/src/frontend/src/handler/handle_privilege.rs b/src/frontend/src/handler/handle_privilege.rs index b677f28b11cb..9e3c46667ff9 100644 --- a/src/frontend/src/handler/handle_privilege.rs +++ b/src/frontend/src/handler/handle_privilege.rs @@ -14,8 +14,8 @@ use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::error::{ErrorCode, Result}; -use risingwave_pb::user::grant_privilege::{ActionWithGrantOption, Object as ProstObject}; -use risingwave_pb::user::GrantPrivilege as ProstPrivilege; +use risingwave_pb::user::grant_privilege::{ActionWithGrantOption, PbObject}; +use risingwave_pb::user::PbGrantPrivilege; use risingwave_sqlparser::ast::{GrantObjects, Privileges, Statement}; use super::RwPgResponse; @@ -31,7 +31,7 @@ fn make_prost_privilege( session: &SessionImpl, privileges: Privileges, objects: GrantObjects, -) -> Result> { +) -> Result> { check_privilege_type(&privileges, &objects)?; let catalog_reader = session.env().catalog_reader(); @@ -46,14 +46,14 @@ fn make_prost_privilege( for db in databases { let database_name = Binder::resolve_database_name(db)?; let database = reader.get_database_by_name(&database_name)?; - grant_objs.push(ProstObject::DatabaseId(database.id())); + grant_objs.push(PbObject::DatabaseId(database.id())); } } GrantObjects::Schemas(schemas) => { for schema in schemas { let schema_name = Binder::resolve_schema_name(schema)?; let schema = reader.get_schema_by_name(session.database(), &schema_name)?; - grant_objs.push(ProstObject::SchemaId(schema.id())); + grant_objs.push(PbObject::SchemaId(schema.id())); } } GrantObjects::Mviews(tables) => { @@ -67,7 +67,7 @@ fn make_prost_privilege( let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name); let (table, _) = reader.get_table_by_name(db_name, schema_path, &table_name)?; - grant_objs.push(ProstObject::TableId(table.id().table_id)); + grant_objs.push(PbObject::TableId(table.id().table_id)); } } GrantObjects::Sources(sources) => { @@ -81,21 +81,21 @@ fn make_prost_privilege( let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name); let (source, _) = reader.get_source_by_name(db_name, schema_path, &source_name)?; - grant_objs.push(ProstObject::SourceId(source.id)); + grant_objs.push(PbObject::SourceId(source.id)); } } GrantObjects::AllSourcesInSchema { schemas } => { for schema in schemas { let schema_name = Binder::resolve_schema_name(schema)?; let schema = reader.get_schema_by_name(session.database(), &schema_name)?; - grant_objs.push(ProstObject::AllSourcesSchemaId(schema.id())); + grant_objs.push(PbObject::AllSourcesSchemaId(schema.id())); } } GrantObjects::AllMviewsInSchema { schemas } => { for schema in schemas { let schema_name = Binder::resolve_schema_name(schema)?; let schema = reader.get_schema_by_name(session.database(), &schema_name)?; - grant_objs.push(ProstObject::AllTablesSchemaId(schema.id())); + grant_objs.push(PbObject::AllTablesSchemaId(schema.id())); } } o => { @@ -120,7 +120,7 @@ fn make_prost_privilege( let mut prost_privileges = vec![]; for objs in grant_objs { - prost_privileges.push(ProstPrivilege { + prost_privileges.push(PbGrantPrivilege { action_with_opts: action_with_opts.clone(), object: Some(objs), }); @@ -261,15 +261,15 @@ mod tests { assert_eq!( user_info.grant_privileges, vec![ - ProstPrivilege { + PbGrantPrivilege { action_with_opts: vec![ActionWithGrantOption { action: Action::Connect as i32, with_grant_option: true, granted_by: session.user_id(), }], - object: Some(ProstObject::DatabaseId(session_database_id)), + object: Some(PbObject::DatabaseId(session_database_id)), }, - ProstPrivilege { + PbGrantPrivilege { action_with_opts: vec![ ActionWithGrantOption { action: Action::Connect as i32, @@ -282,7 +282,7 @@ mod tests { granted_by: DEFAULT_SUPER_USER_ID, } ], - object: Some(ProstObject::DatabaseId(database_id)), + object: Some(PbObject::DatabaseId(database_id)), } ] ); @@ -299,7 +299,7 @@ mod tests { assert!(user_info .grant_privileges .iter() - .filter(|gp| gp.object == Some(ProstObject::DatabaseId(database_id))) + .filter(|gp| gp.object == Some(PbObject::DatabaseId(database_id))) .all(|p| p.action_with_opts.iter().all(|ao| !ao.with_grant_option))); } @@ -313,13 +313,13 @@ mod tests { let user_info = reader.get_user_by_name("user1").unwrap(); assert_eq!( user_info.grant_privileges, - vec![ProstPrivilege { + vec![PbGrantPrivilege { action_with_opts: vec![ActionWithGrantOption { action: Action::Connect as i32, with_grant_option: true, granted_by: session.user_id(), }], - object: Some(ProstObject::DatabaseId(session_database_id)), + object: Some(PbObject::DatabaseId(session_database_id)), }] ); } diff --git a/src/frontend/src/handler/privilege.rs b/src/frontend/src/handler/privilege.rs index e144990898f1..21c0ce5317fc 100644 --- a/src/frontend/src/handler/privilege.rs +++ b/src/frontend/src/handler/privilege.rs @@ -14,7 +14,7 @@ use risingwave_common::error::ErrorCode::PermissionDenied; use risingwave_common::error::Result; -use risingwave_pb::user::grant_privilege::{Action as ProstAction, Object as ProstObject}; +use risingwave_pb::user::grant_privilege::{PbAction, PbObject}; use crate::binder::{BoundStatement, Relation}; use crate::catalog::RelationCatalog; @@ -23,12 +23,12 @@ use crate::user::UserId; pub struct ObjectCheckItem { owner: UserId, - action: ProstAction, - object: ProstObject, + action: PbAction, + object: PbObject, } impl ObjectCheckItem { - pub fn new(owner: UserId, action: ProstAction, object: ProstObject) -> Self { + pub fn new(owner: UserId, action: PbAction, object: PbObject) -> Self { Self { owner, action, @@ -40,7 +40,7 @@ impl ObjectCheckItem { /// resolve privileges in `relation` pub(crate) fn resolve_relation_privileges( relation: &Relation, - action: ProstAction, + action: PbAction, objects: &mut Vec, ) { match relation { @@ -48,7 +48,7 @@ pub(crate) fn resolve_relation_privileges( let item = ObjectCheckItem { owner: source.catalog.owner, action, - object: ProstObject::SourceId(source.catalog.id), + object: PbObject::SourceId(source.catalog.id), }; objects.push(item); } @@ -56,7 +56,7 @@ pub(crate) fn resolve_relation_privileges( let item = ObjectCheckItem { owner: table.table_catalog.owner, action, - object: ProstObject::TableId(table.table_id.table_id), + object: PbObject::TableId(table.table_id.table_id), }; objects.push(item); } @@ -85,36 +85,36 @@ pub(crate) fn resolve_privileges(stmt: &BoundStatement) -> Vec BoundStatement::Insert(ref insert) => { let object = ObjectCheckItem { owner: insert.owner, - action: ProstAction::Insert, - object: ProstObject::TableId(insert.table_id.table_id), + action: PbAction::Insert, + object: PbObject::TableId(insert.table_id.table_id), }; objects.push(object); if let crate::binder::BoundSetExpr::Select(select) = &insert.source.body { if let Some(sub_relation) = &select.from { - resolve_relation_privileges(sub_relation, ProstAction::Select, &mut objects); + resolve_relation_privileges(sub_relation, PbAction::Select, &mut objects); } } } BoundStatement::Delete(ref delete) => { let object = ObjectCheckItem { owner: delete.owner, - action: ProstAction::Delete, - object: ProstObject::TableId(delete.table_id.table_id), + action: PbAction::Delete, + object: PbObject::TableId(delete.table_id.table_id), }; objects.push(object); } BoundStatement::Update(ref update) => { let object = ObjectCheckItem { owner: update.owner, - action: ProstAction::Update, - object: ProstObject::TableId(update.table_id.table_id), + action: PbAction::Update, + object: PbObject::TableId(update.table_id.table_id), }; objects.push(object); } BoundStatement::Query(ref query) => { if let crate::binder::BoundSetExpr::Select(select) = &query.body { if let Some(sub_relation) = &select.from { - resolve_relation_privileges(sub_relation, ProstAction::Select, &mut objects); + resolve_relation_privileges(sub_relation, PbAction::Select, &mut objects); } } } @@ -226,8 +226,8 @@ mod tests { .clone(); let check_items = vec![ObjectCheckItem::new( DEFAULT_SUPER_USER_ID, - ProstAction::Create, - ProstObject::SchemaId(schema.id()), + PbAction::Create, + PbObject::SchemaId(schema.id()), )]; assert!(&session.check_privileges(&check_items).is_ok()); diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index e421973e1735..1b0d34dd29c3 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -19,7 +19,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::DeleteNode; use super::{ - ExprRewritable, LogicalDelete, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ExprRewritable, LogicalDelete, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::optimizer::plan_node::ToLocalBatch; @@ -71,7 +71,7 @@ impl ToDistributedBatch for BatchDelete { } } -impl ToBatchProst for BatchDelete { +impl ToBatchPb for BatchDelete { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::Delete(DeleteNode { table_id: self.logical.table_id().table_id(), diff --git a/src/frontend/src/optimizer/plan_node/batch_exchange.rs b/src/frontend/src/optimizer/plan_node/batch_exchange.rs index 4018e8b83551..d69ae041f65b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/batch_exchange.rs @@ -18,9 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ExchangeNode, MergeSortExchangeNode}; -use super::{ - ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, -}; +use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, DistributionDisplay, Order, OrderDisplay}; @@ -76,7 +74,7 @@ impl ToDistributedBatch for BatchExchange { } /// The serialization of Batch Exchange is default cuz it will be rewritten in scheduler. -impl ToBatchProst for BatchExchange { +impl ToBatchPb for BatchExchange { fn to_batch_prost_body(&self) -> NodeBody { if self.base.order.is_any() { NodeBody::Exchange(ExchangeNode { diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index 99e4d746f41c..86043cd3e1ab 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -22,7 +22,7 @@ use risingwave_pb::batch_plan::ExpandNode; use super::ExprRewritable; use crate::optimizer::plan_node::{ - LogicalExpand, PlanBase, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ToLocalBatch, + LogicalExpand, PlanBase, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, ToLocalBatch, }; use crate::optimizer::property::{Distribution, Order}; use crate::optimizer::PlanRef; @@ -77,7 +77,7 @@ impl ToDistributedBatch for BatchExpand { } } -impl ToBatchProst for BatchExpand { +impl ToBatchPb for BatchExpand { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::Expand(ExpandNode { column_subsets: self diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index 56e51cf3bbc0..bf37f65b1984 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -19,7 +19,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::FilterNode; use super::{ - ExprRewritable, LogicalFilter, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, + ExprRewritable, LogicalFilter, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::{PlanBase, ToLocalBatch}; @@ -75,7 +75,7 @@ impl ToDistributedBatch for BatchFilter { } } -impl ToBatchProst for BatchFilter { +impl ToBatchPb for BatchFilter { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::Filter(FilterNode { search_condition: Some( diff --git a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs index a1da0a482f96..b9f8160cc407 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -19,7 +19,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::GroupTopNNode; use super::{ - ExprRewritable, LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ExprRewritable, LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::optimizer::plan_node::ToLocalBatch; @@ -78,7 +78,7 @@ impl ToDistributedBatch for BatchGroupTopN { } } -impl ToBatchProst for BatchGroupTopN { +impl ToBatchPb for BatchGroupTopN { fn to_batch_prost_body(&self) -> NodeBody { let column_orders = self.logical.topn_order().to_protobuf(); NodeBody::GroupTopN(GroupTopNNode { diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs index 8b1a7c9414fc..83f0ef6ae3d7 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -21,7 +21,7 @@ use risingwave_pb::batch_plan::HashAggNode; use super::generic::{GenericPlanRef, PlanAggCall}; use super::{ - ExprRewritable, LogicalAgg, PlanBase, PlanNodeType, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ExprRewritable, LogicalAgg, PlanBase, PlanNodeType, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::expr::ExprRewriter; @@ -135,7 +135,7 @@ impl ToDistributedBatch for BatchHashAgg { } } -impl ToBatchProst for BatchHashAgg { +impl ToBatchPb for BatchHashAgg { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::HashAgg(HashAggNode { agg_calls: self diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs index df60d927b649..3fcb73c48b56 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -22,8 +22,8 @@ use risingwave_pb::plan_common::JoinType; use super::generic::GenericPlanRef; use super::{ - EqJoinPredicate, ExprRewritable, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, - ToBatchProst, ToDistributedBatch, + EqJoinPredicate, ExprRewritable, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchPb, + ToDistributedBatch, }; use crate::expr::{Expr, ExprRewriter}; use crate::optimizer::plan_node::utils::IndicesDisplay; @@ -218,7 +218,7 @@ impl ToDistributedBatch for BatchHashJoin { } } -impl ToBatchProst for BatchHashJoin { +impl ToBatchPb for BatchHashJoin { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::HashJoin(HashJoinNode { join_type: self.logical.join_type() as i32, diff --git a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs index 49454c4ed8c9..ccfa664f9bc4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -19,7 +19,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HopWindowNode; use super::{ - ExprRewritable, LogicalHopWindow, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ExprRewritable, LogicalHopWindow, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::expr::{Expr, ExprImpl, ExprRewriter}; @@ -116,7 +116,7 @@ impl ToDistributedBatch for BatchHopWindow { } } -impl ToBatchProst for BatchHopWindow { +impl ToBatchPb for BatchHopWindow { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::HopWindow(HopWindowNode { time_col: self.logical.core.time_col.index() as _, diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index 3bf9b4345250..b574c002dd8b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -19,7 +19,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::InsertNode; use super::{ - ExprRewritable, LogicalInsert, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, + ExprRewritable, LogicalInsert, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::optimizer::plan_node::{PlanBase, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -71,7 +71,7 @@ impl ToDistributedBatch for BatchInsert { } } -impl ToBatchProst for BatchInsert { +impl ToBatchPb for BatchInsert { fn to_batch_prost_body(&self) -> NodeBody { let column_indices = self .logical diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index 9a851dde7063..4944d312757b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -19,7 +19,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::LimitNode; use super::{ - ExprRewritable, LogicalLimit, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ExprRewritable, LogicalLimit, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::optimizer::plan_node::ToLocalBatch; @@ -98,7 +98,7 @@ impl ToDistributedBatch for BatchLimit { } } -impl ToBatchProst for BatchLimit { +impl ToBatchPb for BatchLimit { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::Limit(LimitNode { limit: self.logical.limit(), diff --git a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs index 1b3fc89db771..b3c02a220e67 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -25,7 +25,7 @@ use crate::expr::{Expr, ExprRewriter}; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::plan_node::{ EqJoinPredicate, EqJoinPredicateDisplay, LogicalJoin, PlanBase, PlanTreeNodeBinary, - PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ToLocalBatch, + PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, ToLocalBatch, }; use crate::optimizer::property::{Distribution, Order, RequiredDist}; use crate::optimizer::PlanRef; @@ -50,7 +50,7 @@ pub struct BatchLookupJoin { lookup_prefix_len: usize, /// If `distributed_lookup` is true, it will generate `DistributedLookupJoinNode` for - /// `ToBatchProst`. Otherwise, it will generate `LookupJoinNode`. + /// `ToBatchPb`. Otherwise, it will generate `LookupJoinNode`. distributed_lookup: bool, } @@ -192,7 +192,7 @@ impl ToDistributedBatch for BatchLookupJoin { } } -impl ToBatchProst for BatchLookupJoin { +impl ToBatchPb for BatchLookupJoin { fn to_batch_prost_body(&self) -> NodeBody { if self.distributed_lookup { NodeBody::DistributedLookupJoin(DistributedLookupJoinNode { diff --git a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs index 0712f0cd9a7c..c2cdb71e4541 100644 --- a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs @@ -21,7 +21,7 @@ use risingwave_pb::batch_plan::NestedLoopJoinNode; use super::generic::GenericPlanRef; use super::{ - ExprRewritable, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchProst, + ExprRewritable, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchPb, ToDistributedBatch, }; use crate::expr::{Expr, ExprImpl, ExprRewriter}; @@ -127,7 +127,7 @@ impl ToDistributedBatch for BatchNestedLoopJoin { } } -impl ToBatchProst for BatchNestedLoopJoin { +impl ToBatchPb for BatchNestedLoopJoin { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::NestedLoopJoin(NestedLoopJoinNode { join_type: self.logical.join_type() as i32, diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 15140cafc7e9..5a05c341ce97 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -20,7 +20,7 @@ use risingwave_pb::batch_plan::ProjectNode; use risingwave_pb::expr::ExprNode; use super::{ - ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::expr::{Expr, ExprImpl, ExprRewriter}; @@ -83,7 +83,7 @@ impl ToDistributedBatch for BatchProject { } } -impl ToBatchProst for BatchProject { +impl ToBatchPb for BatchProject { fn to_batch_prost_body(&self) -> NodeBody { let select_list = self .logical diff --git a/src/frontend/src/optimizer/plan_node/batch_project_set.rs b/src/frontend/src/optimizer/plan_node/batch_project_set.rs index 5946ad9f5fde..2534777b7e8c 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -22,7 +22,7 @@ use risingwave_pb::batch_plan::ProjectSetNode; use super::ExprRewritable; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ - LogicalProjectSet, PlanBase, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ToLocalBatch, + LogicalProjectSet, PlanBase, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, ToLocalBatch, }; use crate::optimizer::PlanRef; use crate::utils::ColIndexMappingRewriteExt; @@ -77,7 +77,7 @@ impl ToDistributedBatch for BatchProjectSet { // TODO: implement to_distributed_with_required like BatchProject } -impl ToBatchProst for BatchProjectSet { +impl ToBatchPb for BatchProjectSet { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::ProjectSet(ProjectSetNode { select_list: self diff --git a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs index f38d11447c73..0acf25cc5756 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -22,9 +22,9 @@ use risingwave_common::util::scan_range::{is_full_range, ScanRange}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::row_seq_scan_node::ChunkSize; use risingwave_pb::batch_plan::{RowSeqScanNode, SysRowSeqScanNode}; -use risingwave_pb::plan_common::ColumnDesc as ProstColumnDesc; +use risingwave_pb::plan_common::PbColumnDesc; -use super::{ExprRewritable, PlanBase, PlanRef, ToBatchProst, ToDistributedBatch}; +use super::{ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch}; use crate::catalog::ColumnId; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{LogicalScan, ToLocalBatch}; @@ -214,13 +214,13 @@ impl ToDistributedBatch for BatchSeqScan { } } -impl ToBatchProst for BatchSeqScan { +impl ToBatchPb for BatchSeqScan { fn to_batch_prost_body(&self) -> NodeBody { let column_descs = self .logical .column_descs() .iter() - .map(ProstColumnDesc::from) + .map(PbColumnDesc::from) .collect(); if self.logical.is_sys_table() { diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index b22cb4d0da0a..e19cbae85aaf 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -20,8 +20,7 @@ use risingwave_pb::batch_plan::SortAggNode; use super::generic::PlanAggCall; use super::{ - ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, - ToDistributedBatch, + ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{BatchExchange, ToLocalBatch}; @@ -108,7 +107,7 @@ impl ToDistributedBatch for BatchSimpleAgg { } } -impl ToBatchProst for BatchSimpleAgg { +impl ToBatchPb for BatchSimpleAgg { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::SortAgg(SortAggNode { agg_calls: self diff --git a/src/frontend/src/optimizer/plan_node/batch_sort.rs b/src/frontend/src/optimizer/plan_node/batch_sort.rs index f03d81949546..38b268dd1c72 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort.rs @@ -18,9 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortNode; -use super::{ - ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, -}; +use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, OrderDisplay}; @@ -73,7 +71,7 @@ impl ToDistributedBatch for BatchSort { } } -impl ToBatchProst for BatchSort { +impl ToBatchPb for BatchSort { fn to_batch_prost_body(&self) -> NodeBody { let column_orders = self.base.order.to_protobuf(); NodeBody::Sort(SortNode { column_orders }) 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 28973348ed96..c348f655076f 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -22,8 +22,7 @@ use risingwave_pb::expr::ExprNode; use super::generic::{GenericPlanRef, PlanAggCall}; use super::{ - ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, - ToDistributedBatch, + ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::expr::{Expr, ExprImpl, ExprRewriter, InputRef}; use crate::optimizer::plan_node::ToLocalBatch; @@ -109,7 +108,7 @@ impl ToDistributedBatch for BatchSortAgg { } } -impl ToBatchProst for BatchSortAgg { +impl ToBatchPb for BatchSortAgg { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::SortAgg(SortAggNode { agg_calls: self diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index 13d4ab98e0f1..49eb14ada6e5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -20,8 +20,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SourceNode; use super::{ - ExprRewritable, LogicalSource, PlanBase, PlanRef, ToBatchProst, ToDistributedBatch, - ToLocalBatch, + ExprRewritable, LogicalSource, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch, ToLocalBatch, }; use crate::optimizer::property::{Distribution, Order}; @@ -92,7 +91,7 @@ impl ToDistributedBatch for BatchSource { } } -impl ToBatchProst for BatchSource { +impl ToBatchPb for BatchSource { fn to_batch_prost_body(&self) -> NodeBody { let source_catalog = self.logical.source_catalog().unwrap(); NodeBody::Source(SourceNode { diff --git a/src/frontend/src/optimizer/plan_node/batch_table_function.rs b/src/frontend/src/optimizer/plan_node/batch_table_function.rs index 8e3e1568d4e5..b8e7eb979fbe 100644 --- a/src/frontend/src/optimizer/plan_node/batch_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/batch_table_function.rs @@ -18,9 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TableFunctionNode; -use super::{ - ExprRewritable, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchProst, ToDistributedBatch, -}; +use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchPb, ToDistributedBatch}; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::logical_table_function::LogicalTableFunction; use crate::optimizer::plan_node::ToLocalBatch; @@ -68,7 +66,7 @@ impl ToDistributedBatch for BatchTableFunction { } } -impl ToBatchProst for BatchTableFunction { +impl ToBatchPb for BatchTableFunction { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::TableFunction(TableFunctionNode { table_function: Some(self.logical.table_function.to_protobuf()), diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index 28b21bd3e801..e06f962ddae9 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -19,7 +19,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TopNNode; use super::{ - ExprRewritable, LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ExprRewritable, LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::optimizer::plan_node::ToLocalBatch; @@ -88,7 +88,7 @@ impl ToDistributedBatch for BatchTopN { } } -impl ToBatchProst for BatchTopN { +impl ToBatchPb for BatchTopN { fn to_batch_prost_body(&self) -> NodeBody { let column_orders = self.logical.topn_order().to_protobuf(); NodeBody::TopN(TopNNode { diff --git a/src/frontend/src/optimizer/plan_node/batch_union.rs b/src/frontend/src/optimizer/plan_node/batch_union.rs index 77665ce931fb..4d2f415b3cdb 100644 --- a/src/frontend/src/optimizer/plan_node/batch_union.rs +++ b/src/frontend/src/optimizer/plan_node/batch_union.rs @@ -18,7 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UnionNode; -use super::{ExprRewritable, PlanRef, ToBatchProst, ToDistributedBatch}; +use super::{ExprRewritable, PlanRef, ToBatchPb, ToDistributedBatch}; use crate::optimizer::plan_node::{LogicalUnion, PlanBase, PlanTreeNode, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -82,7 +82,7 @@ impl ToDistributedBatch for BatchUnion { } } -impl ToBatchProst for BatchUnion { +impl ToBatchPb for BatchUnion { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::Union(UnionNode {}) } diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index 57eb7ea82396..b2f80cadff77 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -19,7 +19,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UpdateNode; use super::{ - ExprRewritable, LogicalUpdate, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ExprRewritable, LogicalUpdate, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::expr::{Expr, ExprRewriter}; @@ -72,7 +72,7 @@ impl ToDistributedBatch for BatchUpdate { } } -impl ToBatchProst for BatchUpdate { +impl ToBatchPb for BatchUpdate { fn to_batch_prost_body(&self) -> NodeBody { let exprs = self .logical diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index affcddc8e01d..07b6b2b57795 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -20,7 +20,7 @@ use risingwave_pb::batch_plan::values_node::ExprTuple; use risingwave_pb::batch_plan::ValuesNode; use super::{ - ExprRewritable, LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchProst, + ExprRewritable, LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchPb, ToDistributedBatch, }; use crate::expr::{Expr, ExprImpl, ExprRewriter}; @@ -73,7 +73,7 @@ impl ToDistributedBatch for BatchValues { } } -impl ToBatchProst for BatchValues { +impl ToBatchPb for BatchValues { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::Values(ValuesNode { tuples: self diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 931bfa237116..20b3bac69d49 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -20,8 +20,8 @@ use risingwave_common::catalog::{Field, FieldDisplay, Schema}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::{ColumnOrder, ColumnOrderDisplay, OrderType}; use risingwave_expr::expr::AggKind; -use risingwave_pb::expr::AggCall as ProstAggCall; -use risingwave_pb::stream_plan::{agg_call_state, AggCallState as AggCallStateProst}; +use risingwave_pb::expr::PbAggCall; +use risingwave_pb::stream_plan::{agg_call_state, AggCallState as AggCallStatePb}; use super::super::utils::TableCatalogBuilder; use super::{stream, GenericPlanNode, GenericPlanRef}; @@ -124,8 +124,8 @@ pub enum AggCallState { } impl AggCallState { - pub fn into_prost(self, state: &mut BuildFragmentGraphState) -> AggCallStateProst { - AggCallStateProst { + pub fn into_prost(self, state: &mut BuildFragmentGraphState) -> AggCallStatePb { + AggCallStatePb { inner: Some(match self { AggCallState::ResultValue => { agg_call_state::Inner::ResultValueState(agg_call_state::ResultValueState {}) @@ -578,8 +578,8 @@ impl PlanAggCall { }); } - pub fn to_protobuf(&self) -> ProstAggCall { - ProstAggCall { + pub fn to_protobuf(&self) -> PbAggCall { + PbAggCall { r#type: self.agg_kind.to_prost().into(), return_type: Some(self.return_type.to_protobuf()), args: self.inputs.iter().map(InputRef::to_proto).collect(), diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index d4e879cfefcf..fdb7b3d0c9ee 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -41,8 +41,8 @@ pub use logical_source::KAFKA_TIMESTAMP_COLUMN_NAME; use paste::paste; use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result}; -use risingwave_pb::batch_plan::PlanNode as BatchPlanProst; -use risingwave_pb::stream_plan::StreamNode as StreamPlanProst; +use risingwave_pb::batch_plan::PlanNode as BatchPlanPb; +use risingwave_pb::stream_plan::StreamNode as StreamPlanPb; use serde::Serialize; use smallvec::SmallVec; @@ -73,7 +73,7 @@ pub trait PlanNode: + ToBatch + ToStream + ToDistributedBatch - + ToProst + + ToPb + ToLocalBatch + PredicatePushdown + PlanNodeMeta @@ -520,7 +520,7 @@ impl dyn PlanNode { /// /// Note that [`StreamTableScan`] has its own implementation of `to_stream_prost`. We have a /// hook inside to do some ad-hoc thing for [`StreamTableScan`]. - pub fn to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> StreamPlanProst { + pub fn to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> StreamPlanPb { if let Some(stream_table_scan) = self.as_stream_table_scan() { return stream_table_scan.adhoc_to_stream_prost(); } @@ -535,7 +535,7 @@ impl dyn PlanNode { .map(|plan| plan.to_stream_prost(state)) .collect(); // TODO: support pk_indices and operator_id - StreamPlanProst { + StreamPlanPb { input, identity: format!("{}", self), node_body: node, @@ -547,20 +547,20 @@ impl dyn PlanNode { } /// Serialize the plan node and its children to a batch plan proto. - pub fn to_batch_prost(&self) -> BatchPlanProst { + pub fn to_batch_prost(&self) -> BatchPlanPb { self.to_batch_prost_identity(true) } /// Serialize the plan node and its children to a batch plan proto without the identity field /// (for testing). - pub fn to_batch_prost_identity(&self, identity: bool) -> BatchPlanProst { + pub fn to_batch_prost_identity(&self, identity: bool) -> BatchPlanPb { let node_body = Some(self.to_batch_prost_body()); let children = self .inputs() .into_iter() .map(|plan| plan.to_batch_prost_identity(identity)) .collect(); - BatchPlanProst { + BatchPlanPb { children, identity: if identity { format!("{:?}", self) diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index c63795d5fb44..0ecb43df610e 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -422,18 +422,18 @@ impl_node!( TopN ); -use pb_node::NodeBody as ProstNode; +use pb_node::PbNodeBody; #[allow(dead_code)] pub fn to_stream_prost_body( (base, core): &PlanOwned, state: &mut BuildFragmentGraphState, -) -> ProstNode { +) -> PbNodeBody { use pb::*; match core { Node::TableScan(_) => todo!(), Node::IndexScan(_) => todo!(), // ^ need standalone implementations - Node::Exchange(_) => ProstNode::Exchange(ExchangeNode { + Node::Exchange(_) => PbNodeBody::Exchange(ExchangeNode { strategy: Some(DispatchStrategy { r#type: match &base.dist { Distribution::HashShard(_) => DispatcherType::Hash, @@ -459,7 +459,7 @@ pub fn to_stream_prost_body( .with_id(state.gen_table_id_wrapped()); let right_table = infer_right_internal_table_catalog(&me.right.0) .with_id(state.gen_table_id_wrapped()); - ProstNode::DynamicFilter(DynamicFilterNode { + PbNodeBody::DynamicFilter(DynamicFilterNode { left_key: me.left_index as u32, condition, left_table: Some(left_table.to_internal_table_prost()), @@ -482,7 +482,7 @@ pub fn to_stream_prost_body( // TODO: add a separate delta join node in proto, or move fragmenter to frontend so that // we don't need an intermediate representation. - ProstNode::DeltaIndexJoin(DeltaIndexJoinNode { + PbNodeBody::DeltaIndexJoin(DeltaIndexJoinNode { join_type: me.core.join_type as i32, left_key: me .eq_join_predicate @@ -530,7 +530,7 @@ pub fn to_stream_prost_body( use pb::expand_node::Subset; let me = &me.core; - ProstNode::Expand(ExpandNode { + PbNodeBody::Expand(ExpandNode { column_subsets: me .column_subsets .iter() @@ -543,7 +543,7 @@ pub fn to_stream_prost_body( } Node::Filter(me) => { let me = &me.core; - ProstNode::Filter(FilterNode { + PbNodeBody::Filter(FilterNode { search_condition: Some(ExprImpl::from(me.predicate.clone()).to_expr_proto()), }) } @@ -552,7 +552,7 @@ pub fn to_stream_prost_body( let agg_states = me.core.infer_stream_agg_state(base, None); let distinct_dedup_tables = me.core.infer_distinct_dedup_tables(base, None); - ProstNode::GlobalSimpleAgg(SimpleAggNode { + PbNodeBody::GlobalSimpleAgg(SimpleAggNode { agg_calls: me .core .agg_calls @@ -596,14 +596,14 @@ pub fn to_stream_prost_body( order_by: me.core.order.to_protobuf(), }; - ProstNode::GroupTopN(group_topn_node) + PbNodeBody::GroupTopN(group_topn_node) } Node::HashAgg(me) => { let result_table = me.core.infer_result_table(base, me.vnode_col_idx); let agg_states = me.core.infer_stream_agg_state(base, me.vnode_col_idx); let distinct_dedup_tables = me.core.infer_distinct_dedup_tables(base, me.vnode_col_idx); - ProstNode::HashAgg(HashAggNode { + PbNodeBody::HashAgg(HashAggNode { group_key: me.core.group_key.iter().map(|&idx| idx as u32).collect(), agg_calls: me .core @@ -645,7 +645,7 @@ pub fn to_stream_prost_body( .map(|x| x.to_expr_proto()) .collect(); let me = &me.core; - ProstNode::HopWindow(HopWindowNode { + PbNodeBody::HopWindow(HopWindowNode { time_col: me.time_col.index() as _, window_slide: Some(me.window_slide.into()), window_size: Some(me.window_size.into()), @@ -656,7 +656,7 @@ pub fn to_stream_prost_body( } Node::LocalSimpleAgg(me) => { let me = &me.core; - ProstNode::LocalSimpleAgg(SimpleAggNode { + PbNodeBody::LocalSimpleAgg(SimpleAggNode { agg_calls: me.agg_calls.iter().map(PlanAggCall::to_protobuf).collect(), row_count_index: u32::MAX, // this is not used distribution_key: base @@ -672,7 +672,7 @@ pub fn to_stream_prost_body( }) } Node::Materialize(me) => { - ProstNode::Materialize(MaterializeNode { + PbNodeBody::Materialize(MaterializeNode { // We don't need table id for materialize node in frontend. The id will be generated // on meta catalog service. table_id: 0, @@ -688,9 +688,9 @@ pub fn to_stream_prost_body( .iter() .map(ExprImpl::to_project_set_select_item_proto) .collect(); - ProstNode::ProjectSet(ProjectSetNode { select_list }) + PbNodeBody::ProjectSet(ProjectSetNode { select_list }) } - Node::Project(me) => ProstNode::Project(ProjectNode { + Node::Project(me) => PbNodeBody::Project(ProjectNode { select_list: me.core.exprs.iter().map(|x| x.to_expr_proto()).collect(), watermark_input_key: me .watermark_derivations @@ -703,7 +703,7 @@ pub fn to_stream_prost_body( .map(|(_, y)| *y as u32) .collect(), }), - Node::Sink(me) => ProstNode::Sink(SinkNode { + Node::Sink(me) => PbNodeBody::Sink(SinkNode { sink_desc: Some(me.sink_desc.to_proto()), }), Node::Source(me) => { @@ -722,7 +722,7 @@ pub fn to_stream_prost_body( pk_column_ids: me.pk_col_ids.iter().map(Into::into).collect(), properties: me.properties.clone().into_iter().collect(), }); - ProstNode::Source(SourceNode { source_inner }) + PbNodeBody::Source(SourceNode { source_inner }) } Node::TopN(me) => { let me = &me.core; @@ -740,9 +740,9 @@ pub fn to_stream_prost_body( // TODO: support with ties for append only TopN // if me.input.0.append_only && !me.with_ties { - ProstNode::AppendOnlyTopN(topn_node) + PbNodeBody::AppendOnlyTopN(topn_node) } else { - ProstNode::TopN(topn_node) + PbNodeBody::TopN(topn_node) } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_dml.rs b/src/frontend/src/optimizer/plan_node/stream_dml.rs index 1d8d5300ad8a..1d0a62056aae 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dml.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dml.rs @@ -16,7 +16,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use risingwave_common::catalog::{ColumnDesc, INITIAL_TABLE_VERSION_ID}; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -78,10 +78,10 @@ impl PlanTreeNodeUnary for StreamDml { impl_plan_tree_node_for_unary! {StreamDml} impl StreamNode for StreamDml { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { use risingwave_pb::stream_plan::*; - ProstStreamNode::Dml(DmlNode { + PbNodeBody::Dml(DmlNode { table_id: 0, // Meta will fill this table id. table_version_id: INITIAL_TABLE_VERSION_ID, // Meta will fill this version id. column_descs: self.column_descs.iter().map(Into::into).collect(), diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs index 853e5800bcd9..bc70d8d3b2b8 100644 --- a/src/frontend/src/optimizer/plan_node/stream_expand.rs +++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs @@ -16,7 +16,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use risingwave_pb::stream_plan::expand_node::Subset; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ExpandNode; use super::{ExprRewritable, LogicalExpand, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -86,8 +86,8 @@ impl PlanTreeNodeUnary for StreamExpand { impl_plan_tree_node_for_unary! { StreamExpand } impl StreamNode for StreamExpand { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { - ProstStreamNode::Expand(ExpandNode { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { + PbNodeBody::Expand(ExpandNode { column_subsets: self .column_subsets() .iter() diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index 964a9cb9f506..23ecfea0d4cb 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -14,7 +14,7 @@ use std::fmt; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::FilterNode; use super::{ExprRewritable, LogicalFilter, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -73,8 +73,8 @@ impl PlanTreeNodeUnary for StreamFilter { impl_plan_tree_node_for_unary! { StreamFilter } impl StreamNode for StreamFilter { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { - ProstStreamNode::Filter(FilterNode { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { + PbNodeBody::Filter(FilterNode { search_condition: Some(ExprImpl::from(self.predicate().clone()).to_expr_proto()), }) } diff --git a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs index b31e714141e6..9d4a2bddbd9d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs @@ -16,7 +16,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::PlanAggCall; use super::{ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -101,13 +101,13 @@ impl PlanTreeNodeUnary for StreamGlobalSimpleAgg { impl_plan_tree_node_for_unary! { StreamGlobalSimpleAgg } impl StreamNode for StreamGlobalSimpleAgg { - fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> PbNodeBody { use risingwave_pb::stream_plan::*; let result_table = self.logical.infer_result_table(None); let agg_states = self.logical.infer_stream_agg_state(None); let distinct_dedup_tables = self.logical.infer_distinct_dedup_tables(None); - ProstStreamNode::GlobalSimpleAgg(SimpleAggNode { + PbNodeBody::GlobalSimpleAgg(SimpleAggNode { agg_calls: self .agg_calls() .iter() diff --git a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs index 522200ef49d5..0735993b1fc5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs @@ -17,7 +17,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::FieldDisplay; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::{ExprRewritable, LogicalTopN, PlanBase, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::{Order, OrderDisplay}; @@ -90,7 +90,7 @@ impl StreamGroupTopN { } impl StreamNode for StreamGroupTopN { - fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> PbNodeBody { use risingwave_pb::stream_plan::*; let table = self .logical @@ -106,9 +106,9 @@ impl StreamNode for StreamGroupTopN { order_by: self.topn_order().to_protobuf(), }; if self.input().append_only() { - ProstStreamNode::AppendOnlyGroupTopN(group_topn_node) + PbNodeBody::AppendOnlyGroupTopN(group_topn_node) } else { - ProstStreamNode::GroupTopN(group_topn_node) + PbNodeBody::GroupTopN(group_topn_node) } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 5f679432a855..e73644886314 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -17,7 +17,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::FieldDisplay; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::PlanAggCall; use super::{ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -138,13 +138,13 @@ impl PlanTreeNodeUnary for StreamHashAgg { impl_plan_tree_node_for_unary! { StreamHashAgg } impl StreamNode for StreamHashAgg { - fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> PbNodeBody { use risingwave_pb::stream_plan::*; let result_table = self.logical.infer_result_table(self.vnode_col_idx); let agg_states = self.logical.infer_stream_agg_state(self.vnode_col_idx); let distinct_dedup_tables = self.logical.infer_distinct_dedup_tables(self.vnode_col_idx); - ProstStreamNode::HashAgg(HashAggNode { + PbNodeBody::HashAgg(HashAggNode { group_key: self.group_key().iter().map(|idx| *idx as u32).collect(), agg_calls: self .agg_calls() diff --git a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs index 73b455a54b75..221ed3e678d6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -17,7 +17,7 @@ use std::fmt; use itertools::Itertools; use risingwave_common::catalog::FieldDisplay; use risingwave_common::util::column_index_mapping::ColIndexMapping; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::HopWindowNode; use super::{ExprRewritable, LogicalHopWindow, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -118,8 +118,8 @@ impl PlanTreeNodeUnary for StreamHopWindow { impl_plan_tree_node_for_unary! {StreamHopWindow} impl StreamNode for StreamHopWindow { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { - ProstStreamNode::HopWindow(HopWindowNode { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { + PbNodeBody::HopWindow(HopWindowNode { time_col: self.logical.core.time_col.index() as _, window_slide: Some(self.logical.core.window_slide.into()), window_size: Some(self.logical.core.window_size.into()), diff --git a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs index 21d95483d546..32e2bdf8c19d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs @@ -16,7 +16,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::PlanAggCall; use super::{ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -89,9 +89,9 @@ impl PlanTreeNodeUnary for StreamLocalSimpleAgg { impl_plan_tree_node_for_unary! { StreamLocalSimpleAgg } impl StreamNode for StreamLocalSimpleAgg { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { use risingwave_pb::stream_plan::*; - ProstStreamNode::LocalSimpleAgg(SimpleAggNode { + PbNodeBody::LocalSimpleAgg(SimpleAggNode { agg_calls: self .agg_calls() .iter() diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 87809258e5e2..99a3d9d0ca33 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -21,7 +21,7 @@ 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 risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::derive::derive_columns; use super::{reorganize_elements_id, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -296,11 +296,11 @@ impl PlanTreeNodeUnary for StreamMaterialize { impl_plan_tree_node_for_unary! { StreamMaterialize } impl StreamNode for StreamMaterialize { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { use risingwave_pb::stream_plan::*; let handle_pk_conflict_behavior = self.table.conflict_behavior_type(); - ProstStreamNode::Materialize(MaterializeNode { + PbNodeBody::Materialize(MaterializeNode { // We don't need table id for materialize node in frontend. The id will be generated on // meta catalog service. table_id: 0, diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index 5fe185cad742..c79f3614a127 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -17,7 +17,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::FieldDisplay; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ProjectNode; use super::{ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -113,8 +113,8 @@ impl PlanTreeNodeUnary for StreamProject { impl_plan_tree_node_for_unary! {StreamProject} impl StreamNode for StreamProject { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { - ProstStreamNode::Project(ProjectNode { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { + PbNodeBody::Project(ProjectNode { select_list: self .logical .exprs() diff --git a/src/frontend/src/optimizer/plan_node/stream_project_set.rs b/src/frontend/src/optimizer/plan_node/stream_project_set.rs index 24ed8a833db4..6f0a5ee5de7e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project_set.rs @@ -16,7 +16,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ProjectSetNode; use super::{ExprRewritable, LogicalProjectSet, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -84,8 +84,8 @@ impl PlanTreeNodeUnary for StreamProjectSet { impl_plan_tree_node_for_unary! { StreamProjectSet } impl StreamNode for StreamProjectSet { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { - ProstStreamNode::ProjectSet(ProjectSetNode { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { + PbNodeBody::ProjectSet(ProjectSetNode { select_list: self .logical .select_list() diff --git a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs index fe2c59da603e..faad2b5a896f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs +++ b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs @@ -14,7 +14,7 @@ use std::fmt; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -68,10 +68,10 @@ impl PlanTreeNodeUnary for StreamRowIdGen { impl_plan_tree_node_for_unary! {StreamRowIdGen} impl StreamNode for StreamRowIdGen { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { use risingwave_pb::stream_plan::*; - ProstStreamNode::RowIdGen(RowIdGenNode { + PbNodeBody::RowIdGen(RowIdGenNode { row_id_index: self.row_id_index as _, }) } diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index 486694334f0e..e89bfc6010a6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -15,10 +15,8 @@ use std::fmt; use itertools::Itertools; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; -use risingwave_pb::stream_plan::{ - DispatchStrategy, DispatcherType, ExchangeNode, StreamNode as ProstStreamPlan, -}; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use risingwave_pb::stream_plan::{DispatchStrategy, DispatcherType, ExchangeNode, PbStreamNode}; use super::{ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::plan_node::{LogicalShare, PlanBase, PlanTreeNode}; @@ -77,13 +75,13 @@ impl StreamShare { impl_plan_tree_node_for_unary! { StreamShare } impl StreamNode for StreamShare { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { unreachable!("stream scan cannot be converted into a prost body -- call `adhoc_to_stream_prost` instead.") } } impl StreamShare { - pub fn adhoc_to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> ProstStreamPlan { + pub fn adhoc_to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> PbStreamNode { let operator_id = self.base.id.0 as u32; let output_indices = (0..self.schema().len() as u32).collect_vec(); @@ -122,7 +120,7 @@ impl StreamShare { } }; - let node_body = Some(ProstStreamNode::Exchange(ExchangeNode { + let node_body = Some(PbNodeBody::Exchange(ExchangeNode { strategy: Some(dispatch_strategy), })); let input = self @@ -131,7 +129,7 @@ impl StreamShare { .map(|plan| plan.to_stream_prost(state)) .collect(); - let stream_node = ProstStreamPlan { + let stream_node = PbStreamNode { input, identity: format!("{}", self), node_body, diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 87fa8f7a83d2..8139991de51c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -26,7 +26,7 @@ use risingwave_connector::sink::{ SINK_TYPE_APPEND_ONLY, SINK_TYPE_DEBEZIUM, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, SINK_USER_FORCE_APPEND_ONLY_OPTION, }; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use tracing::info; use super::derive::{derive_columns, derive_pk}; @@ -282,10 +282,10 @@ impl fmt::Display for StreamSink { } impl StreamNode for StreamSink { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { use risingwave_pb::stream_plan::*; - ProstStreamNode::Sink(SinkNode { + PbNodeBody::Sink(SinkNode { sink_desc: Some(self.sink_desc.to_proto()), }) } diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index b09cd99d8f48..de298c2a1b3e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -16,8 +16,8 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; -use risingwave_pb::stream_plan::{SourceNode, StreamSource as ProstStreamSource}; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use risingwave_pb::stream_plan::{PbStreamSource, SourceNode}; use super::{ExprRewritable, LogicalSource, PlanBase, StreamNode}; use crate::optimizer::property::Distribution; @@ -80,9 +80,9 @@ impl fmt::Display for StreamSource { } impl StreamNode for StreamSource { - fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> PbNodeBody { let source_catalog = self.logical.source_catalog(); - let source_inner = source_catalog.map(|source_catalog| ProstStreamSource { + let source_inner = source_catalog.map(|source_catalog| PbStreamSource { source_id: source_catalog.id, source_name: source_catalog.name.clone(), state_table: Some( @@ -105,7 +105,7 @@ impl StreamNode for StreamSource { .collect_vec(), properties: source_catalog.properties.clone().into_iter().collect(), }); - ProstStreamNode::Source(SourceNode { source_inner }) + PbNodeBody::Source(SourceNode { source_inner }) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index c9ec715dceeb..303e87c7fef6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -18,8 +18,8 @@ use std::rc::Rc; use itertools::Itertools; use risingwave_common::catalog::{Field, TableDesc}; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; -use risingwave_pb::stream_plan::{ChainType, StreamNode as ProstStreamPlan}; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use risingwave_pb::stream_plan::{ChainType, PbStreamNode}; use super::{ExprRewritable, LogicalScan, PlanBase, PlanNodeId, PlanRef, StreamNode}; use crate::catalog::ColumnId; @@ -149,13 +149,13 @@ impl fmt::Display for StreamTableScan { } impl StreamNode for StreamTableScan { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { unreachable!("stream scan cannot be converted into a prost body -- call `adhoc_to_stream_prost` instead.") } } impl StreamTableScan { - pub fn adhoc_to_stream_prost(&self) -> ProstStreamPlan { + pub fn adhoc_to_stream_prost(&self) -> PbStreamNode { use risingwave_pb::stream_plan::*; let stream_key = self.base.logical_pk.iter().map(|x| *x as u32).collect_vec(); @@ -205,19 +205,19 @@ impl StreamTableScan { column_ids: upstream_column_ids.clone(), }; - ProstStreamPlan { + PbStreamNode { fields: self.schema().to_prost(), input: vec![ // The merge node body will be filled by the `ActorBuilder` on the meta service. - ProstStreamPlan { - node_body: Some(ProstStreamNode::Merge(Default::default())), + PbStreamNode { + node_body: Some(PbNodeBody::Merge(Default::default())), identity: "Upstream".into(), fields: upstream_schema.clone(), stream_key: vec![], // not used ..Default::default() }, - ProstStreamPlan { - node_body: Some(ProstStreamNode::BatchPlan(batch_plan_node)), + PbStreamNode { + node_body: Some(PbNodeBody::BatchPlan(batch_plan_node)), operator_id: self.batch_plan_id.0 as u64, identity: "BatchPlanNode".into(), fields: upstream_schema, @@ -226,7 +226,7 @@ impl StreamTableScan { append_only: true, }, ], - node_body: Some(ProstStreamNode::Chain(ChainNode { + node_body: Some(PbNodeBody::Chain(ChainNode { table_id: self.logical.table_desc().table_id.table_id, chain_type: self.chain_type as i32, // The column indices need to be forwarded to the downstream diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index 5445249b3f61..4c474afe3f56 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -15,7 +15,7 @@ use std::fmt; use fixedbitset::FixedBitSet; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::{ExprRewritable, LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::{Distribution, Order}; @@ -93,7 +93,7 @@ impl PlanTreeNodeUnary for StreamTopN { impl_plan_tree_node_for_unary! { StreamTopN } impl StreamNode for StreamTopN { - fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> PbNodeBody { use risingwave_pb::stream_plan::*; let topn_node = TopNNode { limit: self.limit(), @@ -108,9 +108,9 @@ impl StreamNode for StreamTopN { order_by: self.topn_order().to_protobuf(), }; if self.input().append_only() { - ProstStreamNode::AppendOnlyTopN(topn_node) + PbNodeBody::AppendOnlyTopN(topn_node) } else { - ProstStreamNode::TopN(topn_node) + PbNodeBody::TopN(topn_node) } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index 067e0c4ea87f..e103dc5988ba 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -18,7 +18,7 @@ use std::ops::BitAnd; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::FieldDisplay; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::UnionNode; use super::{ExprRewritable, PlanRef}; @@ -105,8 +105,8 @@ impl PlanTreeNode for StreamUnion { } impl StreamNode for StreamUnion { - fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { - ProstStreamNode::Union(UnionNode {}) + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { + PbNodeBody::Union(UnionNode {}) } } 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 45ef145bfd4f..73cce1d689aa 100644 --- a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs @@ -22,7 +22,7 @@ use risingwave_common::error::RwError; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::WatermarkDesc; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::utils::TableCatalogBuilder; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -141,7 +141,7 @@ pub fn infer_internal_table_catalog(watermark_type: DataType) -> TableCatalog { } impl StreamNode for StreamWatermarkFilter { - fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> ProstStreamNode { + fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> PbNodeBody { use risingwave_pb::stream_plan::*; // TODO(yuhao): allow multiple watermark on source. @@ -150,7 +150,7 @@ impl StreamNode for StreamWatermarkFilter { let table = infer_internal_table_catalog(watermark_type); - ProstStreamNode::WatermarkFilter(WatermarkFilterNode { + PbNodeBody::WatermarkFilter(WatermarkFilterNode { watermark_descs: self.watermark_descs.clone(), tables: vec![table .with_id(state.gen_table_id_wrapped()) diff --git a/src/frontend/src/optimizer/plan_node/to_prost.rs b/src/frontend/src/optimizer/plan_node/to_prost.rs index eb08c91a5e1f..c2b7cad2de0e 100644 --- a/src/frontend/src/optimizer/plan_node/to_prost.rs +++ b/src/frontend/src/optimizer/plan_node/to_prost.rs @@ -21,9 +21,9 @@ use crate::{ for_all_plan_nodes, for_batch_plan_nodes, for_logical_plan_nodes, for_stream_plan_nodes, }; -pub trait ToProst: ToBatchProst + StreamNode {} +pub trait ToPb: ToBatchPb + StreamNode {} -pub trait ToBatchProst { +pub trait ToBatchPb { fn to_batch_prost_body(&self) -> pb_batch_node::NodeBody { unimplemented!() } @@ -38,20 +38,20 @@ pub trait StreamNode { } } -/// impl `ToProst` nodes which have impl `ToBatchProst` and `ToStreamProst`. +/// impl `ToPb` nodes which have impl `ToBatchPb` and `ToStreamPb`. macro_rules! impl_to_prost { ($( { $convention:ident, $name:ident }),*) => { paste!{ - $(impl ToProst for [<$convention $name>] { })* + $(impl ToPb for [<$convention $name>] { })* } } } for_all_plan_nodes! { impl_to_prost } -/// impl a panic `ToBatchProst` for logical and stream node. +/// impl a panic `ToBatchPb` for logical and stream node. macro_rules! ban_to_batch_prost { ($( { $convention:ident, $name:ident }),*) => { paste!{ - $(impl ToBatchProst for [<$convention $name>] { + $(impl ToBatchPb for [<$convention $name>] { fn to_batch_prost_body(&self) -> pb_batch_node::NodeBody { panic!("convert into distributed is only allowed on batch plan") } @@ -61,7 +61,7 @@ macro_rules! ban_to_batch_prost { } for_logical_plan_nodes! { ban_to_batch_prost } for_stream_plan_nodes! { ban_to_batch_prost } -/// impl a panic `ToStreamProst` for logical and batch node. +/// impl a panic `ToStreamPb` for logical and batch node. macro_rules! ban_to_stream_prost { ($( { $convention:ident, $name:ident }),*) => { paste!{ diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs index 1281a70bd111..a870167fd569 100644 --- a/src/frontend/src/optimizer/property/distribution.rs +++ b/src/frontend/src/optimizer/property/distribution.rs @@ -52,7 +52,7 @@ use risingwave_common::catalog::{FieldDisplay, Schema, TableId}; use risingwave_common::error::Result; use risingwave_common::hash::{ParallelUnitId, ParallelUnitMapping}; use risingwave_pb::batch_plan::exchange_info::{ - ConsistentHashInfo, Distribution as DistributionProst, DistributionMode, HashInfo, + ConsistentHashInfo, Distribution as DistributionPb, DistributionMode, HashInfo, }; use risingwave_pb::batch_plan::ExchangeInfo; @@ -131,7 +131,7 @@ impl Distribution { !key.is_empty(), "hash key should not be empty, use `Single` instead" ); - Some(DistributionProst::HashInfo(HashInfo { + Some(DistributionPb::HashInfo(HashInfo { output_count, key: key.iter().map(|num| *num as u32).collect(), })) @@ -155,7 +155,7 @@ impl Distribution { .map(|(i, pu)| (pu, i as u32)) .collect(); - Some(DistributionProst::ConsistentHashInfo(ConsistentHashInfo { + Some(DistributionPb::ConsistentHashInfo(ConsistentHashInfo { vmap: vnode_mapping.iter().map(|x| pu2id_map[&x]).collect_vec(), key: key.iter().map(|num| *num as u32).collect(), })) diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index 98590d3e3504..4a1f4aedb520 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -24,7 +24,7 @@ use petgraph::dot::{Config, Dot}; use petgraph::Graph; use pgwire::pg_server::SessionId; use risingwave_common::array::DataChunk; -use risingwave_pb::batch_plan::{TaskId as TaskIdProst, TaskOutputId as TaskOutputIdProst}; +use risingwave_pb::batch_plan::{TaskId as TaskIdPb, TaskOutputId as TaskOutputIdPb}; use risingwave_pb::common::HostAddress; use risingwave_rpc_client::ComputeClientPoolRef; use tokio::sync::mpsc::{channel, Receiver, Sender}; @@ -359,13 +359,13 @@ impl QueryRunner { /// of shutdown sender so that shutdown receiver won't be triggered. fn send_root_stage_info(&mut self, chunk_rx: Receiver>) { let root_task_output_id = { - let root_task_id_prost = TaskIdProst { + let root_task_id_prost = TaskIdPb { query_id: self.query.query_id.clone().id, stage_id: self.query.root_stage_id(), task_id: ROOT_TASK_ID, }; - TaskOutputIdProst { + TaskOutputIdPb { task_id: Some(root_task_id_prost), output_id: ROOT_TASK_OUTPUT_ID, } diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index 2fcc892bb1a4..d9773fdb566f 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -38,7 +38,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::plan_node::NodeBody::{Delete, Insert, Update}; use risingwave_pb::batch_plan::{ DistributedLookupJoinNode, ExchangeNode, ExchangeSource, MergeSortExchangeNode, PlanFragment, - PlanNode as PlanNodeProst, PlanNode, TaskId as TaskIdProst, TaskOutputId, + PlanNode as PlanNodePb, PlanNode, TaskId as TaskIdPb, TaskOutputId, }; use risingwave_pb::common::{BatchQueryEpoch, HostAddress, WorkerNode}; use risingwave_pb::task_service::{AbortTaskRequest, TaskInfoResponse}; @@ -283,7 +283,7 @@ impl StageExecution { .iter() .map(|(task_id, status_holder)| { let task_output_id = TaskOutputId { - task_id: Some(TaskIdProst { + task_id: Some(TaskIdPb { query_id: self.stage.query_id.id.clone(), stage_id: self.stage.id, task_id: *task_id, @@ -345,7 +345,7 @@ impl StageRunner { .zip_eq_fast(workers.into_iter()) .enumerate() { - let task_id = TaskIdProst { + let task_id = TaskIdPb { query_id: self.stage.query_id.id.clone(), stage_id: self.stage.id, task_id: i as u32, @@ -356,7 +356,7 @@ impl StageRunner { } } else if let Some(source_info) = self.stage.source_info.as_ref() { for (id, split) in source_info.split_info().unwrap().iter().enumerate() { - let task_id = TaskIdProst { + let task_id = TaskIdPb { query_id: self.stage.query_id.id.clone(), stage_id: self.stage.id, task_id: id as u32, @@ -368,7 +368,7 @@ impl StageRunner { } else { for id in 0..self.stage.parallelism.unwrap() { - let task_id = TaskIdProst { + let task_id = TaskIdPb { query_id: self.stage.query_id.id.clone(), stage_id: self.stage.id, task_id: id, @@ -397,9 +397,9 @@ impl StageRunner { while let Some(status_res_inner) = all_streams.next().await { match status_res_inner { Ok(status) => { - use risingwave_pb::task_service::task_info_response::TaskStatus as TaskStatusProst; - match TaskStatusProst::from_i32(status.task_status).unwrap() { - TaskStatusProst::Running => { + use risingwave_pb::task_service::task_info_response::TaskStatus as TaskStatusPb; + match TaskStatusPb::from_i32(status.task_status).unwrap() { + TaskStatusPb::Running => { running_task_cnt += 1; // The task running count should always less or equal than the // registered tasks number. @@ -415,7 +415,7 @@ impl StageRunner { } } - TaskStatusProst::Finished => { + TaskStatusPb::Finished => { finished_task_cnt += 1; assert!(finished_task_cnt <= self.tasks.keys().len()); assert!(running_task_cnt >= finished_task_cnt); @@ -427,7 +427,7 @@ impl StageRunner { break; } } - TaskStatusProst::Aborted => { + TaskStatusPb::Aborted => { // Currently, the only reason that we receive an abort status is that // the task's memory usage is too high so // it's aborted. @@ -446,7 +446,7 @@ impl StageRunner { sent_signal_to_next = true; break; } - TaskStatusProst::Failed => { + TaskStatusPb::Failed => { // Task failed, we should fail whole query error!( "Task {:?} failed, reason: {:?}", @@ -810,7 +810,7 @@ impl StageRunner { async fn schedule_task( &self, - task_id: TaskIdProst, + task_id: TaskIdPb, plan_fragment: PlanFragment, worker: Option, ) -> SchedulerResult>> { @@ -864,7 +864,7 @@ impl StageRunner { task_id: TaskId, partition: Option, identity_id: Rc>, - ) -> PlanNodeProst { + ) -> PlanNodePb { // Generate identity let identity = { let identity_type = execution_plan_node.plan_node_type; @@ -886,7 +886,7 @@ impl StageRunner { let exchange_sources = child_stage.all_exchange_sources_for(task_id); match &execution_plan_node.node { - NodeBody::Exchange(_exchange_node) => PlanNodeProst { + NodeBody::Exchange(_exchange_node) => PlanNodePb { children: vec![], identity, node_body: Some(NodeBody::Exchange(ExchangeNode { @@ -894,7 +894,7 @@ impl StageRunner { input_schema: execution_plan_node.schema.clone(), })), }, - NodeBody::MergeSortExchange(sort_merge_exchange_node) => PlanNodeProst { + NodeBody::MergeSortExchange(sort_merge_exchange_node) => PlanNodePb { children: vec![], identity, node_body: Some(NodeBody::MergeSortExchange(MergeSortExchangeNode { @@ -919,7 +919,7 @@ impl StageRunner { .expect("PartitionInfo should be TablePartitionInfo"); scan_node.vnode_bitmap = Some(partition.vnode_bitmap); scan_node.scan_ranges = partition.scan_ranges; - PlanNodeProst { + PlanNodePb { children: vec![], identity, node_body: Some(NodeBody::RowSeqScan(scan_node)), @@ -935,7 +935,7 @@ impl StageRunner { .into_source() .expect("PartitionInfo should be SourcePartitionInfo"); source_node.split = partition.encode_to_bytes().into(); - PlanNodeProst { + PlanNodePb { children: vec![], identity, node_body: Some(NodeBody::Source(source_node)), @@ -950,7 +950,7 @@ impl StageRunner { }) .collect(); - PlanNodeProst { + PlanNodePb { children, identity, node_body: Some(execution_plan_node.node.clone()), diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index 2e165e2c047c..148de9c23ce9 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -35,8 +35,8 @@ use risingwave_pb::batch_plan::exchange_info::DistributionMode; use risingwave_pb::batch_plan::exchange_source::LocalExecutePlan::Plan; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ - ExchangeInfo, ExchangeSource, LocalExecutePlan, PlanFragment, PlanNode as PlanNodeProst, - TaskId as ProstTaskId, TaskOutputId, + ExchangeInfo, ExchangeSource, LocalExecutePlan, PbTaskId, PlanFragment, PlanNode as PlanNodePb, + TaskOutputId, }; use risingwave_pb::common::WorkerNode; use tokio::sync::mpsc; @@ -203,7 +203,7 @@ impl LocalQueryExecution { execution_plan_node: &ExecutionPlanNode, second_stages: &mut Option>, partition: Option, - ) -> SchedulerResult { + ) -> SchedulerResult { match execution_plan_node.plan_node_type { PlanNodeType::BatchExchange => { let exchange_source_stage_id = execution_plan_node @@ -258,7 +258,7 @@ impl LocalQueryExecution { }; let exchange_source = ExchangeSource { task_output_id: Some(TaskOutputId { - task_id: Some(ProstTaskId { + task_id: Some(PbTaskId { task_id: idx as u32, stage_id: exchange_source_stage_id, query_id: self.query.query_id.id.clone(), @@ -292,7 +292,7 @@ impl LocalQueryExecution { let worker_node = self.front_env.worker_node_manager().next_random()?; let exchange_source = ExchangeSource { task_output_id: Some(TaskOutputId { - task_id: Some(ProstTaskId { + task_id: Some(PbTaskId { task_id: id as u32, stage_id: exchange_source_stage_id, query_id: self.query.query_id.id.clone(), @@ -328,7 +328,7 @@ impl LocalQueryExecution { .map(|(idx, worker_node)| { let exchange_source = ExchangeSource { task_output_id: Some(TaskOutputId { - task_id: Some(ProstTaskId { + task_id: Some(PbTaskId { task_id: idx as u32, stage_id: exchange_source_stage_id, query_id: self.query.query_id.id.clone(), @@ -343,7 +343,7 @@ impl LocalQueryExecution { .collect(); } - Ok(PlanNodeProst { + Ok(PlanNodePb { /// Since all the rest plan is embedded into the exchange node, /// there is no children any more. children: vec![], @@ -367,7 +367,7 @@ impl LocalQueryExecution { _ => unreachable!(), } - Ok(PlanNodeProst { + Ok(PlanNodePb { children: vec![], // TODO: Generate meaningful identify identity: Uuid::new_v4().to_string(), @@ -388,7 +388,7 @@ impl LocalQueryExecution { _ => unreachable!(), } - Ok(PlanNodeProst { + Ok(PlanNodePb { children: vec![], // TODO: Generate meaningful identify identity: Uuid::new_v4().to_string(), @@ -429,7 +429,7 @@ impl LocalQueryExecution { partition, )?; - Ok(PlanNodeProst { + Ok(PlanNodePb { children: vec![left_child], identity: Uuid::new_v4().to_string(), node_body: Some(node_body), @@ -440,9 +440,9 @@ impl LocalQueryExecution { .children .iter() .map(|e| self.convert_plan_node(e, second_stages, partition.clone())) - .collect::>>()?; + .collect::>>()?; - Ok(PlanNodeProst { + Ok(PlanNodePb { children, // TODO: Generate meaningful identify identity: Uuid::new_v4().to_string(), diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index cb3ef5a02610..f36cad5a35c8 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -31,7 +31,7 @@ use risingwave_connector::source::{ConnectorProperties, SplitEnumeratorImpl, Spl use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ExchangeInfo, ScanRange as ScanRangeProto}; use risingwave_pb::common::Buffer; -use risingwave_pb::plan_common::Field as FieldProst; +use risingwave_pb::plan_common::Field as FieldPb; use serde::ser::SerializeStruct; use serde::Serialize; use uuid::Uuid; @@ -70,7 +70,7 @@ pub struct ExecutionPlanNode { pub plan_node_id: PlanNodeId, pub plan_node_type: PlanNodeType, pub node: NodeBody, - pub schema: Vec, + pub schema: Vec, pub children: Vec>, diff --git a/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs b/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs index 1450b373baa2..4574805b0956 100644 --- a/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs +++ b/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs @@ -16,7 +16,7 @@ use std::rc::Rc; use itertools::Itertools; use risingwave_common::error::Result; -use risingwave_pb::plan_common::Field as ProstField; +use risingwave_pb::plan_common::PbField; use risingwave_pb::stream_plan::lookup_node::ArrangementTableId; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ @@ -89,7 +89,7 @@ fn dispatch_consistent_hash_shuffle( fn build_lookup_for_delta_join( state: &mut BuildFragmentGraphState, (exchange_node_arrangement, exchange_node_stream): (&StreamNode, &StreamNode), - (output_fields, output_stream_key): (Vec, Vec), + (output_fields, output_stream_key): (Vec, Vec), lookup_node: LookupNode, ) -> StreamNode { StreamNode { diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 26153d6cef66..1d33ae1d3806 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -32,9 +32,7 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::backup_service::MetaSnapshotMetadata; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ - Database as ProstDatabase, Function as ProstFunction, Index as ProstIndex, - Schema as ProstSchema, Sink as ProstSink, Source as ProstSource, Table as ProstTable, - View as ProstView, + PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, }; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockSnapshot; @@ -195,7 +193,7 @@ pub struct MockCatalogWriter { impl CatalogWriter for MockCatalogWriter { async fn create_database(&self, db_name: &str, owner: UserId) -> Result<()> { let database_id = self.gen_id(); - self.catalog.write().create_database(&ProstDatabase { + self.catalog.write().create_database(&PbDatabase { name: db_name.to_string(), id: database_id, owner, @@ -214,7 +212,7 @@ impl CatalogWriter for MockCatalogWriter { owner: UserId, ) -> Result<()> { let id = self.gen_id(); - self.catalog.write().create_schema(&ProstSchema { + self.catalog.write().create_schema(&PbSchema { id, name: schema_name.to_string(), database_id: db_id, @@ -226,7 +224,7 @@ impl CatalogWriter for MockCatalogWriter { async fn create_materialized_view( &self, - mut table: ProstTable, + mut table: PbTable, _graph: StreamFragmentGraph, ) -> Result<()> { table.id = self.gen_id(); @@ -235,7 +233,7 @@ impl CatalogWriter for MockCatalogWriter { Ok(()) } - async fn create_view(&self, mut view: ProstView) -> Result<()> { + async fn create_view(&self, mut view: PbView) -> Result<()> { view.id = self.gen_id(); self.catalog.write().create_view(&view); self.add_table_or_source_id(view.id, view.schema_id, view.database_id); @@ -244,8 +242,8 @@ impl CatalogWriter for MockCatalogWriter { async fn create_table( &self, - source: Option, - mut table: ProstTable, + source: Option, + mut table: PbTable, graph: StreamFragmentGraph, ) -> Result<()> { if let Some(source) = source { @@ -259,7 +257,7 @@ impl CatalogWriter for MockCatalogWriter { async fn replace_table( &self, - table: ProstTable, + table: PbTable, _graph: StreamFragmentGraph, _mapping: ColIndexMapping, ) -> Result<()> { @@ -267,18 +265,18 @@ impl CatalogWriter for MockCatalogWriter { Ok(()) } - async fn create_source(&self, source: ProstSource) -> Result<()> { + async fn create_source(&self, source: PbSource) -> Result<()> { self.create_source_inner(source).map(|_| ()) } - async fn create_sink(&self, sink: ProstSink, graph: StreamFragmentGraph) -> Result<()> { + async fn create_sink(&self, sink: PbSink, graph: StreamFragmentGraph) -> Result<()> { self.create_sink_inner(sink, graph) } async fn create_index( &self, - mut index: ProstIndex, - mut index_table: ProstTable, + mut index: PbIndex, + mut index_table: PbTable, _graph: StreamFragmentGraph, ) -> Result<()> { index_table.id = self.gen_id(); @@ -295,7 +293,7 @@ impl CatalogWriter for MockCatalogWriter { Ok(()) } - async fn create_function(&self, _function: ProstFunction) -> Result<()> { + async fn create_function(&self, _function: PbFunction) -> Result<()> { todo!() } @@ -402,18 +400,18 @@ impl CatalogWriter for MockCatalogWriter { impl MockCatalogWriter { pub fn new(catalog: Arc>) -> Self { - catalog.write().create_database(&ProstDatabase { + catalog.write().create_database(&PbDatabase { id: 0, name: DEFAULT_DATABASE_NAME.to_string(), owner: DEFAULT_SUPER_USER_ID, }); - catalog.write().create_schema(&ProstSchema { + catalog.write().create_schema(&PbSchema { id: 1, name: DEFAULT_SCHEMA_NAME.to_string(), database_id: 0, owner: DEFAULT_SUPER_USER_ID, }); - catalog.write().create_schema(&ProstSchema { + catalog.write().create_schema(&PbSchema { id: 2, name: PG_CATALOG_SCHEMA_NAME.to_string(), database_id: 0, @@ -493,14 +491,14 @@ impl MockCatalogWriter { .unwrap() } - fn create_source_inner(&self, mut source: ProstSource) -> Result { + fn create_source_inner(&self, mut source: PbSource) -> Result { source.id = self.gen_id(); self.catalog.write().create_source(&source); self.add_table_or_source_id(source.id, source.schema_id, source.database_id); Ok(source.id) } - fn create_sink_inner(&self, mut sink: ProstSink, _graph: StreamFragmentGraph) -> Result<()> { + fn create_sink_inner(&self, mut sink: PbSink, _graph: StreamFragmentGraph) -> Result<()> { sink.id = self.gen_id(); self.catalog.write().create_sink(&sink); self.add_table_or_sink_id(sink.id, sink.schema_id, sink.database_id); diff --git a/src/frontend/src/user/user_privilege.rs b/src/frontend/src/user/user_privilege.rs index 14ade81f0b0d..69d9138f2b6e 100644 --- a/src/frontend/src/user/user_privilege.rs +++ b/src/frontend/src/user/user_privilege.rs @@ -15,10 +15,8 @@ use itertools::Itertools; use risingwave_common::catalog::DEFAULT_SUPER_USER_ID; use risingwave_common::error::{ErrorCode, Result}; -use risingwave_pb::user::grant_privilege::{ - Action as ProstAction, ActionWithGrantOption, Object as ProstObject, -}; -use risingwave_pb::user::GrantPrivilege as ProstPrivilege; +use risingwave_pb::user::grant_privilege::{ActionWithGrantOption, PbAction, PbObject}; +use risingwave_pb::user::PbGrantPrivilege; use risingwave_sqlparser::ast::{Action, GrantObjects, Privileges}; // TODO: add user_privilege mod under user manager and move check and expand logic there, and bitmap @@ -90,31 +88,31 @@ pub fn available_privilege_actions(objects: &GrantObjects) -> Result } #[inline(always)] -pub fn get_prost_action(action: &Action) -> ProstAction { +pub fn get_prost_action(action: &Action) -> PbAction { match action { - Action::Select { .. } => ProstAction::Select, - Action::Insert { .. } => ProstAction::Insert, - Action::Update { .. } => ProstAction::Update, - Action::Delete { .. } => ProstAction::Delete, - Action::Connect => ProstAction::Connect, - Action::Create => ProstAction::Create, + Action::Select { .. } => PbAction::Select, + Action::Insert { .. } => PbAction::Insert, + Action::Update { .. } => PbAction::Update, + Action::Delete { .. } => PbAction::Delete, + Action::Connect => PbAction::Connect, + Action::Create => PbAction::Create, _ => unreachable!(), } } -pub fn available_prost_privilege(object: ProstObject) -> ProstPrivilege { +pub fn available_prost_privilege(object: PbObject) -> PbGrantPrivilege { let actions = match object { - ProstObject::DatabaseId(_) => AVAILABLE_ACTION_ON_DATABASE.to_vec(), - ProstObject::SchemaId(_) => AVAILABLE_ACTION_ON_SCHEMA.to_vec(), - ProstObject::SourceId(_) | ProstObject::AllSourcesSchemaId { .. } => { + PbObject::DatabaseId(_) => AVAILABLE_ACTION_ON_DATABASE.to_vec(), + PbObject::SchemaId(_) => AVAILABLE_ACTION_ON_SCHEMA.to_vec(), + PbObject::SourceId(_) | PbObject::AllSourcesSchemaId { .. } => { AVAILABLE_ACTION_ON_SOURCE.to_vec() } - ProstObject::TableId(_) | ProstObject::AllTablesSchemaId { .. } => { + PbObject::TableId(_) | PbObject::AllTablesSchemaId { .. } => { AVAILABLE_ACTION_ON_MVIEW.to_vec() } - ProstObject::ViewId(_) => AVAILABLE_ACTION_ON_VIEW.to_vec(), - ProstObject::SinkId(_) => AVAILABLE_ACTION_ON_SINK.to_vec(), - ProstObject::FunctionId(_) => AVAILABLE_ACTION_ON_FUNCTION.to_vec(), + PbObject::ViewId(_) => AVAILABLE_ACTION_ON_VIEW.to_vec(), + PbObject::SinkId(_) => AVAILABLE_ACTION_ON_SINK.to_vec(), + PbObject::FunctionId(_) => AVAILABLE_ACTION_ON_FUNCTION.to_vec(), }; let actions = actions .iter() @@ -124,7 +122,7 @@ pub fn available_prost_privilege(object: ProstObject) -> ProstPrivilege { granted_by: DEFAULT_SUPER_USER_ID, }) .collect_vec(); - ProstPrivilege { + PbGrantPrivilege { action_with_opts: actions, object: Some(object), } diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 6a586485bebf..58bdfcaf18fb 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -268,7 +268,7 @@ impl CreateMviewProgressTracker { None } - /// Update the progress of `actor` according to the Prost struct. + /// Update the progress of `actor` according to the Pb struct. /// /// If all actors in this MV have finished, returns the command. pub fn update( diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index 17e3b872913e..4b7617294af3 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -59,7 +59,7 @@ pub(super) mod handlers { use risingwave_pb::catalog::table::TableType; use risingwave_pb::catalog::{Sink, Source, Table}; use risingwave_pb::common::WorkerNode; - use risingwave_pb::meta::{ActorLocation, TableFragments as ProstTableFragments}; + use risingwave_pb::meta::{ActorLocation, PbTableFragments}; use risingwave_pb::monitor_service::StackTraceResponse; use risingwave_pb::stream_plan::StreamActor; use serde_json::json; @@ -203,7 +203,7 @@ pub(super) mod handlers { pub async fn list_fragments( Extension(srv): Extension>, - ) -> Result>> { + ) -> Result>> { use crate::model::MetadataModel; let table_fragments = TableFragments::list(&*srv.meta_store) diff --git a/src/meta/src/error.rs b/src/meta/src/error.rs index 7a3485fb2ede..bba22e5fc706 100644 --- a/src/meta/src/error.rs +++ b/src/meta/src/error.rs @@ -15,7 +15,7 @@ use std::backtrace::Backtrace; use std::sync::Arc; -use risingwave_pb::ProstFieldNotFound; +use risingwave_pb::PbFieldNotFound; use risingwave_rpc_client::error::RpcError; use crate::hummock::error::Error as HummockError; @@ -199,8 +199,8 @@ impl From for tonic::Status { } } -impl From for MetaError { - fn from(e: ProstFieldNotFound) -> Self { +impl From for MetaError { + fn from(e: PbFieldNotFound) -> Self { MetadataModelError::from(e).into() } } diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 70bf0c797f26..9116db32326e 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -161,11 +161,11 @@ macro_rules! read_lock { pub(crate) use read_lock; use risingwave_hummock_sdk::compaction_group::{StateTableId, StaticCompactionGroupId}; use risingwave_hummock_sdk::table_stats::{ - add_prost_table_stats_map, purge_prost_table_stats, ProstTableStatsMap, + add_prost_table_stats_map, purge_prost_table_stats, PbTableStatsMap, }; use risingwave_pb::catalog::Table; use risingwave_pb::hummock::version_update_payload::Payload; -use risingwave_pb::hummock::CompactionGroupInfo as ProstCompactionGroup; +use risingwave_pb::hummock::PbCompactionGroupInfo; /// Acquire write lock of the lock with `lock_name`. /// The macro will use macro `function_name` to get the name of the function of method that calls @@ -1071,7 +1071,7 @@ where &self, context_id: HummockContextId, compact_task: &mut CompactTask, - table_stats_change: Option, + table_stats_change: Option, ) -> Result { let ret = self .report_compact_task_impl(Some(context_id), compact_task, None, table_stats_change) @@ -1093,7 +1093,7 @@ where context_id: Option, compact_task: &mut CompactTask, compaction_guard: Option>, - table_stats_change: Option, + table_stats_change: Option, ) -> Result { let mut compaction_guard = match compaction_guard { None => write_lock!(self, compaction).await, @@ -1345,7 +1345,7 @@ where .await?; // Consume and aggregate table stats. - let mut table_stats_change = ProstTableStatsMap::default(); + let mut table_stats_change = PbTableStatsMap::default(); for s in &mut sstables { add_prost_table_stats_map(&mut table_stats_change, &std::mem::take(&mut s.table_stats)); } @@ -1700,7 +1700,7 @@ where pub async fn init_metadata_for_version_replay( &self, table_catalogs: Vec, - compaction_groups: Vec, + compaction_groups: Vec, ) -> Result<()> { for table in &table_catalogs { table.insert(self.env.meta_store()).await?; diff --git a/src/meta/src/hummock/model/compact_task_assignment.rs b/src/meta/src/hummock/model/compact_task_assignment.rs index a9a0f72429f1..4145182bdd8f 100644 --- a/src/meta/src/hummock/model/compact_task_assignment.rs +++ b/src/meta/src/hummock/model/compact_task_assignment.rs @@ -22,13 +22,13 @@ use crate::model::{MetadataModel, MetadataModelResult}; /// `AssignedCompactTasks` tracks compact tasks assigned to context id. impl MetadataModel for CompactTaskAssignment { type KeyType = HummockCompactionTaskId; - type ProstType = CompactTaskAssignment; + type PbType = CompactTaskAssignment; fn cf_name() -> String { HUMMOCK_COMPACT_TASK_ASSIGNMENT.to_string() } - fn to_protobuf(&self) -> Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { self.clone() } @@ -36,7 +36,7 @@ impl MetadataModel for CompactTaskAssignment { self.encode_to_vec() } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { prost } diff --git a/src/meta/src/hummock/model/compaction_group_config.rs b/src/meta/src/hummock/model/compaction_group_config.rs index 8d16cc01d822..c130c2b36f1e 100644 --- a/src/meta/src/hummock/model/compaction_group_config.rs +++ b/src/meta/src/hummock/model/compaction_group_config.rs @@ -71,17 +71,17 @@ impl From<&CompactionGroup> for risingwave_pb::hummock::CompactionGroup { impl MetadataModel for CompactionGroup { type KeyType = CompactionGroupId; - type ProstType = risingwave_pb::hummock::CompactionGroup; + type PbType = risingwave_pb::hummock::CompactionGroup; fn cf_name() -> String { String::from(HUMMOCK_COMPACTION_GROUP_CONFIG_CF_NAME) } - fn to_protobuf(&self) -> Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { self.borrow().into() } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { prost.borrow().into() } diff --git a/src/meta/src/hummock/model/compaction_status.rs b/src/meta/src/hummock/model/compaction_status.rs index 6ffa1553565f..4f10e8c5091e 100644 --- a/src/meta/src/hummock/model/compaction_status.rs +++ b/src/meta/src/hummock/model/compaction_status.rs @@ -23,17 +23,17 @@ use crate::model::{MetadataModel, MetadataModelResult}; impl MetadataModel for CompactStatus { type KeyType = CompactionGroupId; - type ProstType = risingwave_pb::hummock::CompactStatus; + type PbType = risingwave_pb::hummock::CompactStatus; fn cf_name() -> String { String::from(HUMMOCK_COMPACTION_STATUS_CF_NAME) } - fn to_protobuf(&self) -> Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { self.into() } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { prost.borrow().into() } diff --git a/src/meta/src/hummock/model/pinned_snapshot.rs b/src/meta/src/hummock/model/pinned_snapshot.rs index 8c153844355f..b436b2d9312a 100644 --- a/src/meta/src/hummock/model/pinned_snapshot.rs +++ b/src/meta/src/hummock/model/pinned_snapshot.rs @@ -22,13 +22,13 @@ use crate::model::{MetadataModel, MetadataModelResult}; /// `HummockPinnedSnapshot` tracks pinned snapshots by given context id. impl MetadataModel for HummockPinnedSnapshot { type KeyType = HummockContextId; - type ProstType = HummockPinnedSnapshot; + type PbType = HummockPinnedSnapshot; fn cf_name() -> String { String::from(HUMMOCK_PINNED_SNAPSHOT_CF_NAME) } - fn to_protobuf(&self) -> Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { self.clone() } @@ -36,7 +36,7 @@ impl MetadataModel for HummockPinnedSnapshot { self.encode_to_vec() } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { prost } diff --git a/src/meta/src/hummock/model/pinned_version.rs b/src/meta/src/hummock/model/pinned_version.rs index 020532166ca9..c2a49206ff8e 100644 --- a/src/meta/src/hummock/model/pinned_version.rs +++ b/src/meta/src/hummock/model/pinned_version.rs @@ -22,13 +22,13 @@ use crate::model::{MetadataModel, MetadataModelResult}; /// `HummockPinnedVersion` tracks pinned versions by given context id. impl MetadataModel for HummockPinnedVersion { type KeyType = HummockContextId; - type ProstType = HummockPinnedVersion; + type PbType = HummockPinnedVersion; fn cf_name() -> String { String::from(HUMMOCK_PINNED_VERSION_CF_NAME) } - fn to_protobuf(&self) -> Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { self.clone() } @@ -36,7 +36,7 @@ impl MetadataModel for HummockPinnedVersion { self.encode_to_vec() } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { prost } diff --git a/src/meta/src/hummock/model/version.rs b/src/meta/src/hummock/model/version.rs index 0ecd42ce5d39..d6a85ae745c6 100644 --- a/src/meta/src/hummock/model/version.rs +++ b/src/meta/src/hummock/model/version.rs @@ -22,13 +22,13 @@ use crate::model::{MetadataModel, MetadataModelResult}; /// `HummockVersion` tracks `Sstables` in given version. impl MetadataModel for HummockVersion { type KeyType = HummockVersionId; - type ProstType = HummockVersion; + type PbType = HummockVersion; fn cf_name() -> String { String::from(HUMMOCK_VERSION_CF_NAME) } - fn to_protobuf(&self) -> Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { self.clone() } @@ -36,7 +36,7 @@ impl MetadataModel for HummockVersion { self.encode_to_vec() } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { prost } diff --git a/src/meta/src/hummock/model/version_delta.rs b/src/meta/src/hummock/model/version_delta.rs index 5b2f541a4e5a..a7d97790ce51 100644 --- a/src/meta/src/hummock/model/version_delta.rs +++ b/src/meta/src/hummock/model/version_delta.rs @@ -22,13 +22,13 @@ use crate::model::{MetadataModel, MetadataModelResult}; /// `HummockVersionDelta` tracks delta of `Sstables` in given version based on previous version. impl MetadataModel for HummockVersionDelta { type KeyType = HummockVersionId; - type ProstType = HummockVersionDelta; + type PbType = HummockVersionDelta; fn cf_name() -> String { String::from(HUMMOCK_VERSION_DELTA_CF_NAME) } - fn to_protobuf(&self) -> Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { self.clone() } @@ -36,7 +36,7 @@ impl MetadataModel for HummockVersionDelta { self.encode_to_vec() } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { prost } diff --git a/src/meta/src/hummock/model/version_stats.rs b/src/meta/src/hummock/model/version_stats.rs index 7e3f71e5204d..6a498daa3390 100644 --- a/src/meta/src/hummock/model/version_stats.rs +++ b/src/meta/src/hummock/model/version_stats.rs @@ -23,13 +23,13 @@ use crate::model::{MetadataModel, MetadataModelResult}; /// Currently it only persists one row for latest version. impl MetadataModel for HummockVersionStats { type KeyType = HummockVersionId; - type ProstType = HummockVersionStats; + type PbType = HummockVersionStats; fn cf_name() -> String { String::from(HUMMOCK_VERSION_STATS_CF_NAME) } - fn to_protobuf(&self) -> Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { self.clone() } @@ -37,7 +37,7 @@ impl MetadataModel for HummockVersionStats { self.encode_to_vec() } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { prost } diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 34ad67d8ec0b..8bf63e5acf65 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -24,7 +24,7 @@ use risingwave_hummock_sdk::key::key_with_epoch; use risingwave_hummock_sdk::{ CompactionGroupId, HummockContextId, HummockEpoch, HummockSstableObjectId, LocalSstableInfo, }; -use risingwave_pb::catalog::Table as ProstTable; +use risingwave_pb::catalog::PbTable; use risingwave_pb::common::{HostAddress, WorkerNode, WorkerType}; use risingwave_pb::hummock::compact_task::TaskStatus; use risingwave_pb::hummock::{ @@ -239,7 +239,7 @@ pub fn update_filter_key_extractor_for_table_ids( pub fn update_filter_key_extractor_for_tables( filter_key_extractor_manager_ref: &FilterKeyExtractorManagerRef, - tables: &[ProstTable], + tables: &[PbTable], ) { for table in tables { filter_key_extractor_manager_ref.update( diff --git a/src/meta/src/model/catalog.rs b/src/meta/src/model/catalog.rs index b96e3d5704fa..de3ef793ff25 100644 --- a/src/meta/src/model/catalog.rs +++ b/src/meta/src/model/catalog.rs @@ -37,17 +37,17 @@ macro_rules! impl_model_for_catalog { ($name:ident, $cf:ident, $key_ty:ty, $key_fn:ident) => { impl MetadataModel for $name { type KeyType = $key_ty; - type ProstType = Self; + type PbType = Self; fn cf_name() -> String { $cf.to_string() } - fn to_protobuf(&self) -> Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { self.clone() } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { prost } diff --git a/src/meta/src/model/cluster.rs b/src/meta/src/model/cluster.rs index 50723dcca7f0..17ea0b62b424 100644 --- a/src/meta/src/model/cluster.rs +++ b/src/meta/src/model/cluster.rs @@ -46,17 +46,17 @@ pub struct Worker { impl MetadataModel for Worker { type KeyType = HostAddress; - type ProstType = WorkerNode; + type PbType = WorkerNode; fn cf_name() -> String { WORKER_CF_NAME.to_string() } - fn to_protobuf(&self) -> Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { self.worker_node.clone() } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { Self { worker_node: prost, expire_at: INVALID_EXPIRE_AT, diff --git a/src/meta/src/model/connection.rs b/src/meta/src/model/connection.rs index 7f762ab93120..81ebbf0f86f9 100644 --- a/src/meta/src/model/connection.rs +++ b/src/meta/src/model/connection.rs @@ -21,17 +21,17 @@ const CONNECTION_CF_NAME: &str = "cf/connection"; impl MetadataModel for Connection { type KeyType = u32; - type ProstType = Connection; + type PbType = Connection; fn cf_name() -> String { CONNECTION_CF_NAME.to_string() } - fn to_protobuf(&self) -> Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { self.clone() } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { prost } diff --git a/src/meta/src/model/error.rs b/src/meta/src/model/error.rs index 083efbe8f1c1..e53a6ae3547f 100644 --- a/src/meta/src/model/error.rs +++ b/src/meta/src/model/error.rs @@ -13,7 +13,7 @@ // limitations under the License. use anyhow::anyhow; -use risingwave_pb::ProstFieldNotFound; +use risingwave_pb::PbFieldNotFound; use thiserror::Error; use crate::storage::MetaStoreError; @@ -25,15 +25,15 @@ pub enum MetadataModelError { #[error("Meta store error: {0}")] MetaStoreError(#[from] MetaStoreError), - #[error("Prost decode error: {0}")] - ProstDecode(#[from] prost::DecodeError), + #[error("Pb decode error: {0}")] + PbDecode(#[from] prost::DecodeError), #[error(transparent)] InternalError(anyhow::Error), } -impl From for MetadataModelError { - fn from(p: ProstFieldNotFound) -> Self { +impl From for MetadataModelError { + fn from(p: PbFieldNotFound) -> Self { MetadataModelError::InternalError(anyhow::anyhow!( "Failed to decode prost: field not found `{}`", p.0 diff --git a/src/meta/src/model/mod.rs b/src/meta/src/model/mod.rs index 5acbe36f63a2..fa1b91abbdac 100644 --- a/src/meta/src/model/mod.rs +++ b/src/meta/src/model/mod.rs @@ -63,7 +63,7 @@ mod private { #[async_trait] pub trait MetadataModel: std::fmt::Debug + Sized + private::MetadataModelMarker { /// Serialized prost message type. - type ProstType: Message + Default; + type PbType: Message + Default; /// Serialized key type. type KeyType: Message; @@ -71,7 +71,7 @@ pub trait MetadataModel: std::fmt::Debug + Sized + private::MetadataModelMarker fn cf_name() -> String; /// Serialize to protobuf. - fn to_protobuf(&self) -> Self::ProstType; + fn to_protobuf(&self) -> Self::PbType; /// Serialize to protobuf encoded byte vector. fn to_protobuf_encoded_vec(&self) -> Vec { @@ -79,7 +79,7 @@ pub trait MetadataModel: std::fmt::Debug + Sized + private::MetadataModelMarker } /// Deserialize from protobuf. - fn from_protobuf(prost: Self::ProstType) -> Self; + fn from_protobuf(prost: Self::PbType) -> Self; /// Current record key. fn key(&self) -> MetadataModelResult; @@ -93,7 +93,7 @@ pub trait MetadataModel: std::fmt::Debug + Sized + private::MetadataModelMarker bytes_vec .iter() .map(|(_k, v)| { - Self::ProstType::decode(v.as_slice()) + Self::PbType::decode(v.as_slice()) .map(Self::from_protobuf) .map_err(Into::into) }) @@ -108,7 +108,7 @@ pub trait MetadataModel: std::fmt::Debug + Sized + private::MetadataModelMarker bytes_vec .iter() .map(|(_k, v)| { - Self::ProstType::decode(v.as_slice()) + Self::PbType::decode(v.as_slice()) .map(Self::from_protobuf) .map_err(Into::into) }) @@ -155,7 +155,7 @@ pub trait MetadataModel: std::fmt::Debug + Sized + private::MetadataModelMarker return Ok(None); } }; - let model = Self::from_protobuf(Self::ProstType::decode(byte_vec.as_slice())?); + let model = Self::from_protobuf(Self::PbType::decode(byte_vec.as_slice())?); Ok(Some(model)) } } diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 05e680a03475..4b6d41f0fb9c 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -21,11 +21,10 @@ use risingwave_connector::source::SplitImpl; use risingwave_pb::common::{ParallelUnit, ParallelUnitMapping}; use risingwave_pb::meta::table_fragments::actor_status::ActorState; use risingwave_pb::meta::table_fragments::{ActorStatus, Fragment, State}; -use risingwave_pb::meta::TableFragments as ProstTableFragments; +use risingwave_pb::meta::PbTableFragments; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ - FragmentTypeFlag, SourceNode, StreamActor, StreamEnvironment as ProstStreamEnvironment, - StreamNode, + FragmentTypeFlag, PbStreamEnvironment, SourceNode, StreamActor, StreamNode, }; use super::{ActorId, FragmentId}; @@ -68,13 +67,13 @@ pub struct StreamEnvironment { } impl StreamEnvironment { - pub fn to_protobuf(&self) -> ProstStreamEnvironment { - ProstStreamEnvironment { + pub fn to_protobuf(&self) -> PbStreamEnvironment { + PbStreamEnvironment { timezone: self.timezone.clone().unwrap_or("".into()), } } - pub fn from_protobuf(prost: &ProstStreamEnvironment) -> Self { + pub fn from_protobuf(prost: &PbStreamEnvironment) -> Self { Self { timezone: if prost.get_timezone().is_empty() { None @@ -87,14 +86,14 @@ impl StreamEnvironment { impl MetadataModel for TableFragments { type KeyType = u32; - type ProstType = ProstTableFragments; + type PbType = PbTableFragments; fn cf_name() -> String { TABLE_FRAGMENTS_CF_NAME.to_string() } - fn to_protobuf(&self) -> Self::ProstType { - Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { + Self::PbType { table_id: self.table_id.table_id(), state: self.state as _, fragments: self.fragments.clone().into_iter().collect(), @@ -104,7 +103,7 @@ impl MetadataModel for TableFragments { } } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { let env = StreamEnvironment::from_protobuf(prost.get_env().unwrap()); Self { table_id: TableId::new(prost.table_id), diff --git a/src/meta/src/model/user.rs b/src/meta/src/model/user.rs index b77f0aa16942..1a6cf5392ee4 100644 --- a/src/meta/src/model/user.rs +++ b/src/meta/src/model/user.rs @@ -22,17 +22,17 @@ const USER_INFO_CF_NAME: &str = "cf/user_info"; /// `UserInfo` stores the user information. impl MetadataModel for UserInfo { type KeyType = u32; - type ProstType = UserInfo; + type PbType = UserInfo; fn cf_name() -> String { USER_INFO_CF_NAME.to_string() } - fn to_protobuf(&self) -> Self::ProstType { + fn to_protobuf(&self) -> Self::PbType { self.clone() } - fn from_protobuf(prost: Self::ProstType) -> Self { + fn from_protobuf(prost: Self::PbType) -> Self { prost } diff --git a/src/meta/src/rpc/service/ddl_service.rs b/src/meta/src/rpc/service/ddl_service.rs index 77a0f98be9fb..0186bd6525b4 100644 --- a/src/meta/src/rpc/service/ddl_service.rs +++ b/src/meta/src/rpc/service/ddl_service.rs @@ -24,7 +24,7 @@ use risingwave_connector::source::KAFKA_CONNECTOR; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{connection, Connection}; use risingwave_pb::ddl_service::ddl_service_server::DdlService; -use risingwave_pb::ddl_service::drop_table_request::SourceId as ProstSourceId; +use risingwave_pb::ddl_service::drop_table_request::PbSourceId; use risingwave_pb::ddl_service::*; use risingwave_pb::stream_plan::stream_node::NodeBody; use tonic::{Request, Response, Status}; @@ -457,7 +457,7 @@ where let version = self .ddl_controller .run_command(DdlCommand::DropStreamingJob(StreamingJobId::Table( - source_id.map(|ProstSourceId::Id(id)| id), + source_id.map(|PbSourceId::Id(id)| id), table_id, ))) .await?; diff --git a/src/meta/src/stream/sink.rs b/src/meta/src/stream/sink.rs index d4ad160c7e0a..5fd707941e27 100644 --- a/src/meta/src/stream/sink.rs +++ b/src/meta/src/stream/sink.rs @@ -15,12 +15,12 @@ use anyhow::anyhow; use risingwave_connector::sink::catalog::SinkCatalog; use risingwave_connector::sink::{SinkConfig, SinkImpl}; -use risingwave_pb::catalog::Sink as ProstSinkCatalog; +use risingwave_pb::catalog::PbSink; use crate::{MetaError, MetaResult}; pub async fn validate_sink( - prost_sink_catalog: &ProstSinkCatalog, + prost_sink_catalog: &PbSink, connector_rpc_endpoint: Option, ) -> MetaResult<()> { let sink_catalog = SinkCatalog::from(prost_sink_catalog); diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index 43f5e991b9f1..91ea7647fcd7 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -18,14 +18,14 @@ use std::vec; use itertools::Itertools; use risingwave_common::catalog::{DatabaseId, SchemaId, TableId}; -use risingwave_pb::catalog::Table as ProstTable; +use risingwave_pb::catalog::PbTable; use risingwave_pb::common::{ParallelUnit, PbColumnOrder, PbDirection, PbOrderType, WorkerNode}; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType; use risingwave_pb::expr::agg_call::Type; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::{Add, GreaterThan, InputRef}; -use risingwave_pb::expr::{AggCall, ExprNode, FunctionCall, InputRef as ProstInputRef}; +use risingwave_pb::expr::{AggCall, ExprNode, FunctionCall, PbInputRef}; use risingwave_pb::plan_common::{ColumnCatalog, ColumnDesc, Field}; use risingwave_pb::stream_plan::stream_fragment_graph::{StreamFragment, StreamFragmentEdge}; use risingwave_pb::stream_plan::stream_node::NodeBody; @@ -56,7 +56,7 @@ fn make_inputref(idx: u32) -> ExprNode { fn make_sum_aggcall(idx: u32) -> AggCall { AggCall { r#type: Type::Sum as i32, - args: vec![ProstInputRef { + args: vec![PbInputRef { index: idx, r#type: Some(DataType { type_name: TypeName::Int64 as i32, @@ -114,12 +114,12 @@ fn make_column(column_type: TypeName, column_id: i32) -> ColumnCatalog { } } -fn make_source_internal_table(id: u32) -> ProstTable { +fn make_source_internal_table(id: u32) -> PbTable { let columns = vec![ make_column(TypeName::Varchar, 0), make_column(TypeName::Varchar, 1), ]; - ProstTable { + PbTable { id, schema_id: SchemaId::placeholder().schema_id, database_id: DatabaseId::placeholder().database_id, @@ -135,12 +135,12 @@ fn make_source_internal_table(id: u32) -> ProstTable { } } -fn make_internal_table(id: u32, is_agg_value: bool) -> ProstTable { +fn make_internal_table(id: u32, is_agg_value: bool) -> PbTable { let mut columns = vec![make_column(TypeName::Int64, 0)]; if !is_agg_value { columns.push(make_column(TypeName::Int32, 1)); } - ProstTable { + PbTable { id, schema_id: SchemaId::placeholder().schema_id, database_id: DatabaseId::placeholder().database_id, @@ -157,8 +157,8 @@ fn make_internal_table(id: u32, is_agg_value: bool) -> ProstTable { } } -fn make_empty_table(id: u32) -> ProstTable { - ProstTable { +fn make_empty_table(id: u32) -> PbTable { + PbTable { id, schema_id: SchemaId::placeholder().schema_id, database_id: DatabaseId::placeholder().database_id, @@ -170,7 +170,7 @@ fn make_empty_table(id: u32) -> ProstTable { } } -fn make_materialize_table(id: u32) -> ProstTable { +fn make_materialize_table(id: u32) -> PbTable { make_internal_table(id, true) } diff --git a/src/prost/helpers/src/generate.rs b/src/prost/helpers/src/generate.rs index 7fe949131ca6..a689f7e00a0f 100644 --- a/src/prost/helpers/src/generate.rs +++ b/src/prost/helpers/src/generate.rs @@ -93,11 +93,11 @@ pub fn implement(field: &Field) -> TokenStream2 { if let Some(enum_type) = extract_enum_type_from_field(field) { return quote! { #[inline(always)] - pub fn #getter_fn_name(&self) -> std::result::Result<#enum_type, crate::ProstFieldNotFound> { + pub fn #getter_fn_name(&self) -> std::result::Result<#enum_type, crate::PbFieldNotFound> { if self.#field_name.eq(&0) { - return Err(crate::ProstFieldNotFound(stringify!(#field_name))); + return Err(crate::PbFieldNotFound(stringify!(#field_name))); } - #enum_type::from_i32(self.#field_name).ok_or_else(|| crate::ProstFieldNotFound(stringify!(#field_name))) + #enum_type::from_i32(self.#field_name).ok_or_else(|| crate::PbFieldNotFound(stringify!(#field_name))) } }; }; @@ -110,8 +110,8 @@ pub fn implement(field: &Field) -> TokenStream2 { let ty = extract_type_from_option(data_type); return quote! { #[inline(always)] - pub fn #getter_fn_name(&self) -> std::result::Result<&#ty, crate::ProstFieldNotFound> { - self.#field_name.as_ref().ok_or_else(|| crate::ProstFieldNotFound(stringify!(#field_name))) + pub fn #getter_fn_name(&self) -> std::result::Result<&#ty, crate::PbFieldNotFound> { + self.#field_name.as_ref().ok_or_else(|| crate::PbFieldNotFound(stringify!(#field_name))) } }; } else if ["u32", "u64", "f32", "f64", "i32", "i64", "bool"] diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index e8ad3c7f770d..a6669186cd3f 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -142,10 +142,10 @@ pub mod backup_service_serde; pub mod java_binding_serde; #[derive(Clone, PartialEq, Eq, Debug)] -pub struct ProstFieldNotFound(pub &'static str); +pub struct PbFieldNotFound(pub &'static str); -impl From for tonic::Status { - fn from(e: ProstFieldNotFound) -> Self { +impl From for tonic::Status { + fn from(e: PbFieldNotFound) -> Self { tonic::Status::new(tonic::Code::Internal, e.0) } } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 77cc767bccfe..e3ff92e2f5a8 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -39,9 +39,7 @@ use risingwave_hummock_sdk::{ use risingwave_pb::backup_service::backup_service_client::BackupServiceClient; use risingwave_pb::backup_service::*; use risingwave_pb::catalog::{ - Connection, Database as ProstDatabase, Function as ProstFunction, Index as ProstIndex, - Schema as ProstSchema, Sink as ProstSink, Source as ProstSource, Table as ProstTable, - View as ProstView, + Connection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, }; use risingwave_pb::common::{HostAddress, WorkerType}; use risingwave_pb::ddl_service::ddl_service_client::DdlServiceClient; @@ -56,7 +54,7 @@ use risingwave_pb::meta::heartbeat_service_client::HeartbeatServiceClient; use risingwave_pb::meta::list_table_fragments_response::TableFragmentInfo; use risingwave_pb::meta::meta_member_service_client::MetaMemberServiceClient; use risingwave_pb::meta::notification_service_client::NotificationServiceClient; -use risingwave_pb::meta::reschedule_request::Reschedule as ProstReschedule; +use risingwave_pb::meta::reschedule_request::PbReschedule; use risingwave_pb::meta::scale_service_client::ScaleServiceClient; use risingwave_pb::meta::stream_manager_service_client::StreamManagerServiceClient; use risingwave_pb::meta::system_params_service_client::SystemParamsServiceClient; @@ -261,14 +259,14 @@ impl MetaClient { Ok(()) } - pub async fn create_database(&self, db: ProstDatabase) -> Result<(DatabaseId, CatalogVersion)> { + pub async fn create_database(&self, db: PbDatabase) -> Result<(DatabaseId, CatalogVersion)> { let request = CreateDatabaseRequest { db: Some(db) }; let resp = self.inner.create_database(request).await?; // TODO: handle error in `resp.status` here Ok((resp.database_id, resp.version)) } - pub async fn create_schema(&self, schema: ProstSchema) -> Result<(SchemaId, CatalogVersion)> { + pub async fn create_schema(&self, schema: PbSchema) -> Result<(SchemaId, CatalogVersion)> { let request = CreateSchemaRequest { schema: Some(schema), }; @@ -279,7 +277,7 @@ impl MetaClient { pub async fn create_materialized_view( &self, - table: ProstTable, + table: PbTable, graph: StreamFragmentGraph, ) -> Result<(TableId, CatalogVersion)> { let request = CreateMaterializedViewRequest { @@ -300,7 +298,7 @@ impl MetaClient { Ok(resp.version) } - pub async fn create_source(&self, source: ProstSource) -> Result<(u32, CatalogVersion)> { + pub async fn create_source(&self, source: PbSource) -> Result<(u32, CatalogVersion)> { let request = CreateSourceRequest { source: Some(source), }; @@ -311,7 +309,7 @@ impl MetaClient { pub async fn create_sink( &self, - sink: ProstSink, + sink: PbSink, graph: StreamFragmentGraph, ) -> Result<(u32, CatalogVersion)> { let request = CreateSinkRequest { @@ -325,7 +323,7 @@ impl MetaClient { pub async fn create_function( &self, - function: ProstFunction, + function: PbFunction, ) -> Result<(FunctionId, CatalogVersion)> { let request = CreateFunctionRequest { function: Some(function), @@ -336,8 +334,8 @@ impl MetaClient { pub async fn create_table( &self, - source: Option, - table: ProstTable, + source: Option, + table: PbTable, graph: StreamFragmentGraph, ) -> Result<(TableId, CatalogVersion)> { let request = CreateTableRequest { @@ -352,7 +350,7 @@ impl MetaClient { pub async fn replace_table( &self, - table: ProstTable, + table: PbTable, graph: StreamFragmentGraph, table_col_index_mapping: ColIndexMapping, ) -> Result { @@ -366,7 +364,7 @@ impl MetaClient { Ok(resp.version) } - pub async fn create_view(&self, view: ProstView) -> Result<(u32, CatalogVersion)> { + pub async fn create_view(&self, view: PbView) -> Result<(u32, CatalogVersion)> { let request = CreateViewRequest { view: Some(view) }; let resp = self.inner.create_view(request).await?; // TODO: handle error in `resp.status` here @@ -375,8 +373,8 @@ impl MetaClient { pub async fn create_index( &self, - index: ProstIndex, - table: ProstTable, + index: PbIndex, + table: PbTable, graph: StreamFragmentGraph, ) -> Result<(TableId, CatalogVersion)> { let request = CreateIndexRequest { @@ -588,7 +586,7 @@ impl MetaClient { (join_handle, shutdown_tx) } - pub async fn risectl_list_state_tables(&self) -> Result> { + pub async fn risectl_list_state_tables(&self) -> Result> { let request = RisectlListStateTablesRequest {}; let resp = self.inner.risectl_list_state_tables(request).await?; Ok(resp.tables) @@ -635,7 +633,7 @@ impl MetaClient { Ok(resp) } - pub async fn reschedule(&self, reschedules: HashMap) -> Result { + pub async fn reschedule(&self, reschedules: HashMap) -> Result { let request = RescheduleRequest { reschedules }; let resp = self.inner.reschedule(request).await?; Ok(resp.success) @@ -661,7 +659,7 @@ impl MetaClient { pub async fn init_metadata_for_replay( &self, - tables: Vec, + tables: Vec, compaction_groups: Vec, ) -> Result<()> { let req = InitMetadataForReplayRequest { diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index d6c0036b6fb2..ff8e8d1676a2 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -22,10 +22,8 @@ use risingwave_connector::parser::SpecificParserConfig; use risingwave_connector::source::monitor::SourceMetrics; use risingwave_connector::source::{SourceColumnDesc, SourceFormat}; use risingwave_connector::ConnectorParams; -use risingwave_pb::catalog::StreamSourceInfo as ProstStreamSourceInfo; -use risingwave_pb::plan_common::{ - ColumnCatalog as ProstColumnCatalog, RowFormatType as ProstRowFormatType, -}; +use risingwave_pb::catalog::PbStreamSourceInfo; +use risingwave_pb::plan_common::{PbColumnCatalog, PbRowFormatType}; use crate::connector_source::ConnectorSource; use crate::fs_connector_source::FsConnectorSource; @@ -54,12 +52,12 @@ pub struct FsSourceDesc { #[derive(Clone)] pub struct SourceDescBuilder { - columns: Vec, + columns: Vec, metrics: Arc, pk_column_ids: Vec, row_id_index: Option, properties: HashMap, - source_info: ProstStreamSourceInfo, + source_info: PbStreamSourceInfo, connector_params: ConnectorParams, connector_message_buffer_size: usize, } @@ -67,12 +65,12 @@ pub struct SourceDescBuilder { impl SourceDescBuilder { #[allow(clippy::too_many_arguments)] pub fn new( - columns: Vec, + columns: Vec, metrics: Arc, pk_column_ids: Vec, row_id_index: Option, properties: HashMap, - source_info: ProstStreamSourceInfo, + source_info: PbStreamSourceInfo, connector_params: ConnectorParams, connector_message_buffer_size: usize, ) -> Self { @@ -90,16 +88,16 @@ impl SourceDescBuilder { pub async fn build(self) -> Result { let format = match self.source_info.get_row_format()? { - ProstRowFormatType::Json => SourceFormat::Json, - ProstRowFormatType::Protobuf => SourceFormat::Protobuf, - ProstRowFormatType::DebeziumJson => SourceFormat::DebeziumJson, - ProstRowFormatType::Avro => SourceFormat::Avro, - ProstRowFormatType::Maxwell => SourceFormat::Maxwell, - ProstRowFormatType::CanalJson => SourceFormat::CanalJson, - ProstRowFormatType::Native => SourceFormat::Native, - ProstRowFormatType::DebeziumAvro => SourceFormat::DebeziumAvro, - ProstRowFormatType::UpsertJson => SourceFormat::UpsertJson, - ProstRowFormatType::UpsertAvro => SourceFormat::UpsertAvro, + PbRowFormatType::Json => SourceFormat::Json, + PbRowFormatType::Protobuf => SourceFormat::Protobuf, + PbRowFormatType::DebeziumJson => SourceFormat::DebeziumJson, + PbRowFormatType::Avro => SourceFormat::Avro, + PbRowFormatType::Maxwell => SourceFormat::Maxwell, + PbRowFormatType::CanalJson => SourceFormat::CanalJson, + PbRowFormatType::Native => SourceFormat::Native, + PbRowFormatType::DebeziumAvro => SourceFormat::DebeziumAvro, + PbRowFormatType::UpsertJson => SourceFormat::UpsertJson, + PbRowFormatType::UpsertAvro => SourceFormat::UpsertAvro, _ => unreachable!(), }; @@ -146,8 +144,8 @@ impl SourceDescBuilder { pub async fn build_fs_source_desc(&self) -> Result { let format = match self.source_info.get_row_format()? { - ProstRowFormatType::Csv => SourceFormat::Csv, - ProstRowFormatType::Json => SourceFormat::Json, + PbRowFormatType::Csv => SourceFormat::Csv, + PbRowFormatType::Json => SourceFormat::Json, _ => unreachable!(), }; diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index b2d371431f6c..ab85d6809521 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -41,10 +41,7 @@ use std::hash::Hasher; use itertools::Itertools; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::{HummockSstableObjectId, HummockVersionId}; -use risingwave_pb::backup_service::{ - MetaSnapshotManifest as ProstMetaSnapshotManifest, - MetaSnapshotMetadata as ProstMetaSnapshotMetadata, -}; +use risingwave_pb::backup_service::{PbMetaSnapshotManifest, PbMetaSnapshotMetadata}; use risingwave_pb::hummock::HummockVersion; use serde::{Deserialize, Serialize}; @@ -102,7 +99,7 @@ pub fn xxhash64_verify(data: &[u8], checksum: u64) -> BackupResult<()> { Ok(()) } -impl From<&MetaSnapshotMetadata> for ProstMetaSnapshotMetadata { +impl From<&MetaSnapshotMetadata> for PbMetaSnapshotMetadata { fn from(m: &MetaSnapshotMetadata) -> Self { Self { id: m.id, @@ -113,7 +110,7 @@ impl From<&MetaSnapshotMetadata> for ProstMetaSnapshotMetadata { } } -impl From<&MetaSnapshotManifest> for ProstMetaSnapshotManifest { +impl From<&MetaSnapshotManifest> for PbMetaSnapshotManifest { fn from(m: &MetaSnapshotManifest) -> Self { Self { manifest_id: m.manifest_id, diff --git a/src/storage/hummock_sdk/src/filter_key_extractor.rs b/src/storage/hummock_sdk/src/filter_key_extractor.rs index 15527fc48e6d..a576de5b647e 100644 --- a/src/storage/hummock_sdk/src/filter_key_extractor.rs +++ b/src/storage/hummock_sdk/src/filter_key_extractor.rs @@ -347,9 +347,9 @@ mod tests { use risingwave_common::util::ordered::OrderedRowSerde; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::table::TableType; - use risingwave_pb::catalog::Table as ProstTable; + use risingwave_pb::catalog::PbTable; use risingwave_pb::common::{PbColumnOrder, PbDirection, PbOrderType}; - use risingwave_pb::plan_common::ColumnCatalog as ProstColumnCatalog; + use risingwave_pb::plan_common::PbColumnCatalog; use tokio::task; use super::{DummyFilterKeyExtractor, FilterKeyExtractor, SchemaFilterKeyExtractor}; @@ -377,15 +377,15 @@ mod tests { assert_eq!(full_key, output_key); } - fn build_table_with_prefix_column_num(column_count: u32) -> ProstTable { - ProstTable { + fn build_table_with_prefix_column_num(column_count: u32) -> PbTable { + PbTable { id: 0, schema_id: 0, database_id: 0, name: "test".to_string(), table_type: TableType::Table as i32, columns: vec![ - ProstColumnCatalog { + PbColumnCatalog { column_desc: Some( (&ColumnDesc { data_type: DataType::Int64, @@ -398,7 +398,7 @@ mod tests { ), is_hidden: true, }, - ProstColumnCatalog { + PbColumnCatalog { column_desc: Some( (&ColumnDesc { data_type: DataType::Int64, @@ -411,7 +411,7 @@ mod tests { ), is_hidden: false, }, - ProstColumnCatalog { + PbColumnCatalog { column_desc: Some( (&ColumnDesc { data_type: DataType::Float64, @@ -424,7 +424,7 @@ mod tests { ), is_hidden: false, }, - ProstColumnCatalog { + PbColumnCatalog { column_desc: Some( (&ColumnDesc { data_type: DataType::Varchar, diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index 6db409280e40..32835a6da2fc 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -32,7 +32,7 @@ use risingwave_pb::hummock::SstableInfo; use crate::compaction_group::StaticCompactionGroupId; use crate::key_range::KeyRangeCommon; -use crate::table_stats::{to_prost_table_stats_map, ProstTableStatsMap, TableStatsMap}; +use crate::table_stats::{to_prost_table_stats_map, PbTableStatsMap, TableStatsMap}; pub mod compact; pub mod compaction_group; @@ -131,14 +131,14 @@ impl LocalSstableInfo { pub struct ExtendedSstableInfo { pub compaction_group_id: CompactionGroupId, pub sst_info: SstableInfo, - pub table_stats: ProstTableStatsMap, + pub table_stats: PbTableStatsMap, } impl ExtendedSstableInfo { pub fn new( compaction_group_id: CompactionGroupId, sst_info: SstableInfo, - table_stats: ProstTableStatsMap, + table_stats: PbTableStatsMap, ) -> Self { Self { compaction_group_id, @@ -151,7 +151,7 @@ impl ExtendedSstableInfo { compaction_group_id: CompactionGroupId, sst_info: SstableInfo, ) -> Self { - Self::new(compaction_group_id, sst_info, ProstTableStatsMap::default()) + Self::new(compaction_group_id, sst_info, PbTableStatsMap::default()) } } diff --git a/src/storage/hummock_sdk/src/table_stats.rs b/src/storage/hummock_sdk/src/table_stats.rs index 19732e30cba1..02da4999bebc 100644 --- a/src/storage/hummock_sdk/src/table_stats.rs +++ b/src/storage/hummock_sdk/src/table_stats.rs @@ -15,13 +15,13 @@ use std::borrow::Borrow; use std::collections::{HashMap, HashSet}; -use risingwave_pb::hummock::{HummockVersion, TableStats as ProstTableStats}; +use risingwave_pb::hummock::{HummockVersion, PbTableStats}; use crate::compaction_group::hummock_version_ext::HummockVersionExt; pub type TableStatsMap = HashMap; -pub type ProstTableStatsMap = HashMap; +pub type PbTableStatsMap = HashMap; #[derive(Default, Debug, Clone)] pub struct TableStats { @@ -30,7 +30,7 @@ pub struct TableStats { pub total_key_count: i64, } -impl From<&TableStats> for ProstTableStats { +impl From<&TableStats> for PbTableStats { fn from(value: &TableStats) -> Self { Self { total_key_size: value.total_key_size, @@ -40,14 +40,14 @@ impl From<&TableStats> for ProstTableStats { } } -impl From for ProstTableStats { +impl From for PbTableStats { fn from(value: TableStats) -> Self { (&value).into() } } -impl From<&ProstTableStats> for TableStats { - fn from(value: &ProstTableStats) -> Self { +impl From<&PbTableStats> for TableStats { + fn from(value: &PbTableStats) -> Self { Self { total_key_size: value.total_key_size, total_value_size: value.total_value_size, @@ -64,13 +64,13 @@ impl TableStats { } } -pub fn add_prost_table_stats(this: &mut ProstTableStats, other: &ProstTableStats) { +pub fn add_prost_table_stats(this: &mut PbTableStats, other: &PbTableStats) { this.total_key_size += other.total_key_size; this.total_value_size += other.total_value_size; this.total_key_count += other.total_key_count; } -pub fn add_prost_table_stats_map(this: &mut ProstTableStatsMap, other: &ProstTableStatsMap) { +pub fn add_prost_table_stats_map(this: &mut PbTableStatsMap, other: &PbTableStatsMap) { for (table_id, stats) in other { add_prost_table_stats(this.entry(*table_id).or_default(), stats); } @@ -84,7 +84,7 @@ pub fn add_table_stats_map(this: &mut TableStatsMap, other: &TableStatsMap) { pub fn to_prost_table_stats_map( table_stats: impl Borrow, -) -> HashMap { +) -> HashMap { table_stats .borrow() .iter() @@ -93,7 +93,7 @@ pub fn to_prost_table_stats_map( } pub fn from_prost_table_stats_map( - table_stats: impl Borrow>, + table_stats: impl Borrow>, ) -> HashMap { table_stats .borrow() @@ -103,7 +103,7 @@ pub fn from_prost_table_stats_map( } pub fn purge_prost_table_stats( - table_stats: &mut ProstTableStatsMap, + table_stats: &mut PbTableStatsMap, hummock_version: &HummockVersion, ) { let mut all_tables_in_version: HashSet = HashSet::default(); diff --git a/src/storage/hummock_test/src/test_utils.rs b/src/storage/hummock_test/src/test_utils.rs index 465e4654c15c..da2763ca6cef 100644 --- a/src/storage/hummock_test/src/test_utils.rs +++ b/src/storage/hummock_test/src/test_utils.rs @@ -29,7 +29,7 @@ use risingwave_meta::hummock::test_utils::{ use risingwave_meta::hummock::{HummockManagerRef, MockHummockMetaClient}; use risingwave_meta::manager::MetaSrvEnv; use risingwave_meta::storage::{MemStore, MetaStore}; -use risingwave_pb::catalog::Table as ProstTable; +use risingwave_pb::catalog::PbTable; use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::version_update_payload; use risingwave_rpc_client::HummockMetaClient; @@ -179,7 +179,7 @@ pub async fn register_tables_with_id_for_test( pub async fn register_tables_with_catalog_for_test( filter_key_extractor_manager: &FilterKeyExtractorManagerRef, hummock_manager_ref: &HummockManagerRef, - tables: &[ProstTable], + tables: &[PbTable], ) { update_filter_key_extractor_for_tables(filter_key_extractor_manager, tables); let table_ids = tables.iter().map(|t| t.id).collect_vec(); @@ -207,7 +207,7 @@ impl HummockTestEnv { .await; } - pub async fn register_table(&self, table: ProstTable) { + pub async fn register_table(&self, table: PbTable) { register_tables_with_catalog_for_test( self.storage.filter_key_extractor_manager(), &self.manager, diff --git a/src/stream/src/common/table/test_utils.rs b/src/stream/src/common/table/test_utils.rs index 1d317cf80933..526f6864b3a9 100644 --- a/src/stream/src/common/table/test_utils.rs +++ b/src/stream/src/common/table/test_utils.rs @@ -16,7 +16,7 @@ use itertools::Itertools; 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::catalog::PbTable; use risingwave_pb::common::PbColumnOrder; use risingwave_pb::plan_common::ColumnCatalog; @@ -26,7 +26,7 @@ pub(crate) fn gen_prost_table( order_types: Vec, pk_index: Vec, read_prefix_len_hint: u32, -) -> ProstTable { +) -> PbTable { let col_len = column_descs.len() as i32; gen_prost_table_with_value_indices( table_id, @@ -45,7 +45,7 @@ pub(crate) fn gen_prost_table_with_value_indices( pk_index: Vec, read_prefix_len_hint: u32, value_indices: Vec, -) -> ProstTable { +) -> PbTable { let prost_pk = pk_index .iter() .zip_eq_fast(order_types.iter()) @@ -62,7 +62,7 @@ pub(crate) fn gen_prost_table_with_value_indices( }) .collect(); - ProstTable { + PbTable { id: table_id.table_id(), columns: prost_columns, pk: prost_pk, diff --git a/src/stream/src/error.rs b/src/stream/src/error.rs index d2250cceeb92..7c92a2b4e023 100644 --- a/src/stream/src/error.rs +++ b/src/stream/src/error.rs @@ -16,7 +16,7 @@ use std::backtrace::Backtrace; use risingwave_common::array::ArrayError; use risingwave_expr::ExprError; -use risingwave_pb::ProstFieldNotFound; +use risingwave_pb::PbFieldNotFound; use risingwave_storage::error::StorageError; use crate::executor::StreamExecutorError; @@ -95,8 +95,8 @@ impl From for StreamError { } } -impl From for StreamError { - fn from(err: ProstFieldNotFound) -> Self { +impl From for StreamError { + fn from(err: PbFieldNotFound) -> Self { Self::from(anyhow::anyhow!( "Failed to decode prost: field not found `{}`", err.0 diff --git a/src/stream/src/executor/.!41891!.DS_Store b/src/stream/src/executor/.!41891!.DS_Store new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/src/stream/src/executor/dispatch.rs b/src/stream/src/executor/dispatch.rs index f115985bf63b..4366037dc54a 100644 --- a/src/stream/src/executor/dispatch.rs +++ b/src/stream/src/executor/dispatch.rs @@ -27,8 +27,8 @@ use risingwave_common::buffer::BitmapBuilder; use risingwave_common::hash::{ActorMapping, ExpandedActorMapping, VirtualNode}; use risingwave_common::util::hash_util::Crc32FastBuilder; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_pb::stream_plan::update_mutation::DispatcherUpdate as ProstDispatcherUpdate; -use risingwave_pb::stream_plan::Dispatcher as ProstDispatcher; +use risingwave_pb::stream_plan::update_mutation::PbDispatcherUpdate; +use risingwave_pb::stream_plan::PbDispatcher; use smallvec::{smallvec, SmallVec}; use tracing::event; @@ -106,7 +106,7 @@ impl DispatchExecutorInner { /// Add new dispatchers to the executor. Will check whether their ids are unique. fn add_dispatchers<'a>( &mut self, - new_dispatchers: impl IntoIterator, + new_dispatchers: impl IntoIterator, ) -> StreamResult<()> { let new_dispatchers: Vec<_> = new_dispatchers .into_iter() @@ -136,7 +136,7 @@ impl DispatchExecutorInner { /// Update the dispatcher BEFORE we actually dispatch this barrier. We'll only add the new /// outputs. - fn pre_update_dispatcher(&mut self, update: &ProstDispatcherUpdate) -> StreamResult<()> { + fn pre_update_dispatcher(&mut self, update: &PbDispatcherUpdate) -> StreamResult<()> { let outputs: Vec<_> = update .added_downstream_actor_id .iter() @@ -151,7 +151,7 @@ impl DispatchExecutorInner { /// Update the dispatcher AFTER we dispatch this barrier. We'll remove some outputs and finally /// update the hash mapping. - fn post_update_dispatcher(&mut self, update: &ProstDispatcherUpdate) -> StreamResult<()> { + fn post_update_dispatcher(&mut self, update: &PbDispatcherUpdate) -> StreamResult<()> { let ids = update.removed_downstream_actor_id.iter().copied().collect(); let dispatcher = self.find_dispatcher(update.dispatcher_id); @@ -291,7 +291,7 @@ impl DispatcherImpl { pub fn new( context: &SharedContext, actor_id: ActorId, - dispatcher: &ProstDispatcher, + dispatcher: &PbDispatcher, ) -> StreamResult { let outputs = dispatcher .downstream_actor_id @@ -1035,7 +1035,7 @@ mod tests { let broadcast_dispatcher = DispatcherImpl::new( &ctx, actor_id, - &ProstDispatcher { + &PbDispatcher { r#type: DispatcherType::Broadcast as _, dispatcher_id: broadcast_dispatcher_id, downstream_actor_id: vec![untouched, old], @@ -1048,7 +1048,7 @@ mod tests { let simple_dispatcher = DispatcherImpl::new( &ctx, actor_id, - &ProstDispatcher { + &PbDispatcher { r#type: DispatcherType::Simple as _, dispatcher_id: simple_dispatcher_id, downstream_actor_id: vec![old_simple], @@ -1085,7 +1085,7 @@ mod tests { // 4. Send a configuration change barrier for broadcast dispatcher. let dispatcher_updates = maplit::hashmap! { - actor_id => vec![ProstDispatcherUpdate { + actor_id => vec![PbDispatcherUpdate { actor_id, dispatcher_id: broadcast_dispatcher_id, added_downstream_actor_id: vec![new], @@ -1136,7 +1136,7 @@ mod tests { // 9. Send a configuration change barrier for simple dispatcher. let dispatcher_updates = maplit::hashmap! { - actor_id => vec![ProstDispatcherUpdate { + actor_id => vec![PbDispatcherUpdate { actor_id, dispatcher_id: simple_dispatcher_id, added_downstream_actor_id: vec![new_simple], diff --git a/src/stream/src/executor/error.rs b/src/stream/src/executor/error.rs index eafb17e977ae..88c09bd7d043 100644 --- a/src/stream/src/executor/error.rs +++ b/src/stream/src/executor/error.rs @@ -21,7 +21,7 @@ use risingwave_common::util::value_encoding::error::ValueEncodingError; use risingwave_connector::error::ConnectorError; use risingwave_connector::sink::SinkError; use risingwave_expr::ExprError; -use risingwave_pb::ProstFieldNotFound; +use risingwave_pb::PbFieldNotFound; use risingwave_rpc_client::error::RpcError; use risingwave_storage::error::StorageError; @@ -173,8 +173,8 @@ impl From for StreamExecutorError { } } -impl From for StreamExecutorError { - fn from(err: ProstFieldNotFound) -> Self { +impl From for StreamExecutorError { + fn from(err: PbFieldNotFound) -> Self { Self::from(anyhow::anyhow!( "Failed to decode prost: field not found `{}`", err.0 diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index cf2be19024e3..d0925c5d8588 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -34,16 +34,15 @@ use risingwave_common::util::value_encoding::{deserialize_datum, serialize_datum use risingwave_connector::source::SplitImpl; use risingwave_expr::expr::BoxedExpression; use risingwave_expr::ExprError; -use risingwave_pb::data::{Datum as ProstDatum, Epoch as ProstEpoch}; -use risingwave_pb::expr::InputRef as ProstInputRef; +use risingwave_pb::data::{PbDatum, PbEpoch}; +use risingwave_pb::expr::PbInputRef; use risingwave_pb::stream_plan::add_mutation::Dispatchers; -use risingwave_pb::stream_plan::barrier::Mutation as ProstMutation; +use risingwave_pb::stream_plan::barrier::PbMutation; use risingwave_pb::stream_plan::stream_message::StreamMessage; use risingwave_pb::stream_plan::update_mutation::{DispatcherUpdate, MergeUpdate}; use risingwave_pb::stream_plan::{ - AddMutation, Barrier as ProstBarrier, Dispatcher as ProstDispatcher, PauseMutation, - ResumeMutation, SourceChangeSplitMutation, StopMutation, StreamMessage as ProstStreamMessage, - UpdateMutation, Watermark as ProstWatermark, + AddMutation, PauseMutation, PbBarrier, PbDispatcher, PbStreamMessage, PbWatermark, + ResumeMutation, SourceChangeSplitMutation, StopMutation, UpdateMutation, }; use smallvec::SmallVec; @@ -220,7 +219,7 @@ pub enum Mutation { actor_splits: HashMap>, }, Add { - adds: HashMap>, + adds: HashMap>, // TODO: remove this and use `SourceChangesSplit` after we support multiple mutations. splits: HashMap>, }, @@ -364,9 +363,9 @@ impl Mutation { matches!(self, Mutation::Stop(_)) } - fn to_protobuf(&self) -> ProstMutation { + fn to_protobuf(&self) -> PbMutation { match self { - Mutation::Stop(actors) => ProstMutation::Stop(StopMutation { + Mutation::Stop(actors) => PbMutation::Stop(StopMutation { actors: actors.iter().copied().collect::>(), }), Mutation::Update { @@ -375,7 +374,7 @@ impl Mutation { vnode_bitmaps, dropped_actors, actor_splits, - } => ProstMutation::Update(UpdateMutation { + } => PbMutation::Update(UpdateMutation { dispatcher_update: dispatchers.values().flatten().cloned().collect(), merge_update: merges.values().cloned().collect(), actor_vnode_bitmap_update: vnode_bitmaps @@ -395,7 +394,7 @@ impl Mutation { }) .collect(), }), - Mutation::Add { adds, .. } => ProstMutation::Add(AddMutation { + Mutation::Add { adds, .. } => PbMutation::Add(AddMutation { actor_dispatchers: adds .iter() .map(|(&actor_id, dispatchers)| { @@ -409,37 +408,29 @@ impl Mutation { .collect(), ..Default::default() }), - Mutation::SourceChangeSplit(changes) => { - ProstMutation::Splits(SourceChangeSplitMutation { - actor_splits: changes - .iter() - .map(|(&actor_id, splits)| { - ( - actor_id, - ConnectorSplits { - splits: splits - .clone() - .iter() - .map(ConnectorSplit::from) - .collect(), - }, - ) - }) - .collect(), - }) - } - Mutation::Pause => ProstMutation::Pause(PauseMutation {}), - Mutation::Resume => ProstMutation::Resume(ResumeMutation {}), + Mutation::SourceChangeSplit(changes) => PbMutation::Splits(SourceChangeSplitMutation { + actor_splits: changes + .iter() + .map(|(&actor_id, splits)| { + ( + actor_id, + ConnectorSplits { + splits: splits.clone().iter().map(ConnectorSplit::from).collect(), + }, + ) + }) + .collect(), + }), + Mutation::Pause => PbMutation::Pause(PauseMutation {}), + Mutation::Resume => PbMutation::Resume(ResumeMutation {}), } } - fn from_protobuf(prost: &ProstMutation) -> StreamExecutorResult { + fn from_protobuf(prost: &PbMutation) -> StreamExecutorResult { let mutation = match prost { - ProstMutation::Stop(stop) => { - Mutation::Stop(HashSet::from_iter(stop.get_actors().clone())) - } + PbMutation::Stop(stop) => Mutation::Stop(HashSet::from_iter(stop.get_actors().clone())), - ProstMutation::Update(update) => Mutation::Update { + PbMutation::Update(update) => Mutation::Update { dispatchers: update .dispatcher_update .iter() @@ -472,7 +463,7 @@ impl Mutation { .collect(), }, - ProstMutation::Add(add) => Mutation::Add { + PbMutation::Add(add) => Mutation::Add { adds: add .actor_dispatchers .iter() @@ -496,7 +487,7 @@ impl Mutation { .collect(), }, - ProstMutation::Splits(s) => { + PbMutation::Splits(s) => { let mut change_splits: Vec<(ActorId, Vec)> = Vec::with_capacity(s.actor_splits.len()); for (&actor_id, splits) in &s.actor_splits { @@ -513,15 +504,15 @@ impl Mutation { } Mutation::SourceChangeSplit(change_splits.into_iter().collect()) } - ProstMutation::Pause(_) => Mutation::Pause, - ProstMutation::Resume(_) => Mutation::Resume, + PbMutation::Pause(_) => Mutation::Pause, + PbMutation::Resume(_) => Mutation::Resume, }; Ok(mutation) } } impl Barrier { - pub fn to_protobuf(&self) -> ProstBarrier { + pub fn to_protobuf(&self) -> PbBarrier { let Barrier { epoch, mutation, @@ -529,8 +520,8 @@ impl Barrier { passed_actors, .. }: Barrier = self.clone(); - ProstBarrier { - epoch: Some(ProstEpoch { + PbBarrier { + epoch: Some(PbEpoch { curr: epoch.curr, prev: epoch.prev, }), @@ -541,7 +532,7 @@ impl Barrier { } } - pub fn from_protobuf(prost: &ProstBarrier) -> StreamExecutorResult { + pub fn from_protobuf(prost: &PbBarrier) -> StreamExecutorResult { let mutation = prost .mutation .as_ref() @@ -624,19 +615,19 @@ impl Watermark { .map(|new_col_idx| self.with_idx(new_col_idx)) } - pub fn to_protobuf(&self) -> ProstWatermark { - ProstWatermark { - column: Some(ProstInputRef { + pub fn to_protobuf(&self) -> PbWatermark { + PbWatermark { + column: Some(PbInputRef { index: self.col_idx as _, r#type: Some(self.data_type.to_protobuf()), }), - val: Some(ProstDatum { + val: Some(PbDatum { body: serialize_datum(Some(&self.val)), }), } } - pub fn from_protobuf(prost: &ProstWatermark) -> StreamExecutorResult { + pub fn from_protobuf(prost: &PbWatermark) -> StreamExecutorResult { let col_ref = prost.get_column()?; let data_type = DataType::from(col_ref.get_type()?); let val = deserialize_datum(prost.get_val()?.get_body().as_slice(), &data_type)? @@ -692,7 +683,7 @@ impl Message { ) } - pub fn to_protobuf(&self) -> ProstStreamMessage { + pub fn to_protobuf(&self) -> PbStreamMessage { let prost = match self { Self::Chunk(stream_chunk) => { let prost_stream_chunk = stream_chunk.to_protobuf(); @@ -701,12 +692,12 @@ impl Message { Self::Barrier(barrier) => StreamMessage::Barrier(barrier.clone().to_protobuf()), Self::Watermark(watermark) => StreamMessage::Watermark(watermark.to_protobuf()), }; - ProstStreamMessage { + PbStreamMessage { stream_message: Some(prost), } } - pub fn from_protobuf(prost: &ProstStreamMessage) -> StreamExecutorResult { + pub fn from_protobuf(prost: &PbStreamMessage) -> StreamExecutorResult { let res = match prost.get_stream_message()? { StreamMessage::StreamChunk(chunk) => Message::Chunk(StreamChunk::from_protobuf(chunk)?), StreamMessage::Barrier(barrier) => Message::Barrier(Barrier::from_protobuf(barrier)?), diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 0558b70e6f5e..4da91f1376d7 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -499,7 +499,7 @@ mod tests { 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; + use risingwave_pb::plan_common::PbRowFormatType; use risingwave_source::connector_test_utils::create_source_desc_builder; use risingwave_storage::memory::MemoryStateStore; use tokio::sync::mpsc::unbounded_channel; @@ -519,7 +519,7 @@ mod tests { let pk_column_ids = vec![0]; let pk_indices = vec![0]; let source_info = StreamSourceInfo { - row_format: ProstRowFormatType::Native as i32, + row_format: PbRowFormatType::Native as i32, ..Default::default() }; let (barrier_tx, barrier_rx) = unbounded_channel::(); @@ -610,7 +610,7 @@ mod tests { let pk_column_ids = vec![0]; let pk_indices = vec![0_usize]; let source_info = StreamSourceInfo { - row_format: ProstRowFormatType::Native as i32, + row_format: PbRowFormatType::Native as i32, ..Default::default() }; let properties = convert_args!(hashmap!( diff --git a/src/stream/src/executor/source/state_table_handler.rs b/src/stream/src/executor/source/state_table_handler.rs index 1377120a075b..0b7a15bcb632 100644 --- a/src/stream/src/executor/source/state_table_handler.rs +++ b/src/stream/src/executor/source/state_table_handler.rs @@ -27,7 +27,7 @@ use risingwave_common::{bail, row}; use risingwave_connector::source::{SplitId, SplitImpl, SplitMetaData}; use risingwave_hummock_sdk::key::next_key; use risingwave_pb::catalog::table::TableType; -use risingwave_pb::catalog::Table as ProstTable; +use risingwave_pb::catalog::PbTable; use risingwave_pb::common::{PbColumnOrder, PbDirection, PbOrderType}; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType; @@ -46,7 +46,7 @@ pub struct SourceStateTableHandler { } impl SourceStateTableHandler { - pub async fn from_table_catalog(table_catalog: &ProstTable, store: S) -> Self { + pub async fn from_table_catalog(table_catalog: &PbTable, store: S) -> Self { // The state of source should not be cleaned up by retention_seconds assert!(!table_catalog .properties @@ -199,7 +199,7 @@ impl SourceStateTableHandler { // align with schema defined in `LogicalSource::infer_internal_table_catalog`. The function is used // for test purpose and should not be used in production. -pub fn default_source_internal_table(id: u32) -> ProstTable { +pub fn default_source_internal_table(id: u32) -> PbTable { let make_column = |column_type: TypeName, column_id: i32| -> ColumnCatalog { ColumnCatalog { column_desc: Some(ColumnDesc { @@ -218,7 +218,7 @@ pub fn default_source_internal_table(id: u32) -> ProstTable { make_column(TypeName::Varchar, 0), make_column(TypeName::Jsonb, 1), ]; - ProstTable { + PbTable { id, schema_id: SchemaId::placeholder().schema_id, database_id: DatabaseId::placeholder().database_id, diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 3116aba68164..548117bc8f12 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -16,7 +16,7 @@ use std::collections::{HashMap, HashSet}; use anyhow::anyhow; use prometheus::HistogramTimer; -use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress as ProstCreateMviewProgress; +use risingwave_pb::stream_service::barrier_complete_response::PbCreateMviewProgress; use tokio::sync::mpsc::UnboundedSender; use tokio::sync::oneshot; use tokio::sync::oneshot::Receiver; @@ -42,7 +42,7 @@ pub const ENABLE_BARRIER_AGGREGATION: bool = false; /// Collect result of some barrier on current compute node. Will be reported to the meta service. #[derive(Debug)] pub struct CollectResult { - pub create_mview_progress: Vec, + pub create_mview_progress: Vec, } enum BarrierState { diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index aab5ab40a790..86865d036943 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -37,7 +37,7 @@ use risingwave_meta::hummock::test_utils::setup_compute_env_with_config; use risingwave_meta::hummock::MockHummockMetaClient; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_object_store::object::parse_remote_object_store; -use risingwave_pb::catalog::Table as ProstTable; +use risingwave_pb::catalog::PbTable; use risingwave_pb::hummock::{CompactionConfig, CompactionGroupInfo}; use risingwave_pb::meta::SystemParams; use risingwave_rpc_client::HummockMetaClient; @@ -102,7 +102,7 @@ async fn compaction_test( worker_node.id, )); - let delete_key_table = ProstTable { + let delete_key_table = PbTable { id: 1, schema_id: 1, database_id: 1, diff --git a/src/tests/simulation/src/ctl_ext.rs b/src/tests/simulation/src/ctl_ext.rs index 562ee1345737..1fe319e7a779 100644 --- a/src/tests/simulation/src/ctl_ext.rs +++ b/src/tests/simulation/src/ctl_ext.rs @@ -24,7 +24,7 @@ use rand::seq::{IteratorRandom, SliceRandom}; use rand::Rng; use risingwave_common::hash::ParallelUnitId; use risingwave_pb::meta::table_fragments::fragment::FragmentDistributionType; -use risingwave_pb::meta::table_fragments::Fragment as ProstFragment; +use risingwave_pb::meta::table_fragments::PbFragment; use risingwave_pb::meta::GetClusterInfoResponse; use risingwave_pb::stream_plan::StreamNode; @@ -35,10 +35,10 @@ use crate::cluster::Cluster; pub mod predicate { use super::*; - trait Predicate = Fn(&ProstFragment) -> bool + Send + 'static; + trait Predicate = Fn(&PbFragment) -> bool + Send + 'static; pub type BoxedPredicate = Box; - fn root(fragment: &ProstFragment) -> &StreamNode { + fn root(fragment: &PbFragment) -> &StreamNode { fragment.actors.first().unwrap().nodes.as_ref().unwrap() } @@ -58,7 +58,7 @@ pub mod predicate { /// There're exactly `n` operators whose identity contains `s` in the fragment. pub fn identity_contains_n(n: usize, s: impl Into) -> BoxedPredicate { let s: String = s.into(); - let p = move |f: &ProstFragment| { + let p = move |f: &PbFragment| { count(root(f), &|n| { n.identity.to_lowercase().contains(&s.to_lowercase()) }) == n @@ -69,7 +69,7 @@ pub mod predicate { /// There exists operators whose identity contains `s` in the fragment. pub fn identity_contains(s: impl Into) -> BoxedPredicate { let s: String = s.into(); - let p = move |f: &ProstFragment| { + let p = move |f: &PbFragment| { any(root(f), &|n| { n.identity.to_lowercase().contains(&s.to_lowercase()) }) @@ -80,7 +80,7 @@ pub mod predicate { /// There does not exist any operator whose identity contains `s` in the fragment. pub fn no_identity_contains(s: impl Into) -> BoxedPredicate { let s: String = s.into(); - let p = move |f: &ProstFragment| { + let p = move |f: &PbFragment| { all(root(f), &|n| { !n.identity.to_lowercase().contains(&s.to_lowercase()) }) @@ -90,7 +90,7 @@ pub mod predicate { /// There're `n` upstream fragments of the fragment. pub fn upstream_fragment_count(n: usize) -> BoxedPredicate { - let p = move |f: &ProstFragment| f.upstream_fragment_ids.len() == n; + let p = move |f: &PbFragment| f.upstream_fragment_ids.len() == n; Box::new(p) } @@ -103,7 +103,7 @@ pub mod predicate { /// The fragment with the given id. pub fn id(id: u32) -> BoxedPredicate { - let p = move |f: &ProstFragment| f.fragment_id == id; + let p = move |f: &PbFragment| f.fragment_id == id; Box::new(p) } }