From 3120ef390280ad509c895a4ba5bb15a130bd5bd0 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Tue, 21 Mar 2023 14:56:52 -0400 Subject: [PATCH 01/17] introduce and Signed-off-by: Clearlove --- proto/stream_plan.proto | 9 ++ src/frontend/src/optimizer/mod.rs | 7 +- .../optimizer/plan_node/logical_row_id_gen.rs | 119 ++++++++++++++++++ .../src/optimizer/plan_node/logical_source.rs | 6 +- .../src/optimizer/plan_node/logical_values.rs | 30 +++-- src/frontend/src/optimizer/plan_node/mod.rs | 8 ++ .../optimizer/plan_node/stream_row_id_gen.rs | 39 +++--- .../src/optimizer/plan_node/stream_values.rs | 94 ++++++++++++++ src/stream/src/from_proto/mod.rs | 2 +- src/stream/src/from_proto/values.rs | 0 10 files changed, 277 insertions(+), 37 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs create mode 100644 src/frontend/src/optimizer/plan_node/stream_values.rs create mode 100644 src/stream/src/from_proto/values.rs diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index f865ab992e014..8ccfb3d715407 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -511,6 +511,14 @@ message NowNode { catalog.Table state_table = 1; } +message ValuesNode { + message ExprTuple { + repeated expr.ExprNode cells = 1; + } + repeated ExprTuple tuples = 1; + repeated plan_common.Field fields = 2; +} + message StreamNode { oneof node_body { SourceNode source = 100; @@ -545,6 +553,7 @@ message StreamNode { NowNode now = 129; GroupTopNNode append_only_group_top_n = 130; TemporalJoinNode temporal_join = 131; + ValuesNode values = 132; } // The id for the operator. This is local per mview. // TODO: should better be a uint32. diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 91dac433c14ce..228ad14d95960 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -41,8 +41,8 @@ use risingwave_pb::catalog::WatermarkDesc; use self::heuristic_optimizer::ApplyOrder; use self::plan_node::{ - BatchProject, Convention, LogicalProject, StreamDml, StreamMaterialize, StreamProject, - StreamRowIdGen, StreamSink, StreamWatermarkFilter, + BatchProject, Convention, LogicalProject, LogicalRowIdGen, StreamDml, StreamMaterialize, + StreamProject, StreamRowIdGen, StreamSink, StreamWatermarkFilter, }; use self::plan_visitor::has_batch_exchange; #[cfg(debug_assertions)] @@ -397,7 +397,8 @@ impl PlanRoot { // Add RowIDGen node if needed. if let Some(row_id_index) = row_id_index { - stream_plan = StreamRowIdGen::new(stream_plan, row_id_index).into(); + let logical_row_id_gen = LogicalRowIdGen::new(stream_plan, row_id_index); + stream_plan = StreamRowIdGen::new(logical_row_id_gen).into(); } let conflict_behavior = match append_only { diff --git a/src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs new file mode 100644 index 0000000000000..7976037076c6b --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs @@ -0,0 +1,119 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{fmt, vec}; + +use risingwave_common::bail; +use risingwave_common::error::Result; + +use super::{ + ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PlanTreeNodeUnary, + PredicatePushdown, StreamRowIdGen, ToBatch, ToStream, +}; +use crate::optimizer::plan_node::{ + ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, +}; +use crate::optimizer::property::FunctionalDependencySet; +use crate::utils::{ColIndexMapping, Condition}; + +/// `LogicalRowIdGen` builds rows according to a list of expressions +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LogicalRowIdGen { + pub base: PlanBase, + input: PlanRef, + row_id_index: usize, +} + +impl LogicalRowIdGen { + pub fn new(input: PlanRef, row_id: usize) -> Self { + let schema = input.schema().clone(); + let functional_dependency = FunctionalDependencySet::new(schema.len()); + let base = PlanBase::new_logical(input.ctx(), schema, vec![row_id], functional_dependency); + Self { + base, + input, + row_id_index: row_id, + } + } + + pub fn create(input: PlanRef, row_id: usize) -> Result { + Ok(Self::new(input, row_id)) + } + + pub fn row_id_index(&self) -> usize { + self.row_id_index + } +} + +impl fmt::Display for LogicalRowIdGen { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!( + f, + "LogicalRowIdGen {{ row_id_index: {} }}", + self.row_id_index + ) + } +} + +impl ExprRewritable for LogicalRowIdGen {} + +impl PlanTreeNodeUnary for LogicalRowIdGen { + fn input(&self) -> PlanRef { + self.input.clone() + } + + fn clone_with_input(&self, input: PlanRef) -> Self { + Self::new(input, self.row_id_index) + } +} + +impl_plan_tree_node_for_unary! {LogicalRowIdGen} + +impl PredicatePushdown for LogicalRowIdGen { + fn predicate_pushdown( + &self, + predicate: Condition, + _ctx: &mut PredicatePushdownContext, + ) -> PlanRef { + LogicalFilter::create(self.clone().into(), predicate) + } +} + +impl ColPrunable for LogicalRowIdGen { + fn prune_col(&self, _required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { + self.clone().into() + } +} + +impl ToBatch for LogicalRowIdGen { + fn to_batch(&self) -> Result { + bail!("`LogicalRowIdGen` can only be converted to stream") + } +} + +impl ToStream for LogicalRowIdGen { + fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { + let stream_input = self.input().to_stream(ctx)?; + let new_logical = self.clone_with_input(stream_input); + Ok(StreamRowIdGen::new(new_logical).into()) + } + + fn logical_rewrite_for_stream( + &self, + ctx: &mut RewriteStreamContext, + ) -> Result<(PlanRef, ColIndexMapping)> { + let (input, input_col_change) = self.input().logical_rewrite_for_stream(ctx)?; + Ok((self.clone_with_input(input).into(), input_col_change)) + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 5300db3ca2610..131dfe2de3392 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -32,7 +32,8 @@ use crate::catalog::ColumnId; use crate::expr::{Expr, ExprImpl, ExprType}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::{ - ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, + ColumnPruningContext, LogicalRowIdGen, PredicatePushdownContext, RewriteStreamContext, + ToStreamContext, }; use crate::utils::{ColIndexMapping, Condition}; use crate::TableCatalog; @@ -352,7 +353,8 @@ impl ToStream for LogicalSource { assert!(!(self.core.gen_row_id && self.core.for_table)); if let Some(row_id_index) = self.core.row_id_index && self.core.gen_row_id { - plan = StreamRowIdGen::new(plan, row_id_index).into(); + let logical_row_id_gen = LogicalRowIdGen::new(plan, row_id_index); + plan = StreamRowIdGen::new(logical_row_id_gen).into(); } Ok(plan) } diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index f86015fe59839..38a1ac7853455 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -15,12 +15,13 @@ use std::sync::Arc; use std::{fmt, vec}; -use risingwave_common::catalog::Schema; -use risingwave_common::error::{ErrorCode, Result, RwError}; +use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::error::Result; +use risingwave_common::types::DataType; use super::{ BatchValues, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, - ToBatch, ToStream, + StreamValues, ToBatch, ToStream, LogicalRowIdGen, }; use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::optimizer_context::OptimizerContextRef; @@ -132,20 +133,27 @@ impl ToBatch for LogicalValues { impl ToStream for LogicalValues { fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { - Err(RwError::from(ErrorCode::NotImplemented( - "Stream values executor is unimplemented!".to_string(), - None.into(), - ))) + Ok(StreamValues::new(self.clone()).into()) } fn logical_rewrite_for_stream( &self, _ctx: &mut RewriteStreamContext, ) -> Result<(PlanRef, ColIndexMapping)> { - Err(RwError::from(ErrorCode::NotImplemented( - "Stream values executor is unimplemented!".to_string(), - None.into(), - ))) + let row_id_index = self.schema().len(); + let col_index_mapping = ColIndexMapping::identity(row_id_index); + let ctx = self.ctx().clone(); + let mut schema = self.schema().clone(); + schema.fields.push(Field { + data_type: DataType::Int64, + name: "_row_id".to_string(), + sub_fields: vec![], + type_name: "int64".to_string(), + }); + let rows = self.rows().clone().to_owned(); + let logical_values = Self::create(rows, schema, ctx); + let logical_row_id_gen = LogicalRowIdGen::new(logical_values, row_id_index); + Ok((logical_row_id_gen.into(), col_index_mapping)) } } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index d4e879cfefcf2..b2a393f00a54b 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -638,6 +638,7 @@ mod logical_now; mod logical_over_agg; mod logical_project; mod logical_project_set; +mod logical_row_id_gen; mod logical_scan; mod logical_share; mod logical_source; @@ -667,6 +668,7 @@ mod stream_sink; mod stream_source; mod stream_table_scan; mod stream_topn; +mod stream_values; mod stream_watermark_filter; mod derive; @@ -713,6 +715,7 @@ pub use logical_now::LogicalNow; pub use logical_over_agg::{LogicalOverAgg, PlanWindowFunction}; pub use logical_project::LogicalProject; pub use logical_project_set::LogicalProjectSet; +pub use logical_row_id_gen::LogicalRowIdGen; pub use logical_scan::LogicalScan; pub use logical_share::LogicalShare; pub use logical_source::LogicalSource; @@ -745,6 +748,7 @@ pub use stream_table_scan::StreamTableScan; pub use stream_temporal_join::StreamTemporalJoin; pub use stream_topn::StreamTopN; pub use stream_union::StreamUnion; +pub use stream_values::StreamValues; pub use stream_watermark_filter::StreamWatermarkFilter; use crate::expr::{ExprImpl, ExprRewriter, InputRef, Literal}; @@ -791,6 +795,7 @@ macro_rules! for_all_plan_nodes { , { Logical, OverAgg } , { Logical, Share } , { Logical, Now } + , { Logical, RowIdGen} // , { Logical, Sort } we don't need a LogicalSort, just require the Order , { Batch, SimpleAgg } , { Batch, HashAgg } @@ -841,6 +846,7 @@ macro_rules! for_all_plan_nodes { , { Stream, Share } , { Stream, WatermarkFilter } , { Stream, TemporalJoin } + , { Stream, Values} } }; } @@ -872,6 +878,7 @@ macro_rules! for_logical_plan_nodes { , { Logical, OverAgg } , { Logical, Share } , { Logical, Now } + , { Logical, RowIdGen} // , { Logical, Sort} not sure if we will support Order by clause in subquery/view/MV // if we don't support that, we don't need LogicalSort, just require the Order at the top of query } @@ -941,6 +948,7 @@ macro_rules! for_stream_plan_nodes { , { Stream, Share } , { Stream, WatermarkFilter } , { Stream, TemporalJoin } + , { Stream, Values } } }; } 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 fe2c59da603eb..b7de4ca15d9e5 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,34 +14,31 @@ use std::fmt; +use fixedbitset::FixedBitSet; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; -use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, LogicalRowIdGen, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamRowIdGen { pub base: PlanBase, - input: PlanRef, - row_id_index: usize, + logical: LogicalRowIdGen, } impl StreamRowIdGen { - pub fn new(input: PlanRef, row_id_index: usize) -> Self { + pub fn new(logical: LogicalRowIdGen) -> Self { + let watermark_columns = FixedBitSet::with_capacity(logical.schema().len()); let base = PlanBase::new_stream( - input.ctx(), - input.schema().clone(), - input.logical_pk().to_vec(), - input.functional_dependency().clone(), - input.distribution().clone(), - input.append_only(), - input.watermark_columns().clone(), + logical.ctx(), + logical.schema().clone(), + logical.logical_pk().to_vec(), + logical.functional_dependency().clone(), + logical.distribution().clone(), + logical.append_only(), + watermark_columns, ); - Self { - base, - input, - row_id_index, - } + Self { base, logical } } } @@ -50,18 +47,20 @@ impl fmt::Display for StreamRowIdGen { write!( f, "StreamRowIdGen {{ row_id_index: {} }}", - self.row_id_index + self.logical.row_id_index() ) } } impl PlanTreeNodeUnary for StreamRowIdGen { fn input(&self) -> PlanRef { - self.input.clone() + self.logical.input() } fn clone_with_input(&self, input: PlanRef) -> Self { - Self::new(input, self.row_id_index) + Self::new( + self.logical.clone_with_input(input), + ) } } @@ -72,7 +71,7 @@ impl StreamNode for StreamRowIdGen { use risingwave_pb::stream_plan::*; ProstStreamNode::RowIdGen(RowIdGenNode { - row_id_index: self.row_id_index as _, + row_id_index: self.logical.row_id_index() as _, }) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs new file mode 100644 index 0000000000000..ad66e79282918 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/stream_values.rs @@ -0,0 +1,94 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt; + +use fixedbitset::FixedBitSet; +use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::values_node::ExprTuple; +use risingwave_pb::stream_plan::ValuesNode; + +use super::{ExprRewritable, LogicalValues, PlanBase, StreamNode}; +use crate::expr::{Expr, ExprImpl}; +use crate::optimizer::property::Distribution; +use crate::stream_fragmenter::BuildFragmentGraphState; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct StreamValues { + pub base: PlanBase, + logical: LogicalValues, +} + +impl_plan_tree_node_for_leaf! { StreamValues } + +impl StreamValues { + pub fn new(logical: LogicalValues) -> Self { + Self::with_dist(logical, Distribution::Single) + } + + pub fn with_dist(logical: LogicalValues, dist: Distribution) -> Self { + let ctx = logical.ctx(); + let mut watermark_columns = FixedBitSet::with_capacity(logical.schema().len()); + watermark_columns.set(0, true); + let base = PlanBase::new_stream( + ctx, + logical.schema().clone(), + logical.logical_pk().to_vec(), + logical.functional_dependency().clone(), + dist, + false, + watermark_columns, + ); + Self { base, logical } + } + + pub fn logical(&self) -> &LogicalValues { + &self.logical + } + + fn row_to_protobuf(&self, row: &[ExprImpl]) -> ExprTuple { + let cells = row.iter().map(|x| x.to_expr_proto()).collect(); + ExprTuple { cells } + } +} + +impl fmt::Display for StreamValues { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("StreamValues") + .field("rows", &self.logical.rows()) + .finish() + } +} + +impl StreamNode for StreamValues { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { + ProstStreamNode::Values(ValuesNode { + tuples: self + .logical + .rows() + .iter() + .map(|row| self.row_to_protobuf(row)) + .collect(), + fields: self + .logical + .schema() + .fields() + .iter() + .map(|f| f.to_prost()) + .collect(), + }) + } +} + +impl ExprRewritable for StreamValues {} diff --git a/src/stream/src/from_proto/mod.rs b/src/stream/src/from_proto/mod.rs index 818d25b6e2698..7ea2e2acb8c14 100644 --- a/src/stream/src/from_proto/mod.rs +++ b/src/stream/src/from_proto/mod.rs @@ -105,7 +105,7 @@ macro_rules! build_executor { <$data_type>::new_boxed_executor($source, node, $store, $stream).await }, )* - NodeBody::Exchange(_) | NodeBody::DeltaIndexJoin(_) => unreachable!() + NodeBody::Exchange(_) | NodeBody::DeltaIndexJoin(_) | NodeBody::Values(_) => unreachable!() } } } diff --git a/src/stream/src/from_proto/values.rs b/src/stream/src/from_proto/values.rs new file mode 100644 index 0000000000000..e69de29bb2d1d From 59edb429eb9b3b73c004f9cf9a029f3ad36fe945 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Thu, 23 Mar 2023 10:52:41 -0400 Subject: [PATCH 02/17] runable Signed-off-by: Clearlove --- proto/stream_plan.proto | 1 + src/frontend/src/lib.rs | 1 + .../src/optimizer/plan_node/logical_values.rs | 48 +++- .../src/optimizer/plan_node/stream_values.rs | 3 +- src/frontend/src/stream_fragmenter/mod.rs | 5 + src/meta/src/model/stream.rs | 5 +- src/stream/src/executor/mod.rs | 1 + src/stream/src/executor/values.rs | 247 ++++++++++++++++++ src/stream/src/from_proto/mod.rs | 5 +- src/stream/src/from_proto/values.rs | 65 +++++ src/stream/src/lib.rs | 1 + 11 files changed, 370 insertions(+), 12 deletions(-) create mode 100644 src/stream/src/executor/values.rs diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 8ccfb3d715407..c8e064ec1ad74 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -641,6 +641,7 @@ enum FragmentTypeFlag { SINK = 4; NOW = 8; CHAIN_NODE = 16; + VALUES = 32; } // The environment associated with a stream plan diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 3e521d7b147be..40fe0eb67b3a3 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -31,6 +31,7 @@ #![feature(slice_internals)] #![feature(min_specialization)] #![feature(is_some_and)] +#![feature(extend_one)] #![recursion_limit = "256"] #[macro_use] diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index 38a1ac7853455..32af0aee6278d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -15,20 +15,23 @@ use std::sync::Arc; use std::{fmt, vec}; +use itertools::Itertools; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::Result; -use risingwave_common::types::DataType; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::util::iter_util::ZipEqFast; +use tracing::debug; use super::{ - BatchValues, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, - StreamValues, ToBatch, ToStream, LogicalRowIdGen, + BatchValues, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, + PredicatePushdown, StreamValues, ToBatch, ToStream, }; -use crate::expr::{Expr, ExprImpl, ExprRewriter}; +use crate::expr::{Expr, ExprImpl, ExprRewriter, Literal}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; -use crate::optimizer::property::FunctionalDependencySet; +use crate::optimizer::property::{FunctionalDependencySet, RequiredDist, Order}; use crate::utils::{ColIndexMapping, Condition}; /// `LogicalValues` builds rows according to a list of expressions @@ -54,6 +57,26 @@ impl LogicalValues { } } + /// Used only by `LogicalValues.rewrite_logical_for_stream, set the `_row_id` column as pk + fn new_with_pk( + rows: Vec>, + schema: Schema, + ctx: OptimizerContextRef, + pk_index: usize, + ) -> Self { + for exprs in &rows { + for (i, expr) in exprs.iter().enumerate() { + assert_eq!(schema.fields()[i].data_type(), expr.return_type()) + } + } + let functional_dependency = FunctionalDependencySet::new(schema.len()); + let base = PlanBase::new_logical(ctx, schema, vec![pk_index], functional_dependency); + Self { + rows: rows.into(), + base, + } + } + /// Create a [`LogicalValues`] node. Used by planner. pub fn create(rows: Vec>, schema: Schema, ctx: OptimizerContextRef) -> PlanRef { // No additional checks after binder. @@ -141,7 +164,7 @@ impl ToStream for LogicalValues { _ctx: &mut RewriteStreamContext, ) -> Result<(PlanRef, ColIndexMapping)> { let row_id_index = self.schema().len(); - let col_index_mapping = ColIndexMapping::identity(row_id_index); + let col_index_mapping = ColIndexMapping::identity_or_none(row_id_index, row_id_index + 1); let ctx = self.ctx().clone(); let mut schema = self.schema().clone(); schema.fields.push(Field { @@ -151,9 +174,16 @@ impl ToStream for LogicalValues { type_name: "int64".to_string(), }); let rows = self.rows().clone().to_owned(); - let logical_values = Self::create(rows, schema, ctx); - let logical_row_id_gen = LogicalRowIdGen::new(logical_values, row_id_index); - Ok((logical_row_id_gen.into(), col_index_mapping)) + let row_with_id = (0..rows.len()) + .into_iter() + .zip_eq_fast(rows.into_iter()) + .map(|(i, mut r)| { + r.extend_one(Literal::new(Some(ScalarImpl::Int64(i as i64)), DataType::Int64).into()); + r + }) + .collect_vec(); + let logical_values = Self::new_with_pk(row_with_id, schema, ctx, row_id_index); + Ok((logical_values.into(), col_index_mapping)) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs index ad66e79282918..c97c6ad34f1fc 100644 --- a/src/frontend/src/optimizer/plan_node/stream_values.rs +++ b/src/frontend/src/optimizer/plan_node/stream_values.rs @@ -40,7 +40,7 @@ impl StreamValues { pub fn with_dist(logical: LogicalValues, dist: Distribution) -> Self { let ctx = logical.ctx(); let mut watermark_columns = FixedBitSet::with_capacity(logical.schema().len()); - watermark_columns.set(0, true); + (0..(logical.schema().len()-1)).into_iter().for_each(|i| watermark_columns.set(i, true)); let base = PlanBase::new_stream( ctx, logical.schema().clone(), @@ -67,6 +67,7 @@ impl fmt::Display for StreamValues { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("StreamValues") .field("rows", &self.logical.rows()) + .field("schema", &self.logical.schema()) .finish() } } diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index 6457969dd51fe..7a07f318a8e1b 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -263,6 +263,11 @@ fn build_fragment( current_fragment.requires_singleton = true; } + NodeBody::Values(_) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::Values as u32; + current_fragment.requires_singleton = true; + } + _ => {} }; diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 9480a528fe619..7be05e521b64b 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -259,7 +259,10 @@ impl TableFragments { /// Returns barrier inject actor ids. pub fn barrier_inject_actor_ids(&self) -> Vec { Self::filter_actor_ids(self, |fragment_type_mask| { - (fragment_type_mask & (FragmentTypeFlag::Source as u32 | FragmentTypeFlag::Now as u32)) + (fragment_type_mask + & (FragmentTypeFlag::Source as u32 + | FragmentTypeFlag::Now as u32 + | FragmentTypeFlag::Values as u32)) != 0 }) } diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index 27e8d7583b371..f49d013671f75 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -93,6 +93,7 @@ mod union; mod watermark; mod watermark_filter; mod wrapper; +pub mod values; mod backfill; #[cfg(test)] diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs new file mode 100644 index 0000000000000..5e31f429ab988 --- /dev/null +++ b/src/stream/src/executor/values.rs @@ -0,0 +1,247 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::vec; + +use await_tree::InstrumentAwait; +use futures::StreamExt; +use futures_async_stream::try_stream; +use risingwave_common::array::{DataChunk, Op, StreamChunk}; +use risingwave_common::catalog::Schema; +use risingwave_common::ensure; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_expr::expr::BoxedExpression; +use tokio::sync::mpsc::UnboundedReceiver; + +use super::{ + Barrier, BoxedMessageStream, Executor, Message, PkIndices, PkIndicesRef, StreamExecutorError, +}; + +pub struct ValuesExecutor { + /// Receiver of barrier channel. + barrier_receiver: UnboundedReceiver, + + rows: vec::IntoIter>, + pk_indices: PkIndices, + identity: String, + schema: Schema, +} + +impl ValuesExecutor { + pub fn new( + rows: Vec>, + schema: Schema, + barrier_receiver: UnboundedReceiver, + executor_id: u64, + ) -> Self { + Self { + barrier_receiver, + rows: rows.into_iter(), + pk_indices: vec![schema.len()], + identity: format!("ValuesExecutor {:X}", executor_id), + schema, + } + } + + #[try_stream(ok = Message, error = StreamExecutorError)] + async fn into_stream(self) { + let Self { + mut barrier_receiver, + schema, + mut rows, + .. + } = self; + let barrier = barrier_receiver + .recv() + .instrument_await("values_executor_recv_first_barrier") + .await + .unwrap(); + + let emit = !barrier.is_resume(); + + yield Message::Barrier(barrier); + // If it's failover, do not evaluate rows (assume they have been yielded) + if emit { + let cardinality = schema.len(); + ensure!(cardinality > 0); + while !rows.is_empty() { + // We need a one row chunk rather than an empty chunk because constant + // expression's eval result is same size as input chunk + // cardinality. + let one_row_chunk = DataChunk::new_dummy(1); + + let chunk_size = 1024_usize.min(rows.len()); + let mut array_builders = schema.create_array_builders(chunk_size); + for row in rows.by_ref().take(chunk_size) { + for (expr, builder) in row.into_iter().zip_eq_fast(&mut array_builders) { + let out = expr.eval(&one_row_chunk).await?; + builder.append_array(&out); + } + } + + let columns: Vec<_> = array_builders + .into_iter() + .map(|b| b.finish().into()) + .collect(); + + let chunk = DataChunk::new(columns, chunk_size); + let ops = vec![Op::Insert; chunk_size]; + + let stream_chunk = StreamChunk::from_parts(ops, chunk); + yield Message::Chunk(stream_chunk); + } + } + + while let Some(barrier) = barrier_receiver.recv().await { + yield Message::Barrier(barrier); + } + } +} + +impl Executor for ValuesExecutor { + fn execute(self: Box) -> BoxedMessageStream { + self.into_stream().boxed() + } + + fn schema(&self) -> &Schema { + &self.schema + } + + fn pk_indices(&self) -> PkIndicesRef<'_> { + &self.pk_indices + } + + fn identity(&self) -> &str { + self.identity.as_str() + } +} + +#[cfg(test)] +mod tests { + use futures::StreamExt; + use risingwave_common::array; + use risingwave_common::array::{ + ArrayImpl, I16Array, I32Array, I64Array, StructArray, StructValue, + }; + use risingwave_common::catalog::{Field, Schema}; + use risingwave_common::types::{DataType, ScalarImpl}; + use risingwave_expr::expr::{BoxedExpression, LiteralExpression}; + use risingwave_pb::stream_plan::Dispatcher; + use tokio::sync::mpsc::unbounded_channel; + + use super::ValuesExecutor; + use crate::executor::{Barrier, Executor, Message, Mutation}; + + #[tokio::test] + async fn test_values() { + let (tx, barrier_receiver) = unbounded_channel(); + let value = StructValue::new(vec![Some(1.into()), Some(2.into()), Some(3.into())]); + let exprs = vec![ + Box::new(LiteralExpression::new( + DataType::Int16, + Some(ScalarImpl::Int16(1)), + )) as BoxedExpression, + Box::new(LiteralExpression::new( + DataType::Int32, + Some(ScalarImpl::Int32(2)), + )), + Box::new(LiteralExpression::new( + DataType::Int64, + Some(ScalarImpl::Int64(3)), + )), + Box::new(LiteralExpression::new( + DataType::new_struct( + vec![DataType::Int32, DataType::Int32, DataType::Int32], + vec![], + ), + Some(ScalarImpl::Struct(value)), + )) as BoxedExpression, + Box::new(LiteralExpression::new( + DataType::Int64, + Some(ScalarImpl::Int64(0)), + )) as BoxedExpression, + ]; + let fields = exprs + .iter() // for each column + .map(|col| Field::unnamed(col.return_type())) + .collect::>(); + let values_executor_struct = + ValuesExecutor::new(vec![exprs], Schema { fields }, barrier_receiver, 10005); + let mut values_executor = Box::new(values_executor_struct).execute(); + + // Init barrier + let first_message = Barrier::new_test_barrier(1).with_mutation(Mutation::Add { + adds: maplit::hashmap! { + 0 => vec![Dispatcher { + downstream_actor_id: vec![1], + ..Default::default() + }], + }, + splits: Default::default(), + }); + tx.send(first_message).unwrap(); + + assert!(matches!( + values_executor.next().await.unwrap().unwrap(), + Message::Barrier { .. } + )); + + // Consume the barrier + let values_msg = values_executor.next().await.unwrap().unwrap(); + + let result = values_msg + .into_chunk() + .unwrap() + .compact() + .data_chunk() + .to_owned(); + + let array: ArrayImpl = StructArray::from_slices( + &[true], + vec![ + array! { I32Array, [Some(1)] }.into(), + array! { I32Array, [Some(2)] }.into(), + array! { I32Array, [Some(3)] }.into(), + ], + vec![DataType::Int32, DataType::Int32, DataType::Int32], + ) + .into(); + + assert_eq!( + *result.column_at(0).array(), + array! {I16Array, [Some(1_i16)]}.into() + ); + assert_eq!( + *result.column_at(1).array(), + array! {I32Array, [Some(2)]}.into() + ); + assert_eq!( + *result.column_at(2).array(), + array! {I64Array, [Some(3)]}.into() + ); + assert_eq!(*result.column_at(3).array(), array); + assert_eq!( + *result.column_at(4).array(), + array! {I64Array, [Some(0)]}.into() + ); + + // ValueExecutor should simply forward following barriers + tx.send(Barrier::new_test_barrier(2)).unwrap(); + + assert!(matches!( + values_executor.next().await.unwrap().unwrap(), + Message::Barrier { .. } + )); + } +} diff --git a/src/stream/src/from_proto/mod.rs b/src/stream/src/from_proto/mod.rs index 7ea2e2acb8c14..337affb594f4f 100644 --- a/src/stream/src/from_proto/mod.rs +++ b/src/stream/src/from_proto/mod.rs @@ -44,6 +44,7 @@ mod top_n; mod top_n_appendonly; mod union; mod watermark_filter; +mod values; // import for submodules use itertools::Itertools; @@ -82,6 +83,7 @@ use self::union::*; use self::watermark_filter::WatermarkFilterBuilder; use crate::error::StreamResult; use crate::executor::{BoxedExecutor, Executor, ExecutorInfo}; +use crate::from_proto::values::ValuesExecutorBuilder; use crate::task::{ExecutorParams, LocalStreamManagerCore}; #[async_trait::async_trait] @@ -105,7 +107,7 @@ macro_rules! build_executor { <$data_type>::new_boxed_executor($source, node, $store, $stream).await }, )* - NodeBody::Exchange(_) | NodeBody::DeltaIndexJoin(_) | NodeBody::Values(_) => unreachable!() + NodeBody::Exchange(_) | NodeBody::DeltaIndexJoin(_) => unreachable!() } } } @@ -152,5 +154,6 @@ pub async fn create_executor( NodeBody::RowIdGen => RowIdGenExecutorBuilder, NodeBody::Now => NowExecutorBuilder, NodeBody::TemporalJoin => TemporalJoinExecutorBuilder, + NodeBody::Values => ValuesExecutorBuilder, } } diff --git a/src/stream/src/from_proto/values.rs b/src/stream/src/from_proto/values.rs index e69de29bb2d1d..71bb599e51f6e 100644 --- a/src/stream/src/from_proto/values.rs +++ b/src/stream/src/from_proto/values.rs @@ -0,0 +1,65 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use itertools::Itertools; +use risingwave_common::catalog::{Field, Schema}; +use risingwave_expr::expr::{build_from_prost}; +use risingwave_pb::stream_plan::ValuesNode; +use risingwave_storage::StateStore; +use tokio::sync::mpsc::unbounded_channel; + +use super::ExecutorBuilder; +use crate::error::StreamResult; +use crate::executor::values::ValuesExecutor; +use crate::executor::BoxedExecutor; +use crate::task::{ExecutorParams, LocalStreamManagerCore}; + +pub struct ValuesExecutorBuilder; + +#[async_trait::async_trait] +impl ExecutorBuilder for ValuesExecutorBuilder { + type Node = ValuesNode; + + async fn new_boxed_executor( + params: ExecutorParams, + node: &ValuesNode, + _store: impl StateStore, + stream: &mut LocalStreamManagerCore, + ) -> StreamResult { + let (sender, barrier_receiver) = unbounded_channel(); + stream + .context + .lock_barrier_manager() + .register_sender(params.actor_context.id, sender); + debug!("ValuesExecutor registers {} at lock barrier manager", params.actor_context.id); + let rows = node + .get_tuples() + .iter() + .map(|tuple| { + tuple + .get_cells() + .iter() + .map(|node| build_from_prost(node).unwrap()) + .collect_vec() + }) + .collect_vec(); + let schema = Schema::new(node.get_fields().iter().map(Field::from).collect_vec()); + Ok(Box::new(ValuesExecutor::new( + rows, + schema, + barrier_receiver, + params.executor_id, + ))) + } +} diff --git a/src/stream/src/lib.rs b/src/stream/src/lib.rs index aea2f732e72d3..e982487b41471 100644 --- a/src/stream/src/lib.rs +++ b/src/stream/src/lib.rs @@ -39,6 +39,7 @@ #![feature(btree_drain_filter)] #![feature(bound_map)] #![feature(iter_order_by)] +#![feature(exact_size_is_empty)] #[macro_use] extern crate tracing; From f8b40b4b98a758682560f0b94fddf991fad169a4 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Thu, 23 Mar 2023 12:30:41 -0400 Subject: [PATCH 03/17] comment Signed-off-by: Clearlove --- src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs | 5 ++++- src/frontend/src/optimizer/plan_node/logical_values.rs | 3 +-- src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs | 2 ++ src/frontend/src/optimizer/plan_node/stream_values.rs | 4 +++- src/stream/src/executor/values.rs | 5 ++++- src/stream/src/from_proto/values.rs | 5 +++-- 6 files changed, 17 insertions(+), 7 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs index 7976037076c6b..9930312968371 100644 --- a/src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs +++ b/src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs @@ -27,7 +27,7 @@ use crate::optimizer::plan_node::{ use crate::optimizer::property::FunctionalDependencySet; use crate::utils::{ColIndexMapping, Condition}; -/// `LogicalRowIdGen` builds rows according to a list of expressions +/// `LogicalRowIdGen` generates `_row_id` at `row_id_index` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalRowIdGen { pub base: PlanBase, @@ -36,6 +36,7 @@ pub struct LogicalRowIdGen { } impl LogicalRowIdGen { + /// Build a `LogicalRowIdGen` node given `input` and desired `row_id` index pub fn new(input: PlanRef, row_id: usize) -> Self { let schema = input.schema().clone(); let functional_dependency = FunctionalDependencySet::new(schema.len()); @@ -47,10 +48,12 @@ impl LogicalRowIdGen { } } + /// Build a `LogicalRowIdGen` node given `input` and desired `row_id` index, shall be `Ok` pub fn create(input: PlanRef, row_id: usize) -> Result { Ok(Self::new(input, row_id)) } + /// `row_id_index` for the node pub fn row_id_index(&self) -> usize { self.row_id_index } diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index 32af0aee6278d..75044e860318a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -20,7 +20,6 @@ use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::Result; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::iter_util::ZipEqFast; -use tracing::debug; use super::{ BatchValues, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, @@ -31,7 +30,7 @@ use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; -use crate::optimizer::property::{FunctionalDependencySet, RequiredDist, Order}; +use crate::optimizer::property::{FunctionalDependencySet}; use crate::utils::{ColIndexMapping, Condition}; /// `LogicalValues` builds rows according to a list of expressions 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 b7de4ca15d9e5..f5a45abe9bcc0 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 @@ -20,6 +20,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::{ExprRewritable, LogicalRowIdGen, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; +/// `StreamRowIdGen` holds a stream `PlanBase` and a `LogicalRowIdGen` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamRowIdGen { pub base: PlanBase, @@ -27,6 +28,7 @@ pub struct StreamRowIdGen { } impl StreamRowIdGen { + /// Create a `StreamRowIdGen` with intermediate `LogicalRowIdGen` pub fn new(logical: LogicalRowIdGen) -> Self { let watermark_columns = FixedBitSet::with_capacity(logical.schema().len()); let base = PlanBase::new_stream( diff --git a/src/frontend/src/optimizer/plan_node/stream_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs index c97c6ad34f1fc..0f3567daec0a6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_values.rs +++ b/src/frontend/src/optimizer/plan_node/stream_values.rs @@ -24,6 +24,7 @@ use crate::expr::{Expr, ExprImpl}; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; +/// `StreamValues` implements `LogicalValues.to_stream()` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamValues { pub base: PlanBase, @@ -33,11 +34,12 @@ pub struct StreamValues { impl_plan_tree_node_for_leaf! { StreamValues } impl StreamValues { + /// StreamValues should enforce `Distribution::Single` pub fn new(logical: LogicalValues) -> Self { Self::with_dist(logical, Distribution::Single) } - pub fn with_dist(logical: LogicalValues, dist: Distribution) -> Self { + fn with_dist(logical: LogicalValues, dist: Distribution) -> Self { let ctx = logical.ctx(); let mut watermark_columns = FixedBitSet::with_capacity(logical.schema().len()); (0..(logical.schema().len()-1)).into_iter().for_each(|i| watermark_columns.set(i, true)); diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 5e31f429ab988..8377a1ec1751b 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -28,8 +28,10 @@ use super::{ Barrier, BoxedMessageStream, Executor, Message, PkIndices, PkIndicesRef, StreamExecutorError, }; +/// Execute `values` in stream. As is a leaf, current workaround holds a `barrier_executor`. +/// May refractor with `BarrierRecvExecutor` in the near future. pub struct ValuesExecutor { - /// Receiver of barrier channel. + // Receiver of barrier channel. barrier_receiver: UnboundedReceiver, rows: vec::IntoIter>, @@ -39,6 +41,7 @@ pub struct ValuesExecutor { } impl ValuesExecutor { + /// Currently hard-code the `pk_indices` as the last column. pub fn new( rows: Vec>, schema: Schema, diff --git a/src/stream/src/from_proto/values.rs b/src/stream/src/from_proto/values.rs index 71bb599e51f6e..4892c7b2c8228 100644 --- a/src/stream/src/from_proto/values.rs +++ b/src/stream/src/from_proto/values.rs @@ -14,7 +14,7 @@ use itertools::Itertools; use risingwave_common::catalog::{Field, Schema}; -use risingwave_expr::expr::{build_from_prost}; +use risingwave_expr::expr::build_from_prost; use risingwave_pb::stream_plan::ValuesNode; use risingwave_storage::StateStore; use tokio::sync::mpsc::unbounded_channel; @@ -25,6 +25,8 @@ use crate::executor::values::ValuesExecutor; use crate::executor::BoxedExecutor; use crate::task::{ExecutorParams, LocalStreamManagerCore}; +/// Build a `ValuesExecutor` for stream. As is a leaf, current workaround registers a `sender` for this executor. +/// May refractor with `BarrierRecvExecutor` in the near future. pub struct ValuesExecutorBuilder; #[async_trait::async_trait] @@ -42,7 +44,6 @@ impl ExecutorBuilder for ValuesExecutorBuilder { .context .lock_barrier_manager() .register_sender(params.actor_context.id, sender); - debug!("ValuesExecutor registers {} at lock barrier manager", params.actor_context.id); let rows = node .get_tuples() .iter() From 2f22567be561c528d4a923f48829dff32d1275b6 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Thu, 23 Mar 2023 13:30:18 -0400 Subject: [PATCH 04/17] update planner test Signed-off-by: Clearlove --- .../planner_test/tests/testdata/array.yaml | 44 ++++++------- .../tests/testdata/array_access.yaml | 2 +- .../tests/testdata/basic_query.yaml | 3 + .../testdata/common_table_expressions.yaml | 18 +++++- .../planner_test/tests/testdata/explain.yaml | 14 ++--- .../planner_test/tests/testdata/expr.yaml | 42 ++++++------- .../planner_test/tests/testdata/insert.yaml | 2 +- .../tests/testdata/nexmark_source.yaml | 12 ++-- .../tests/testdata/pg_catalog.yaml | 4 +- .../planner_test/tests/testdata/share.yaml | 12 ++-- .../tests/testdata/struct_query.yaml | 4 +- .../planner_test/tests/testdata/subquery.yaml | 13 ++-- .../tests/testdata/subquery_expr.yaml | 34 +++++----- .../testdata/subquery_expr_correlated.yaml | 6 +- .../tests/testdata/time_window.yaml | 2 +- .../planner_test/tests/testdata/types.yaml | 12 ++-- .../planner_test/tests/testdata/union.yaml | 6 +- .../tests/testdata/watermark.yaml | 62 +++++++++---------- .../src/optimizer/plan_node/logical_values.rs | 1 - .../optimizer/plan_node/stream_row_id_gen.rs | 4 +- .../src/optimizer/plan_node/stream_values.rs | 1 - 21 files changed, 158 insertions(+), 140 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/array.yaml b/src/frontend/planner_test/tests/testdata/array.yaml index bc4bb13311ca9..72989cb0f325c 100644 --- a/src/frontend/planner_test/tests/testdata/array.yaml +++ b/src/frontend/planner_test/tests/testdata/array.yaml @@ -2,13 +2,13 @@ - sql: | values (ARRAY['foo', 'bar']); logical_plan: | - LogicalValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } + LogicalValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]] } batch_plan: | BatchValues { rows: [[ARRAY[foo, bar]:List { datatype: Varchar }]] } - sql: | values (ARRAY[1, 2+3, 4*5+1]); logical_plan: | - LogicalValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Int32 }] } } + LogicalValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]] } batch_plan: | BatchValues { rows: [[ARRAY[1, 5, 21]:List { datatype: Int32 }]] } - sql: | @@ -25,7 +25,7 @@ select ARRAY[null]; logical_plan: | LogicalProject { exprs: [Array(null:Varchar) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select ARRAY[]; binder_error: |- @@ -35,12 +35,12 @@ select ARRAY[]::int[]; logical_plan: | LogicalProject { exprs: [Array::List { datatype: Int32 } as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select ARRAY[]::int[][]; logical_plan: | LogicalProject { exprs: [Array::List { datatype: List { datatype: Int32 } } as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select ARRAY[]::int; binder_error: |- @@ -50,21 +50,21 @@ select array_cat(array[66], array[123]); logical_plan: | LogicalProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32)) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } batch_plan: | BatchValues { rows: [[ARRAY[66, 123]:List { datatype: Int32 }]] } - sql: | select array_cat(array[array[66]], array[233]); logical_plan: | LogicalProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32)) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } batch_plan: | BatchValues { rows: [[ARRAY[{66}, {233}]:List { datatype: List { datatype: Int32 } }]] } - sql: | select array_cat(array[233], array[array[66]]); logical_plan: | LogicalProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32))) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } batch_plan: | BatchValues { rows: [[ARRAY[{233}, {66}]:List { datatype: List { datatype: Int32 } }]] } - sql: | @@ -80,7 +80,7 @@ select array_append(array[66], 123); logical_plan: | LogicalProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } batch_plan: | BatchValues { rows: [[ARRAY[66, 123]:List { datatype: Int32 }]] } - sql: | @@ -91,12 +91,12 @@ select array_append(array[array[66]], array[233]); logical_plan: | LogicalProject { exprs: [ArrayAppend(Array(Array(66:Int32)), Array(233:Int32)) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select array_prepend(123, array[66]); logical_plan: | LogicalProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32)) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } batch_plan: | BatchValues { rows: [[ARRAY[123, 66]:List { datatype: Int32 }]] } - sql: | @@ -106,7 +106,7 @@ select array_prepend(array[233], array[array[66]]); logical_plan: | LogicalProject { exprs: [ArrayPrepend(Array(233:Int32), Array(Array(66:Int32))) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: string from/to varchar[] in implicit context sql: | values (array['a', 'b']), ('{c,' || 'd}'); @@ -121,62 +121,62 @@ select ('{c,' || 'd}')::varchar[]; logical_plan: | LogicalProject { exprs: [ConcatOp('{c,':Varchar, 'd}':Varchar)::List { datatype: Varchar } as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: unknown to varchar[] in implicit context sql: | values (array['a', 'b']), ('{c,d}'); logical_plan: | - LogicalValues { rows: [[Array('a':Varchar, 'b':Varchar)], ['{c,d}':Varchar::List { datatype: Varchar }]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } + LogicalValues { rows: [[Array('a':Varchar, 'b':Varchar)], ['{c,d}':Varchar::List { datatype: Varchar }]] } - name: unknown to varchar[] in assign context sql: | create table t (v1 varchar[]); insert into t values ('{c,d}'); logical_plan: | LogicalInsert { table: t } - └─LogicalValues { rows: [['{c,d}':Varchar::List { datatype: Varchar }]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } + └─LogicalValues { rows: [['{c,d}':Varchar::List { datatype: Varchar }]] } - name: unknown to varchar[] in explicit context sql: | select ('{c,d}')::varchar[]; logical_plan: | LogicalProject { exprs: ['{c,d}':Varchar::List { datatype: Varchar } as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: varchar[] to string in assign context sql: | create table t (v1 varchar); insert into t values (array['a', 'b']); logical_plan: | LogicalInsert { table: t } - └─LogicalValues { rows: [[Array('a':Varchar, 'b':Varchar)::Varchar]], schema: Schema { fields: [*VALUES*_0.column_0:Varchar] } } + └─LogicalValues { rows: [[Array('a':Varchar, 'b':Varchar)::Varchar]] } - name: varchar[] to string in explicit context sql: | select array['a', 'b']::varchar; logical_plan: | LogicalProject { exprs: [Array('a':Varchar, 'b':Varchar)::Varchar as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: bool[] cast in explicit context sql: | select array[1, true]::bool[]; logical_plan: | LogicalProject { exprs: [Array(1:Int32::Boolean, true:Boolean) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: bool[][] cast in explicit context sql: | select array[array[1, true]]::bool[][]; logical_plan: | LogicalProject { exprs: [Array(Array(1:Int32::Boolean, true:Boolean)) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: compare with null sql: | select null = array[1]; logical_plan: | LogicalProject { exprs: [(null:List { datatype: Int32 } = Array(1:Int32)) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: compare with literal sql: | select array[1] = '{1}'; logical_plan: | LogicalProject { exprs: [(Array(1:Int32) = '{1}':Varchar::List { datatype: Int32 }) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: compare with different type sql: | select array[1] = array[1.2]; diff --git a/src/frontend/planner_test/tests/testdata/array_access.yaml b/src/frontend/planner_test/tests/testdata/array_access.yaml index d1adfeb18d699..e0691ee5bccae 100644 --- a/src/frontend/planner_test/tests/testdata/array_access.yaml +++ b/src/frontend/planner_test/tests/testdata/array_access.yaml @@ -3,7 +3,7 @@ select (ARRAY['foo', 'bar'])[1]; logical_plan: | LogicalProject { exprs: [ArrayAccess(Array('foo':Varchar, 'bar':Varchar), 1:Int32) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | create table t(i int[]); select min(i) from t where i[1]>2; diff --git a/src/frontend/planner_test/tests/testdata/basic_query.yaml b/src/frontend/planner_test/tests/testdata/basic_query.yaml index b856a6050082c..9acbf51a3dad0 100644 --- a/src/frontend/planner_test/tests/testdata/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/basic_query.yaml @@ -2,6 +2,9 @@ - sql: values (11, 22), (33+(1+2), 44); batch_plan: | BatchValues { rows: [[11:Int32, 22:Int32], [36:Int32, 44:Int32]] } + stream_plan: | + StreamMaterialize { columns: [*VALUES*_0.column_0, *VALUES*_0.column_1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [*VALUES*_0.column_0, *VALUES*_0.column_1] } + └─StreamValues { rows: [[11:Int32, 22:Int32, 0:Int64], [(33:Int32 + (1:Int32 + 2:Int32)), 44:Int32, 1:Int64]] } - sql: select * from t binder_error: 'Catalog error: table or source not found: t' - sql: | diff --git a/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml b/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml index f4271f5e3fa8f..0198b45f1bc7e 100644 --- a/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml +++ b/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml @@ -54,7 +54,7 @@ | └─LogicalProject { exprs: [t1.x] } | └─LogicalScan { table: t1, columns: [t1.x, t1._row_id] } └─LogicalProject { exprs: [0.1:Decimal] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: Ensure we can bind CTE with aliases in both table name and columns sql: | create table t1 (x int, y int); @@ -68,3 +68,19 @@ └─LogicalShare { id = 2 } └─LogicalProject { exprs: [t1.x, t1.y] } └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } +- sql: | + create table t (v int, c varchar); + with dict(abbr, real) as (values ('cn', 'China'), ('us', 'United States')) select * from t join dict on t.c = dict.abbr; + logical_plan: | + LogicalProject { exprs: [t.v, t.c, *VALUES*_0.column_0, *VALUES*_0.column_1] } + └─LogicalJoin { type: Inner, on: (t.c = *VALUES*_0.column_0), output: all } + ├─LogicalScan { table: t, columns: [t.v, t.c, t._row_id] } + └─LogicalShare { id = 1 } + └─LogicalValues { rows: [['cn':Varchar, 'China':Varchar], ['us':Varchar, 'United States':Varchar]] } + stream_plan: | + StreamMaterialize { columns: [v, c, abbr, real, t._row_id(hidden), _row_id(hidden)], pk_columns: [t._row_id, _row_id, c], pk_conflict: "no check" } + └─StreamHashJoin { type: Inner, predicate: t.c = *VALUES*_0.column_0, output: [t.v, t.c, *VALUES*_0.column_0, *VALUES*_0.column_1, t._row_id, _row_id] } + ├─StreamExchange { dist: HashShard(t.c) } + | └─StreamTableScan { table: t, columns: [t.v, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(*VALUES*_0.column_0) } + └─StreamValues { rows: [['cn':Varchar, 'China':Varchar, 0:Int64], ['us':Varchar, 'United States':Varchar, 1:Int64]] } diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index 881d01f1c6a0a..5fc2788694a83 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -4,38 +4,38 @@ Begin: LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } Predicate Push Down: LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } Predicate Push Down: LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } Predicate Push Down: LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } Prune Columns: LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } Predicate Push Down: LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } Project Remove: apply TrivialProjectToValuesRule 1 time(s) - LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [1:Int32:Int32] } } + LogicalValues { rows: [[1:Int32]] } Inline Session Timezone: diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index fc134d82d6eb0..013ab5554349e 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -4,13 +4,13 @@ select int '1'; logical_plan: | LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: bind typed literal - bool sql: | SELECT bool 't' logical_plan: | LogicalProject { exprs: [true:Boolean] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | values(must_be_unimplemented_func(1)); binder_error: |- @@ -36,31 +36,31 @@ SELECT 1 between 2 and 3 logical_plan: | LogicalProject { exprs: [((1:Int32 >= 2:Int32) AND (1:Int32 <= 3:Int32)) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: null eq null sql: | SELECT null = null; logical_plan: | LogicalProject { exprs: [(null:Varchar = null:Varchar) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: null lt null sql: | SELECT null < null; logical_plan: | LogicalProject { exprs: [(null:Varchar < null:Varchar) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: bind is distinct from sql: | SELECT 1 IS DISTINCT FROM 2 logical_plan: | LogicalProject { exprs: [IsDistinctFrom(1:Int32, 2:Int32) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: bind is not distinct from sql: | SELECT 1 IS NOT DISTINCT FROM 2 logical_plan: | LogicalProject { exprs: [IsNotDistinctFrom(1:Int32, 2:Int32) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: in-list with aligned types sql: | SELECT 1::real in (3, 1.0, 2); @@ -346,27 +346,27 @@ select 1 < SOME(null); logical_plan: | LogicalProject { exprs: [Some((1:Int32 < null:List { datatype: Int32 })) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select 1 < ANY(null); logical_plan: | LogicalProject { exprs: [Some((1:Int32 < null:List { datatype: Int32 })) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select 1 < ALL(null); logical_plan: | LogicalProject { exprs: [All((1:Int32 < null:List { datatype: Int32 })) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select 1 < ALL('{2,3}'); logical_plan: | LogicalProject { exprs: [All((1:Int32 < '{2,3}':Varchar::List { datatype: Int32 })) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select 1 < SOME(null::integer[]); logical_plan: | LogicalProject { exprs: [Some((1:Int32 < null:List { datatype: Int32 })) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select 1 < SOME(null::varchar[]); binder_error: |- @@ -391,20 +391,20 @@ select 1 < SOME(array[null]::integer[]); logical_plan: | LogicalProject { exprs: [Some((1:Int32 < Array(null:Int32))) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select 1 < SOME(array[1, 2]); logical_plan: | LogicalProject { exprs: [Some((1:Int32 < Array(1:Int32, 2:Int32))) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select 1 < SOME((select array[1]) || array[2]); logical_plan: | LogicalProject { exprs: [Some((1:Int32 < ArrayCat($expr1, Array(2:Int32)))) as $expr2] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + ├─LogicalValues { rows: [[]] } └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } batch_plan: | BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10035, ARRAY[2]:List { datatype: Int32 }))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } @@ -414,20 +414,20 @@ select 1 < ALL(array[null]::integer[]); logical_plan: | LogicalProject { exprs: [All((1:Int32 < Array(null:Int32))) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select 1 < ALL(array[1, 2]); logical_plan: | LogicalProject { exprs: [All((1:Int32 < Array(1:Int32, 2:Int32))) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | select 1 < ALL((select array[1]) || array[2]); logical_plan: | LogicalProject { exprs: [All((1:Int32 < ArrayCat($expr1, Array(2:Int32)))) as $expr2] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + ├─LogicalValues { rows: [[]] } └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } batch_plan: | BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10035, ARRAY[2]:List { datatype: Int32 }))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } @@ -519,7 +519,7 @@ select ARRAY[1, null] t; logical_plan: | LogicalProject { exprs: [Array(1:Int32, null:Int32) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - name: regression (#7641) -fuzzing test failed at Bind error,types Boolean and Varchar cannot be matched sql: | select false >= 'LN1O0QP1yi' NOT IN (md5('4SeUPZhUbH')) diff --git a/src/frontend/planner_test/tests/testdata/insert.yaml b/src/frontend/planner_test/tests/testdata/insert.yaml index 423f8e0e97f7e..5e85c942707ce 100644 --- a/src/frontend/planner_test/tests/testdata/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/insert.yaml @@ -205,7 +205,7 @@ logical_plan: | LogicalProject { exprs: [*VALUES*_0.column_0, *VALUES*_0.column_1, *VALUES*_0.column_0, (*VALUES*_0.column_0 + *VALUES*_0.column_1) as $expr1] } └─LogicalInsert { table: t, returning: true } - └─LogicalValues { rows: [[0:Int32, 1:Int32], [1:Int32, 2:Int32]], schema: Schema { fields: [*VALUES*_0.column_0:Int32, *VALUES*_0.column_1:Int32] } } + └─LogicalValues { rows: [[0:Int32, 1:Int32], [1:Int32, 2:Int32]] } batch_plan: | BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [*VALUES*_0.column_0, *VALUES*_0.column_1, *VALUES*_0.column_0, (*VALUES*_0.column_0 + *VALUES*_0.column_1) as $expr1] } diff --git a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml index 9e2b78cf7450f..1b680e26a185d 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -344,7 +344,7 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [auction, count, window_start] } - | └─StreamShare { id = 11 } + | └─StreamShare { id = 12 } | └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -356,7 +356,7 @@ └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [auction, window_start, count] } - └─StreamShare { id = 11 } + └─StreamShare { id = 12 } └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -468,7 +468,7 @@ └─StreamHashJoin { type: Inner, predicate: price = max(price), output: all } ├─StreamExchange { dist: HashShard(price) } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } - | └─StreamShare { id = 4 } + | └─StreamShare { id = 5 } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -477,7 +477,7 @@ └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [max(price), count] } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } - └─StreamShare { id = 4 } + └─StreamShare { id = 5 } └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -1291,7 +1291,7 @@ | | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } | └─StreamExchange { dist: HashShard(auction) } | └─StreamProject { exprs: [auction, _row_id] } - | └─StreamShare { id = 8 } + | └─StreamShare { id = 10 } | └─StreamProject { exprs: [auction, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -1302,7 +1302,7 @@ └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count] } └─StreamExchange { dist: HashShard(auction) } └─StreamProject { exprs: [auction, _row_id] } - └─StreamShare { id = 8 } + └─StreamShare { id = 10 } └─StreamProject { exprs: [auction, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/pg_catalog.yaml b/src/frontend/planner_test/tests/testdata/pg_catalog.yaml index 06ba337441bfd..4e85889110d17 100644 --- a/src/frontend/planner_test/tests/testdata/pg_catalog.yaml +++ b/src/frontend/planner_test/tests/testdata/pg_catalog.yaml @@ -25,7 +25,7 @@ logical_plan: | LogicalProject { exprs: [pg_user.name] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + ├─LogicalValues { rows: [[]] } └─LogicalProject { exprs: [pg_user.name] } └─LogicalFilter { predicate: (1:Int32 = pg_user.usesysid) } └─LogicalScan { table: pg_user, columns: [pg_user.usesysid, pg_user.name, pg_user.usecreatedb, pg_user.usesuper, pg_user.passwd] } @@ -39,6 +39,6 @@ select 'pg_namespace'::regclass logical_plan: | LogicalProject { exprs: [2:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } batch_plan: | BatchValues { rows: [[2:Int32]] } diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index 4d06322b550b7..5d85e0f9a1193 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -43,7 +43,7 @@ ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } | └─StreamFilter { predicate: (initial_bid = 1:Int32) } - | └─StreamShare { id = 6 } + | └─StreamShare { id = 7 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 10 } @@ -51,7 +51,7 @@ └─StreamExchange { dist: HashShard(id) } └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } - └─StreamShare { id = 6 } + └─StreamShare { id = 7 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 10 } @@ -115,7 +115,7 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [auction, count, window_start] } - | └─StreamShare { id = 11 } + | └─StreamShare { id = 12 } | └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -127,7 +127,7 @@ └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [auction, window_start, count] } - └─StreamShare { id = 11 } + └─StreamShare { id = 12 } └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -192,13 +192,13 @@ └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id] } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } - | └─StreamShare { id = 4 } + | └─StreamShare { id = 5 } | └─StreamProject { exprs: [id, _row_id] } | └─StreamRowIdGen { row_id_index: 10 } | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } └─StreamExchange { dist: HashShard(id) } └─StreamProject { exprs: [id, _row_id] } - └─StreamShare { id = 4 } + └─StreamShare { id = 5 } └─StreamProject { exprs: [id, _row_id] } └─StreamRowIdGen { row_id_index: 10 } └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/struct_query.yaml b/src/frontend/planner_test/tests/testdata/struct_query.yaml index 3ba6dbe2fcb8c..4bb02dc383177 100644 --- a/src/frontend/planner_test/tests/testdata/struct_query.yaml +++ b/src/frontend/planner_test/tests/testdata/struct_query.yaml @@ -341,7 +341,7 @@ insert into s values (1,2,(1,2,(1,2,null))); logical_plan: | LogicalInsert { table: s } - └─LogicalValues { rows: [[1:Int32, 2:Int32, Row(1:Int32, 2:Int32, Row(1:Int32, 2:Int32, null:Int32))]], schema: Schema { fields: [*VALUES*_0.column_0:Int32, *VALUES*_0.column_1:Int32, *VALUES*_0.column_2:Struct(StructType { fields: [Int32, Int32, Struct(StructType { fields: [Int32, Int32, Int32], field_names: ["v1", "v2", "v3"] })], field_names: ["v1", "v2", "v3"] })] } } + └─LogicalValues { rows: [[1:Int32, 2:Int32, Row(1:Int32, 2:Int32, Row(1:Int32, 2:Int32, null:Int32))]] } create_table_with_connector: row_format: protobuf name: s @@ -399,4 +399,4 @@ select CASE WHEN false THEN ROW(0, INTERVAL '1') WHEN true THEN ROW(1.1, INTERVAL '1') ELSE ROW(1, INTERVAL '1') END; logical_plan: | LogicalProject { exprs: [Case(false:Boolean, Row(0:Int32::Decimal, '00:00:01':Interval), true:Boolean, Row(1.1:Decimal, '00:00:01':Interval), Row(1:Int32::Decimal, '00:00:01':Interval)) as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } diff --git a/src/frontend/planner_test/tests/testdata/subquery.yaml b/src/frontend/planner_test/tests/testdata/subquery.yaml index 6b66130ad691c..2cb8bde0b2b5d 100644 --- a/src/frontend/planner_test/tests/testdata/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery.yaml @@ -215,7 +215,7 @@ └─LogicalProject { exprs: [CorrelatedInputRef { index: 0, correlated_id: 1 } as $expr2] } └─LogicalAgg { group_key: [$expr1], aggs: [] } └─LogicalProject { exprs: [CorrelatedInputRef { index: 0, correlated_id: 1 } as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(auction.date_time, auction.date_time), output: all } ├─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: all } @@ -227,7 +227,7 @@ ├─LogicalAgg { group_key: [auction.date_time], aggs: [] } | └─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time] } | └─LogicalScan { table: auction, columns: [auction.date_time], predicate: IsNotNull(auction.date_time) } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } batch_plan: | BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: LeftSemi, predicate: auction.date_time IS NOT DISTINCT FROM auction.date_time, output: all } @@ -247,8 +247,9 @@ | └─BatchScan { table: auction, columns: [auction.date_time], distribution: SomeShard } └─BatchValues { rows: [[]] } stream_error: |- - Feature is not yet implemented: Stream values executor is unimplemented! - No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml + Not supported: streaming nested-loop join + HINT: The non-equal join in the query requires a nested-loop join executor, which could be very expensive to run. Consider rewriting the query to use dynamic filter as a substitute if possible. + See also: https://github.com/risingwavelabs/rfcs/blob/main/rfcs/0033-dynamic-filter.md - sql: | CREATE TABLE t (v int); SELECT 1 FROM t AS t_inner WHERE EXISTS ( SELECT 1 HAVING t_inner.v > 1); @@ -260,13 +261,13 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 0, correlated_id: 1 } > 1:Int32) } └─LogicalAgg { aggs: [] } └─LogicalProject { exprs: [] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftSemi, on: true, output: all } ├─LogicalScan { table: t, output_columns: [], required_columns: [t.v], predicate: (t.v > 1:Int32) } └─LogicalAgg { aggs: [] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | create table a (a1 int, a2 int); create table b (b1 int, b2 int); diff --git a/src/frontend/planner_test/tests/testdata/subquery_expr.yaml b/src/frontend/planner_test/tests/testdata/subquery_expr.yaml index 6ca7ba07b2e02..e98ea69db858e 100644 --- a/src/frontend/planner_test/tests/testdata/subquery_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery_expr.yaml @@ -4,13 +4,13 @@ logical_plan: | LogicalProject { exprs: [1:Int32] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + ├─LogicalValues { rows: [[]] } └─LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } optimized_logical_plan_for_batch: | LogicalJoin { type: LeftOuter, on: true, output: all } - ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - └─LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [1:Int32:Int32] } } + ├─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[1:Int32]] } - sql: | create table t(x int); select (select x from t), 1 from t; @@ -81,11 +81,11 @@ LogicalProject { exprs: [t.x, 1:Int32] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } ├─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - | ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + | ├─LogicalValues { rows: [[]] } | └─LogicalProject { exprs: [t.x] } | └─LogicalScan { table: t, columns: [t.x, t._row_id] } └─LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } optimizer_error: 'internal error: Scalar subquery might produce more than one row.' - sql: | create table t(x int); @@ -119,13 +119,13 @@ logical_plan: | LogicalProject { exprs: [1:Int32] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + ├─LogicalValues { rows: [[]] } └─LogicalProject { exprs: [t.x] } └─LogicalScan { table: t, columns: [t.x, t._row_id] } optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftSemi, on: true, output: all } - ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + ├─LogicalValues { rows: [[]] } └─LogicalScan { table: t, columns: [] } - sql: | create table t(x int); @@ -133,13 +133,13 @@ logical_plan: | LogicalProject { exprs: [1:Int32] } └─LogicalApply { type: LeftAnti, on: true, correlated_id: 1 } - ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + ├─LogicalValues { rows: [[]] } └─LogicalProject { exprs: [t.x] } └─LogicalScan { table: t, columns: [t.x, t._row_id] } optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftAnti, on: true, output: all } - ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + ├─LogicalValues { rows: [[]] } └─LogicalScan { table: t, columns: [] } - sql: | create table t1(x int); @@ -177,8 +177,8 @@ LogicalProject { exprs: [1:Int32] } └─LogicalFilter { predicate: (1:Int32 > 0:Int32) } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - └─LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [*VALUES*_0.column_0:Int32] } } + ├─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[1:Int32]] } - sql: | select 1 where (not exists (values (1))) and (1>0 or exists (values (1))) logical_plan: | @@ -186,23 +186,23 @@ └─LogicalFilter { predicate: ((1:Int32 > 0:Int32) OR $expr1) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true } ├─LogicalApply { type: LeftAnti, on: true, correlated_id: 1 } - | ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - | └─LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [*VALUES*_0.column_0:Int32] } } + | ├─LogicalValues { rows: [[]] } + | └─LogicalValues { rows: [[1:Int32]] } └─LogicalProject { exprs: [(count >= 1:Int32) as $expr1] } └─LogicalAgg { aggs: [count] } - └─LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [*VALUES*_1.column_0:Int32] } } + └─LogicalValues { rows: [[1:Int32]] } - sql: | select a + 1, b::varchar, c from (values (1, 2, 3), (4, 5, 6)) t(a, b, c); logical_plan: | LogicalProject { exprs: [(*VALUES*_0.column_0 + 1:Int32) as $expr1, *VALUES*_0.column_1::Varchar as $expr2, *VALUES*_0.column_2] } - └─LogicalValues { rows: [[1:Int32, 2:Int32, 3:Int32], [4:Int32, 5:Int32, 6:Int32]], schema: Schema { fields: [*VALUES*_0.column_0:Int32, *VALUES*_0.column_1:Int32, *VALUES*_0.column_2:Int32] } } + └─LogicalValues { rows: [[1:Int32, 2:Int32, 3:Int32], [4:Int32, 5:Int32, 6:Int32]] } - sql: | select sum(a), max(b + c + 10), string_agg(c::varchar || '~', ',') from (values (1, 2, 3), (4, 5, 6)) as t(a, b, c); logical_plan: | LogicalProject { exprs: [sum(*VALUES*_0.column_0), max($expr1), string_agg($expr2, ',':Varchar)] } └─LogicalAgg { aggs: [sum(*VALUES*_0.column_0), max($expr1), string_agg($expr2, ',':Varchar)] } └─LogicalProject { exprs: [*VALUES*_0.column_0, ((*VALUES*_0.column_1 + *VALUES*_0.column_2) + 10:Int32) as $expr1, ConcatOp(*VALUES*_0.column_2::Varchar, '~':Varchar) as $expr2, ',':Varchar] } - └─LogicalValues { rows: [[1:Int32, 2:Int32, 3:Int32], [4:Int32, 5:Int32, 6:Int32]], schema: Schema { fields: [*VALUES*_0.column_0:Int32, *VALUES*_0.column_1:Int32, *VALUES*_0.column_2:Int32] } } + └─LogicalValues { rows: [[1:Int32, 2:Int32, 3:Int32], [4:Int32, 5:Int32, 6:Int32]] } - sql: | select 1 + (select 2 from t); binder_error: 'Catalog error: table or source not found: t' diff --git a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml index d45221df79b6c..85a137fbbec5e 100644 --- a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml @@ -40,7 +40,7 @@ └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true } ├─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } └─LogicalProject { exprs: [CorrelatedInputRef { index: 1, correlated_id: 1 } as $expr1] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -64,7 +64,7 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: t3, columns: [t3.x, t3.y, t3._row_id] } └─LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -679,7 +679,7 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalAgg { group_key: [t1.a], aggs: [] } | └─LogicalScan { table: t1, columns: [t1.a] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: | create table t1(x int, y int); create table t2(x int, y int); diff --git a/src/frontend/planner_test/tests/testdata/time_window.yaml b/src/frontend/planner_test/tests/testdata/time_window.yaml index db258b5eca7a6..e81d6b72fdc8b 100644 --- a/src/frontend/planner_test/tests/testdata/time_window.yaml +++ b/src/frontend/planner_test/tests/testdata/time_window.yaml @@ -206,7 +206,7 @@ LogicalProject { exprs: [*VALUES*_0.column_0, $expr1, $expr2] } └─LogicalProject { exprs: [*VALUES*_0.column_0, TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) as $expr1, (TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) + '00:00:10':Interval) as $expr2] } └─LogicalShare { id = 1 } - └─LogicalValues { rows: [['2020-01-01 12:00:00':Timestamp]], schema: Schema { fields: [*VALUES*_0.column_0:Timestamp] } } + └─LogicalValues { rows: [['2020-01-01 12:00:00':Timestamp]] } batch_plan: | BatchProject { exprs: [*VALUES*_0.column_0, TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) as $expr1, (TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) + '00:00:10':Interval) as $expr2] } └─BatchValues { rows: [['2020-01-01 12:00:00':Timestamp]] } diff --git a/src/frontend/planner_test/tests/testdata/types.yaml b/src/frontend/planner_test/tests/testdata/types.yaml index 34d561c046fc2..6868a4558c17b 100644 --- a/src/frontend/planner_test/tests/testdata/types.yaml +++ b/src/frontend/planner_test/tests/testdata/types.yaml @@ -5,19 +5,19 @@ No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml - sql: values ('1'::float); logical_plan: | - LogicalValues { rows: [[1:Float64]], schema: Schema { fields: [*VALUES*_0.column_0:Float64] } } + LogicalValues { rows: [[1:Float64]] } - sql: values ('1'::float(24)); logical_plan: | - LogicalValues { rows: [[1:Float32]], schema: Schema { fields: [*VALUES*_0.column_0:Float32] } } + LogicalValues { rows: [[1:Float32]] } - sql: values ('1'::float(25)); logical_plan: | - LogicalValues { rows: [[1:Float64]], schema: Schema { fields: [*VALUES*_0.column_0:Float64] } } + LogicalValues { rows: [[1:Float64]] } - sql: values ('1'::float(53)); logical_plan: | - LogicalValues { rows: [[1:Float64]], schema: Schema { fields: [*VALUES*_0.column_0:Float64] } } + LogicalValues { rows: [[1:Float64]] } - sql: values (''::timestamp with time zone); logical_plan: | - LogicalValues { rows: [['':Varchar::Timestamptz]], schema: Schema { fields: [*VALUES*_0.column_0:Timestamptz] } } + LogicalValues { rows: [['':Varchar::Timestamptz]] } - sql: values (''::time with time zone); binder_error: |- Feature is not yet implemented: unsupported data type: TIME WITH TIME ZONE @@ -25,7 +25,7 @@ - sql: select '1'::iNt2; logical_plan: | LogicalProject { exprs: [1:Int16] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[]] } - sql: select ''::"iNt2"; binder_error: |- Feature is not yet implemented: unsupported data type: "iNt2" diff --git a/src/frontend/planner_test/tests/testdata/union.yaml b/src/frontend/planner_test/tests/testdata/union.yaml index 0dfee81d93421..2243512f844c9 100644 --- a/src/frontend/planner_test/tests/testdata/union.yaml +++ b/src/frontend/planner_test/tests/testdata/union.yaml @@ -206,20 +206,20 @@ - sql: | select 1 union all select 1 optimized_logical_plan_for_batch: | - LogicalValues { rows: [[1:Int32], [1:Int32]], schema: Schema { fields: [1:Int32:Int32] } } + LogicalValues { rows: [[1:Int32], [1:Int32]] } batch_plan: | BatchValues { rows: [[1:Int32], [1:Int32]] } - sql: | select 1 union all select 2 union all select 3 union all select 4 union all select 5 optimized_logical_plan_for_batch: | - LogicalValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32]], schema: Schema { fields: [1:Int32:Int32] } } + LogicalValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32]] } batch_plan: | BatchValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32]] } - sql: | select 1 union select 2 union select 3 union select 4 union select 5 union select 5 optimized_logical_plan_for_batch: | LogicalAgg { group_key: [1:Int32], aggs: [] } - └─LogicalValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32], [5:Int32]], schema: Schema { fields: [1:Int32:Int32] } } + └─LogicalValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32], [5:Int32]] } batch_plan: | BatchExchange { order: [], dist: Single } └─BatchHashAgg { group_key: [1:Int32], aggs: [] } diff --git a/src/frontend/planner_test/tests/testdata/watermark.yaml b/src/frontend/planner_test/tests/testdata/watermark.yaml index 020d8e3a4c0a3..53a026d9649d5 100644 --- a/src/frontend/planner_test/tests/testdata/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/watermark.yaml @@ -7,20 +7,20 @@ LogicalProject { exprs: [(v1 - '00:00:02':Interval) as $expr1] } └─LogicalSource { source: t, columns: [v1, _row_id], time_range: [(Unbounded, Unbounded)] } stream_plan: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [v1] } + StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1, _row_id] } └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } └─StreamSource { source: "t", columns: ["v1", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [v1] } + StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } materialized table: 4294967294 StreamExchange Hash([1]) from 1 Fragment 1 - StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1, _row_id], output_watermarks: [$expr1] } + StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1, _row_id] } StreamRowIdGen { row_id_index: 1 } StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } StreamSource { source: "t", columns: ["v1", "_row_id"] } @@ -32,7 +32,7 @@ sql: | explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only with (connector = 'kafka', kafka.topic = 'kafka_3_partition_topic', kafka.brokers = '127.0.0.1:1234', kafka.scan.startup.mode='earliest') ROW FORMAT JSON; explain_output: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [v1] } + StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } @@ -42,7 +42,7 @@ sql: | explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only; explain_output: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [v1] } + StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } └─StreamExchange { dist: HashShard(_row_id) } └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } @@ -53,9 +53,9 @@ create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select count(v2) from t group by ts, v1; stream_plan: | - StreamMaterialize { columns: [count, t.ts(hidden), t.v1(hidden)], pk_columns: [t.ts, t.v1], pk_conflict: "no check", watermark_columns: [t.ts(hidden)] } - └─StreamProject { exprs: [count(t.v2), t.ts, t.v1], output_watermarks: [t.ts] } - └─StreamAppendOnlyHashAgg { group_key: [t.ts, t.v1], aggs: [count(t.v2), count], output_watermarks: [t.ts] } + StreamMaterialize { columns: [count, t.ts(hidden), t.v1(hidden)], pk_columns: [t.ts, t.v1], pk_conflict: "no check" } + └─StreamProject { exprs: [count(t.v2), t.ts, t.v1] } + └─StreamAppendOnlyHashAgg { group_key: [t.ts, t.v1], aggs: [count(t.v2), count] } └─StreamExchange { dist: HashShard(t.ts, t.v1) } └─StreamTableScan { table: t, columns: [t.ts, t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: hash join @@ -64,8 +64,8 @@ create table t2 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select t1.ts as t1_ts, t2.ts as ts2, t1.v1 as t1_v1, t1.v2 as t1_v2, t2.v1 as t2_v1, t2.v2 as t2_v2 from t1, t2 where t1.ts = t2.ts; stream_plan: | - StreamMaterialize { columns: [t1_ts, ts2, t1_v1, t1_v2, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, t1_ts], pk_conflict: "no check", watermark_columns: [t1_ts, ts2] } - └─StreamAppendOnlyHashJoin { type: Inner, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t2.ts, t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } + StreamMaterialize { columns: [t1_ts, ts2, t1_v1, t1_v2, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, t1_ts], pk_conflict: "no check" } + └─StreamAppendOnlyHashJoin { type: Inner, predicate: t1.ts = t2.ts, output: [t1.ts, t2.ts, t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.ts) } | └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard(t2.ts) } @@ -76,13 +76,13 @@ create table t2 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select * from t1 Union all select * from t2; stream_plan: | - StreamMaterialize { columns: [ts, v1, v2, t1._row_id(hidden), null:Int64(hidden), 0:Int32(hidden)], pk_columns: [t1._row_id, null:Int64, 0:Int32], pk_conflict: "no check", watermark_columns: [ts] } - └─StreamUnion { all: true, output_watermarks: [t1.ts] } + StreamMaterialize { columns: [ts, v1, v2, t1._row_id(hidden), null:Int64(hidden), 0:Int32(hidden)], pk_columns: [t1._row_id, null:Int64, 0:Int32], pk_conflict: "no check" } + └─StreamUnion { all: true } ├─StreamExchange { dist: HashShard(t1._row_id, null:Int64, 0:Int32) } - | └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, null:Int64, 0:Int32], output_watermarks: [t1.ts] } + | └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, null:Int64, 0:Int32] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard(null:Int64, t2._row_id, 1:Int32) } - └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, null:Int64, t2._row_id, 1:Int32], output_watermarks: [t2.ts] } + └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, null:Int64, t2._row_id, 1:Int32] } └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: union sql: | @@ -90,32 +90,32 @@ create table t2 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select * from t1 Union select * from t2; stream_plan: | - StreamMaterialize { columns: [ts, v1, v2], pk_columns: [ts, v1, v2], pk_conflict: "no check", watermark_columns: [ts] } - └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2], output_watermarks: [t1.ts] } - └─StreamAppendOnlyHashAgg { group_key: [t1.ts, t1.v1, t1.v2], aggs: [count], output_watermarks: [t1.ts] } + StreamMaterialize { columns: [ts, v1, v2], pk_columns: [ts, v1, v2], pk_conflict: "no check" } + └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2] } + └─StreamAppendOnlyHashAgg { group_key: [t1.ts, t1.v1, t1.v2], aggs: [count] } └─StreamExchange { dist: HashShard(t1.ts, t1.v1, t1.v2) } - └─StreamUnion { all: true, output_watermarks: [t1.ts] } + └─StreamUnion { all: true } ├─StreamExchange { dist: HashShard(t1._row_id, null:Int64, 0:Int32) } - | └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, null:Int64, 0:Int32], output_watermarks: [t1.ts] } + | └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, null:Int64, 0:Int32] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard(null:Int64, t2._row_id, 1:Int32) } - └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, null:Int64, t2._row_id, 1:Int32], output_watermarks: [t2.ts] } + └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, null:Int64, t2._row_id, 1:Int32] } └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: tumble sql: | create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select * from tumble(t, ts, interval '3' minute); stream_plan: | - StreamMaterialize { columns: [ts, v1, v2, window_start, window_end, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check", watermark_columns: [ts, window_start, window_end] } - └─StreamProject { exprs: [t.ts, t.v1, t.v2, TumbleStart(t.ts, '00:03:00':Interval) as $expr1, (AtTimeZone((AtTimeZone(TumbleStart(t.ts, '00:03:00':Interval), 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '00:03:00':Interval) as $expr2, t._row_id], output_watermarks: [t.ts, $expr1, $expr2] } + StreamMaterialize { columns: [ts, v1, v2, window_start, window_end, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } + └─StreamProject { exprs: [t.ts, t.v1, t.v2, TumbleStart(t.ts, '00:03:00':Interval) as $expr1, (AtTimeZone((AtTimeZone(TumbleStart(t.ts, '00:03:00':Interval), 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '00:03:00':Interval) as $expr2, t._row_id] } └─StreamTableScan { table: t, columns: [t.ts, t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: hop all sql: | create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select * from hop(t, ts, interval '1' minute, interval '3' minute); stream_plan: | - StreamMaterialize { columns: [ts, v1, v2, window_start, window_end, t._row_id(hidden)], pk_columns: [t._row_id, window_start, window_end], pk_conflict: "no check", watermark_columns: [ts, window_start, window_end] } - └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [t.ts, t.v1, t.v2, window_start, window_end, t._row_id], output_watermarks: [t.ts, window_start, window_end] } + StreamMaterialize { columns: [ts, v1, v2, window_start, window_end, t._row_id(hidden)], pk_columns: [t._row_id, window_start, window_end], pk_conflict: "no check" } + └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [t.ts, t.v1, t.v2, window_start, window_end, t._row_id] } └─StreamFilter { predicate: IsNotNull(t.ts) } └─StreamTableScan { table: t, columns: [t.ts, t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: hop ts @@ -123,8 +123,8 @@ create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select ts from hop(t, ts, interval '1' minute, interval '3' minute); stream_plan: | - StreamMaterialize { columns: [ts, window_start(hidden), t._row_id(hidden)], pk_columns: [t._row_id, window_start], pk_conflict: "no check", watermark_columns: [ts, window_start(hidden)] } - └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [t.ts, window_start, t._row_id], output_watermarks: [t.ts, window_start] } + StreamMaterialize { columns: [ts, window_start(hidden), t._row_id(hidden)], pk_columns: [t._row_id, window_start], pk_conflict: "no check" } + └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [t.ts, window_start, t._row_id] } └─StreamFilter { predicate: IsNotNull(t.ts) } └─StreamTableScan { table: t, columns: [t.ts, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: hop start @@ -132,8 +132,8 @@ create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select window_end from hop(t, ts, interval '1' minute, interval '3' minute); stream_plan: | - StreamMaterialize { columns: [window_end, t._row_id(hidden)], pk_columns: [t._row_id, window_end], pk_conflict: "no check", watermark_columns: [window_end] } - └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [window_end, t._row_id], output_watermarks: [window_end] } + StreamMaterialize { columns: [window_end, t._row_id(hidden)], pk_columns: [t._row_id, window_end], pk_conflict: "no check" } + └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [window_end, t._row_id] } └─StreamFilter { predicate: IsNotNull(t.ts) } └─StreamTableScan { table: t, columns: [t.ts, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: hop end @@ -141,7 +141,7 @@ create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select window_start from hop(t, ts, interval '1' minute, interval '3' minute); stream_plan: | - StreamMaterialize { columns: [window_start, t._row_id(hidden)], pk_columns: [t._row_id, window_start], pk_conflict: "no check", watermark_columns: [window_start] } - └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [window_start, t._row_id], output_watermarks: [window_start] } + StreamMaterialize { columns: [window_start, t._row_id(hidden)], pk_columns: [t._row_id, window_start], pk_conflict: "no check" } + └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [window_start, t._row_id] } └─StreamFilter { predicate: IsNotNull(t.ts) } └─StreamTableScan { table: t, columns: [t.ts, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index 75044e860318a..4252e2f35cafa 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -94,7 +94,6 @@ impl fmt::Display for LogicalValues { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("LogicalValues") .field("rows", &self.rows) - .field("schema", &self.schema()) .finish() } } 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 f5a45abe9bcc0..98799fd07f71f 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 @@ -36,8 +36,8 @@ impl StreamRowIdGen { logical.schema().clone(), logical.logical_pk().to_vec(), logical.functional_dependency().clone(), - logical.distribution().clone(), - logical.append_only(), + logical.input().distribution().clone(), + logical.input().append_only(), watermark_columns, ); Self { base, logical } diff --git a/src/frontend/src/optimizer/plan_node/stream_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs index 0f3567daec0a6..15fa19f911a36 100644 --- a/src/frontend/src/optimizer/plan_node/stream_values.rs +++ b/src/frontend/src/optimizer/plan_node/stream_values.rs @@ -69,7 +69,6 @@ impl fmt::Display for StreamValues { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("StreamValues") .field("rows", &self.logical.rows()) - .field("schema", &self.logical.schema()) .finish() } } From 0e058588498793acfb9896ccd66404d9a328a972 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Thu, 23 Mar 2023 13:40:24 -0400 Subject: [PATCH 05/17] e2e tests Signed-off-by: Clearlove --- e2e_test/streaming/values.slt | 37 +++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) create mode 100644 e2e_test/streaming/values.slt diff --git a/e2e_test/streaming/values.slt b/e2e_test/streaming/values.slt new file mode 100644 index 0000000000000..c07ec20edc826 --- /dev/null +++ b/e2e_test/streaming/values.slt @@ -0,0 +1,37 @@ +statement ok +create materialized view mvb as values (233, 'risingwave'), (233, 'risingwave'); + +statement ok +flush; + +query IR +select * from mvb; +---- +233 risingwave +233 risingwave + +statement ok +create table t (v int, c varchar); + +statement ok +insert into t values (1, 'China'), (0, 'United States'); + +statement ok +create materialized view mv as +with dict(abbr, real) as (values ('cn', 'China'), ('us', 'United States')) +select * from t join dict on t.c = dict.real; + +query IRRR +select * from mv order by v; +---- +0 United States us United States +1 China cn China + +statement ok +drop materialized view mvb; + +statement ok +drop materialized view mv; + +statement ok +drop table t; From ad78363cf80903882060c0ec76705fa80ab19845 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Thu, 23 Mar 2023 13:55:39 -0400 Subject: [PATCH 06/17] check Signed-off-by: Clearlove --- .../src/optimizer/plan_node/logical_values.rs | 17 +++++++------- .../optimizer/plan_node/stream_row_id_gen.rs | 4 +--- .../src/optimizer/plan_node/stream_values.rs | 4 ++-- src/stream/src/executor/mod.rs | 2 +- src/stream/src/executor/values.rs | 23 +++++++++++++++---- src/stream/src/from_proto/mod.rs | 2 +- src/stream/src/from_proto/values.rs | 5 ++-- 7 files changed, 35 insertions(+), 22 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index 4252e2f35cafa..f20cf62457418 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -22,15 +22,15 @@ use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::iter_util::ZipEqFast; use super::{ - BatchValues, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, - PredicatePushdown, StreamValues, ToBatch, ToStream, + BatchValues, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, + StreamValues, ToBatch, ToStream, }; use crate::expr::{Expr, ExprImpl, ExprRewriter, Literal}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; -use crate::optimizer::property::{FunctionalDependencySet}; +use crate::optimizer::property::FunctionalDependencySet; use crate::utils::{ColIndexMapping, Condition}; /// `LogicalValues` builds rows according to a list of expressions @@ -56,7 +56,7 @@ impl LogicalValues { } } - /// Used only by `LogicalValues.rewrite_logical_for_stream, set the `_row_id` column as pk + /// Used only by `LogicalValues.rewrite_logical_for_stream`, set the `_row_id` column as pk fn new_with_pk( rows: Vec>, schema: Schema, @@ -163,7 +163,7 @@ impl ToStream for LogicalValues { ) -> Result<(PlanRef, ColIndexMapping)> { let row_id_index = self.schema().len(); let col_index_mapping = ColIndexMapping::identity_or_none(row_id_index, row_id_index + 1); - let ctx = self.ctx().clone(); + let ctx = self.ctx(); let mut schema = self.schema().clone(); schema.fields.push(Field { data_type: DataType::Int64, @@ -171,12 +171,13 @@ impl ToStream for LogicalValues { sub_fields: vec![], type_name: "int64".to_string(), }); - let rows = self.rows().clone().to_owned(); + let rows = self.rows().to_vec(); let row_with_id = (0..rows.len()) - .into_iter() .zip_eq_fast(rows.into_iter()) .map(|(i, mut r)| { - r.extend_one(Literal::new(Some(ScalarImpl::Int64(i as i64)), DataType::Int64).into()); + r.extend_one( + Literal::new(Some(ScalarImpl::Int64(i as i64)), DataType::Int64).into(), + ); r }) .collect_vec(); 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 98799fd07f71f..8f529ea55a271 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 @@ -60,9 +60,7 @@ impl PlanTreeNodeUnary for StreamRowIdGen { } fn clone_with_input(&self, input: PlanRef) -> Self { - Self::new( - self.logical.clone_with_input(input), - ) + Self::new(self.logical.clone_with_input(input)) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs index 15fa19f911a36..537f25972e936 100644 --- a/src/frontend/src/optimizer/plan_node/stream_values.rs +++ b/src/frontend/src/optimizer/plan_node/stream_values.rs @@ -34,7 +34,7 @@ pub struct StreamValues { impl_plan_tree_node_for_leaf! { StreamValues } impl StreamValues { - /// StreamValues should enforce `Distribution::Single` + /// `StreamValues` should enforce `Distribution::Single` pub fn new(logical: LogicalValues) -> Self { Self::with_dist(logical, Distribution::Single) } @@ -42,7 +42,7 @@ impl StreamValues { fn with_dist(logical: LogicalValues, dist: Distribution) -> Self { let ctx = logical.ctx(); let mut watermark_columns = FixedBitSet::with_capacity(logical.schema().len()); - (0..(logical.schema().len()-1)).into_iter().for_each(|i| watermark_columns.set(i, true)); + (0..(logical.schema().len() - 1)).for_each(|i| watermark_columns.set(i, true)); let base = PlanBase::new_stream( ctx, logical.schema().clone(), diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index f49d013671f75..563c4eae6c049 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -90,10 +90,10 @@ pub mod subtask; mod temporal_join; mod top_n; mod union; +pub mod values; mod watermark; mod watermark_filter; mod wrapper; -pub mod values; mod backfill; #[cfg(test)] diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 8377a1ec1751b..81881a3caf007 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -25,12 +25,14 @@ use risingwave_expr::expr::BoxedExpression; use tokio::sync::mpsc::UnboundedReceiver; use super::{ - Barrier, BoxedMessageStream, Executor, Message, PkIndices, PkIndicesRef, StreamExecutorError, + ActorContextRef, Barrier, BoxedMessageStream, Executor, Message, PkIndices, PkIndicesRef, + StreamExecutorError, }; /// Execute `values` in stream. As is a leaf, current workaround holds a `barrier_executor`. /// May refractor with `BarrierRecvExecutor` in the near future. pub struct ValuesExecutor { + ctx: ActorContextRef, // Receiver of barrier channel. barrier_receiver: UnboundedReceiver, @@ -43,12 +45,14 @@ pub struct ValuesExecutor { impl ValuesExecutor { /// Currently hard-code the `pk_indices` as the last column. pub fn new( + ctx: ActorContextRef, rows: Vec>, schema: Schema, barrier_receiver: UnboundedReceiver, executor_id: u64, ) -> Self { Self { + ctx, barrier_receiver, rows: rows.into_iter(), pk_indices: vec![schema.len()], @@ -88,7 +92,11 @@ impl ValuesExecutor { let mut array_builders = schema.create_array_builders(chunk_size); for row in rows.by_ref().take(chunk_size) { for (expr, builder) in row.into_iter().zip_eq_fast(&mut array_builders) { - let out = expr.eval(&one_row_chunk).await?; + let out = expr + .eval_infallible(&one_row_chunk, |err| { + self.ctx.on_compute_error(err, self.identity.as_str()) + }) + .await; builder.append_array(&out); } } @@ -144,7 +152,7 @@ mod tests { use tokio::sync::mpsc::unbounded_channel; use super::ValuesExecutor; - use crate::executor::{Barrier, Executor, Message, Mutation}; + use crate::executor::{ActorContext, Barrier, Executor, Message, Mutation}; #[tokio::test] async fn test_values() { @@ -179,8 +187,13 @@ mod tests { .iter() // for each column .map(|col| Field::unnamed(col.return_type())) .collect::>(); - let values_executor_struct = - ValuesExecutor::new(vec![exprs], Schema { fields }, barrier_receiver, 10005); + let values_executor_struct = ValuesExecutor::new( + ActorContext::create(1), + vec![exprs], + Schema { fields }, + barrier_receiver, + 10005, + ); let mut values_executor = Box::new(values_executor_struct).execute(); // Init barrier diff --git a/src/stream/src/from_proto/mod.rs b/src/stream/src/from_proto/mod.rs index 337affb594f4f..f3b127de7fce3 100644 --- a/src/stream/src/from_proto/mod.rs +++ b/src/stream/src/from_proto/mod.rs @@ -43,8 +43,8 @@ mod temporal_join; mod top_n; mod top_n_appendonly; mod union; -mod watermark_filter; mod values; +mod watermark_filter; // import for submodules use itertools::Itertools; diff --git a/src/stream/src/from_proto/values.rs b/src/stream/src/from_proto/values.rs index 4892c7b2c8228..69a60480c15d3 100644 --- a/src/stream/src/from_proto/values.rs +++ b/src/stream/src/from_proto/values.rs @@ -25,8 +25,8 @@ use crate::executor::values::ValuesExecutor; use crate::executor::BoxedExecutor; use crate::task::{ExecutorParams, LocalStreamManagerCore}; -/// Build a `ValuesExecutor` for stream. As is a leaf, current workaround registers a `sender` for this executor. -/// May refractor with `BarrierRecvExecutor` in the near future. +/// Build a `ValuesExecutor` for stream. As is a leaf, current workaround registers a `sender` for +/// this executor. May refractor with `BarrierRecvExecutor` in the near future. pub struct ValuesExecutorBuilder; #[async_trait::async_trait] @@ -57,6 +57,7 @@ impl ExecutorBuilder for ValuesExecutorBuilder { .collect_vec(); let schema = Schema::new(node.get_fields().iter().map(Field::from).collect_vec()); Ok(Box::new(ValuesExecutor::new( + params.actor_context, rows, schema, barrier_receiver, From 3dfe63c7afc594ca7a89cc653cfcf3ede2a8b624 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Thu, 23 Mar 2023 14:10:26 -0400 Subject: [PATCH 07/17] update planner test --- .../tests/testdata/predicate_pushdown.yaml | 2 +- .../tests/testdata/watermark.yaml | 110 +++++++++--------- 2 files changed, 54 insertions(+), 58 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml index 2aee875e9d2bf..370a5c420e112 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -10,7 +10,7 @@ └─LogicalProject { exprs: [t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id] } optimized_logical_plan_for_batch: | - LogicalValues { rows: [], schema: Schema { fields: [min(t.v1):Int32] } } + LogicalValues { rows: [] } - name: filter should not transpose limit sql: | create table t(v1 int, v2 int, v3 int, v4 int); diff --git a/src/frontend/planner_test/tests/testdata/watermark.yaml b/src/frontend/planner_test/tests/testdata/watermark.yaml index 94c9d878505e3..2888ee086dcec 100644 --- a/src/frontend/planner_test/tests/testdata/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/watermark.yaml @@ -7,53 +7,49 @@ LogicalProject { exprs: [(v1 - '00:00:02':Interval) as $expr1] } └─LogicalSource { source: t, columns: [v1, _row_id], time_range: [(Unbounded, Unbounded)] } stream_plan: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1, _row_id] } - └─StreamRowIdGen { row_id_index: 1 } - └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } - └─StreamSource { source: "t", columns: ["v1", "_row_id"] } - stream_dist_plan: | + StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [v1] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamRowIdGen { row_id_index: 1 } + └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } + └─StreamSource { source: "t", columns: ["v1", "_row_id"] } + stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } - materialized table: 4294967294 - StreamExchange Hash([1]) from 1 + StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [v1] } { materialized table: 4294967294 } + └── StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1, _row_id], output_watermarks: [$expr1] } + └── StreamRowIdGen { row_id_index: 1 } + └── StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } + └── StreamSource { source: "t", columns: ["v1", "_row_id"] } { source state table: 1 } - Fragment 1 - StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1, _row_id] } - StreamRowIdGen { row_id_index: 1 } - StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } - StreamSource { source: "t", columns: ["v1", "_row_id"] } - source state table: 1 + Table 1 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + + Table 4294967294 { columns: [ v1, _row_id ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - name: watermark on append only table with source sql: | explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only with (connector = 'kafka', kafka.topic = 'kafka_3_partition_topic', kafka.brokers = '127.0.0.1:1234', kafka.scan.startup.mode='earliest') ROW FORMAT JSON; explain_output: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 1 } - └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } - └─StreamDml { columns: [v1, _row_id] } - └─StreamSource { source: "t", columns: ["v1", "_row_id"] } + StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [v1] } + └─StreamRowIdGen { row_id_index: 1 } + └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } + └─StreamDml { columns: [v1, _row_id] } + └─StreamSource { source: "t", columns: ["v1", "_row_id"] } - name: watermark on append only table without source sql: | explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only; explain_output: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check" } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 1 } - └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } - └─StreamDml { columns: [v1, _row_id] } - └─StreamSource + StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [v1] } + └─StreamRowIdGen { row_id_index: 1 } + └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } + └─StreamDml { columns: [v1, _row_id] } + └─StreamSource - name: hash agg sql: | create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select count(v2) from t group by ts, v1; stream_plan: | - StreamMaterialize { columns: [count, t.ts(hidden), t.v1(hidden)], pk_columns: [t.ts, t.v1], pk_conflict: "no check" } - └─StreamProject { exprs: [count(t.v2), t.ts, t.v1] } - └─StreamAppendOnlyHashAgg { group_key: [t.ts, t.v1], aggs: [count(t.v2), count] } + StreamMaterialize { columns: [count, t.ts(hidden), t.v1(hidden)], pk_columns: [t.ts, t.v1], pk_conflict: "no check", watermark_columns: [t.ts(hidden)] } + └─StreamProject { exprs: [count(t.v2), t.ts, t.v1], output_watermarks: [t.ts] } + └─StreamAppendOnlyHashAgg { group_key: [t.ts, t.v1], aggs: [count(t.v2), count], output_watermarks: [t.ts] } └─StreamExchange { dist: HashShard(t.ts, t.v1) } └─StreamTableScan { table: t, columns: [t.ts, t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: hash join @@ -62,8 +58,8 @@ create table t2 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select t1.ts as t1_ts, t2.ts as ts2, t1.v1 as t1_v1, t1.v2 as t1_v2, t2.v1 as t2_v1, t2.v2 as t2_v2 from t1, t2 where t1.ts = t2.ts; stream_plan: | - StreamMaterialize { columns: [t1_ts, ts2, t1_v1, t1_v2, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, t1_ts], pk_conflict: "no check" } - └─StreamAppendOnlyHashJoin { type: Inner, predicate: t1.ts = t2.ts, output: [t1.ts, t2.ts, t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } + StreamMaterialize { columns: [t1_ts, ts2, t1_v1, t1_v2, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, t1_ts], pk_conflict: "no check", watermark_columns: [t1_ts, ts2] } + └─StreamAppendOnlyHashJoin { type: Inner, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t2.ts, t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.ts) } | └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard(t2.ts) } @@ -86,13 +82,13 @@ create table t2 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select * from t1 Union all select * from t2; stream_plan: | - StreamMaterialize { columns: [ts, v1, v2, t1._row_id(hidden), null:Int64(hidden), 0:Int32(hidden)], pk_columns: [t1._row_id, null:Int64, 0:Int32], pk_conflict: "no check" } - └─StreamUnion { all: true } - ├─StreamExchange { dist: HashShard(t1._row_id, null:Int64, 0:Int32) } - | └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, null:Int64, 0:Int32] } + StreamMaterialize { columns: [ts, v1, v2, t1._row_id(hidden), null:Serial(hidden), 0:Int32(hidden)], pk_columns: [t1._row_id, null:Serial, 0:Int32], pk_conflict: "no check", watermark_columns: [ts] } + └─StreamUnion { all: true, output_watermarks: [t1.ts] } + ├─StreamExchange { dist: HashShard(t1._row_id, null:Serial, 0:Int32) } + | └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, null:Serial, 0:Int32], output_watermarks: [t1.ts] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(null:Int64, t2._row_id, 1:Int32) } - └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, null:Int64, t2._row_id, 1:Int32] } + └─StreamExchange { dist: HashShard(null:Serial, t2._row_id, 1:Int32) } + └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, null:Serial, t2._row_id, 1:Int32], output_watermarks: [t2.ts] } └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: union sql: | @@ -100,32 +96,32 @@ create table t2 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select * from t1 Union select * from t2; stream_plan: | - StreamMaterialize { columns: [ts, v1, v2], pk_columns: [ts, v1, v2], pk_conflict: "no check" } - └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2] } - └─StreamAppendOnlyHashAgg { group_key: [t1.ts, t1.v1, t1.v2], aggs: [count] } + StreamMaterialize { columns: [ts, v1, v2], pk_columns: [ts, v1, v2], pk_conflict: "no check", watermark_columns: [ts] } + └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2], output_watermarks: [t1.ts] } + └─StreamAppendOnlyHashAgg { group_key: [t1.ts, t1.v1, t1.v2], aggs: [count], output_watermarks: [t1.ts] } └─StreamExchange { dist: HashShard(t1.ts, t1.v1, t1.v2) } - └─StreamUnion { all: true } - ├─StreamExchange { dist: HashShard(t1._row_id, null:Int64, 0:Int32) } - | └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, null:Int64, 0:Int32] } + └─StreamUnion { all: true, output_watermarks: [t1.ts] } + ├─StreamExchange { dist: HashShard(t1._row_id, null:Serial, 0:Int32) } + | └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, null:Serial, 0:Int32], output_watermarks: [t1.ts] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(null:Int64, t2._row_id, 1:Int32) } - └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, null:Int64, t2._row_id, 1:Int32] } + └─StreamExchange { dist: HashShard(null:Serial, t2._row_id, 1:Int32) } + └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, null:Serial, t2._row_id, 1:Int32], output_watermarks: [t2.ts] } └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: tumble sql: | create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select * from tumble(t, ts, interval '3' minute); stream_plan: | - StreamMaterialize { columns: [ts, v1, v2, window_start, window_end, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check" } - └─StreamProject { exprs: [t.ts, t.v1, t.v2, TumbleStart(t.ts, '00:03:00':Interval) as $expr1, (AtTimeZone((AtTimeZone(TumbleStart(t.ts, '00:03:00':Interval), 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '00:03:00':Interval) as $expr2, t._row_id] } + StreamMaterialize { columns: [ts, v1, v2, window_start, window_end, t._row_id(hidden)], pk_columns: [t._row_id], pk_conflict: "no check", watermark_columns: [ts, window_start, window_end] } + └─StreamProject { exprs: [t.ts, t.v1, t.v2, TumbleStart(t.ts, '00:03:00':Interval) as $expr1, (AtTimeZone((AtTimeZone(TumbleStart(t.ts, '00:03:00':Interval), 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '00:03:00':Interval) as $expr2, t._row_id], output_watermarks: [t.ts, $expr1, $expr2] } └─StreamTableScan { table: t, columns: [t.ts, t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: hop all sql: | create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select * from hop(t, ts, interval '1' minute, interval '3' minute); stream_plan: | - StreamMaterialize { columns: [ts, v1, v2, window_start, window_end, t._row_id(hidden)], pk_columns: [t._row_id, window_start, window_end], pk_conflict: "no check" } - └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [t.ts, t.v1, t.v2, window_start, window_end, t._row_id] } + StreamMaterialize { columns: [ts, v1, v2, window_start, window_end, t._row_id(hidden)], pk_columns: [t._row_id, window_start, window_end], pk_conflict: "no check", watermark_columns: [ts, window_start, window_end] } + └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [t.ts, t.v1, t.v2, window_start, window_end, t._row_id], output_watermarks: [t.ts, window_start, window_end] } └─StreamFilter { predicate: IsNotNull(t.ts) } └─StreamTableScan { table: t, columns: [t.ts, t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: hop ts @@ -133,8 +129,8 @@ create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select ts from hop(t, ts, interval '1' minute, interval '3' minute); stream_plan: | - StreamMaterialize { columns: [ts, window_start(hidden), t._row_id(hidden)], pk_columns: [t._row_id, window_start], pk_conflict: "no check" } - └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [t.ts, window_start, t._row_id] } + StreamMaterialize { columns: [ts, window_start(hidden), t._row_id(hidden)], pk_columns: [t._row_id, window_start], pk_conflict: "no check", watermark_columns: [ts, window_start(hidden)] } + └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [t.ts, window_start, t._row_id], output_watermarks: [t.ts, window_start] } └─StreamFilter { predicate: IsNotNull(t.ts) } └─StreamTableScan { table: t, columns: [t.ts, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: hop start @@ -142,8 +138,8 @@ create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select window_end from hop(t, ts, interval '1' minute, interval '3' minute); stream_plan: | - StreamMaterialize { columns: [window_end, t._row_id(hidden)], pk_columns: [t._row_id, window_end], pk_conflict: "no check" } - └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [window_end, t._row_id] } + StreamMaterialize { columns: [window_end, t._row_id(hidden)], pk_columns: [t._row_id, window_end], pk_conflict: "no check", watermark_columns: [window_end] } + └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [window_end, t._row_id], output_watermarks: [window_end] } └─StreamFilter { predicate: IsNotNull(t.ts) } └─StreamTableScan { table: t, columns: [t.ts, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: hop end @@ -151,7 +147,7 @@ create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select window_start from hop(t, ts, interval '1' minute, interval '3' minute); stream_plan: | - StreamMaterialize { columns: [window_start, t._row_id(hidden)], pk_columns: [t._row_id, window_start], pk_conflict: "no check" } - └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [window_start, t._row_id] } + StreamMaterialize { columns: [window_start, t._row_id(hidden)], pk_columns: [t._row_id, window_start], pk_conflict: "no check", watermark_columns: [window_start] } + └─StreamHopWindow { time_col: t.ts, slide: 00:01:00, size: 00:03:00, output: [window_start, t._row_id], output_watermarks: [window_start] } └─StreamFilter { predicate: IsNotNull(t.ts) } └─StreamTableScan { table: t, columns: [t.ts, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } From b51655c60e8d63fe7c3dab8752112966a678bd50 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Thu, 23 Mar 2023 23:35:48 -0400 Subject: [PATCH 08/17] rename variants in FragmentTypeFlag to avoid collision Signed-off-by: Clearlove --- proto/stream_plan.proto | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index be88a0e19e4e5..19b9d5457050a 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -640,14 +640,14 @@ message StreamActor { } enum FragmentTypeFlag { - FRAGMENT_UNSPECIFIED = 0; - SOURCE = 1; - MVIEW = 2; - SINK = 4; - NOW = 8; // TODO: Remove this and insert a `BarrierRecv` instead. - CHAIN_NODE = 16; - BARRIER_RECV = 32; - VALUES = 64; + FRAGMENT_TYPE_FLAG_FRAGMENT_UNSPECIFIED = 0; + FRAGMENT_TYPE_FLAG_SOURCE = 1; + FRAGMENT_TYPE_FLAG_MVIEW = 2; + FRAGMENT_TYPE_FLAG_SINK = 4; + FRAGMENT_TYPE_FLAG_NOW = 8; // TODO: Remove this and insert a `BarrierRecv` instead. + FRAGMENT_TYPE_FLAG_CHAIN_NODE = 16; + FRAGMENT_TYPE_FLAG_BARRIER_RECV = 32; + FRAGMENT_TYPE_FLAG_VALUES = 64; } // The environment associated with a stream plan From f3c0587550ed6b839925238d52942dd91bb7ac7f Mon Sep 17 00:00:00 2001 From: Clearlove Date: Thu, 23 Mar 2023 23:44:30 -0400 Subject: [PATCH 09/17] check Signed-off-by: Clearlove --- src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 7304c81507e24..c516ea4a6f021 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 @@ -16,9 +16,9 @@ use std::fmt; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; -use crate::{optimizer::property::Distribution, stream_fragmenter::BuildFragmentGraphState}; - use super::{ExprRewritable, LogicalRowIdGen, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use crate::optimizer::property::Distribution; +use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamRowIdGen` holds a stream `PlanBase` and a `LogicalRowIdGen` #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -72,7 +72,7 @@ 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) -> ProstStreamNode { use risingwave_pb::stream_plan::*; ProstStreamNode::RowIdGen(RowIdGenNode { From b8901ae84663d8b2cddd192356ec56ad34a4e530 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Tue, 28 Mar 2023 15:28:06 -0400 Subject: [PATCH 10/17] remove logical_row_id_gen Signed-off-by: Clearlove --- src/frontend/src/optimizer/mod.rs | 7 +- .../optimizer/plan_node/logical_row_id_gen.rs | 122 ------------------ .../src/optimizer/plan_node/logical_source.rs | 6 +- src/frontend/src/optimizer/plan_node/mod.rs | 4 - .../optimizer/plan_node/stream_row_id_gen.rs | 49 +++---- 5 files changed, 32 insertions(+), 156 deletions(-) delete mode 100644 src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 228ad14d95960..91dac433c14ce 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -41,8 +41,8 @@ use risingwave_pb::catalog::WatermarkDesc; use self::heuristic_optimizer::ApplyOrder; use self::plan_node::{ - BatchProject, Convention, LogicalProject, LogicalRowIdGen, StreamDml, StreamMaterialize, - StreamProject, StreamRowIdGen, StreamSink, StreamWatermarkFilter, + BatchProject, Convention, LogicalProject, StreamDml, StreamMaterialize, StreamProject, + StreamRowIdGen, StreamSink, StreamWatermarkFilter, }; use self::plan_visitor::has_batch_exchange; #[cfg(debug_assertions)] @@ -397,8 +397,7 @@ impl PlanRoot { // Add RowIDGen node if needed. if let Some(row_id_index) = row_id_index { - let logical_row_id_gen = LogicalRowIdGen::new(stream_plan, row_id_index); - stream_plan = StreamRowIdGen::new(logical_row_id_gen).into(); + stream_plan = StreamRowIdGen::new(stream_plan, row_id_index).into(); } let conflict_behavior = match append_only { diff --git a/src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs deleted file mode 100644 index 9930312968371..0000000000000 --- a/src/frontend/src/optimizer/plan_node/logical_row_id_gen.rs +++ /dev/null @@ -1,122 +0,0 @@ -// Copyright 2023 RisingWave Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::{fmt, vec}; - -use risingwave_common::bail; -use risingwave_common::error::Result; - -use super::{ - ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, StreamRowIdGen, ToBatch, ToStream, -}; -use crate::optimizer::plan_node::{ - ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, -}; -use crate::optimizer::property::FunctionalDependencySet; -use crate::utils::{ColIndexMapping, Condition}; - -/// `LogicalRowIdGen` generates `_row_id` at `row_id_index` -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct LogicalRowIdGen { - pub base: PlanBase, - input: PlanRef, - row_id_index: usize, -} - -impl LogicalRowIdGen { - /// Build a `LogicalRowIdGen` node given `input` and desired `row_id` index - pub fn new(input: PlanRef, row_id: usize) -> Self { - let schema = input.schema().clone(); - let functional_dependency = FunctionalDependencySet::new(schema.len()); - let base = PlanBase::new_logical(input.ctx(), schema, vec![row_id], functional_dependency); - Self { - base, - input, - row_id_index: row_id, - } - } - - /// Build a `LogicalRowIdGen` node given `input` and desired `row_id` index, shall be `Ok` - pub fn create(input: PlanRef, row_id: usize) -> Result { - Ok(Self::new(input, row_id)) - } - - /// `row_id_index` for the node - pub fn row_id_index(&self) -> usize { - self.row_id_index - } -} - -impl fmt::Display for LogicalRowIdGen { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!( - f, - "LogicalRowIdGen {{ row_id_index: {} }}", - self.row_id_index - ) - } -} - -impl ExprRewritable for LogicalRowIdGen {} - -impl PlanTreeNodeUnary for LogicalRowIdGen { - fn input(&self) -> PlanRef { - self.input.clone() - } - - fn clone_with_input(&self, input: PlanRef) -> Self { - Self::new(input, self.row_id_index) - } -} - -impl_plan_tree_node_for_unary! {LogicalRowIdGen} - -impl PredicatePushdown for LogicalRowIdGen { - fn predicate_pushdown( - &self, - predicate: Condition, - _ctx: &mut PredicatePushdownContext, - ) -> PlanRef { - LogicalFilter::create(self.clone().into(), predicate) - } -} - -impl ColPrunable for LogicalRowIdGen { - fn prune_col(&self, _required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { - self.clone().into() - } -} - -impl ToBatch for LogicalRowIdGen { - fn to_batch(&self) -> Result { - bail!("`LogicalRowIdGen` can only be converted to stream") - } -} - -impl ToStream for LogicalRowIdGen { - fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { - let stream_input = self.input().to_stream(ctx)?; - let new_logical = self.clone_with_input(stream_input); - Ok(StreamRowIdGen::new(new_logical).into()) - } - - fn logical_rewrite_for_stream( - &self, - ctx: &mut RewriteStreamContext, - ) -> Result<(PlanRef, ColIndexMapping)> { - let (input, input_col_change) = self.input().logical_rewrite_for_stream(ctx)?; - Ok((self.clone_with_input(input).into(), input_col_change)) - } -} diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 131dfe2de3392..5300db3ca2610 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -32,8 +32,7 @@ use crate::catalog::ColumnId; use crate::expr::{Expr, ExprImpl, ExprType}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::{ - ColumnPruningContext, LogicalRowIdGen, PredicatePushdownContext, RewriteStreamContext, - ToStreamContext, + ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; use crate::utils::{ColIndexMapping, Condition}; use crate::TableCatalog; @@ -353,8 +352,7 @@ impl ToStream for LogicalSource { assert!(!(self.core.gen_row_id && self.core.for_table)); if let Some(row_id_index) = self.core.row_id_index && self.core.gen_row_id { - let logical_row_id_gen = LogicalRowIdGen::new(plan, row_id_index); - plan = StreamRowIdGen::new(logical_row_id_gen).into(); + plan = StreamRowIdGen::new(plan, row_id_index).into(); } Ok(plan) } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index c766aabfae9b0..65cd475e7ee52 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -638,7 +638,6 @@ mod logical_now; mod logical_over_agg; mod logical_project; mod logical_project_set; -mod logical_row_id_gen; mod logical_scan; mod logical_share; mod logical_source; @@ -715,7 +714,6 @@ pub use logical_now::LogicalNow; pub use logical_over_agg::{LogicalOverAgg, PlanWindowFunction}; pub use logical_project::LogicalProject; pub use logical_project_set::LogicalProjectSet; -pub use logical_row_id_gen::LogicalRowIdGen; pub use logical_scan::LogicalScan; pub use logical_share::LogicalShare; pub use logical_source::LogicalSource; @@ -795,7 +793,6 @@ macro_rules! for_all_plan_nodes { , { Logical, OverAgg } , { Logical, Share } , { Logical, Now } - , { Logical, RowIdGen} // , { Logical, Sort } we don't need a LogicalSort, just require the Order , { Batch, SimpleAgg } , { Batch, HashAgg } @@ -878,7 +875,6 @@ macro_rules! for_logical_plan_nodes { , { Logical, OverAgg } , { Logical, Share } , { Logical, Now } - , { Logical, RowIdGen} // , { Logical, Sort} not sure if we will support Order by clause in subquery/view/MV // if we don't support that, we don't need LogicalSort, just require the Order at the top of query } 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 c516ea4a6f021..c703428a94d66 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,38 +14,43 @@ use std::fmt; -use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; -use super::{ExprRewritable, LogicalRowIdGen, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use crate::optimizer::plan_node::stream::StreamPlanRef; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; -/// `StreamRowIdGen` holds a stream `PlanBase` and a `LogicalRowIdGen` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamRowIdGen { pub base: PlanBase, - logical: LogicalRowIdGen, + input: PlanRef, + row_id_index: usize, } impl StreamRowIdGen { - /// Create a `StreamRowIdGen` with intermediate `LogicalRowIdGen` - pub fn new(logical: LogicalRowIdGen) -> Self { - let distribution = if logical.input().append_only() { + pub fn new(input: PlanRef, row_id_index: usize) -> Self { + let distribution = if input.append_only() { // remove exchange for append only source - Distribution::HashShard(vec![logical.row_id_index()]) + Distribution::HashShard(vec![row_id_index]) } else { - logical.input().distribution().clone() + input.distribution().clone() }; + let base = PlanBase::new_stream( - logical.ctx(), - logical.schema().clone(), - logical.logical_pk().to_vec(), - logical.functional_dependency().clone(), + input.ctx(), + input.schema().clone(), + input.logical_pk().to_vec(), + input.functional_dependency().clone(), distribution, - logical.input().append_only(), - logical.input().watermark_columns().clone(), + input.append_only(), + input.watermark_columns().clone(), ); - Self { base, logical } + Self { + base, + input, + row_id_index, + } } } @@ -54,29 +59,29 @@ impl fmt::Display for StreamRowIdGen { write!( f, "StreamRowIdGen {{ row_id_index: {} }}", - self.logical.row_id_index() + self.row_id_index ) } } impl PlanTreeNodeUnary for StreamRowIdGen { fn input(&self) -> PlanRef { - self.logical.input() + self.input.clone() } fn clone_with_input(&self, input: PlanRef) -> Self { - Self::new(self.logical.clone_with_input(input)) + Self::new(input, self.row_id_index) } } 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 { - row_id_index: self.logical.row_id_index() as _, + PbNodeBody::RowIdGen(RowIdGenNode { + row_id_index: self.row_id_index as _, }) } } From 856635eed89924809fdedf7d0ebf5e3ba8b04fc9 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Tue, 28 Mar 2023 15:45:19 -0400 Subject: [PATCH 11/17] other refractor Signed-off-by: Clearlove --- .../src/optimizer/plan_node/logical_values.rs | 21 ++++++++----------- .../src/optimizer/plan_node/stream_values.rs | 6 +----- src/stream/src/executor/values.rs | 14 ++++++------- 3 files changed, 16 insertions(+), 25 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index f20cf62457418..08100aaff1445 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -94,6 +94,7 @@ impl fmt::Display for LogicalValues { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("LogicalValues") .field("rows", &self.rows) + .field("schema", &self.schema()) .finish() } } @@ -165,19 +166,15 @@ impl ToStream for LogicalValues { let col_index_mapping = ColIndexMapping::identity_or_none(row_id_index, row_id_index + 1); let ctx = self.ctx(); let mut schema = self.schema().clone(); - schema.fields.push(Field { - data_type: DataType::Int64, - name: "_row_id".to_string(), - sub_fields: vec![], - type_name: "int64".to_string(), - }); - let rows = self.rows().to_vec(); - let row_with_id = (0..rows.len()) - .zip_eq_fast(rows.into_iter()) + schema + .fields + .push(Field::with_name(DataType::Int64, "_row_id")); + let rows = self.rows().to_owned(); + let row_with_id = rows + .into_iter() + .enumerate() .map(|(i, mut r)| { - r.extend_one( - Literal::new(Some(ScalarImpl::Int64(i as i64)), DataType::Int64).into(), - ); + r.push(Literal::new(Some(ScalarImpl::Int64(i as i64)), DataType::Int64).into()); r }) .collect_vec(); diff --git a/src/frontend/src/optimizer/plan_node/stream_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs index 537f25972e936..e6587fac61a56 100644 --- a/src/frontend/src/optimizer/plan_node/stream_values.rs +++ b/src/frontend/src/optimizer/plan_node/stream_values.rs @@ -36,10 +36,6 @@ impl_plan_tree_node_for_leaf! { StreamValues } impl StreamValues { /// `StreamValues` should enforce `Distribution::Single` pub fn new(logical: LogicalValues) -> Self { - Self::with_dist(logical, Distribution::Single) - } - - fn with_dist(logical: LogicalValues, dist: Distribution) -> Self { let ctx = logical.ctx(); let mut watermark_columns = FixedBitSet::with_capacity(logical.schema().len()); (0..(logical.schema().len() - 1)).for_each(|i| watermark_columns.set(i, true)); @@ -48,7 +44,7 @@ impl StreamValues { logical.schema().clone(), logical.logical_pk().to_vec(), logical.functional_dependency().clone(), - dist, + Distribution::Single, false, watermark_columns, ); diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 81881a3caf007..8b2822cacff16 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -29,6 +29,8 @@ use super::{ StreamExecutorError, }; +const DEFAULT_CHUNK_SIZE: usize = 1024; + /// Execute `values` in stream. As is a leaf, current workaround holds a `barrier_executor`. /// May refractor with `BarrierRecvExecutor` in the near future. pub struct ValuesExecutor { @@ -75,7 +77,7 @@ impl ValuesExecutor { .await .unwrap(); - let emit = !barrier.is_resume(); + let emit = !barrier.is_newly_added(self.ctx.id); yield Message::Barrier(barrier); // If it's failover, do not evaluate rows (assume they have been yielded) @@ -88,7 +90,7 @@ impl ValuesExecutor { // cardinality. let one_row_chunk = DataChunk::new_dummy(1); - let chunk_size = 1024_usize.min(rows.len()); + let chunk_size = DEFAULT_CHUNK_SIZE.min(rows.len()); let mut array_builders = schema.create_array_builders(chunk_size); for row in rows.by_ref().take(chunk_size) { for (expr, builder) in row.into_iter().zip_eq_fast(&mut array_builders) { @@ -198,12 +200,8 @@ mod tests { // Init barrier let first_message = Barrier::new_test_barrier(1).with_mutation(Mutation::Add { - adds: maplit::hashmap! { - 0 => vec![Dispatcher { - downstream_actor_id: vec![1], - ..Default::default() - }], - }, + adds: Default::default(), + added_actors: maplit::hashset! {1}, splits: Default::default(), }); tx.send(first_message).unwrap(); From 9cd021f6ec1d37f308f0d8d76b731d5dec93e255 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Tue, 28 Mar 2023 15:47:28 -0400 Subject: [PATCH 12/17] planner test Signed-off-by: Clearlove --- .../planner_test/tests/testdata/array.yaml | 44 +++++++++---------- .../tests/testdata/array_access.yaml | 2 +- .../tests/testdata/basic_query.yaml | 2 +- .../testdata/common_table_expressions.yaml | 6 +-- .../planner_test/tests/testdata/explain.yaml | 14 +++--- .../planner_test/tests/testdata/expr.yaml | 42 +++++++++--------- .../planner_test/tests/testdata/insert.yaml | 2 +- .../tests/testdata/nexmark_source.yaml | 12 ++--- .../tests/testdata/pg_catalog.yaml | 4 +- .../tests/testdata/predicate_pushdown.yaml | 2 +- .../planner_test/tests/testdata/share.yaml | 12 ++--- .../tests/testdata/struct_query.yaml | 4 +- .../planner_test/tests/testdata/subquery.yaml | 8 ++-- .../tests/testdata/subquery_expr.yaml | 34 +++++++------- .../testdata/subquery_expr_correlated.yaml | 6 +-- .../tests/testdata/time_window.yaml | 2 +- .../planner_test/tests/testdata/types.yaml | 12 ++--- .../planner_test/tests/testdata/union.yaml | 6 +-- .../src/optimizer/plan_node/logical_values.rs | 1 - 19 files changed, 107 insertions(+), 108 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/array.yaml b/src/frontend/planner_test/tests/testdata/array.yaml index 72989cb0f325c..bc4bb13311ca9 100644 --- a/src/frontend/planner_test/tests/testdata/array.yaml +++ b/src/frontend/planner_test/tests/testdata/array.yaml @@ -2,13 +2,13 @@ - sql: | values (ARRAY['foo', 'bar']); logical_plan: | - LogicalValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]] } + LogicalValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } batch_plan: | BatchValues { rows: [[ARRAY[foo, bar]:List { datatype: Varchar }]] } - sql: | values (ARRAY[1, 2+3, 4*5+1]); logical_plan: | - LogicalValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]] } + LogicalValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Int32 }] } } batch_plan: | BatchValues { rows: [[ARRAY[1, 5, 21]:List { datatype: Int32 }]] } - sql: | @@ -25,7 +25,7 @@ select ARRAY[null]; logical_plan: | LogicalProject { exprs: [Array(null:Varchar) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select ARRAY[]; binder_error: |- @@ -35,12 +35,12 @@ select ARRAY[]::int[]; logical_plan: | LogicalProject { exprs: [Array::List { datatype: Int32 } as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select ARRAY[]::int[][]; logical_plan: | LogicalProject { exprs: [Array::List { datatype: List { datatype: Int32 } } as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select ARRAY[]::int; binder_error: |- @@ -50,21 +50,21 @@ select array_cat(array[66], array[123]); logical_plan: | LogicalProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32)) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | BatchValues { rows: [[ARRAY[66, 123]:List { datatype: Int32 }]] } - sql: | select array_cat(array[array[66]], array[233]); logical_plan: | LogicalProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32)) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | BatchValues { rows: [[ARRAY[{66}, {233}]:List { datatype: List { datatype: Int32 } }]] } - sql: | select array_cat(array[233], array[array[66]]); logical_plan: | LogicalProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32))) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | BatchValues { rows: [[ARRAY[{233}, {66}]:List { datatype: List { datatype: Int32 } }]] } - sql: | @@ -80,7 +80,7 @@ select array_append(array[66], 123); logical_plan: | LogicalProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | BatchValues { rows: [[ARRAY[66, 123]:List { datatype: Int32 }]] } - sql: | @@ -91,12 +91,12 @@ select array_append(array[array[66]], array[233]); logical_plan: | LogicalProject { exprs: [ArrayAppend(Array(Array(66:Int32)), Array(233:Int32)) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select array_prepend(123, array[66]); logical_plan: | LogicalProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32)) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | BatchValues { rows: [[ARRAY[123, 66]:List { datatype: Int32 }]] } - sql: | @@ -106,7 +106,7 @@ select array_prepend(array[233], array[array[66]]); logical_plan: | LogicalProject { exprs: [ArrayPrepend(Array(233:Int32), Array(Array(66:Int32))) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: string from/to varchar[] in implicit context sql: | values (array['a', 'b']), ('{c,' || 'd}'); @@ -121,62 +121,62 @@ select ('{c,' || 'd}')::varchar[]; logical_plan: | LogicalProject { exprs: [ConcatOp('{c,':Varchar, 'd}':Varchar)::List { datatype: Varchar } as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: unknown to varchar[] in implicit context sql: | values (array['a', 'b']), ('{c,d}'); logical_plan: | - LogicalValues { rows: [[Array('a':Varchar, 'b':Varchar)], ['{c,d}':Varchar::List { datatype: Varchar }]] } + LogicalValues { rows: [[Array('a':Varchar, 'b':Varchar)], ['{c,d}':Varchar::List { datatype: Varchar }]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } - name: unknown to varchar[] in assign context sql: | create table t (v1 varchar[]); insert into t values ('{c,d}'); logical_plan: | LogicalInsert { table: t } - └─LogicalValues { rows: [['{c,d}':Varchar::List { datatype: Varchar }]] } + └─LogicalValues { rows: [['{c,d}':Varchar::List { datatype: Varchar }]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } - name: unknown to varchar[] in explicit context sql: | select ('{c,d}')::varchar[]; logical_plan: | LogicalProject { exprs: ['{c,d}':Varchar::List { datatype: Varchar } as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: varchar[] to string in assign context sql: | create table t (v1 varchar); insert into t values (array['a', 'b']); logical_plan: | LogicalInsert { table: t } - └─LogicalValues { rows: [[Array('a':Varchar, 'b':Varchar)::Varchar]] } + └─LogicalValues { rows: [[Array('a':Varchar, 'b':Varchar)::Varchar]], schema: Schema { fields: [*VALUES*_0.column_0:Varchar] } } - name: varchar[] to string in explicit context sql: | select array['a', 'b']::varchar; logical_plan: | LogicalProject { exprs: [Array('a':Varchar, 'b':Varchar)::Varchar as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: bool[] cast in explicit context sql: | select array[1, true]::bool[]; logical_plan: | LogicalProject { exprs: [Array(1:Int32::Boolean, true:Boolean) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: bool[][] cast in explicit context sql: | select array[array[1, true]]::bool[][]; logical_plan: | LogicalProject { exprs: [Array(Array(1:Int32::Boolean, true:Boolean)) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: compare with null sql: | select null = array[1]; logical_plan: | LogicalProject { exprs: [(null:List { datatype: Int32 } = Array(1:Int32)) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: compare with literal sql: | select array[1] = '{1}'; logical_plan: | LogicalProject { exprs: [(Array(1:Int32) = '{1}':Varchar::List { datatype: Int32 }) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: compare with different type sql: | select array[1] = array[1.2]; diff --git a/src/frontend/planner_test/tests/testdata/array_access.yaml b/src/frontend/planner_test/tests/testdata/array_access.yaml index e0691ee5bccae..d1adfeb18d699 100644 --- a/src/frontend/planner_test/tests/testdata/array_access.yaml +++ b/src/frontend/planner_test/tests/testdata/array_access.yaml @@ -3,7 +3,7 @@ select (ARRAY['foo', 'bar'])[1]; logical_plan: | LogicalProject { exprs: [ArrayAccess(Array('foo':Varchar, 'bar':Varchar), 1:Int32) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | create table t(i int[]); select min(i) from t where i[1]>2; diff --git a/src/frontend/planner_test/tests/testdata/basic_query.yaml b/src/frontend/planner_test/tests/testdata/basic_query.yaml index c318164c2b427..70753af118575 100644 --- a/src/frontend/planner_test/tests/testdata/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/basic_query.yaml @@ -3,7 +3,7 @@ batch_plan: | BatchValues { rows: [[11:Int32, 22:Int32], [36:Int32, 44:Int32]] } stream_plan: | - StreamMaterialize { columns: [*VALUES*_0.column_0, *VALUES*_0.column_1, _row_id(hidden)], pk_columns: [_row_id], pk_conflict: "no check", watermark_columns: [*VALUES*_0.column_0, *VALUES*_0.column_1] } + StreamMaterialize { columns: [*VALUES*_0.column_0, *VALUES*_0.column_1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: "NoCheck", watermark_columns: [*VALUES*_0.column_0, *VALUES*_0.column_1] } └─StreamValues { rows: [[11:Int32, 22:Int32, 0:Int64], [(33:Int32 + (1:Int32 + 2:Int32)), 44:Int32, 1:Int64]] } - sql: select * from t binder_error: 'Catalog error: table or source not found: t' diff --git a/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml b/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml index c05b78c0d993b..eff991d029409 100644 --- a/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml +++ b/src/frontend/planner_test/tests/testdata/common_table_expressions.yaml @@ -54,7 +54,7 @@ | └─LogicalProject { exprs: [t1.x] } | └─LogicalScan { table: t1, columns: [t1.x, t1._row_id] } └─LogicalProject { exprs: [0.1:Decimal] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: Ensure we can bind CTE with aliases in both table name and columns sql: | create table t1 (x int, y int); @@ -76,9 +76,9 @@ └─LogicalJoin { type: Inner, on: (t.c = *VALUES*_0.column_0), output: all } ├─LogicalScan { table: t, columns: [t.v, t.c, t._row_id] } └─LogicalShare { id = 1 } - └─LogicalValues { rows: [['cn':Varchar, 'China':Varchar], ['us':Varchar, 'United States':Varchar]] } + └─LogicalValues { rows: [['cn':Varchar, 'China':Varchar], ['us':Varchar, 'United States':Varchar]], schema: Schema { fields: [*VALUES*_0.column_0:Varchar, *VALUES*_0.column_1:Varchar] } } stream_plan: | - StreamMaterialize { columns: [v, c, abbr, real, t._row_id(hidden), _row_id(hidden)], pk_columns: [t._row_id, _row_id, c], pk_conflict: "no check" } + StreamMaterialize { columns: [v, c, abbr, real, t._row_id(hidden), _row_id(hidden)], stream_key: [t._row_id, _row_id, c], pk_columns: [t._row_id, _row_id, c], pk_conflict: "NoCheck" } └─StreamHashJoin { type: Inner, predicate: t.c = *VALUES*_0.column_0, output: [t.v, t.c, *VALUES*_0.column_0, *VALUES*_0.column_1, t._row_id, _row_id] } ├─StreamExchange { dist: HashShard(t.c) } | └─StreamTableScan { table: t, columns: [t.v, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index 2ca965fef587b..4758621a43dbc 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -4,38 +4,38 @@ Begin: LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } Predicate Push Down: LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } Predicate Push Down: LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } Predicate Push Down: LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } Prune Columns: LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } Predicate Push Down: LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } Project Remove: apply TrivialProjectToValuesRule 1 time(s) - LogicalValues { rows: [[1:Int32]] } + LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [1:Int32:Int32] } } Inline Session Timezone: diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 1f10656a595bc..e3d6ddc38b72b 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -4,13 +4,13 @@ select int '1'; logical_plan: | LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: bind typed literal - bool sql: | SELECT bool 't' logical_plan: | LogicalProject { exprs: [true:Boolean] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | values(must_be_unimplemented_func(1)); binder_error: |- @@ -36,31 +36,31 @@ SELECT 1 between 2 and 3 logical_plan: | LogicalProject { exprs: [((1:Int32 >= 2:Int32) AND (1:Int32 <= 3:Int32)) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: null eq null sql: | SELECT null = null; logical_plan: | LogicalProject { exprs: [(null:Varchar = null:Varchar) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: null lt null sql: | SELECT null < null; logical_plan: | LogicalProject { exprs: [(null:Varchar < null:Varchar) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: bind is distinct from sql: | SELECT 1 IS DISTINCT FROM 2 logical_plan: | LogicalProject { exprs: [IsDistinctFrom(1:Int32, 2:Int32) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: bind is not distinct from sql: | SELECT 1 IS NOT DISTINCT FROM 2 logical_plan: | LogicalProject { exprs: [IsNotDistinctFrom(1:Int32, 2:Int32) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: in-list with aligned types sql: | SELECT 1::real in (3, 1.0, 2); @@ -344,27 +344,27 @@ select 1 < SOME(null); logical_plan: | LogicalProject { exprs: [Some((1:Int32 < null:List { datatype: Int32 })) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select 1 < ANY(null); logical_plan: | LogicalProject { exprs: [Some((1:Int32 < null:List { datatype: Int32 })) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select 1 < ALL(null); logical_plan: | LogicalProject { exprs: [All((1:Int32 < null:List { datatype: Int32 })) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select 1 < ALL('{2,3}'); logical_plan: | LogicalProject { exprs: [All((1:Int32 < '{2,3}':Varchar::List { datatype: Int32 })) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select 1 < SOME(null::integer[]); logical_plan: | LogicalProject { exprs: [Some((1:Int32 < null:List { datatype: Int32 })) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select 1 < SOME(null::varchar[]); binder_error: |- @@ -385,20 +385,20 @@ select 1 < SOME(array[null]::integer[]); logical_plan: | LogicalProject { exprs: [Some((1:Int32 < Array(null:Int32))) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select 1 < SOME(array[1, 2]); logical_plan: | LogicalProject { exprs: [Some((1:Int32 < Array(1:Int32, 2:Int32))) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select 1 < SOME((select array[1]) || array[2]); logical_plan: | LogicalProject { exprs: [Some((1:Int32 < ArrayCat($expr1, Array(2:Int32)))) as $expr2] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalValues { rows: [[]] } + ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10037, ARRAY[2]:List { datatype: Int32 }))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } @@ -408,20 +408,20 @@ select 1 < ALL(array[null]::integer[]); logical_plan: | LogicalProject { exprs: [All((1:Int32 < Array(null:Int32))) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select 1 < ALL(array[1, 2]); logical_plan: | LogicalProject { exprs: [All((1:Int32 < Array(1:Int32, 2:Int32))) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select 1 < ALL((select array[1]) || array[2]); logical_plan: | LogicalProject { exprs: [All((1:Int32 < ArrayCat($expr1, Array(2:Int32)))) as $expr2] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalValues { rows: [[]] } + ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10037, ARRAY[2]:List { datatype: Int32 }))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } @@ -513,7 +513,7 @@ select ARRAY[1, null] t; logical_plan: | LogicalProject { exprs: [Array(1:Int32, null:Int32) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: regression (#7641) -fuzzing test failed at Bind error,types Boolean and Varchar cannot be matched sql: | select false >= 'LN1O0QP1yi' NOT IN (md5('4SeUPZhUbH')) diff --git a/src/frontend/planner_test/tests/testdata/insert.yaml b/src/frontend/planner_test/tests/testdata/insert.yaml index 5e85c942707ce..423f8e0e97f7e 100644 --- a/src/frontend/planner_test/tests/testdata/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/insert.yaml @@ -205,7 +205,7 @@ logical_plan: | LogicalProject { exprs: [*VALUES*_0.column_0, *VALUES*_0.column_1, *VALUES*_0.column_0, (*VALUES*_0.column_0 + *VALUES*_0.column_1) as $expr1] } └─LogicalInsert { table: t, returning: true } - └─LogicalValues { rows: [[0:Int32, 1:Int32], [1:Int32, 2:Int32]] } + └─LogicalValues { rows: [[0:Int32, 1:Int32], [1:Int32, 2:Int32]], schema: Schema { fields: [*VALUES*_0.column_0:Int32, *VALUES*_0.column_1:Int32] } } batch_plan: | BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [*VALUES*_0.column_0, *VALUES*_0.column_1, *VALUES*_0.column_0, (*VALUES*_0.column_0 + *VALUES*_0.column_1) as $expr1] } diff --git a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml index 12004b909ede8..24228d1179e06 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -382,7 +382,7 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [auction, count, window_start] } - | └─StreamShare { id = 12 } + | └─StreamShare { id = 11 } | └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -394,7 +394,7 @@ └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [auction, window_start, count] } - └─StreamShare { id = 12 } + └─StreamShare { id = 11 } └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -511,7 +511,7 @@ └─StreamHashJoin { type: Inner, predicate: price = max(price), output: all } ├─StreamExchange { dist: HashShard(price) } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } - | └─StreamShare { id = 5 } + | └─StreamShare { id = 4 } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -520,7 +520,7 @@ └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [max(price), count] } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } - └─StreamShare { id = 5 } + └─StreamShare { id = 4 } └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -1417,7 +1417,7 @@ | | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } | └─StreamExchange { dist: HashShard(auction) } | └─StreamProject { exprs: [auction, _row_id] } - | └─StreamShare { id = 10 } + | └─StreamShare { id = 8 } | └─StreamProject { exprs: [auction, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -1428,7 +1428,7 @@ └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count] } └─StreamExchange { dist: HashShard(auction) } └─StreamProject { exprs: [auction, _row_id] } - └─StreamShare { id = 10 } + └─StreamShare { id = 8 } └─StreamProject { exprs: [auction, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/pg_catalog.yaml b/src/frontend/planner_test/tests/testdata/pg_catalog.yaml index 4e85889110d17..06ba337441bfd 100644 --- a/src/frontend/planner_test/tests/testdata/pg_catalog.yaml +++ b/src/frontend/planner_test/tests/testdata/pg_catalog.yaml @@ -25,7 +25,7 @@ logical_plan: | LogicalProject { exprs: [pg_user.name] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalValues { rows: [[]] } + ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalProject { exprs: [pg_user.name] } └─LogicalFilter { predicate: (1:Int32 = pg_user.usesysid) } └─LogicalScan { table: pg_user, columns: [pg_user.usesysid, pg_user.name, pg_user.usecreatedb, pg_user.usesuper, pg_user.passwd] } @@ -39,6 +39,6 @@ select 'pg_namespace'::regclass logical_plan: | LogicalProject { exprs: [2:Int32] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | BatchValues { rows: [[2:Int32]] } diff --git a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml index 07ef66e8dd710..f69991326119f 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -10,7 +10,7 @@ └─LogicalProject { exprs: [t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id] } optimized_logical_plan_for_batch: | - LogicalValues { rows: [] } + LogicalValues { rows: [], schema: Schema { fields: [min(t.v1):Int32] } } - name: filter should not transpose limit sql: | create table t(v1 int, v2 int, v3 int, v4 int); diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index bbf8bd46e0c8a..ea3c8af16dcbe 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -43,7 +43,7 @@ ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } | └─StreamFilter { predicate: (initial_bid = 1:Int32) } - | └─StreamShare { id = 7 } + | └─StreamShare { id = 6 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 10 } @@ -51,7 +51,7 @@ └─StreamExchange { dist: HashShard(id) } └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } - └─StreamShare { id = 7 } + └─StreamShare { id = 6 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 10 } @@ -115,7 +115,7 @@ └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } ├─StreamExchange { dist: HashShard(window_start) } | └─StreamProject { exprs: [auction, count, window_start] } - | └─StreamShare { id = 12 } + | └─StreamShare { id = 11 } | └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -127,7 +127,7 @@ └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } └─StreamExchange { dist: HashShard(window_start) } └─StreamProject { exprs: [auction, window_start, count] } - └─StreamShare { id = 12 } + └─StreamShare { id = 11 } └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count] } └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } @@ -192,13 +192,13 @@ └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id] } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } - | └─StreamShare { id = 5 } + | └─StreamShare { id = 4 } | └─StreamProject { exprs: [id, _row_id] } | └─StreamRowIdGen { row_id_index: 10 } | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } └─StreamExchange { dist: HashShard(id) } └─StreamProject { exprs: [id, _row_id] } - └─StreamShare { id = 5 } + └─StreamShare { id = 4 } └─StreamProject { exprs: [id, _row_id] } └─StreamRowIdGen { row_id_index: 10 } └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/struct_query.yaml b/src/frontend/planner_test/tests/testdata/struct_query.yaml index cf402c0d3b858..9668fa9a2a586 100644 --- a/src/frontend/planner_test/tests/testdata/struct_query.yaml +++ b/src/frontend/planner_test/tests/testdata/struct_query.yaml @@ -341,7 +341,7 @@ insert into s values (1,2,(1,2,(1,2,null))); logical_plan: | LogicalInsert { table: s } - └─LogicalValues { rows: [[1:Int32, 2:Int32, Row(1:Int32, 2:Int32, Row(1:Int32, 2:Int32, null:Int32))]] } + └─LogicalValues { rows: [[1:Int32, 2:Int32, Row(1:Int32, 2:Int32, Row(1:Int32, 2:Int32, null:Int32))]], schema: Schema { fields: [*VALUES*_0.column_0:Int32, *VALUES*_0.column_1:Int32, *VALUES*_0.column_2:Struct(StructType { fields: [Int32, Int32, Struct(StructType { fields: [Int32, Int32, Int32], field_names: ["v1", "v2", "v3"] })], field_names: ["v1", "v2", "v3"] })] } } create_table_with_connector: row_format: protobuf name: s @@ -399,4 +399,4 @@ select CASE WHEN false THEN ROW(0, INTERVAL '1') WHEN true THEN ROW(1.1, INTERVAL '1') ELSE ROW(1, INTERVAL '1') END; logical_plan: | LogicalProject { exprs: [Case(false:Boolean, Row(0:Int32::Decimal, '00:00:01':Interval), true:Boolean, Row(1.1:Decimal, '00:00:01':Interval), Row(1:Int32::Decimal, '00:00:01':Interval)) as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } diff --git a/src/frontend/planner_test/tests/testdata/subquery.yaml b/src/frontend/planner_test/tests/testdata/subquery.yaml index 2cb8bde0b2b5d..d2937d06d79ac 100644 --- a/src/frontend/planner_test/tests/testdata/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery.yaml @@ -215,7 +215,7 @@ └─LogicalProject { exprs: [CorrelatedInputRef { index: 0, correlated_id: 1 } as $expr2] } └─LogicalAgg { group_key: [$expr1], aggs: [] } └─LogicalProject { exprs: [CorrelatedInputRef { index: 0, correlated_id: 1 } as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } optimized_logical_plan_for_batch: | LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(auction.date_time, auction.date_time), output: all } ├─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: all } @@ -227,7 +227,7 @@ ├─LogicalAgg { group_key: [auction.date_time], aggs: [] } | └─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time] } | └─LogicalScan { table: auction, columns: [auction.date_time], predicate: IsNotNull(auction.date_time) } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: LeftSemi, predicate: auction.date_time IS NOT DISTINCT FROM auction.date_time, output: all } @@ -261,13 +261,13 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 0, correlated_id: 1 } > 1:Int32) } └─LogicalAgg { aggs: [] } └─LogicalProject { exprs: [] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftSemi, on: true, output: all } ├─LogicalScan { table: t, output_columns: [], required_columns: [t.v], predicate: (t.v > 1:Int32) } └─LogicalAgg { aggs: [] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | create table a (a1 int, a2 int); create table b (b1 int, b2 int); diff --git a/src/frontend/planner_test/tests/testdata/subquery_expr.yaml b/src/frontend/planner_test/tests/testdata/subquery_expr.yaml index e98ea69db858e..6ca7ba07b2e02 100644 --- a/src/frontend/planner_test/tests/testdata/subquery_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery_expr.yaml @@ -4,13 +4,13 @@ logical_plan: | LogicalProject { exprs: [1:Int32] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalValues { rows: [[]] } + ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } optimized_logical_plan_for_batch: | LogicalJoin { type: LeftOuter, on: true, output: all } - ├─LogicalValues { rows: [[]] } - └─LogicalValues { rows: [[1:Int32]] } + ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [1:Int32:Int32] } } - sql: | create table t(x int); select (select x from t), 1 from t; @@ -81,11 +81,11 @@ LogicalProject { exprs: [t.x, 1:Int32] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } ├─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - | ├─LogicalValues { rows: [[]] } + | ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } | └─LogicalProject { exprs: [t.x] } | └─LogicalScan { table: t, columns: [t.x, t._row_id] } └─LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } optimizer_error: 'internal error: Scalar subquery might produce more than one row.' - sql: | create table t(x int); @@ -119,13 +119,13 @@ logical_plan: | LogicalProject { exprs: [1:Int32] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalValues { rows: [[]] } + ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalProject { exprs: [t.x] } └─LogicalScan { table: t, columns: [t.x, t._row_id] } optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftSemi, on: true, output: all } - ├─LogicalValues { rows: [[]] } + ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalScan { table: t, columns: [] } - sql: | create table t(x int); @@ -133,13 +133,13 @@ logical_plan: | LogicalProject { exprs: [1:Int32] } └─LogicalApply { type: LeftAnti, on: true, correlated_id: 1 } - ├─LogicalValues { rows: [[]] } + ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalProject { exprs: [t.x] } └─LogicalScan { table: t, columns: [t.x, t._row_id] } optimized_logical_plan_for_batch: | LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftAnti, on: true, output: all } - ├─LogicalValues { rows: [[]] } + ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalScan { table: t, columns: [] } - sql: | create table t1(x int); @@ -177,8 +177,8 @@ LogicalProject { exprs: [1:Int32] } └─LogicalFilter { predicate: (1:Int32 > 0:Int32) } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalValues { rows: [[]] } - └─LogicalValues { rows: [[1:Int32]] } + ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + └─LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [*VALUES*_0.column_0:Int32] } } - sql: | select 1 where (not exists (values (1))) and (1>0 or exists (values (1))) logical_plan: | @@ -186,23 +186,23 @@ └─LogicalFilter { predicate: ((1:Int32 > 0:Int32) OR $expr1) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true } ├─LogicalApply { type: LeftAnti, on: true, correlated_id: 1 } - | ├─LogicalValues { rows: [[]] } - | └─LogicalValues { rows: [[1:Int32]] } + | ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + | └─LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [*VALUES*_0.column_0:Int32] } } └─LogicalProject { exprs: [(count >= 1:Int32) as $expr1] } └─LogicalAgg { aggs: [count] } - └─LogicalValues { rows: [[1:Int32]] } + └─LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [*VALUES*_1.column_0:Int32] } } - sql: | select a + 1, b::varchar, c from (values (1, 2, 3), (4, 5, 6)) t(a, b, c); logical_plan: | LogicalProject { exprs: [(*VALUES*_0.column_0 + 1:Int32) as $expr1, *VALUES*_0.column_1::Varchar as $expr2, *VALUES*_0.column_2] } - └─LogicalValues { rows: [[1:Int32, 2:Int32, 3:Int32], [4:Int32, 5:Int32, 6:Int32]] } + └─LogicalValues { rows: [[1:Int32, 2:Int32, 3:Int32], [4:Int32, 5:Int32, 6:Int32]], schema: Schema { fields: [*VALUES*_0.column_0:Int32, *VALUES*_0.column_1:Int32, *VALUES*_0.column_2:Int32] } } - sql: | select sum(a), max(b + c + 10), string_agg(c::varchar || '~', ',') from (values (1, 2, 3), (4, 5, 6)) as t(a, b, c); logical_plan: | LogicalProject { exprs: [sum(*VALUES*_0.column_0), max($expr1), string_agg($expr2, ',':Varchar)] } └─LogicalAgg { aggs: [sum(*VALUES*_0.column_0), max($expr1), string_agg($expr2, ',':Varchar)] } └─LogicalProject { exprs: [*VALUES*_0.column_0, ((*VALUES*_0.column_1 + *VALUES*_0.column_2) + 10:Int32) as $expr1, ConcatOp(*VALUES*_0.column_2::Varchar, '~':Varchar) as $expr2, ',':Varchar] } - └─LogicalValues { rows: [[1:Int32, 2:Int32, 3:Int32], [4:Int32, 5:Int32, 6:Int32]] } + └─LogicalValues { rows: [[1:Int32, 2:Int32, 3:Int32], [4:Int32, 5:Int32, 6:Int32]], schema: Schema { fields: [*VALUES*_0.column_0:Int32, *VALUES*_0.column_1:Int32, *VALUES*_0.column_2:Int32] } } - sql: | select 1 + (select 2 from t); binder_error: 'Catalog error: table or source not found: t' diff --git a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml index e6f414327d2bb..22386033deee8 100644 --- a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml @@ -40,7 +40,7 @@ └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true } ├─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } └─LogicalProject { exprs: [CorrelatedInputRef { index: 1, correlated_id: 1 } as $expr1] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -64,7 +64,7 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: t3, columns: [t3.x, t3.y, t3._row_id] } └─LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -679,7 +679,7 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalAgg { group_key: [t1.a], aggs: [] } | └─LogicalScan { table: t1, columns: [t1.a] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | create table t1(x int, y int); create table t2(x int, y int); diff --git a/src/frontend/planner_test/tests/testdata/time_window.yaml b/src/frontend/planner_test/tests/testdata/time_window.yaml index c23cfed1c7667..f16bd8310347f 100644 --- a/src/frontend/planner_test/tests/testdata/time_window.yaml +++ b/src/frontend/planner_test/tests/testdata/time_window.yaml @@ -206,7 +206,7 @@ LogicalProject { exprs: [*VALUES*_0.column_0, $expr1, $expr2] } └─LogicalProject { exprs: [*VALUES*_0.column_0, TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) as $expr1, (TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) + '00:00:10':Interval) as $expr2] } └─LogicalShare { id = 1 } - └─LogicalValues { rows: [['2020-01-01 12:00:00':Timestamp]] } + └─LogicalValues { rows: [['2020-01-01 12:00:00':Timestamp]], schema: Schema { fields: [*VALUES*_0.column_0:Timestamp] } } batch_plan: | BatchProject { exprs: [*VALUES*_0.column_0, TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) as $expr1, (TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) + '00:00:10':Interval) as $expr2] } └─BatchValues { rows: [['2020-01-01 12:00:00':Timestamp]] } diff --git a/src/frontend/planner_test/tests/testdata/types.yaml b/src/frontend/planner_test/tests/testdata/types.yaml index 6868a4558c17b..34d561c046fc2 100644 --- a/src/frontend/planner_test/tests/testdata/types.yaml +++ b/src/frontend/planner_test/tests/testdata/types.yaml @@ -5,19 +5,19 @@ No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml - sql: values ('1'::float); logical_plan: | - LogicalValues { rows: [[1:Float64]] } + LogicalValues { rows: [[1:Float64]], schema: Schema { fields: [*VALUES*_0.column_0:Float64] } } - sql: values ('1'::float(24)); logical_plan: | - LogicalValues { rows: [[1:Float32]] } + LogicalValues { rows: [[1:Float32]], schema: Schema { fields: [*VALUES*_0.column_0:Float32] } } - sql: values ('1'::float(25)); logical_plan: | - LogicalValues { rows: [[1:Float64]] } + LogicalValues { rows: [[1:Float64]], schema: Schema { fields: [*VALUES*_0.column_0:Float64] } } - sql: values ('1'::float(53)); logical_plan: | - LogicalValues { rows: [[1:Float64]] } + LogicalValues { rows: [[1:Float64]], schema: Schema { fields: [*VALUES*_0.column_0:Float64] } } - sql: values (''::timestamp with time zone); logical_plan: | - LogicalValues { rows: [['':Varchar::Timestamptz]] } + LogicalValues { rows: [['':Varchar::Timestamptz]], schema: Schema { fields: [*VALUES*_0.column_0:Timestamptz] } } - sql: values (''::time with time zone); binder_error: |- Feature is not yet implemented: unsupported data type: TIME WITH TIME ZONE @@ -25,7 +25,7 @@ - sql: select '1'::iNt2; logical_plan: | LogicalProject { exprs: [1:Int16] } - └─LogicalValues { rows: [[]] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: select ''::"iNt2"; binder_error: |- Feature is not yet implemented: unsupported data type: "iNt2" diff --git a/src/frontend/planner_test/tests/testdata/union.yaml b/src/frontend/planner_test/tests/testdata/union.yaml index 4eef96a626883..af635f1b3fa28 100644 --- a/src/frontend/planner_test/tests/testdata/union.yaml +++ b/src/frontend/planner_test/tests/testdata/union.yaml @@ -235,20 +235,20 @@ - sql: | select 1 union all select 1 optimized_logical_plan_for_batch: | - LogicalValues { rows: [[1:Int32], [1:Int32]] } + LogicalValues { rows: [[1:Int32], [1:Int32]], schema: Schema { fields: [1:Int32:Int32] } } batch_plan: | BatchValues { rows: [[1:Int32], [1:Int32]] } - sql: | select 1 union all select 2 union all select 3 union all select 4 union all select 5 optimized_logical_plan_for_batch: | - LogicalValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32]] } + LogicalValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32]], schema: Schema { fields: [1:Int32:Int32] } } batch_plan: | BatchValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32]] } - sql: | select 1 union select 2 union select 3 union select 4 union select 5 union select 5 optimized_logical_plan_for_batch: | LogicalAgg { group_key: [1:Int32], aggs: [] } - └─LogicalValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32], [5:Int32]] } + └─LogicalValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32], [5:Int32]], schema: Schema { fields: [1:Int32:Int32] } } batch_plan: | BatchExchange { order: [], dist: Single } └─BatchHashAgg { group_key: [1:Int32], aggs: [] } diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index 08100aaff1445..9bdf673a9bf97 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -19,7 +19,6 @@ use itertools::Itertools; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::Result; use risingwave_common::types::{DataType, ScalarImpl}; -use risingwave_common::util::iter_util::ZipEqFast; use super::{ BatchValues, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, From eff23bb118b68546e86c968fac77e0735516c22a Mon Sep 17 00:00:00 2001 From: Clearlove Date: Tue, 28 Mar 2023 15:51:54 -0400 Subject: [PATCH 13/17] more refractor Signed-off-by: Clearlove --- src/frontend/src/optimizer/plan_node/mod.rs | 2 +- src/stream/src/executor/mod.rs | 3 ++- src/stream/src/executor/values.rs | 1 - src/stream/src/from_proto/values.rs | 3 +-- 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 65cd475e7ee52..500962f9c3956 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -843,7 +843,7 @@ macro_rules! for_all_plan_nodes { , { Stream, Share } , { Stream, WatermarkFilter } , { Stream, TemporalJoin } - , { Stream, Values} + , { Stream, Values } } }; } diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index dac2ab6c01b94..ad522996e26f4 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -90,7 +90,7 @@ pub mod subtask; mod temporal_join; mod top_n; mod union; -pub mod values; +mod values; mod watermark; mod watermark_filter; mod wrapper; @@ -135,6 +135,7 @@ pub use top_n::{ AppendOnlyGroupTopNExecutor, AppendOnlyTopNExecutor, GroupTopNExecutor, TopNExecutor, }; pub use union::UnionExecutor; +pub use values::ValuesExecutor; pub use watermark_filter::WatermarkFilterExecutor; pub use wrapper::WrapperExecutor; diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 8b2822cacff16..5219a053dc741 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -150,7 +150,6 @@ mod tests { use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_expr::expr::{BoxedExpression, LiteralExpression}; - use risingwave_pb::stream_plan::Dispatcher; use tokio::sync::mpsc::unbounded_channel; use super::ValuesExecutor; diff --git a/src/stream/src/from_proto/values.rs b/src/stream/src/from_proto/values.rs index 69a60480c15d3..01f02c92ac47f 100644 --- a/src/stream/src/from_proto/values.rs +++ b/src/stream/src/from_proto/values.rs @@ -21,8 +21,7 @@ use tokio::sync::mpsc::unbounded_channel; use super::ExecutorBuilder; use crate::error::StreamResult; -use crate::executor::values::ValuesExecutor; -use crate::executor::BoxedExecutor; +use crate::executor::{ValuesExecutor, BoxedExecutor}; use crate::task::{ExecutorParams, LocalStreamManagerCore}; /// Build a `ValuesExecutor` for stream. As is a leaf, current workaround registers a `sender` for From d122214e76fea972a37971af9a4a57f504250353 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Tue, 28 Mar 2023 15:53:59 -0400 Subject: [PATCH 14/17] check --- src/stream/src/from_proto/values.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/stream/src/from_proto/values.rs b/src/stream/src/from_proto/values.rs index 01f02c92ac47f..b12208a329982 100644 --- a/src/stream/src/from_proto/values.rs +++ b/src/stream/src/from_proto/values.rs @@ -21,7 +21,7 @@ use tokio::sync::mpsc::unbounded_channel; use super::ExecutorBuilder; use crate::error::StreamResult; -use crate::executor::{ValuesExecutor, BoxedExecutor}; +use crate::executor::{BoxedExecutor, ValuesExecutor}; use crate::task::{ExecutorParams, LocalStreamManagerCore}; /// Build a `ValuesExecutor` for stream. As is a leaf, current workaround registers a `sender` for From 988dbb8dd7d2e2e27ee651fccb15bb5a215cf9cb Mon Sep 17 00:00:00 2001 From: Clearlove Date: Tue, 28 Mar 2023 16:06:25 -0400 Subject: [PATCH 15/17] use streamexecutortestext --- src/stream/src/executor/values.rs | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 5219a053dc741..2efb55bd052b5 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -77,7 +77,7 @@ impl ValuesExecutor { .await .unwrap(); - let emit = !barrier.is_newly_added(self.ctx.id); + let emit = barrier.is_newly_added(self.ctx.id); yield Message::Barrier(barrier); // If it's failover, do not evaluate rows (assume they have been yielded) @@ -153,7 +153,8 @@ mod tests { use tokio::sync::mpsc::unbounded_channel; use super::ValuesExecutor; - use crate::executor::{ActorContext, Barrier, Executor, Message, Mutation}; + use crate::executor::test_utils::StreamExecutorTestExt; + use crate::executor::{ActorContext, Barrier, Executor, Mutation}; #[tokio::test] async fn test_values() { @@ -206,8 +207,8 @@ mod tests { tx.send(first_message).unwrap(); assert!(matches!( - values_executor.next().await.unwrap().unwrap(), - Message::Barrier { .. } + values_executor.next_unwrap_ready_barrier().unwrap(), + Barrier { .. } )); // Consume the barrier @@ -253,8 +254,8 @@ mod tests { tx.send(Barrier::new_test_barrier(2)).unwrap(); assert!(matches!( - values_executor.next().await.unwrap().unwrap(), - Message::Barrier { .. } + values_executor.next_unwrap_ready_barrier().unwrap(), + Barrier { .. } )); } } From 2fb8d74bc5db5143e083f8b60197257c30b772cf Mon Sep 17 00:00:00 2001 From: Clearlove Date: Tue, 28 Mar 2023 23:34:21 -0400 Subject: [PATCH 16/17] report create mv progress Signed-off-by: Clearlove --- src/stream/src/executor/values.rs | 18 ++++++++++++++++-- src/stream/src/from_proto/values.rs | 4 ++++ 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 2efb55bd052b5..592527c81a8e6 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -28,6 +28,7 @@ use super::{ ActorContextRef, Barrier, BoxedMessageStream, Executor, Message, PkIndices, PkIndicesRef, StreamExecutorError, }; +use crate::task::CreateMviewProgress; const DEFAULT_CHUNK_SIZE: usize = 1024; @@ -37,6 +38,7 @@ pub struct ValuesExecutor { ctx: ActorContextRef, // Receiver of barrier channel. barrier_receiver: UnboundedReceiver, + progress: CreateMviewProgress, rows: vec::IntoIter>, pk_indices: PkIndices, @@ -48,6 +50,7 @@ impl ValuesExecutor { /// Currently hard-code the `pk_indices` as the last column. pub fn new( ctx: ActorContextRef, + progress: CreateMviewProgress, rows: Vec>, schema: Schema, barrier_receiver: UnboundedReceiver, @@ -55,6 +58,7 @@ impl ValuesExecutor { ) -> Self { Self { ctx, + progress, barrier_receiver, rows: rows.into_iter(), pk_indices: vec![schema.len()], @@ -66,6 +70,7 @@ impl ValuesExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] async fn into_stream(self) { let Self { + mut progress, mut barrier_receiver, schema, mut rows, @@ -117,6 +122,7 @@ impl ValuesExecutor { } while let Some(barrier) = barrier_receiver.recv().await { + progress.finish(barrier.epoch.curr); yield Message::Barrier(barrier); } } @@ -142,6 +148,8 @@ impl Executor for ValuesExecutor { #[cfg(test)] mod tests { + use std::sync::Arc; + use futures::StreamExt; use risingwave_common::array; use risingwave_common::array::{ @@ -155,9 +163,14 @@ mod tests { use super::ValuesExecutor; use crate::executor::test_utils::StreamExecutorTestExt; use crate::executor::{ActorContext, Barrier, Executor, Mutation}; + use crate::task::{CreateMviewProgress, LocalBarrierManager}; #[tokio::test] async fn test_values() { + let barrier_manager = LocalBarrierManager::for_test(); + let progress = + CreateMviewProgress::for_test(Arc::new(parking_lot::Mutex::new(barrier_manager))); + let actor_id = progress.actor_id(); let (tx, barrier_receiver) = unbounded_channel(); let value = StructValue::new(vec![Some(1.into()), Some(2.into()), Some(3.into())]); let exprs = vec![ @@ -190,7 +203,8 @@ mod tests { .map(|col| Field::unnamed(col.return_type())) .collect::>(); let values_executor_struct = ValuesExecutor::new( - ActorContext::create(1), + ActorContext::create(actor_id), + progress, vec![exprs], Schema { fields }, barrier_receiver, @@ -201,7 +215,7 @@ mod tests { // Init barrier let first_message = Barrier::new_test_barrier(1).with_mutation(Mutation::Add { adds: Default::default(), - added_actors: maplit::hashset! {1}, + added_actors: maplit::hashset! {actor_id}, splits: Default::default(), }); tx.send(first_message).unwrap(); diff --git a/src/stream/src/from_proto/values.rs b/src/stream/src/from_proto/values.rs index b12208a329982..efb5532fa62c6 100644 --- a/src/stream/src/from_proto/values.rs +++ b/src/stream/src/from_proto/values.rs @@ -38,6 +38,9 @@ impl ExecutorBuilder for ValuesExecutorBuilder { _store: impl StateStore, stream: &mut LocalStreamManagerCore, ) -> StreamResult { + let progress = stream + .context + .register_create_mview_progress(params.actor_context.id); let (sender, barrier_receiver) = unbounded_channel(); stream .context @@ -57,6 +60,7 @@ impl ExecutorBuilder for ValuesExecutorBuilder { let schema = Schema::new(node.get_fields().iter().map(Field::from).collect_vec()); Ok(Box::new(ValuesExecutor::new( params.actor_context, + progress, rows, schema, barrier_receiver, From c9f76dac86cd27a4bc08b84909051187dfc3c3a4 Mon Sep 17 00:00:00 2001 From: Clearlove Date: Wed, 29 Mar 2023 13:25:55 -0400 Subject: [PATCH 17/17] register values' actor to meta --- src/meta/src/barrier/command.rs | 7 ++++++- src/meta/src/model/stream.rs | 7 +++++++ src/stream/src/executor/values.rs | 11 ++++++++++- src/stream/src/from_proto/values.rs | 6 +++--- 4 files changed, 26 insertions(+), 5 deletions(-) diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 2f21003b0ca05..8a14e91bade2b 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -458,10 +458,15 @@ where /// returns an empty set. pub fn actors_to_track(&self) -> HashSet { match &self.command { - Command::CreateStreamingJob { dispatchers, .. } => dispatchers + Command::CreateStreamingJob { + dispatchers, + table_fragments, + .. + } => dispatchers .values() .flatten() .flat_map(|dispatcher| dispatcher.downstream_actor_id.iter().copied()) + .chain(table_fragments.values_actor_ids().into_iter()) .collect(), _ => Default::default(), diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index a77a55e18821c..9509e9265fce5 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -274,6 +274,13 @@ impl TableFragments { }) } + /// Returns values actor ids. + pub fn values_actor_ids(&self) -> Vec { + Self::filter_actor_ids(self, |fragment_type_mask| { + (fragment_type_mask & FragmentTypeFlag::Values as u32) != 0 + }) + } + /// Returns the fragment with the `Mview` type flag. pub fn mview_fragment(&self) -> Option { self.fragments diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 592527c81a8e6..ace9c0861f965 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -122,7 +122,9 @@ impl ValuesExecutor { } while let Some(barrier) = barrier_receiver.recv().await { - progress.finish(barrier.epoch.curr); + if emit { + progress.finish(barrier.epoch.curr); + } yield Message::Barrier(barrier); } } @@ -271,5 +273,12 @@ mod tests { values_executor.next_unwrap_ready_barrier().unwrap(), Barrier { .. } )); + + tx.send(Barrier::new_test_barrier(3)).unwrap(); + + assert!(matches!( + values_executor.next_unwrap_ready_barrier().unwrap(), + Barrier { .. } + )); } } diff --git a/src/stream/src/from_proto/values.rs b/src/stream/src/from_proto/values.rs index efb5532fa62c6..6857e1fc56ef9 100644 --- a/src/stream/src/from_proto/values.rs +++ b/src/stream/src/from_proto/values.rs @@ -38,14 +38,14 @@ impl ExecutorBuilder for ValuesExecutorBuilder { _store: impl StateStore, stream: &mut LocalStreamManagerCore, ) -> StreamResult { - let progress = stream - .context - .register_create_mview_progress(params.actor_context.id); let (sender, barrier_receiver) = unbounded_channel(); stream .context .lock_barrier_manager() .register_sender(params.actor_context.id, sender); + let progress = stream + .context + .register_create_mview_progress(params.actor_context.id); let rows = node .get_tuples() .iter()