diff --git a/Cargo.lock b/Cargo.lock index 3c360279d27ba..32cb936bdaae3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5495,8 +5495,7 @@ dependencies = [ [[package]] name = "ordered-multimap" version = "0.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ed8acf08e98e744e5384c8bc63ceb0364e68a6854187221c18df61c4797690e" +source = "git+https://github.com/risingwavelabs/ordered-multimap-rs.git?rev=19c743f#19c743f3e3d106c99ba37628f06a2ca6faa2284f" dependencies = [ "dlv-list", "hashbrown 0.13.2", @@ -11205,8 +11204,3 @@ dependencies = [ "libc", "pkg-config", ] - -[[patch.unused]] -name = "ordered-multimap" -version = "0.7.0" -source = "git+https://github.com/risingwavelabs/ordered-multimap-rs.git#c3151128a6f19ca778d1fd1b7e22fd0190fe9e51" diff --git a/Cargo.toml b/Cargo.toml index 31f27f8cb6a43..f8a9b7d0e2fa5 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -244,4 +244,4 @@ etcd-client = { git = "https://github.com/risingwavelabs/etcd-client.git", rev = # Patch for coverage_attribute. # https://github.com/sgodwincs/dlv-list-rs/pull/19#issuecomment-1774786289 dlv-list = { git = "https://github.com/sgodwincs/dlv-list-rs.git", rev = "5bbc5d0" } -ordered-multimap = { git = "https://github.com/risingwavelabs/ordered-multimap-rs.git", dev = "c315112" } +ordered-multimap = { git = "https://github.com/risingwavelabs/ordered-multimap-rs.git", rev = "19c743f" } diff --git a/ci/scripts/deterministic-recovery-test.sh b/ci/scripts/deterministic-recovery-test.sh index 6514fe1f7c0c3..c5f89a2bbc7e0 100755 --- a/ci/scripts/deterministic-recovery-test.sh +++ b/ci/scripts/deterministic-recovery-test.sh @@ -11,6 +11,7 @@ chmod +x ./risingwave_simulation export RUST_LOG="info,\ risingwave_meta::barrier::recovery=debug,\ +risingwave_meta::manager::catalog=debug,\ risingwave_meta::rpc::ddl_controller=debug,\ risingwave_meta::barrier::mod=debug,\ risingwave_simulation=debug" diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index 89aa99a1c8b5d..d25c94daf2670 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -2,7 +2,7 @@ version: "3" services: compactor-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}" command: - compactor-node - "--listen-addr" @@ -37,7 +37,7 @@ services: timeout: 5s retries: 5 compute-node-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}" command: - compute-node - "--listen-addr" @@ -122,7 +122,7 @@ services: timeout: 5s retries: 5 frontend-node-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}" command: - frontend-node - "--listen-addr" @@ -179,7 +179,7 @@ services: timeout: 5s retries: 5 meta-node-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}" command: - meta-node - "--listen-addr" @@ -295,7 +295,7 @@ services: timeout: 5s retries: 5 connector-node: - image: ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0} + image: ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0} entrypoint: "/risingwave/bin/connector-node/start-service.sh" ports: - 50051 diff --git a/integration_tests/redis-sink/create_sink.sql b/integration_tests/redis-sink/create_sink.sql index 03bfc2d0b0df1..2ba9ba67feb39 100644 --- a/integration_tests/redis-sink/create_sink.sql +++ b/integration_tests/redis-sink/create_sink.sql @@ -3,19 +3,13 @@ FROM bhv_mv WITH ( primary_key = 'user_id', connector = 'redis', - type = 'append-only', - force_append_only='true', redis.url= 'redis://127.0.0.1:6379/', -); +)FORMAT PLAIN ENCODE JSON(force_append_only='true'); CREATE SINK bhv_redis_sink_2 FROM bhv_mv WITH ( primary_key = 'user_id', connector = 'redis', - type = 'append-only', - force_append_only='true', redis.url= 'redis://127.0.0.1:6379/', - redis.keyformat='user_id:{user_id}', - redis.valueformat='username:{username},event_timestamp{event_timestamp}' -); \ No newline at end of file +)FORMAT PLAIN ENCODE TEMPLATE(force_append_only='true', key_format = 'UserID:{user_id}', value_format = 'TargetID:{target_id},EventTimestamp{event_timestamp}'); \ No newline at end of file diff --git a/proto/expr.proto b/proto/expr.proto index 769532d8dbe19..2f252d67c8400 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -348,6 +348,7 @@ message AggCall { MODE = 24; LAST_VALUE = 25; GROUPING = 26; + INTERNAL_LAST_SEEN_VALUE = 27; } Type type = 1; repeated InputRef args = 2; diff --git a/proto/plan_common.proto b/proto/plan_common.proto index a88242a572693..d4c7a2e04f138 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -106,6 +106,7 @@ enum EncodeType { ENCODE_TYPE_PROTOBUF = 4; ENCODE_TYPE_JSON = 5; ENCODE_TYPE_BYTES = 6; + ENCODE_TYPE_TEMPLATE = 7; } enum RowFormatType { diff --git a/src/batch/src/executor/aggregation/filter.rs b/src/batch/src/executor/aggregation/filter.rs index 2db2320ed3534..9cfbeabffe417 100644 --- a/src/batch/src/executor/aggregation/filter.rs +++ b/src/batch/src/executor/aggregation/filter.rs @@ -75,7 +75,7 @@ impl AggregateFunction for Filter { mod tests { use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_expr::aggregate::{build_append_only, AggCall}; - use risingwave_expr::expr::{build_from_pretty, Expression, LiteralExpression}; + use risingwave_expr::expr::{build_from_pretty, ExpressionBoxExt, LiteralExpression}; use super::*; diff --git a/src/batch/src/executor/project_set.rs b/src/batch/src/executor/project_set.rs index 670933a6bb50c..fa3dfac917e8a 100644 --- a/src/batch/src/executor/project_set.rs +++ b/src/batch/src/executor/project_set.rs @@ -171,7 +171,7 @@ mod tests { use risingwave_common::catalog::{Field, Schema}; use risingwave_common::test_prelude::*; use risingwave_common::types::DataType; - use risingwave_expr::expr::{Expression, InputRefExpression, LiteralExpression}; + use risingwave_expr::expr::{ExpressionBoxExt, InputRefExpression, LiteralExpression}; use risingwave_expr::table_function::repeat; use super::*; diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index c18dd7d10a92c..ca3a09e7f2eda 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -132,6 +132,7 @@ pub enum SinkEncode { Json, Protobuf, Avro, + Template, } impl SinkFormatDesc { @@ -177,6 +178,7 @@ impl SinkFormatDesc { SinkEncode::Json => E::Json, SinkEncode::Protobuf => E::Protobuf, SinkEncode::Avro => E::Avro, + SinkEncode::Template => E::Template, }; let options = self .options @@ -212,6 +214,7 @@ impl TryFrom for SinkFormatDesc { let encode = match value.encode() { E::Json => SinkEncode::Json, E::Protobuf => SinkEncode::Protobuf, + E::Template => SinkEncode::Template, E::Avro => SinkEncode::Avro, e @ (E::Unspecified | E::Native | E::Csv | E::Bytes) => { return Err(SinkError::Config(anyhow!( diff --git a/src/connector/src/sink/encoder/template.rs b/src/connector/src/sink/encoder/template.rs index 85f085989b6c4..97d8271f9e83a 100644 --- a/src/connector/src/sink/encoder/template.rs +++ b/src/connector/src/sink/encoder/template.rs @@ -12,11 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashSet; + +use regex::Regex; use risingwave_common::catalog::Schema; use risingwave_common::row::Row; use risingwave_common::types::ToText; use super::{Result, RowEncoder}; +use crate::sink::SinkError; /// Encode a row according to a specified string template `user_id:{user_id}` pub struct TemplateEncoder { @@ -34,6 +38,24 @@ impl TemplateEncoder { template, } } + + pub fn check_string_format(format: &str, set: &HashSet) -> Result<()> { + // We will check if the string inside {} corresponds to a column name in rw. + // In other words, the content within {} should exclusively consist of column names from rw, + // which means '{{column_name}}' or '{{column_name1},{column_name2}}' would be incorrect. + let re = Regex::new(r"\{([^}]*)\}").unwrap(); + if !re.is_match(format) { + return Err(SinkError::Redis( + "Can't find {} in key_format or value_format".to_string(), + )); + } + for capture in re.captures_iter(format) { + if let Some(inner_content) = capture.get(1) && !set.contains(inner_content.as_str()){ + return Err(SinkError::Redis(format!("Can't find field({:?}) in key_format or value_format",inner_content.as_str()))) + } + } + Ok(()) + } } impl RowEncoder for TemplateEncoder { diff --git a/src/connector/src/sink/formatter/mod.rs b/src/connector/src/sink/formatter/mod.rs index a7463f7e3b306..17cb708292890 100644 --- a/src/connector/src/sink/formatter/mod.rs +++ b/src/connector/src/sink/formatter/mod.rs @@ -29,6 +29,7 @@ pub use upsert::UpsertFormatter; use super::catalog::{SinkEncode, SinkFormat, SinkFormatDesc}; use super::encoder::template::TemplateEncoder; use super::encoder::KafkaConnectParams; +use super::redis::{KEY_FORMAT, VALUE_FORMAT}; use crate::sink::encoder::{JsonEncoder, ProtoEncoder, TimestampHandlingMode}; /// Transforms a `StreamChunk` into a sequence of key-value pairs according a specific format, @@ -92,7 +93,7 @@ impl SinkFormatterImpl { let key_encoder = (!pk_indices.is_empty()).then(|| { JsonEncoder::new( schema.clone(), - Some(pk_indices), + Some(pk_indices.clone()), TimestampHandlingMode::Milli, ) }); @@ -115,6 +116,28 @@ impl SinkFormatterImpl { Ok(SinkFormatterImpl::AppendOnlyProto(formatter)) } SinkEncode::Avro => err_unsupported(), + SinkEncode::Template => { + let key_format = format_desc.options.get(KEY_FORMAT).ok_or_else(|| { + SinkError::Config(anyhow!( + "Cannot find 'key_format',please set it or use JSON" + )) + })?; + let value_format = + format_desc.options.get(VALUE_FORMAT).ok_or_else(|| { + SinkError::Config(anyhow!( + "Cannot find 'redis_value_format',please set it or use JSON" + )) + })?; + let key_encoder = TemplateEncoder::new( + schema.clone(), + Some(pk_indices), + key_format.clone(), + ); + let val_encoder = TemplateEncoder::new(schema, None, value_format.clone()); + Ok(SinkFormatterImpl::AppendOnlyTemplate( + AppendOnlyFormatter::new(Some(key_encoder), val_encoder), + )) + } } } SinkFormat::Debezium => { @@ -131,85 +154,66 @@ impl SinkFormatterImpl { ))) } SinkFormat::Upsert => { - if format_desc.encode != SinkEncode::Json { - return err_unsupported(); - } + match format_desc.encode { + SinkEncode::Json => { + let mut key_encoder = JsonEncoder::new( + schema.clone(), + Some(pk_indices), + TimestampHandlingMode::Milli, + ); + let mut val_encoder = + JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); - let mut key_encoder = JsonEncoder::new( - schema.clone(), - Some(pk_indices), - TimestampHandlingMode::Milli, - ); - let mut val_encoder = JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); - - if let Some(s) = format_desc.options.get("schemas.enable") { - match s.to_lowercase().parse::() { - Ok(true) => { - let kafka_connect = KafkaConnectParams { - schema_name: format!("{}.{}", db_name, sink_from_name), - }; - key_encoder = key_encoder.with_kafka_connect(kafka_connect.clone()); - val_encoder = val_encoder.with_kafka_connect(kafka_connect); - } - Ok(false) => {} - _ => { - return Err(SinkError::Config(anyhow!( - "schemas.enable is expected to be `true` or `false`, got {}", - s - ))); - } + if let Some(s) = format_desc.options.get("schemas.enable") { + match s.to_lowercase().parse::() { + Ok(true) => { + let kafka_connect = KafkaConnectParams { + schema_name: format!("{}.{}", db_name, sink_from_name), + }; + key_encoder = + key_encoder.with_kafka_connect(kafka_connect.clone()); + val_encoder = val_encoder.with_kafka_connect(kafka_connect); + } + Ok(false) => {} + _ => { + return Err(SinkError::Config(anyhow!( + "schemas.enable is expected to be `true` or `false`, got {}", + s + ))); + } + } + }; + + // Initialize the upsert_stream + let formatter = UpsertFormatter::new(key_encoder, val_encoder); + Ok(SinkFormatterImpl::UpsertJson(formatter)) } - }; - - // Initialize the upsert_stream - let formatter = UpsertFormatter::new(key_encoder, val_encoder); - Ok(SinkFormatterImpl::UpsertJson(formatter)) - } - } - } - - pub fn new_with_redis( - schema: Schema, - pk_indices: Vec, - is_append_only: bool, - key_format: Option, - value_format: Option, - ) -> Result { - match (key_format, value_format) { - (Some(k), Some(v)) => { - let key_encoder = TemplateEncoder::new( - schema.clone(), - Some(pk_indices), - k, - ); - let val_encoder = - TemplateEncoder::new(schema, None, v); - if is_append_only { - Ok(SinkFormatterImpl::AppendOnlyTemplate(AppendOnlyFormatter::new(Some(key_encoder), val_encoder))) - } else { - Ok(SinkFormatterImpl::UpsertTemplate(UpsertFormatter::new(key_encoder, val_encoder))) - } - } - (None, None) => { - let key_encoder = JsonEncoder::new( - schema.clone(), - Some(pk_indices), - TimestampHandlingMode::Milli, - ); - let val_encoder = JsonEncoder::new( - schema, - None, - TimestampHandlingMode::Milli, - ); - if is_append_only { - Ok(SinkFormatterImpl::AppendOnlyJson(AppendOnlyFormatter::new(Some(key_encoder), val_encoder))) - } else { - Ok(SinkFormatterImpl::UpsertJson(UpsertFormatter::new(key_encoder, val_encoder))) + SinkEncode::Template => { + let key_format = format_desc.options.get(KEY_FORMAT).ok_or_else(|| { + SinkError::Config(anyhow!( + "Cannot find 'key_format',please set it or use JSON" + )) + })?; + let value_format = + format_desc.options.get(VALUE_FORMAT).ok_or_else(|| { + SinkError::Config(anyhow!( + "Cannot find 'redis_value_format',please set it or use JSON" + )) + })?; + let key_encoder = TemplateEncoder::new( + schema.clone(), + Some(pk_indices), + key_format.clone(), + ); + let val_encoder = TemplateEncoder::new(schema, None, value_format.clone()); + Ok(SinkFormatterImpl::UpsertTemplate(UpsertFormatter::new( + key_encoder, + val_encoder, + ))) + } + _ => err_unsupported(), } } - _ => { - Err(SinkError::Encode("Please provide template formats for both key and value, or choose the JSON format.".to_string())) - } } } } diff --git a/src/connector/src/sink/redis.rs b/src/connector/src/sink/redis.rs index cc8ff74d0c9c5..6120075a049df 100644 --- a/src/connector/src/sink/redis.rs +++ b/src/connector/src/sink/redis.rs @@ -18,29 +18,27 @@ use anyhow::anyhow; use async_trait::async_trait; use redis::aio::Connection; use redis::{Client as RedisClient, Pipeline}; -use regex::Regex; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; use serde_derive::{Deserialize, Serialize}; use serde_with::serde_as; +use super::catalog::SinkFormatDesc; +use super::encoder::template::TemplateEncoder; use super::formatter::SinkFormatterImpl; use super::writer::FormattedSink; -use super::{SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; +use super::{SinkError, SinkParam}; use crate::dispatch_sink_formatter_impl; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; pub const REDIS_SINK: &str = "redis"; - +pub const KEY_FORMAT: &str = "key_format"; +pub const VALUE_FORMAT: &str = "value_format"; #[derive(Deserialize, Serialize, Debug, Clone)] pub struct RedisCommon { #[serde(rename = "redis.url")] pub url: String, - #[serde(rename = "redis.keyformat")] - pub key_format: Option, - #[serde(rename = "redis.valueformat")] - pub value_format: Option, } impl RedisCommon { @@ -54,23 +52,13 @@ impl RedisCommon { pub struct RedisConfig { #[serde(flatten)] pub common: RedisCommon, - - pub r#type: String, // accept "append-only" or "upsert" } impl RedisConfig { pub fn from_hashmap(properties: HashMap) -> Result { let config = serde_json::from_value::(serde_json::to_value(properties).unwrap()) - .map_err(|e| SinkError::Config(anyhow!(e)))?; - if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { - return Err(SinkError::Config(anyhow!( - "`{}` must be {}, or {}", - SINK_TYPE_OPTION, - SINK_TYPE_APPEND_ONLY, - SINK_TYPE_UPSERT - ))); - } + .map_err(|e| SinkError::Config(anyhow!("{:?}", e)))?; Ok(config) } } @@ -79,28 +67,10 @@ impl RedisConfig { pub struct RedisSink { config: RedisConfig, schema: Schema, - is_append_only: bool, pk_indices: Vec, -} - -fn check_string_format(format: &Option, set: &HashSet) -> Result<()> { - if let Some(format) = format { - // We will check if the string inside {} corresponds to a column name in rw. - // In other words, the content within {} should exclusively consist of column names from rw, - // which means '{{column_name}}' or '{{column_name1},{column_name2}}' would be incorrect. - let re = Regex::new(r"\{([^}]*)\}").unwrap(); - if !re.is_match(format) { - return Err(SinkError::Redis( - "Can't find {} in key_format or value_format".to_string(), - )); - } - for capture in re.captures_iter(format) { - if let Some(inner_content) = capture.get(1) && !set.contains(inner_content.as_str()){ - return Err(SinkError::Redis(format!("Can't find field({:?}) in key_format or value_format",inner_content.as_str()))) - } - } - } - Ok(()) + format_desc: SinkFormatDesc, + db_name: String, + sink_from_name: String, } #[async_trait] @@ -117,8 +87,12 @@ impl TryFrom for RedisSink { Ok(Self { config, schema: param.schema(), - is_append_only: param.sink_type.is_append_only(), pk_indices: param.downstream_pk, + format_desc: param + .format_desc + .ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?, + db_name: param.db_name, + sink_from_name: param.sink_from_name, }) } } @@ -134,7 +108,9 @@ impl Sink for RedisSink { self.config.clone(), self.schema.clone(), self.pk_indices.clone(), - self.is_append_only, + &self.format_desc, + self.db_name.clone(), + self.sink_from_name.clone(), ) .await? .into_log_sinker(writer_param.sink_metrics)) @@ -157,8 +133,23 @@ impl Sink for RedisSink { .filter(|(k, _)| self.pk_indices.contains(k)) .map(|(_, v)| v.name.clone()) .collect(); - check_string_format(&self.config.common.key_format, &pk_set)?; - check_string_format(&self.config.common.value_format, &all_set)?; + if matches!( + self.format_desc.encode, + super::catalog::SinkEncode::Template + ) { + let key_format = self.format_desc.options.get(KEY_FORMAT).ok_or_else(|| { + SinkError::Config(anyhow!( + "Cannot find 'key_format',please set it or use JSON" + )) + })?; + let value_format = self.format_desc.options.get(VALUE_FORMAT).ok_or_else(|| { + SinkError::Config(anyhow!( + "Cannot find 'value_format',please set it or use JSON" + )) + })?; + TemplateEncoder::check_string_format(key_format, &pk_set)?; + TemplateEncoder::check_string_format(value_format, &all_set)?; + } Ok(()) } } @@ -166,7 +157,6 @@ impl Sink for RedisSink { pub struct RedisSinkWriter { epoch: u64, schema: Schema, - is_append_only: bool, pk_indices: Vec, formatter: SinkFormatterImpl, payload_writer: RedisSinkPayloadWriter, @@ -220,21 +210,23 @@ impl RedisSinkWriter { config: RedisConfig, schema: Schema, pk_indices: Vec, - is_append_only: bool, + format_desc: &SinkFormatDesc, + db_name: String, + sink_from_name: String, ) -> Result { let payload_writer = RedisSinkPayloadWriter::new(config.clone()).await?; - let formatter = SinkFormatterImpl::new_with_redis( + let formatter = SinkFormatterImpl::new( + format_desc, schema.clone(), pk_indices.clone(), - is_append_only, - config.common.key_format, - config.common.value_format, - )?; + db_name, + sink_from_name, + ) + .await?; Ok(Self { schema, pk_indices, - is_append_only, epoch: 0, formatter, payload_writer, @@ -242,24 +234,22 @@ impl RedisSinkWriter { } #[cfg(test)] - pub fn mock( + pub async fn mock( schema: Schema, pk_indices: Vec, - is_append_only: bool, - key_format: Option, - value_format: Option, + format_desc: &SinkFormatDesc, ) -> Result { - let formatter = SinkFormatterImpl::new_with_redis( + let formatter = SinkFormatterImpl::new( + format_desc, schema.clone(), pk_indices.clone(), - is_append_only, - key_format, - value_format, - )?; + "d1".to_string(), + "t1".to_string(), + ) + .await?; Ok(Self { schema, pk_indices, - is_append_only, epoch: 0, formatter, payload_writer: RedisSinkPayloadWriter::mock(), @@ -290,6 +280,8 @@ impl SinkWriter for RedisSinkWriter { #[cfg(test)] mod test { + use std::collections::BTreeMap; + use rdkafka::message::FromBytes; use risingwave_common::array::{Array, I32Array, Op, StreamChunk, Utf8Array}; use risingwave_common::catalog::{Field, Schema}; @@ -297,6 +289,7 @@ mod test { use risingwave_common::util::iter_util::ZipEqDebug; use super::*; + use crate::sink::catalog::{SinkEncode, SinkFormat}; #[tokio::test] async fn test_write() { @@ -315,8 +308,15 @@ mod test { }, ]); - let mut redis_sink_writer = - RedisSinkWriter::mock(schema, vec![0], true, None, None).unwrap(); + let format_desc = SinkFormatDesc { + format: SinkFormat::AppendOnly, + encode: SinkEncode::Json, + options: BTreeMap::default(), + }; + + let mut redis_sink_writer = RedisSinkWriter::mock(schema, vec![0], &format_desc) + .await + .unwrap(); let chunk_a = StreamChunk::new( vec![Op::Insert, Op::Insert, Op::Insert], @@ -367,14 +367,21 @@ mod test { }, ]); - let mut redis_sink_writer = RedisSinkWriter::mock( - schema, - vec![0], - true, - Some("key-{id}".to_string()), - Some("values:{id:{id},name:{name}}".to_string()), - ) - .unwrap(); + let mut btree_map = BTreeMap::default(); + btree_map.insert(KEY_FORMAT.to_string(), "key-{id}".to_string()); + btree_map.insert( + VALUE_FORMAT.to_string(), + "values:{id:{id},name:{name}}".to_string(), + ); + let format_desc = SinkFormatDesc { + format: SinkFormat::AppendOnly, + encode: SinkEncode::Template, + options: btree_map, + }; + + let mut redis_sink_writer = RedisSinkWriter::mock(schema, vec![0], &format_desc) + .await + .unwrap(); let chunk_a = StreamChunk::new( vec![Op::Insert, Op::Insert, Op::Insert], diff --git a/src/expr/core/src/aggregate/def.rs b/src/expr/core/src/aggregate/def.rs index f71bfd454a415..964ec46c9f9c4 100644 --- a/src/expr/core/src/aggregate/def.rs +++ b/src/expr/core/src/aggregate/def.rs @@ -233,6 +233,9 @@ pub enum AggKind { PercentileDisc, Mode, Grouping, + + /// Return last seen one of the input values. + InternalLastSeenValue, } impl AggKind { @@ -264,6 +267,7 @@ impl AggKind { PbType::PercentileDisc => Ok(AggKind::PercentileDisc), PbType::Mode => Ok(AggKind::Mode), PbType::Grouping => Ok(AggKind::Grouping), + PbType::InternalLastSeenValue => Ok(AggKind::InternalLastSeenValue), PbType::Unspecified => bail!("Unrecognized agg."), } } @@ -294,8 +298,9 @@ impl AggKind { Self::VarSamp => PbType::VarSamp, Self::PercentileCont => PbType::PercentileCont, Self::PercentileDisc => PbType::PercentileDisc, - Self::Grouping => PbType::Grouping, Self::Mode => PbType::Mode, + Self::Grouping => PbType::Grouping, + Self::InternalLastSeenValue => PbType::InternalLastSeenValue, } } } @@ -422,6 +427,7 @@ pub mod agg_kinds { | AggKind::BoolAnd | AggKind::BoolOr | AggKind::ApproxCountDistinct + | AggKind::InternalLastSeenValue }; } pub use single_value_state; @@ -450,7 +456,11 @@ impl AggKind { /// Get the total phase agg kind from the partial phase agg kind. pub fn partial_to_total(self) -> Option { match self { - AggKind::BitXor | AggKind::Min | AggKind::Max | AggKind::Sum => Some(self), + AggKind::BitXor + | AggKind::Min + | AggKind::Max + | AggKind::Sum + | AggKind::InternalLastSeenValue => Some(self), AggKind::Sum0 | AggKind::Count => Some(AggKind::Sum0), agg_kinds::simply_cannot_two_phase!() => None, agg_kinds::rewritten!() => None, diff --git a/src/expr/core/src/expr/build.rs b/src/expr/core/src/expr/build.rs index 1ea03bd36f42a..7dffbcd42d66b 100644 --- a/src/expr/core/src/expr/build.rs +++ b/src/expr/core/src/expr/build.rs @@ -27,8 +27,13 @@ use super::expr_in::InExpression; use super::expr_some_all::SomeAllExpression; use super::expr_udf::UdfExpression; use super::expr_vnode::VnodeExpression; -use super::wrapper::{Checked, EvalErrorReport, NonStrict}; -use crate::expr::{BoxedExpression, Expression, InputRefExpression, LiteralExpression}; +use super::wrapper::checked::Checked; +use super::wrapper::non_strict::NonStrict; +use super::wrapper::EvalErrorReport; +use super::NonStrictExpression; +use crate::expr::{ + BoxedExpression, Expression, ExpressionBoxExt, InputRefExpression, LiteralExpression, +}; use crate::sig::FUNCTION_REGISTRY; use crate::{bail, ExprError, Result}; @@ -41,8 +46,10 @@ pub fn build_from_prost(prost: &ExprNode) -> Result { pub fn build_non_strict_from_prost( prost: &ExprNode, error_report: impl EvalErrorReport + 'static, -) -> Result { - ExprBuilder::new_non_strict(error_report).build(prost) +) -> Result { + ExprBuilder::new_non_strict(error_report) + .build(prost) + .map(NonStrictExpression) } /// Build an expression from protobuf with possibly some wrappers attached to each node. @@ -153,7 +160,7 @@ impl BuildBoxed for E { prost: &ExprNode, build_child: impl Fn(&ExprNode) -> Result, ) -> Result { - Self::build(prost, build_child).map(Expression::boxed) + Self::build(prost, build_child).map(ExpressionBoxExt::boxed) } } @@ -217,9 +224,9 @@ pub fn build_func_non_strict( ret_type: DataType, children: Vec, error_report: impl EvalErrorReport + 'static, -) -> Result { +) -> Result { let expr = build_func(func, ret_type, children)?; - let wrapped = ExprBuilder::new_non_strict(error_report).wrap(expr); + let wrapped = NonStrictExpression(ExprBuilder::new_non_strict(error_report).wrap(expr)); Ok(wrapped) } diff --git a/src/expr/core/src/expr/mod.rs b/src/expr/core/src/expr/mod.rs index 37e0104371a3e..48a46f640bf7b 100644 --- a/src/expr/core/src/expr/mod.rs +++ b/src/expr/core/src/expr/mod.rs @@ -58,7 +58,7 @@ pub use self::build::*; pub use self::expr_input_ref::InputRefExpression; pub use self::expr_literal::LiteralExpression; pub use self::value::{ValueImpl, ValueRef}; -pub use self::wrapper::EvalErrorReport; +pub use self::wrapper::*; pub use super::{ExprError, Result}; /// Interface of an expression. @@ -67,6 +67,7 @@ pub use super::{ExprError, Result}; /// should be implemented. Prefer calling and implementing `eval_v2` instead of `eval` if possible, /// to gain the performance benefit of scalar expression. #[async_trait::async_trait] +#[auto_impl::auto_impl(&, Box)] pub trait Expression: std::fmt::Debug + Sync + Send { /// Get the return data type. fn return_type(&self) -> DataType; @@ -101,23 +102,77 @@ pub trait Expression: std::fmt::Debug + Sync + Send { fn eval_const(&self) -> Result { Err(ExprError::NotConstant) } +} +/// An owned dynamically typed [`Expression`]. +pub type BoxedExpression = Box; + +/// Extension trait for boxing expressions. +/// +/// This is not directly made into [`Expression`] trait because... +/// - an expression does not have to be `'static`, +/// - and for the ease of `auto_impl`. +#[easy_ext::ext(ExpressionBoxExt)] +impl E { /// Wrap the expression in a Box. - fn boxed(self) -> BoxedExpression - where - Self: Sized + Send + 'static, - { + pub fn boxed(self) -> BoxedExpression { Box::new(self) } } -// TODO: make this an extension, or implement it on a `NonStrict` newtype. -impl dyn Expression { +/// An type-safe wrapper that indicates the inner expression can be evaluated in a non-strict +/// manner, i.e., developers can directly call `eval_infallible` and `eval_row_infallible` without +/// checking the result. +/// +/// This is usually created by non-strict build functions like [`crate::expr::build_non_strict_from_prost`] +/// and [`crate::expr::build_func_non_strict`]. It can also be created directly by +/// [`NonStrictExpression::new_topmost`], where only the evaluation of the topmost level expression +/// node is non-strict and should be treated as a TODO. +/// +/// Compared to [`crate::expr::wrapper::non_strict::NonStrict`], this is more like an indicator +/// applied on the root of an expression tree, while the latter is a wrapper that can be applied on +/// each node of the tree and actually changes the behavior. As a result, [`NonStrictExpression`] +/// does not implement [`Expression`] trait and instead deals directly with developers. +#[derive(Debug)] +pub struct NonStrictExpression(E); + +impl NonStrictExpression +where + E: Expression, +{ + /// Create a non-strict expression directly wrapping the given expression. + /// + /// Should only be used in tests as evaluation may panic. + pub fn for_test(inner: E) -> NonStrictExpression + where + E: 'static, + { + NonStrictExpression(inner.boxed()) + } + + /// Create a non-strict expression from the given expression, where only the evaluation of the + /// topmost level expression node is non-strict (which is subtly different from + /// [`crate::expr::build_non_strict_from_prost`] where every node is non-strict). + /// + /// This should be used as a TODO. + pub fn new_topmost( + inner: E, + error_report: impl EvalErrorReport, + ) -> NonStrictExpression { + let inner = wrapper::non_strict::NonStrict::new(inner, error_report); + NonStrictExpression(inner) + } + + /// Get the return data type. + pub fn return_type(&self) -> DataType { + self.0.return_type() + } + /// Evaluate the expression in vectorized execution and assert it succeeds. Returns an array. /// /// Use with expressions built in non-strict mode. pub async fn eval_infallible(&self, input: &DataChunk) -> ArrayRef { - self.eval(input).await.expect("evaluation failed") + self.0.eval(input).await.expect("evaluation failed") } /// Evaluate the expression in row-based execution and assert it succeeds. Returns a nullable @@ -125,38 +180,17 @@ impl dyn Expression { /// /// Use with expressions built in non-strict mode. pub async fn eval_row_infallible(&self, input: &OwnedRow) -> Datum { - self.eval_row(input).await.expect("evaluation failed") - } -} - -/// An owned dynamically typed [`Expression`]. -pub type BoxedExpression = Box; - -// TODO: avoid the overhead of extra boxing. -#[async_trait::async_trait] -impl Expression for BoxedExpression { - fn return_type(&self) -> DataType { - (**self).return_type() + self.0.eval_row(input).await.expect("evaluation failed") } - async fn eval(&self, input: &DataChunk) -> Result { - (**self).eval(input).await - } - - async fn eval_v2(&self, input: &DataChunk) -> Result { - (**self).eval_v2(input).await - } - - async fn eval_row(&self, input: &OwnedRow) -> Result { - (**self).eval_row(input).await - } - - fn eval_const(&self) -> Result { - (**self).eval_const() + /// Unwrap the inner expression. + pub fn into_inner(self) -> E { + self.0 } - fn boxed(self) -> BoxedExpression { - self + /// Get a reference to the inner expression. + pub fn inner(&self) -> &E { + &self.0 } } diff --git a/src/expr/core/src/expr/wrapper/checked.rs b/src/expr/core/src/expr/wrapper/checked.rs index 1e049ad481010..b3b1375c4fa82 100644 --- a/src/expr/core/src/expr/wrapper/checked.rs +++ b/src/expr/core/src/expr/wrapper/checked.rs @@ -22,7 +22,7 @@ use crate::expr::{Expression, ValueImpl}; /// A wrapper of [`Expression`] that does extra checks after evaluation. #[derive(Debug)] -pub struct Checked(pub E); +pub(crate) struct Checked(pub E); // TODO: avoid the overhead of extra boxing. #[async_trait] diff --git a/src/expr/core/src/expr/wrapper/mod.rs b/src/expr/core/src/expr/wrapper/mod.rs index 48241d05de45c..16988a050ad8d 100644 --- a/src/expr/core/src/expr/wrapper/mod.rs +++ b/src/expr/core/src/expr/wrapper/mod.rs @@ -12,8 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod checked; -mod non_strict; +pub(crate) mod checked; +pub(crate) mod non_strict; -pub use checked::Checked; -pub use non_strict::{EvalErrorReport, NonStrict}; +pub use non_strict::{EvalErrorReport, LogReport}; diff --git a/src/expr/core/src/expr/wrapper/non_strict.rs b/src/expr/core/src/expr/wrapper/non_strict.rs index 0859cea27aa49..782456023cdf7 100644 --- a/src/expr/core/src/expr/wrapper/non_strict.rs +++ b/src/expr/core/src/expr/wrapper/non_strict.rs @@ -23,7 +23,7 @@ use crate::expr::{Expression, ValueImpl}; use crate::ExprError; /// Report an error during evaluation. -#[auto_impl(Arc)] +#[auto_impl(&, Arc)] pub trait EvalErrorReport: Clone + Send + Sync { /// Perform the error reporting. /// @@ -42,11 +42,21 @@ impl EvalErrorReport for ! { } } +/// Log the error to report an error during evaluation. +#[derive(Clone)] +pub struct LogReport; + +impl EvalErrorReport for LogReport { + fn report(&self, error: ExprError) { + tracing::error!(%error, "failed to evaluate expression"); + } +} + /// A wrapper of [`Expression`] that evaluates in a non-strict way. Basically... /// - When an error occurs during chunk-level evaluation, recompute in row-based execution and pad /// with NULL for each failed row. /// - Report all error occurred during row-level evaluation to the [`EvalErrorReport`]. -pub struct NonStrict { +pub(crate) struct NonStrict { inner: E, report: R, } diff --git a/src/expr/impl/src/aggregate/general.rs b/src/expr/impl/src/aggregate/general.rs index de1331c524063..f47c94d45f24d 100644 --- a/src/expr/impl/src/aggregate/general.rs +++ b/src/expr/impl/src/aggregate/general.rs @@ -62,6 +62,15 @@ fn last_value(_: T, input: T) -> T { input } +#[aggregate("internal_last_seen_value(*) -> auto", state = "ref")] +fn internal_last_seen_value(state: T, input: T, retract: bool) -> T { + if retract { + state + } else { + input + } +} + /// Note the following corner cases: /// /// ```slt diff --git a/src/expr/impl/src/scalar/cast.rs b/src/expr/impl/src/scalar/cast.rs index 889cc43fe6b18..c173c76c330c5 100644 --- a/src/expr/impl/src/scalar/cast.rs +++ b/src/expr/impl/src/scalar/cast.rs @@ -22,7 +22,9 @@ use risingwave_common::cast; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, Int256, IntoOrdered, JsonbRef, ToText, F64}; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_expr::expr::{build_func, Context, Expression, InputRefExpression}; +use risingwave_expr::expr::{ + build_func, Context, Expression, ExpressionBoxExt, InputRefExpression, +}; use risingwave_expr::{function, ExprError, Result}; use risingwave_pb::expr::expr_node::PbType; diff --git a/src/expr/impl/src/table_function/generate_series.rs b/src/expr/impl/src/table_function/generate_series.rs index 586fa60de02c2..dfa09b0e215b8 100644 --- a/src/expr/impl/src/table_function/generate_series.rs +++ b/src/expr/impl/src/table_function/generate_series.rs @@ -159,7 +159,7 @@ mod tests { use risingwave_common::array::DataChunk; use risingwave_common::types::test_utils::IntervalTestExt; use risingwave_common::types::{DataType, Decimal, Interval, ScalarImpl, Timestamp}; - use risingwave_expr::expr::{BoxedExpression, Expression, LiteralExpression}; + use risingwave_expr::expr::{BoxedExpression, ExpressionBoxExt, LiteralExpression}; use risingwave_expr::table_function::build; use risingwave_expr::ExprError; use risingwave_pb::expr::table_function::PbType; diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index aefb4df98ef4e..baa77dc79c89b 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1395,20 +1395,20 @@ sq_1.col_2; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [max(max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))))] } - └─BatchHashAgg { group_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], aggs: [max(max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))))] } - └─BatchExchange { order: [], dist: HashShard(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))) } - └─BatchHashAgg { group_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], aggs: [max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)))] } - └─BatchSortAgg { group_key: [lineitem.l_orderkey], aggs: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))] } + └─BatchProject { exprs: [max(max(internal_last_seen_value(lineitem.l_commitdate)))] } + └─BatchHashAgg { group_key: [internal_last_seen_value(lineitem.l_commitdate)], aggs: [max(max(internal_last_seen_value(lineitem.l_commitdate)))] } + └─BatchExchange { order: [], dist: HashShard(internal_last_seen_value(lineitem.l_commitdate)) } + └─BatchHashAgg { group_key: [internal_last_seen_value(lineitem.l_commitdate)], aggs: [max(internal_last_seen_value(lineitem.l_commitdate))] } + └─BatchSortAgg { group_key: [lineitem.l_orderkey], aggs: [internal_last_seen_value(lineitem.l_commitdate)] } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_commitdate], distribution: UpstreamHashShard(lineitem.l_orderkey) } stream_plan: |- - StreamMaterialize { columns: [col_0, first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))(hidden)], stream_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], pk_columns: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)))), first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))] } - └─StreamHashAgg { group_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], aggs: [max(max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)))), count] } - └─StreamExchange { dist: HashShard(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))) } - └─StreamHashAgg { group_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)), $expr1], aggs: [max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))), count] } - └─StreamProject { exprs: [lineitem.l_orderkey, first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)), Vnode(lineitem.l_orderkey) as $expr1] } - └─StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)), count] } + StreamMaterialize { columns: [col_0, internal_last_seen_value(lineitem.l_commitdate)(hidden)], stream_key: [internal_last_seen_value(lineitem.l_commitdate)], pk_columns: [internal_last_seen_value(lineitem.l_commitdate)], pk_conflict: NoCheck } + └─StreamProject { exprs: [max(max(internal_last_seen_value(lineitem.l_commitdate))), internal_last_seen_value(lineitem.l_commitdate)] } + └─StreamHashAgg { group_key: [internal_last_seen_value(lineitem.l_commitdate)], aggs: [max(max(internal_last_seen_value(lineitem.l_commitdate))), count] } + └─StreamExchange { dist: HashShard(internal_last_seen_value(lineitem.l_commitdate)) } + └─StreamHashAgg { group_key: [internal_last_seen_value(lineitem.l_commitdate), $expr1], aggs: [max(internal_last_seen_value(lineitem.l_commitdate)), count] } + └─StreamProject { exprs: [lineitem.l_orderkey, internal_last_seen_value(lineitem.l_commitdate), Vnode(lineitem.l_orderkey) as $expr1] } + └─StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [internal_last_seen_value(lineitem.l_commitdate), count] } └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_commitdate], pk: [lineitem.l_orderkey], dist: UpstreamHashShard(lineitem.l_orderkey) } - name: two phase agg on hop window input should use two phase agg sql: | diff --git a/src/frontend/planner_test/tests/testdata/output/append_only.yaml b/src/frontend/planner_test/tests/testdata/output/append_only.yaml index 184abd564c32b..d693d3fc942df 100644 --- a/src/frontend/planner_test/tests/testdata/output/append_only.yaml +++ b/src/frontend/planner_test/tests/testdata/output/append_only.yaml @@ -14,11 +14,12 @@ select t1.v1 as id, v2, v3 from t1 join t2 on t1.v1=t2.v1; stream_plan: |- StreamMaterialize { columns: [id, v2, v3, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, id], pk_columns: [t1._row_id, t2._row_id, id], pk_conflict: NoCheck } - └─StreamHashJoin [append_only] { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v3, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.v1) } - └─StreamTableScan { table: t2, columns: [t2.v1, t2.v3, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) } + └─StreamHashJoin [append_only] { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v3, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v1) } + └─StreamTableScan { table: t2, columns: [t2.v1, t2.v3, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - sql: | create table t1 (v1 int, v2 int) append only; select v1 from t1 order by v1 limit 3 offset 3; diff --git a/src/frontend/planner_test/tests/testdata/output/basic_query.yaml b/src/frontend/planner_test/tests/testdata/output/basic_query.yaml index fde09972bb66b..ce6724dc91c37 100644 --- a/src/frontend/planner_test/tests/testdata/output/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/output/basic_query.yaml @@ -234,9 +234,10 @@ └─BatchValues { rows: [] } stream_plan: |- StreamMaterialize { columns: [v, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, v], pk_columns: [t._row_id, t._row_id#1, v], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.v = t.v, output: [t.v, t._row_id, t._row_id] } - ├─StreamExchange { dist: HashShard(t.v) } - │ └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(t.v) } - └─StreamFilter { predicate: false:Boolean } - └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v, t._row_id, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.v = t.v, output: [t.v, t._row_id, t._row_id] } + ├─StreamExchange { dist: HashShard(t.v) } + │ └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v) } + └─StreamFilter { predicate: false:Boolean } + └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml b/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml index 236bc31b2503e..2d1b0951089e8 100644 --- a/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml @@ -60,9 +60,9 @@ select t2.c, t2.d, count(distinct t.a) from t join t2 on t.a = t2.c group by t2.c, t2.d; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [first_value(t2.c order_by(t2.c ASC))], aggs: [first_value(first_value(t2.d order_by(t2.d ASC)) order_by(first_value(t2.d order_by(t2.d ASC)) ASC)), count(t.a)] } - └─BatchExchange { order: [], dist: HashShard(first_value(t2.c order_by(t2.c ASC))) } - └─BatchHashAgg { group_key: [t.a], aggs: [first_value(t2.c order_by(t2.c ASC)), first_value(t2.d order_by(t2.d ASC))] } + └─BatchHashAgg { group_key: [internal_last_seen_value(t2.c)], aggs: [internal_last_seen_value(internal_last_seen_value(t2.d)), count(t.a)] } + └─BatchExchange { order: [], dist: HashShard(internal_last_seen_value(t2.c)) } + └─BatchHashAgg { group_key: [t.a], aggs: [internal_last_seen_value(t2.c), internal_last_seen_value(t2.d)] } └─BatchLookupJoin { type: Inner, predicate: t.a = t2.c, output: [t2.c, t2.d, t.a] } └─BatchExchange { order: [], dist: UpstreamHashShard(t.a) } └─BatchScan { table: t, columns: [t.a], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml index 6f4f8a673c996..e7196f7cf4fea 100644 --- a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml @@ -134,141 +134,145 @@ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id(hidden), stock.s_w_id(hidden), min(stock.s_quantity)(hidden), $expr2(hidden), region.r_regionkey(hidden), supplier.s_nationkey(hidden)], stream_key: [stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_columns: [n_name, s_name, i_id, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [item.i_id, item.i_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_i_id, stock.s_w_id, min(stock.s_quantity)] } - │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND min(stock.s_quantity) = stock.s_quantity AND stock.s_i_id = stock.s_i_id, output: [item.i_id, item.i_name, stock.s_i_id, stock.s_w_id, stock.s_i_id, min(stock.s_quantity)] } - │ ├─StreamProject { exprs: [stock.s_i_id, min(stock.s_quantity)] } - │ │ └─StreamHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity), count] } - │ │ └─StreamExchange { dist: HashShard(stock.s_i_id) } - │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } - │ │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] } - │ │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } - │ │ │ │ └─StreamProject { exprs: [region.r_regionkey] } - │ │ │ │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } - │ │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - │ │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } - │ │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } - │ │ │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ │ │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ │ └─StreamExchange { dist: HashShard($expr1) } - │ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } - │ │ └─StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } - │ └─StreamHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all } - │ ├─StreamExchange { dist: HashShard(item.i_id) } - │ │ └─StreamProject { exprs: [item.i_id, item.i_name] } - │ │ └─StreamFilter { predicate: Like(item.i_data, '%b':Varchar) } - │ │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } - │ └─StreamExchange { dist: HashShard(stock.s_i_id) } - │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } - └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] } - ├─StreamExchange { dist: HashShard(region.r_regionkey) } - │ └─StreamProject { exprs: [region.r_regionkey] } - │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } - │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - └─StreamExchange { dist: HashShard(nation.n_regionkey) } - └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } - ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - └─StreamExchange { dist: HashShard(nation.n_nationkey) } - └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey) } + └─StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [item.i_id, item.i_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_i_id, stock.s_w_id, min(stock.s_quantity)] } + │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND min(stock.s_quantity) = stock.s_quantity AND stock.s_i_id = stock.s_i_id, output: [item.i_id, item.i_name, stock.s_i_id, stock.s_w_id, stock.s_i_id, min(stock.s_quantity)] } + │ ├─StreamProject { exprs: [stock.s_i_id, min(stock.s_quantity)] } + │ │ └─StreamHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity), count] } + │ │ └─StreamExchange { dist: HashShard(stock.s_i_id) } + │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } + │ │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] } + │ │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } + │ │ │ │ └─StreamProject { exprs: [region.r_regionkey] } + │ │ │ │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } + │ │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + │ │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } + │ │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } + │ │ │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ │ │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ │ └─StreamExchange { dist: HashShard($expr1) } + │ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } + │ │ └─StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ └─StreamHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all } + │ ├─StreamExchange { dist: HashShard(item.i_id) } + │ │ └─StreamProject { exprs: [item.i_id, item.i_name] } + │ │ └─StreamFilter { predicate: Like(item.i_data, '%b':Varchar) } + │ │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } + │ └─StreamExchange { dist: HashShard(stock.s_i_id) } + │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] } + ├─StreamExchange { dist: HashShard(region.r_regionkey) } + │ └─StreamProject { exprs: [region.r_regionkey] } + │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } + │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + └─StreamExchange { dist: HashShard(nation.n_regionkey) } + └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } + ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(nation.n_nationkey) } + └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id(hidden), stock.s_w_id(hidden), min(stock.s_quantity)(hidden), $expr2(hidden), region.r_regionkey(hidden), supplier.s_nationkey(hidden)], stream_key: [stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_columns: [n_name, s_name, i_id, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([2]) from 1 - └── StreamExchange Hash([0]) from 11 + └── StreamExchange Hash([8, 9, 10, 11, 12, 13]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([2]) from 2 + └── StreamExchange Hash([0]) from 12 + + Fragment 2 StreamProject { exprs: [item.i_id, item.i_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_i_id, stock.s_w_id, min(stock.s_quantity)] } └── StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND min(stock.s_quantity) = stock.s_quantity AND stock.s_i_id = stock.s_i_id, output: [item.i_id, item.i_name, stock.s_i_id, stock.s_w_id, stock.s_i_id, min(stock.s_quantity)] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } ├── StreamProject { exprs: [stock.s_i_id, min(stock.s_quantity)] } │ └── StreamHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity), count] } { intermediate state table: 9, state tables: [ 8 ], distinct tables: [] } - │ └── StreamExchange Hash([0]) from 2 + │ └── StreamExchange Hash([0]) from 3 └── StreamHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all } { left table: 26, right table: 28, left degree table: 27, right degree table: 29 } - ├── StreamExchange Hash([0]) from 9 - └── StreamExchange Hash([0]) from 10 - - Fragment 2 - StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } { left table: 10, right table: 12, left degree table: 11, right degree table: 13 } - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([2]) from 8 + ├── StreamExchange Hash([0]) from 10 + └── StreamExchange Hash([0]) from 11 Fragment 3 - StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 } + StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } { left table: 10, right table: 12, left degree table: 11, right degree table: 13 } ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([1]) from 5 + └── StreamExchange Hash([2]) from 9 Fragment 4 + StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 } + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([1]) from 6 + + Fragment 5 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 18 } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } { left table: 19, right table: 21, left degree table: 20, right degree table: 22 } - ├── StreamExchange Hash([1]) from 6 - └── StreamExchange Hash([0]) from 7 + ├── StreamExchange Hash([1]) from 7 + └── StreamExchange Hash([0]) from 8 - Fragment 6 + Fragment 7 Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 23 } ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 Chain { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 24 } ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } └── StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) } └── Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 25 } ├── Upstream └── BatchPlanNode - Fragment 9 + Fragment 10 StreamProject { exprs: [item.i_id, item.i_name] } └── StreamFilter { predicate: Like(item.i_data, '%b':Varchar) } └── Chain { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 30 } ├── Upstream └── BatchPlanNode - Fragment 10 + Fragment 11 Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 31 } ├── Upstream └── BatchPlanNode - Fragment 11 + Fragment 12 StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] } { left table: 32, right table: 34, left degree table: 33, right degree table: 35 } - ├── StreamExchange Hash([0]) from 12 - └── StreamExchange Hash([6]) from 13 + ├── StreamExchange Hash([0]) from 13 + └── StreamExchange Hash([6]) from 14 - Fragment 12 + Fragment 13 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 36 } ├── Upstream └── BatchPlanNode - Fragment 13 + Fragment 14 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } { left table: 37, right table: 39, left degree table: 38, right degree table: 40 } - ├── StreamExchange Hash([3]) from 14 - └── StreamExchange Hash([0]) from 15 + ├── StreamExchange Hash([3]) from 15 + └── StreamExchange Hash([0]) from 16 - Fragment 14 + Fragment 15 Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 41 } ├── Upstream └── BatchPlanNode - Fragment 15 + Fragment 16 Chain { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 42 } ├── Upstream └── BatchPlanNode @@ -359,7 +363,7 @@ Table 42 { columns: [ vnode, n_nationkey, nation_backfill_finished, nation_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 11 ], read pk prefix len hint: 9 } + Table 4294967294 { columns: [ s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 8, 9, 10, 11, 12, 13 ], read pk prefix len hint: 9 } - id: ch_q3 before: @@ -2496,59 +2500,63 @@ └─LogicalScan { table: revenue1, columns: [revenue1.total_revenue] } stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no(hidden)], stream_key: [s_suppkey, revenue1.supplier_no, total_revenue], pk_columns: [s_suppkey, revenue1.supplier_no, total_revenue], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } - ├─StreamExchange { dist: HashShard(revenue1.total_revenue) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } - │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard($expr1) } - │ └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no::Int64 as $expr1, revenue1.supplier_no] } - │ └─StreamTableScan { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } - └─StreamExchange { dist: HashShard(max(max(revenue1.total_revenue))) } - └─StreamProject { exprs: [max(max(revenue1.total_revenue))] } - └─StreamSimpleAgg { aggs: [max(max(revenue1.total_revenue)), count] } - └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr2], aggs: [max(revenue1.total_revenue), count] } - └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no, Vnode(revenue1.supplier_no) as $expr2] } - └─StreamTableScan { table: revenue1, columns: [revenue1.total_revenue, revenue1.supplier_no], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey, revenue1.total_revenue, revenue1.supplier_no) } + └─StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } + ├─StreamExchange { dist: HashShard(revenue1.total_revenue) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } + │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard($expr1) } + │ └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no::Int64 as $expr1, revenue1.supplier_no] } + │ └─StreamTableScan { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } + └─StreamExchange { dist: HashShard(max(max(revenue1.total_revenue))) } + └─StreamProject { exprs: [max(max(revenue1.total_revenue))] } + └─StreamSimpleAgg { aggs: [max(max(revenue1.total_revenue)), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [$expr2], aggs: [max(revenue1.total_revenue), count] } + └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no, Vnode(revenue1.supplier_no) as $expr2] } + └─StreamTableScan { table: revenue1, columns: [revenue1.total_revenue, revenue1.supplier_no], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no(hidden)], stream_key: [s_suppkey, revenue1.supplier_no, total_revenue], pk_columns: [s_suppkey, revenue1.supplier_no, total_revenue], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([4]) from 1 - └── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0, 4, 5]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([4]) from 2 + └── StreamExchange Hash([0]) from 5 + + Fragment 2 StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } ├── left table: 4 ├── right table: 6 ├── left degree table: 5 ├── right degree table: 7 - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([1]) from 3 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([1]) from 4 - Fragment 2 + Fragment 3 Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 8 } ├── Upstream └── BatchPlanNode - Fragment 3 + Fragment 4 StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no::Int64 as $expr1, revenue1.supplier_no] } └── Chain { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } { state table: 9 } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [max(max(revenue1.total_revenue))] } └── StreamSimpleAgg { aggs: [max(max(revenue1.total_revenue)), count] } { intermediate state table: 11, state tables: [ 10 ], distinct tables: [] } - └── StreamExchange Single from 5 + └── StreamExchange Single from 6 - Fragment 5 + Fragment 6 StreamHashAgg { group_key: [$expr2], aggs: [max(revenue1.total_revenue), count] } { intermediate state table: 13, state tables: [ 12 ], distinct tables: [] } └── StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no, Vnode(revenue1.supplier_no) as $expr2] } └── Chain { table: revenue1, columns: [revenue1.total_revenue, revenue1.supplier_no], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } { state table: 14 } @@ -2590,7 +2598,7 @@ Table 14 { columns: [ vnode, supplier_no, revenue1_backfill_finished, revenue1_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no ], primary key: [ $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 4 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no ], primary key: [ $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 4, 5 ], read pk prefix len hint: 3 } - id: ch_q16 before: @@ -3174,58 +3182,62 @@ └─BatchScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_delivery_d], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [supplier.s_suppkey, supplier.s_nationkey], pk_columns: [s_name, supplier.s_suppkey, supplier.s_nationkey], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } - ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } - │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ └─StreamProject { exprs: [nation.n_nationkey] } - │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamProject { exprs: [((stock.s_i_id * stock.s_w_id) % 10000:Int32)::Int64 as $expr1, stock.s_i_id, stock.s_w_id, stock.s_quantity] } - └─StreamFilter { predicate: ((2:Int32 * stock.s_quantity) > sum(order_line.ol_quantity)) } - └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, stock.s_quantity, sum(order_line.ol_quantity)] } - └─StreamHashAgg { group_key: [stock.s_i_id, stock.s_w_id, stock.s_quantity], aggs: [sum(order_line.ol_quantity), count] } - └─StreamHashJoin { type: LeftSemi, predicate: stock.s_i_id = item.i_id, output: all } - ├─StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, stock.s_quantity, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - │ ├─StreamExchange { dist: HashShard(stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } - │ └─StreamExchange { dist: HashShard(order_line.ol_i_id) } - │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - │ └─StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } - │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } - └─StreamExchange { dist: HashShard(item.i_id) } - └─StreamProject { exprs: [item.i_id] } - └─StreamFilter { predicate: Like(item.i_data, 'co%':Varchar) } - └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey, supplier.s_nationkey) } + └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } + ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } + │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ └─StreamProject { exprs: [nation.n_nationkey] } + │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [((stock.s_i_id * stock.s_w_id) % 10000:Int32)::Int64 as $expr1, stock.s_i_id, stock.s_w_id, stock.s_quantity] } + └─StreamFilter { predicate: ((2:Int32 * stock.s_quantity) > sum(order_line.ol_quantity)) } + └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, stock.s_quantity, sum(order_line.ol_quantity)] } + └─StreamHashAgg { group_key: [stock.s_i_id, stock.s_w_id, stock.s_quantity], aggs: [sum(order_line.ol_quantity), count] } + └─StreamHashJoin { type: LeftSemi, predicate: stock.s_i_id = item.i_id, output: all } + ├─StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, stock.s_quantity, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + │ ├─StreamExchange { dist: HashShard(stock.s_i_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ └─StreamExchange { dist: HashShard(order_line.ol_i_id) } + │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + │ └─StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } + │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └─StreamExchange { dist: HashShard(item.i_id) } + └─StreamProject { exprs: [item.i_id] } + └─StreamFilter { predicate: Like(item.i_data, 'co%':Varchar) } + └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [supplier.s_suppkey, supplier.s_nationkey], pk_columns: [s_name, supplier.s_suppkey, supplier.s_nationkey], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([2, 3]) from 1 Fragment 1 - StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } - ├── StreamExchange Hash([3]) from 2 - └── StreamExchange Hash([0]) from 3 + StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 5 Fragment 2 + StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } + ├── StreamExchange Hash([3]) from 3 + └── StreamExchange Hash([0]) from 4 + + Fragment 3 Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 8 } ├── Upstream └── BatchPlanNode - Fragment 3 + Fragment 4 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 9 } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [((stock.s_i_id * stock.s_w_id) % 10000:Int32)::Int64 as $expr1, stock.s_i_id, stock.s_w_id, stock.s_quantity] } └── StreamFilter { predicate: ((2:Int32 * stock.s_quantity) > sum(order_line.ol_quantity)) } └── StreamProject { exprs: [stock.s_i_id, stock.s_w_id, stock.s_quantity, sum(order_line.ol_quantity)] } @@ -3236,16 +3248,16 @@ │ ├── right table: 17 │ ├── left degree table: 16 │ ├── right degree table: 18 - │ ├── StreamExchange Hash([0]) from 5 - │ └── StreamExchange Hash([0]) from 6 - └── StreamExchange Hash([0]) from 7 + │ ├── StreamExchange Hash([0]) from 6 + │ └── StreamExchange Hash([0]) from 7 + └── StreamExchange Hash([0]) from 8 - Fragment 5 + Fragment 6 Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 19 } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } └── Chain { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } @@ -3253,7 +3265,7 @@ ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamProject { exprs: [item.i_id] } └── StreamFilter { predicate: Like(item.i_data, 'co%':Varchar) } └── Chain { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 21 } @@ -3304,7 +3316,7 @@ Table 21 { columns: [ vnode, i_id, item_backfill_finished, item_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 } - id: ch_q21 before: diff --git a/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml b/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml index 9c961429276a3..dece27002b19b 100644 --- a/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml +++ b/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml @@ -23,11 +23,12 @@ └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } stream_plan: |- StreamMaterialize { columns: [v3, v4, v1, t2._row_id(hidden), t1._row_id(hidden)], stream_key: [t2._row_id, t1._row_id, v3], pk_columns: [t2._row_id, t1._row_id, v3], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t2._row_id, t1._row_id] } - ├─StreamExchange { dist: HashShard(t2.v3) } - │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - └─StreamExchange { dist: HashShard(t1.v1) } - └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v3, t2._row_id, t1._row_id) } + └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t2._row_id, t1._row_id] } + ├─StreamExchange { dist: HashShard(t2.v3) } + │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1) } + └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | create table t1 (v1 int, v2 int); create table t2 (v3 int, v4 int); @@ -79,8 +80,9 @@ └─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)], 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) } - └─StreamExchange { dist: HashShard(*VALUES*_0.column_0) } - └─StreamValues { rows: [['cn':Varchar, 'China':Varchar, 0:Int64], ['us':Varchar, 'United States':Varchar, 1:Int64]] } + └─StreamExchange { dist: HashShard(t.c, t._row_id, _row_id) } + └─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/output/distribution_derive.yaml b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml index 85d76188f3e76..818fd88b30a20 100644 --- a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml @@ -963,31 +963,35 @@ └─BatchScan { table: a, columns: [a.k1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], stream_key: [ak1.a._row_id, ak1.k1], pk_columns: [ak1.a._row_id, ak1.k1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } - ├─StreamExchange { dist: HashShard(ak1.k1) } - │ └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } - └─StreamHashAgg { group_key: [a.k1], aggs: [count] } - └─StreamExchange { dist: HashShard(a.k1) } - └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1) } + └─StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } + ├─StreamExchange { dist: HashShard(ak1.k1) } + │ └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } + └─StreamHashAgg { group_key: [a.k1], aggs: [count] } + └─StreamExchange { dist: HashShard(a.k1) } + └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], stream_key: [ak1.a._row_id, ak1.k1], pk_columns: [ak1.a._row_id, ak1.k1], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([2, 3]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 4 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -1022,7 +1026,7 @@ ├── columns: [ v, bv, ak1.a._row_id, ak1.k1, a.k1 ] ├── primary key: [ $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] - ├── distribution key: [ 3 ] + ├── distribution key: [ 2, 3 ] └── read pk prefix len hint: 2 - id: aggk1_join_Ak1_onk1 @@ -1054,31 +1058,35 @@ └─BatchScan { table: a, columns: [a.k1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden)], stream_key: [a.k1, ak1.a._row_id], pk_columns: [a.k1, ak1.a._row_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] } - ├─StreamHashAgg { group_key: [a.k1], aggs: [count] } - │ └─StreamExchange { dist: HashShard(a.k1) } - │ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } - └─StreamExchange { dist: HashShard(ak1.k1) } - └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } + └─StreamExchange { dist: HashShard(a.k1, ak1.a._row_id) } + └─StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] } + ├─StreamHashAgg { group_key: [a.k1], aggs: [count] } + │ └─StreamExchange { dist: HashShard(a.k1) } + │ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + └─StreamExchange { dist: HashShard(ak1.k1) } + └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden)], stream_key: [a.k1, ak1.a._row_id], pk_columns: [a.k1, ak1.a._row_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 4, state tables: [], distinct tables: [] } - │ └── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([2, 3]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 4, state tables: [], distinct tables: [] } + │ └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 5 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -1113,7 +1121,7 @@ ├── columns: [ v, bv, a.k1, ak1.a._row_id ] ├── primary key: [ $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] - ├── distribution key: [ 2 ] + ├── distribution key: [ 2, 3 ] └── read pk prefix len hint: 2 - id: aggk1_join_aggk1_onk1 @@ -1156,33 +1164,37 @@ └─BatchScan { table: b, columns: [b.k1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], stream_key: [a.k1], pk_columns: [a.k1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } - ├─StreamHashAgg { group_key: [a.k1], aggs: [count] } - │ └─StreamExchange { dist: HashShard(a.k1) } - │ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } - └─StreamHashAgg { group_key: [b.k1], aggs: [count] } - └─StreamExchange { dist: HashShard(b.k1) } - └─StreamTableScan { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } + └─StreamExchange { dist: HashShard(a.k1) } + └─StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } + ├─StreamHashAgg { group_key: [a.k1], aggs: [count] } + │ └─StreamExchange { dist: HashShard(a.k1) } + │ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + └─StreamHashAgg { group_key: [b.k1], aggs: [count] } + └─StreamExchange { dist: HashShard(b.k1) } + └─StreamTableScan { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], stream_key: [a.k1], pk_columns: [a.k1], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 4, state tables: [], distinct tables: [] } - │ └── StreamExchange Hash([0]) from 1 - └── StreamHashAgg { group_key: [b.k1], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 4, state tables: [], distinct tables: [] } + │ └── StreamExchange Hash([0]) from 2 + └── StreamHashAgg { group_key: [b.k1], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 5 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } { state table: 7 } ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml b/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml index a340014298c47..ab282ebe3858a 100644 --- a/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml @@ -124,16 +124,17 @@ └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: |- StreamMaterialize { columns: [v1, max, t1._row_id(hidden)], stream_key: [t1._row_id, v1], pk_columns: [t1._row_id, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = max(max(t2.v2)), output: [t1.v1, max(max(t2.v2)), t1._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(max(max(t2.v2))) } - └─StreamProject { exprs: [max(max(t2.v2))] } - └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] } - └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [max(t2.v2), count] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } - └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = max(max(t2.v2)), output: [t1.v1, max(max(t2.v2)), t1._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(max(max(t2.v2))) } + └─StreamProject { exprs: [max(max(t2.v2))] } + └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [$expr1], aggs: [max(t2.v2), count] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } + └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Dynamic filter join on unequal types sql: | create table t1 (v1 int); diff --git a/src/frontend/planner_test/tests/testdata/output/except.yaml b/src/frontend/planner_test/tests/testdata/output/except.yaml index 204a1814b8db7..1e27a7b74c0f0 100644 --- a/src/frontend/planner_test/tests/testdata/output/except.yaml +++ b/src/frontend/planner_test/tests/testdata/output/except.yaml @@ -108,20 +108,20 @@ create table t2 (a int, b numeric, c bigint, primary key(a)); select * from t1 except select * from t2; optimized_logical_plan_for_batch: |- - LogicalAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } + LogicalAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } └─LogicalJoin { type: LeftAnti, on: IsNotDistinctFrom(t1.a, t2.a) AND IsNotDistinctFrom(t1.b, t2.b) AND IsNotDistinctFrom(t1.c, t2.c), output: all } ├─LogicalScan { table: t1, columns: [t1.a, t1.b, t1.c] } └─LogicalScan { table: t2, columns: [t2.a, t2.b, t2.c] } batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } + └─BatchHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } └─BatchLookupJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } └─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: UpstreamHashShard(t1.a) } stream_plan: |- StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.a, first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } - └─StreamHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC)), count] } + └─StreamProject { exprs: [t1.a, internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } + └─StreamHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c), count] } └─StreamExchange { dist: HashShard(t1.a) } └─StreamHashJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } ├─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) } @@ -130,83 +130,70 @@ └─StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } { materialized table: 4294967294 } - └── StreamProject { exprs: [t1.a, first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } - └── StreamHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC)), count] } - ├── intermediate state table: 2 - ├── state tables: [ 0, 1 ] + StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } + ├── materialized table: 4294967294 + └── StreamProject { exprs: [t1.a, internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } + └── StreamHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c), count] } + ├── intermediate state table: 0 + ├── state tables: [] ├── distinct tables: [] └── StreamExchange Hash([0]) from 1 Fragment 1 StreamHashJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } - ├── left table: 3 - ├── right table: 5 - ├── left degree table: 4 - ├── right degree table: 6 + ├── left table: 1 + ├── right table: 3 + ├── left degree table: 2 + ├── right degree table: 4 ├── StreamExchange Hash([0, 1, 2]) from 2 └── StreamExchange Hash([0, 1, 2]) from 3 Fragment 2 - Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 7 } + Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 8 } + Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 6 } ├── Upstream └── BatchPlanNode Table 0 - ├── columns: [ t1_a, t1_b, t1_c ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 - - Table 1 - ├── columns: [ t1_a, t1_c, t1_b ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 - - Table 2 - ├── columns: [ t1_a, first_value(t1_b order_by(t1_b ASC)), first_value(t1_c order_by(t1_c ASC)), count ] + ├── columns: [ t1_a, internal_last_seen_value(t1_b), internal_last_seen_value(t1_c), count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 + Table 1 ├── columns: [ t1_a, t1_b, t1_c ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 4 + Table 2 ├── columns: [ t1_a, t1_b, t1_c, _degree ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 5 + Table 3 ├── columns: [ t2_a, t2_b, t2_c ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 6 + Table 4 ├── columns: [ t2_a, t2_b, t2_c, _degree ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 7 + Table 5 ├── columns: [ vnode, a, t1_backfill_finished, t1_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] @@ -214,7 +201,7 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 8 + Table 6 ├── columns: [ vnode, a, t2_backfill_finished, t2_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] diff --git a/src/frontend/planner_test/tests/testdata/output/intersect.yaml b/src/frontend/planner_test/tests/testdata/output/intersect.yaml index c203f1f953814..91839346824ec 100644 --- a/src/frontend/planner_test/tests/testdata/output/intersect.yaml +++ b/src/frontend/planner_test/tests/testdata/output/intersect.yaml @@ -108,20 +108,20 @@ create table t2 (a int, b numeric, c bigint, primary key(a)); select * from t1 intersect select * from t2; optimized_logical_plan_for_batch: |- - LogicalAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } + LogicalAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } └─LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(t1.a, t2.a) AND IsNotDistinctFrom(t1.b, t2.b) AND IsNotDistinctFrom(t1.c, t2.c), output: all } ├─LogicalScan { table: t1, columns: [t1.a, t1.b, t1.c] } └─LogicalScan { table: t2, columns: [t2.a, t2.b, t2.c] } batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } + └─BatchHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } └─BatchLookupJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } └─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: UpstreamHashShard(t1.a) } stream_plan: |- StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.a, first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } - └─StreamHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC)), count] } + └─StreamProject { exprs: [t1.a, internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } + └─StreamHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c), count] } └─StreamExchange { dist: HashShard(t1.a) } └─StreamHashJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } ├─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) } @@ -130,83 +130,70 @@ └─StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } { materialized table: 4294967294 } - └── StreamProject { exprs: [t1.a, first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } - └── StreamHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC)), count] } - ├── intermediate state table: 2 - ├── state tables: [ 0, 1 ] + StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } + ├── materialized table: 4294967294 + └── StreamProject { exprs: [t1.a, internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } + └── StreamHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c), count] } + ├── intermediate state table: 0 + ├── state tables: [] ├── distinct tables: [] └── StreamExchange Hash([0]) from 1 Fragment 1 StreamHashJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } - ├── left table: 3 - ├── right table: 5 - ├── left degree table: 4 - ├── right degree table: 6 + ├── left table: 1 + ├── right table: 3 + ├── left degree table: 2 + ├── right degree table: 4 ├── StreamExchange Hash([0, 1, 2]) from 2 └── StreamExchange Hash([0, 1, 2]) from 3 Fragment 2 - Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 7 } + Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 8 } + Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 6 } ├── Upstream └── BatchPlanNode Table 0 - ├── columns: [ t1_a, t1_b, t1_c ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 - - Table 1 - ├── columns: [ t1_a, t1_c, t1_b ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 - - Table 2 - ├── columns: [ t1_a, first_value(t1_b order_by(t1_b ASC)), first_value(t1_c order_by(t1_c ASC)), count ] + ├── columns: [ t1_a, internal_last_seen_value(t1_b), internal_last_seen_value(t1_c), count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 + Table 1 ├── columns: [ t1_a, t1_b, t1_c ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 4 + Table 2 ├── columns: [ t1_a, t1_b, t1_c, _degree ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 5 + Table 3 ├── columns: [ t2_a, t2_b, t2_c ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 6 + Table 4 ├── columns: [ t2_a, t2_b, t2_c, _degree ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 7 + Table 5 ├── columns: [ vnode, a, t1_backfill_finished, t1_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] @@ -214,7 +201,7 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 8 + Table 6 ├── columns: [ vnode, a, t2_backfill_finished, t2_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] diff --git a/src/frontend/planner_test/tests/testdata/output/join.yaml b/src/frontend/planner_test/tests/testdata/output/join.yaml index 4ef01cc84ab47..a61d2a0d73327 100644 --- a/src/frontend/planner_test/tests/testdata/output/join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/join.yaml @@ -14,14 +14,15 @@ └─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id] } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1, t3._row_id], pk_columns: [t1._row_id, t2._row_id, v1, t3._row_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = t3.v5, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] } - ├─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } - │ ├─StreamExchange { dist: HashShard(t1.v1) } - │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ └─StreamExchange { dist: HashShard(t2.v3) } - │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - └─StreamExchange { dist: HashShard(t3.v5) } - └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id, t3._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = t3.v5, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] } + ├─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } + │ ├─StreamExchange { dist: HashShard(t1.v1) } + │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: HashShard(t2.v3) } + │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t3.v5) } + └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } - name: self join sql: | create table t (v1 int, v2 int); @@ -33,11 +34,12 @@ └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } stream_plan: |- StreamMaterialize { columns: [t1v1, t2v1, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, t1v1], pk_columns: [t._row_id, t._row_id#1, t1v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v1, t._row_id, t._row_id] } - ├─StreamExchange { dist: HashShard(t.v1) } - │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(t.v1) } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1, t._row_id, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v1, t._row_id, t._row_id] } + ├─StreamExchange { dist: HashShard(t.v1) } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1) } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t1 (v1 int, v2 int); create table t2 (v1 int, v2 int); @@ -65,15 +67,16 @@ └─BatchScan { table: t3, columns: [t3.v1, t3.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t3_v1, t3_v2, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1, t3._row_id, t2_v2], pk_columns: [t1._row_id, t2._row_id, t1_v1, t3._row_id, t2_v2], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t2.v2 = t3.v2, output: [t1.v1, t1.v2, t2.v1, t2.v2, t3.v1, t3.v2, t1._row_id, t2._row_id, t3._row_id] } - ├─StreamExchange { dist: HashShard(t2.v2) } - │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } - │ ├─StreamExchange { dist: HashShard(t1.v1) } - │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ └─StreamExchange { dist: HashShard(t2.v1) } - │ └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - └─StreamExchange { dist: HashShard(t3.v2) } - └─StreamTableScan { table: t3, columns: [t3.v1, t3.v2, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t2.v2, t1._row_id, t2._row_id, t3._row_id) } + └─StreamHashJoin { type: Inner, predicate: t2.v2 = t3.v2, output: [t1.v1, t1.v2, t2.v1, t2.v2, t3.v1, t3.v2, t1._row_id, t2._row_id, t3._row_id] } + ├─StreamExchange { dist: HashShard(t2.v2) } + │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } + │ ├─StreamExchange { dist: HashShard(t1.v1) } + │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: HashShard(t2.v1) } + │ └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t3.v2) } + └─StreamTableScan { table: t3, columns: [t3.v1, t3.v2, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } - sql: | create table t1 (v1 int, v2 int); create table t2 (v1 int, v2 int); @@ -93,11 +96,12 @@ └─BatchScan { table: t2, columns: [t2.v1, t2.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [t1_v2, t2_v2, t1._row_id(hidden), t1.v1(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1.v1], pk_columns: [t1._row_id, t2._row_id, t1.v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v2, t2.v2, t1._row_id, t1.v1, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.v1) } - └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1._row_id, t1.v1, t2._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v2, t2.v2, t1._row_id, t1.v1, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v1) } + └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - sql: | create table t1 (v1 int, v2 int); create table t2 (v1 int, v2 int); @@ -154,11 +158,12 @@ └─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) } stream_plan: |- StreamMaterialize { columns: [ix, iix, i.t._row_id(hidden), i.t._row_id#1(hidden)], stream_key: [i.t._row_id, i.t._row_id#1, ix], pk_columns: [i.t._row_id, i.t._row_id#1, ix], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id] } - ├─StreamExchange { dist: HashShard(i.x) } - │ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } - └─StreamExchange { dist: HashShard(i.x) } - └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } + └─StreamExchange { dist: HashShard(i.x, i.t._row_id, i.t._row_id) } + └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id] } + ├─StreamExchange { dist: HashShard(i.x) } + │ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } + └─StreamExchange { dist: HashShard(i.x) } + └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } - name: Left & right has same SomeShard distribution. There should still be exchanges below hash join sql: | create table t(x int); @@ -170,11 +175,12 @@ └─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) } stream_plan: |- StreamMaterialize { columns: [ix, tx, i.t._row_id(hidden), t._row_id(hidden)], stream_key: [i.t._row_id, t._row_id, ix], pk_columns: [i.t._row_id, t._row_id, ix], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: i.x = t.x, output: [i.x, t.x, i.t._row_id, t._row_id] } - ├─StreamExchange { dist: HashShard(i.x) } - │ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } - └─StreamExchange { dist: HashShard(t.x) } - └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(i.x, i.t._row_id, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: i.x = t.x, output: [i.x, t.x, i.t._row_id, t._row_id] } + ├─StreamExchange { dist: HashShard(i.x) } + │ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } + └─StreamExchange { dist: HashShard(t.x) } + └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: Left & right has same HashShard distribution. There should be no exchange below hash join sql: | create table t(x int); @@ -628,12 +634,13 @@ └─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr1(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, $expr1], pk_columns: [t1._row_id, t2._row_id, $expr1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: $expr1 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr1, t2._row_id] } - ├─StreamExchange { dist: HashShard($expr1) } - │ └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.v2) } - └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1._row_id, $expr1, t2._row_id) } + └─StreamHashJoin { type: Inner, predicate: $expr1 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr1, t2._row_id] } + ├─StreamExchange { dist: HashShard($expr1) } + │ └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v2) } + └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Repeated columns in project should not interfere with join result (https://github.com/risingwavelabs/risingwave/issues/8216) sql: | create table t(x int); @@ -652,39 +659,43 @@ select t1.src p1, t1.dst p2, t2.dst p3 from t t1, t t2, t t3 where t1.dst = t2.src and t2.src = t3.dst and t3.dst = t1.src; stream_plan: |- StreamMaterialize { columns: [p1, p2, p3, t._row_id(hidden), t._row_id#1(hidden), t.src(hidden), t._row_id#2(hidden)], stream_key: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_columns: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] } - ├─StreamExchange { dist: HashShard(t.src) } - │ └─StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] } - │ ├─StreamExchange { dist: HashShard(t.dst) } - │ │ └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - │ └─StreamExchange { dist: HashShard(t.src) } - │ └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(t.dst) } - └─StreamTableScan { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.src, t.dst, t._row_id, t._row_id, t.src, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] } + ├─StreamExchange { dist: HashShard(t.src) } + │ └─StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] } + │ ├─StreamExchange { dist: HashShard(t.dst) } + │ │ └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamExchange { dist: HashShard(t.src) } + │ └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.dst) } + └─StreamTableScan { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [p1, p2, p3, t._row_id(hidden), t._row_id#1(hidden), t.src(hidden), t._row_id#2(hidden)], stream_key: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_columns: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0, 1, 3, 4, 5, 6]) from 1 Fragment 1 - StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } - ├── StreamExchange Hash([1]) from 2 - └── StreamExchange Hash([0]) from 3 + StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 5 Fragment 2 + StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } + ├── StreamExchange Hash([1]) from 3 + └── StreamExchange Hash([0]) from 4 + + Fragment 3 Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 8 } ├── Upstream └── BatchPlanNode - Fragment 3 + Fragment 4 Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 9 } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 Chain { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 10 } ├── Upstream └── BatchPlanNode @@ -711,5 +722,5 @@ Table 10 { columns: [ vnode, _row_id, t_backfill_finished, t_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ p1, p2, p3, t._row_id, t._row_id#1, t.src, t._row_id#2 ], primary key: [ $3 ASC, $4 ASC, $1 ASC, $6 ASC, $5 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 6 } + Table 4294967294 { columns: [ p1, p2, p3, t._row_id, t._row_id#1, t.src, t._row_id#2 ], primary key: [ $3 ASC, $4 ASC, $1 ASC, $6 ASC, $5 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 1, 3, 4, 5, 6 ], read pk prefix len hint: 6 } diff --git a/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml b/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml index b24fc18c6b513..31c53d02a9a18 100644 --- a/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml +++ b/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml @@ -34,19 +34,20 @@ └─BatchScan { table: t4, columns: [t4.v7, t4.v8], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, v7, v8, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden), t4._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1, t3._row_id, v2, t4._row_id, v5], pk_columns: [t1._row_id, t2._row_id, v1, t3._row_id, v2, t4._row_id, v5], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] } - ├─StreamExchange { dist: HashShard(t3.v5) } - │ └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] } - │ ├─StreamExchange { dist: HashShard(t1.v2) } - │ │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } - │ │ ├─StreamExchange { dist: HashShard(t1.v1) } - │ │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ │ └─StreamExchange { dist: HashShard(t2.v3) } - │ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - │ └─StreamExchange { dist: HashShard(t3.v6) } - │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } - └─StreamExchange { dist: HashShard(t4.v7) } - └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1.v2, t3.v5, t1._row_id, t2._row_id, t3._row_id, t4._row_id) } + └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] } + ├─StreamExchange { dist: HashShard(t3.v5) } + │ └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] } + │ ├─StreamExchange { dist: HashShard(t1.v2) } + │ │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } + │ │ ├─StreamExchange { dist: HashShard(t1.v1) } + │ │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ └─StreamExchange { dist: HashShard(t2.v3) } + │ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + │ └─StreamExchange { dist: HashShard(t3.v6) } + │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + └─StreamExchange { dist: HashShard(t4.v7) } + └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } - name: bushy tree join ordering sql: | create table t1 (v1 int, v2 int); @@ -81,19 +82,20 @@ └─BatchScan { table: t4, columns: [t4.v7, t4.v8], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, v7, v8, t2._row_id(hidden), t1._row_id(hidden), t4._row_id(hidden), t3._row_id(hidden)], stream_key: [t2._row_id, t1._row_id, v3, t4._row_id, t3._row_id, v7, v2], pk_columns: [t2._row_id, t1._row_id, v3, t4._row_id, t3._row_id, v7, v2], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t2._row_id, t1._row_id, t4._row_id, t3._row_id] } - ├─StreamExchange { dist: HashShard(t1.v2) } - │ └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t1.v2, t2._row_id, t1._row_id] } - │ ├─StreamExchange { dist: HashShard(t2.v3) } - │ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - │ └─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t3.v6) } - └─StreamHashJoin { type: Inner, predicate: t4.v7 = t3.v5, output: [t4.v7, t4.v8, t3.v5, t3.v6, t4._row_id, t3._row_id] } - ├─StreamExchange { dist: HashShard(t4.v7) } - │ └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } - └─StreamExchange { dist: HashShard(t3.v5) } - └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + └─StreamExchange { dist: HashShard(t1.v2, t2.v3, t4.v7, t2._row_id, t1._row_id, t4._row_id, t3._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t2._row_id, t1._row_id, t4._row_id, t3._row_id] } + ├─StreamExchange { dist: HashShard(t1.v2) } + │ └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t1.v2, t2._row_id, t1._row_id] } + │ ├─StreamExchange { dist: HashShard(t2.v3) } + │ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + │ └─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t3.v6) } + └─StreamHashJoin { type: Inner, predicate: t4.v7 = t3.v5, output: [t4.v7, t4.v8, t3.v5, t3.v6, t4._row_id, t3._row_id] } + ├─StreamExchange { dist: HashShard(t4.v7) } + │ └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } + └─StreamExchange { dist: HashShard(t3.v5) } + └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } - name: bushy tree join ordering manually sql: | set rw_enable_join_ordering = false; @@ -128,19 +130,20 @@ └─BatchScan { table: t4, columns: [t4.v7, t4.v8], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, v7, v8, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden), t4._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1, t3._row_id, t4._row_id, v5, v2], pk_columns: [t1._row_id, t2._row_id, v1, t3._row_id, t4._row_id, v5, v2], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] } - ├─StreamExchange { dist: HashShard(t1.v2) } - │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } - │ ├─StreamExchange { dist: HashShard(t1.v1) } - │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ └─StreamExchange { dist: HashShard(t2.v3) } - │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - └─StreamExchange { dist: HashShard(t3.v6) } - └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t3.v5, t3.v6, t4.v7, t4.v8, t3._row_id, t4._row_id] } - ├─StreamExchange { dist: HashShard(t3.v5) } - │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } - └─StreamExchange { dist: HashShard(t4.v7) } - └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1.v2, t3.v5, t1._row_id, t2._row_id, t3._row_id, t4._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] } + ├─StreamExchange { dist: HashShard(t1.v2) } + │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } + │ ├─StreamExchange { dist: HashShard(t1.v1) } + │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: HashShard(t2.v3) } + │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t3.v6) } + └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t3.v5, t3.v6, t4.v7, t4.v8, t3._row_id, t4._row_id] } + ├─StreamExchange { dist: HashShard(t3.v5) } + │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + └─StreamExchange { dist: HashShard(t4.v7) } + └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } - name: right deep tree join ordering manually sql: | set rw_enable_join_ordering = false; @@ -175,16 +178,17 @@ └─BatchScan { table: t4, columns: [t4.v7, t4.v8], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, v7, v8, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden), t4._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t3._row_id, t4._row_id, v5, v4, v1], pk_columns: [t1._row_id, t2._row_id, t3._row_id, t4._row_id, v5, v4, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.v3) } - └─StreamHashJoin { type: Inner, predicate: t2.v4 = t3.v6, output: [t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t2._row_id, t3._row_id, t4._row_id] } - ├─StreamExchange { dist: HashShard(t2.v4) } - │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - └─StreamExchange { dist: HashShard(t3.v6) } - └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t3.v5, t3.v6, t4.v7, t4.v8, t3._row_id, t4._row_id] } - ├─StreamExchange { dist: HashShard(t3.v5) } - │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } - └─StreamExchange { dist: HashShard(t4.v7) } - └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t2.v4, t3.v5, t1._row_id, t2._row_id, t3._row_id, t4._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v3) } + └─StreamHashJoin { type: Inner, predicate: t2.v4 = t3.v6, output: [t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t2._row_id, t3._row_id, t4._row_id] } + ├─StreamExchange { dist: HashShard(t2.v4) } + │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t3.v6) } + └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t3.v5, t3.v6, t4.v7, t4.v8, t3._row_id, t4._row_id] } + ├─StreamExchange { dist: HashShard(t3.v5) } + │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + └─StreamExchange { dist: HashShard(t4.v7) } + └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml b/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml index 8e63beb9798c1..85bfb1a6cda36 100644 --- a/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml @@ -42,22 +42,23 @@ └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), all_sales._row_id(hidden), salesperson.id(hidden), all_sales.amount(hidden), salesperson.id#1(hidden)], stream_key: [salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount], pk_columns: [salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM salesperson.id AND all_sales.amount = max(all_sales.amount), output: [salesperson.name, max(all_sales.amount), all_sales.customer_name, salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount, salesperson.id] } - ├─StreamHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id, output: [salesperson.id, salesperson.name, all_sales.customer_name, all_sales.amount, salesperson._row_id, all_sales._row_id] } - │ ├─StreamExchange { dist: HashShard(salesperson.id) } - │ │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } - │ └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } - │ └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } - └─StreamProject { exprs: [salesperson.id, max(all_sales.amount)] } - └─StreamHashAgg { group_key: [salesperson.id], aggs: [max(all_sales.amount), count] } - └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.id, all_sales.amount, all_sales._row_id] } - ├─StreamProject { exprs: [salesperson.id] } - │ └─StreamHashAgg { group_key: [salesperson.id], aggs: [count] } - │ └─StreamExchange { dist: HashShard(salesperson.id) } - │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } - └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } - └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } - └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } + └─StreamExchange { dist: HashShard(salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount) } + └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM salesperson.id AND all_sales.amount = max(all_sales.amount), output: [salesperson.name, max(all_sales.amount), all_sales.customer_name, salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount, salesperson.id] } + ├─StreamHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id, output: [salesperson.id, salesperson.name, all_sales.customer_name, all_sales.amount, salesperson._row_id, all_sales._row_id] } + │ ├─StreamExchange { dist: HashShard(salesperson.id) } + │ │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } + │ └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } + │ └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } + └─StreamProject { exprs: [salesperson.id, max(all_sales.amount)] } + └─StreamHashAgg { group_key: [salesperson.id], aggs: [max(all_sales.amount), count] } + └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.id, all_sales.amount, all_sales._row_id] } + ├─StreamProject { exprs: [salesperson.id] } + │ └─StreamHashAgg { group_key: [salesperson.id], aggs: [count] } + │ └─StreamExchange { dist: HashShard(salesperson.id) } + │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } + └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } + └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } + └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } - name: lateral join 2 sql: | create table all_sales (salesperson_id int, customer_name varchar, amount int ); @@ -87,14 +88,15 @@ └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), salesperson.id(hidden), all_sales.salesperson_id(hidden)], stream_key: [salesperson._row_id, salesperson.id], pk_columns: [salesperson._row_id, salesperson.id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] } - ├─StreamExchange { dist: HashShard(salesperson.id) } - │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } - └─StreamGroupTopN { order: [all_sales.amount DESC], limit: 1, offset: 0, group_key: [all_sales.salesperson_id] } - └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } - └─StreamProject { exprs: [all_sales.salesperson_id, all_sales.amount, all_sales.customer_name, all_sales._row_id] } - └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } - └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } + └─StreamExchange { dist: HashShard(salesperson._row_id, salesperson.id) } + └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] } + ├─StreamExchange { dist: HashShard(salesperson.id) } + │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } + └─StreamGroupTopN { order: [all_sales.amount DESC], limit: 1, offset: 0, group_key: [all_sales.salesperson_id] } + └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } + └─StreamProject { exprs: [all_sales.salesperson_id, all_sales.amount, all_sales.customer_name, all_sales._row_id] } + └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } + └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } - name: lateral join 2 (left join) sql: | create table all_sales (salesperson_id int, customer_name varchar, amount int ); @@ -124,14 +126,15 @@ └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), salesperson.id(hidden), all_sales.salesperson_id(hidden)], stream_key: [salesperson._row_id, salesperson.id], pk_columns: [salesperson._row_id, salesperson.id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] } - ├─StreamExchange { dist: HashShard(salesperson.id) } - │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } - └─StreamGroupTopN { order: [all_sales.amount DESC], limit: 1, offset: 0, group_key: [all_sales.salesperson_id] } - └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } - └─StreamProject { exprs: [all_sales.salesperson_id, all_sales.amount, all_sales.customer_name, all_sales._row_id] } - └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } - └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } + └─StreamExchange { dist: HashShard(salesperson._row_id, salesperson.id) } + └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] } + ├─StreamExchange { dist: HashShard(salesperson.id) } + │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } + └─StreamGroupTopN { order: [all_sales.amount DESC], limit: 1, offset: 0, group_key: [all_sales.salesperson_id] } + └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } + └─StreamProject { exprs: [all_sales.salesperson_id, all_sales.amount, all_sales.customer_name, all_sales._row_id] } + └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } + └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } - name: lateral join 2 (right join) should throw an error sql: | create table all_sales (salesperson_id int, customer_name varchar, amount int ); @@ -165,14 +168,15 @@ └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, arr, unnest, t._row_id(hidden), t.arr(hidden), projected_row_id(hidden)], stream_key: [t._row_id, projected_row_id, arr], pk_columns: [t._row_id, projected_row_id, arr], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, Unnest($0), t._row_id, t.arr, projected_row_id] } - ├─StreamExchange { dist: HashShard(t.arr) } - │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.arr] } - └─StreamHashAgg { group_key: [t.arr], aggs: [count] } - └─StreamExchange { dist: HashShard(t.arr) } - └─StreamTableScan { table: t, columns: [t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.arr, t._row_id, projected_row_id) } + └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, Unnest($0), t._row_id, t.arr, projected_row_id] } + ├─StreamExchange { dist: HashShard(t.arr) } + │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProjectSet { select_list: [$0, Unnest($0)] } + └─StreamProject { exprs: [t.arr] } + └─StreamHashAgg { group_key: [t.arr], aggs: [count] } + └─StreamExchange { dist: HashShard(t.arr) } + └─StreamTableScan { table: t, columns: [t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: https://github.com/risingwavelabs/risingwave/issues/12298 sql: | create table t1(c varchar, n varchar, id varchar, d varchar); diff --git a/src/frontend/planner_test/tests/testdata/output/mv_on_mv.yaml b/src/frontend/planner_test/tests/testdata/output/mv_on_mv.yaml index 2f7d9e5e75b3b..6838ddb331939 100644 --- a/src/frontend/planner_test/tests/testdata/output/mv_on_mv.yaml +++ b/src/frontend/planner_test/tests/testdata/output/mv_on_mv.yaml @@ -12,8 +12,9 @@ select m1.v1 as m1v1, m1.v2 as m1v2, m2.v1 as m2v1, m2.v2 as m2v2 from m1 join m2 on m1.v1 = m2.v1; stream_plan: |- StreamMaterialize { columns: [m1v1, m1v2, m2v1, m2v2, m1.t1._row_id(hidden), m2.t1._row_id(hidden)], stream_key: [m1.t1._row_id, m2.t1._row_id, m1v1], pk_columns: [m1.t1._row_id, m2.t1._row_id, m1v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: m1.v1 = m2.v1, output: [m1.v1, m1.v2, m2.v1, m2.v2, m1.t1._row_id, m2.t1._row_id] } - ├─StreamExchange { dist: HashShard(m1.v1) } - │ └─StreamTableScan { table: m1, columns: [m1.v1, m1.v2, m1.t1._row_id], pk: [m1.t1._row_id], dist: UpstreamHashShard(m1.t1._row_id) } - └─StreamExchange { dist: HashShard(m2.v1) } - └─StreamTableScan { table: m2, columns: [m2.v1, m2.v2, m2.t1._row_id], pk: [m2.t1._row_id], dist: UpstreamHashShard(m2.t1._row_id) } + └─StreamExchange { dist: HashShard(m1.v1, m1.t1._row_id, m2.t1._row_id) } + └─StreamHashJoin { type: Inner, predicate: m1.v1 = m2.v1, output: [m1.v1, m1.v2, m2.v1, m2.v2, m1.t1._row_id, m2.t1._row_id] } + ├─StreamExchange { dist: HashShard(m1.v1) } + │ └─StreamTableScan { table: m1, columns: [m1.v1, m1.v2, m1.t1._row_id], pk: [m1.t1._row_id], dist: UpstreamHashShard(m1.t1._row_id) } + └─StreamExchange { dist: HashShard(m2.v1) } + └─StreamTableScan { table: m2, columns: [m2.v1, m2.v2, m2.t1._row_id], pk: [m2.t1._row_id], dist: UpstreamHashShard(m2.t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index f4b9e28ce0775..8d452bf45bc36 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -181,34 +181,38 @@ └─BatchScan { table: auction, columns: [auction.id, auction.seller, auction.category], distribution: UpstreamHashShard(auction.id) } stream_plan: |- StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], stream_key: [id, auction.seller], pk_columns: [id, auction.seller], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] } - ├─StreamExchange { dist: HashShard(auction.seller) } - │ └─StreamProject { exprs: [auction.id, auction.seller] } - │ └─StreamFilter { predicate: (auction.category = 10:Int32) } - │ └─StreamTableScan { table: auction, columns: [auction.id, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } - └─StreamExchange { dist: HashShard(person.id) } - └─StreamFilter { predicate: (((person.state = 'or':Varchar) OR (person.state = 'id':Varchar)) OR (person.state = 'ca':Varchar)) } - └─StreamTableScan { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) } + └─StreamExchange { dist: HashShard(auction.id, auction.seller) } + └─StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] } + ├─StreamExchange { dist: HashShard(auction.seller) } + │ └─StreamProject { exprs: [auction.id, auction.seller] } + │ └─StreamFilter { predicate: (auction.category = 10:Int32) } + │ └─StreamTableScan { table: auction, columns: [auction.id, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + └─StreamExchange { dist: HashShard(person.id) } + └─StreamFilter { predicate: (((person.state = 'or':Varchar) OR (person.state = 'id':Varchar)) OR (person.state = 'ca':Varchar)) } + └─StreamTableScan { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], stream_key: [id, auction.seller], pk_columns: [id, auction.seller], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([1]) from 1 - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([3, 4]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([1]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 StreamProject { exprs: [auction.id, auction.seller] } └── StreamFilter { predicate: (auction.category = 10:Int32) } └── Chain { table: auction, columns: [auction.id, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 4 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 StreamFilter { predicate: (((person.state = 'or':Varchar) OR (person.state = 'id':Varchar)) OR (person.state = 'ca':Varchar)) } └── Chain { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 5 } ├── Upstream @@ -242,7 +246,7 @@ ├── columns: [ name, city, state, id, auction.seller, person.id ] ├── primary key: [ $3 ASC, $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5 ] - ├── distribution key: [ 4 ] + ├── distribution key: [ 3, 4 ] └── read pk prefix len hint: 2 - id: nexmark_q4 @@ -834,9 +838,9 @@ AND P.endtime = A.endtime; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, first_value(person.name order_by(person.name ASC)), $expr1] } + └─BatchHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1] } ├─BatchExchange { order: [], dist: HashShard(person.id, $expr1, $expr2) } - │ └─BatchHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [first_value(person.name order_by(person.name ASC))] } + │ └─BatchHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name)] } │ └─BatchProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } │ └─BatchProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] } │ └─BatchScan { table: person, columns: [person.id, person.name, person.date_time], distribution: UpstreamHashShard(person.id) } @@ -847,50 +851,54 @@ └─BatchScan { table: auction, columns: [auction.date_time, auction.seller], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2], pk_columns: [id, starttime, $expr2], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, first_value(person.name order_by(person.name ASC)), $expr1, $expr2, auction.seller, $expr3, $expr4] } - ├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) } - │ └─StreamProject { exprs: [person.id, $expr1, $expr2, first_value(person.name order_by(person.name ASC))] } - │ └─StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [first_value(person.name order_by(person.name ASC)), count] } - │ └─StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } - │ └─StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] } - │ └─StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } - └─StreamProject { exprs: [auction.seller, $expr3, $expr4] } - └─StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } - └─StreamExchange { dist: HashShard(auction.seller, $expr3, $expr4) } - └─StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] } - └─StreamProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, auction.id] } - └─StreamTableScan { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + └─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) } + └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] } + ├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) } + │ └─StreamProject { exprs: [person.id, $expr1, $expr2, internal_last_seen_value(person.name)] } + │ └─StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] } + │ └─StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } + │ └─StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] } + │ └─StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } + └─StreamProject { exprs: [auction.seller, $expr3, $expr4] } + └─StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } + └─StreamExchange { dist: HashShard(auction.seller, $expr3, $expr4) } + └─StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] } + └─StreamProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, auction.id] } + └─StreamTableScan { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2], pk_columns: [id, starttime, $expr2], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, first_value(person.name order_by(person.name ASC)), $expr1, $expr2, auction.seller, $expr3, $expr4] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0, 1, 2]) from 1 - └── StreamProject { exprs: [auction.seller, $expr3, $expr4] } - └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { intermediate state table: 7, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0, 1, 2]) from 2 + └── StreamExchange Hash([0, 2, 3]) from 1 Fragment 1 - StreamProject { exprs: [person.id, $expr1, $expr2, first_value(person.name order_by(person.name ASC))] } - └── StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [first_value(person.name order_by(person.name ASC)), count] } { intermediate state table: 5, state tables: [ 4 ], distinct tables: [] } + StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0, 1, 2]) from 2 + └── StreamProject { exprs: [auction.seller, $expr3, $expr4] } + └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0, 1, 2]) from 3 + + Fragment 2 + StreamProject { exprs: [person.id, $expr1, $expr2, internal_last_seen_value(person.name)] } + └── StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] } { intermediate state table: 4, state tables: [], distinct tables: [] } └── StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } └── StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] } - └── Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 6 } + └── Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 5 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] } └── StreamProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, auction.id] } - └── Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 8 } + └── Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } ├── Upstream └── BatchPlanNode - Table 0 { columns: [ person_id, $expr1, $expr2, first_value(person_name order_by(person_name ASC)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 0 { columns: [ person_id, $expr1, $expr2, internal_last_seen_value(person_name) ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } Table 1 { columns: [ person_id, $expr1, $expr2, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } @@ -898,17 +906,20 @@ Table 3 { columns: [ auction_seller, $expr3, $expr4, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4 { columns: [ person_id, $expr1, $expr2, person_name ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 3 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - - Table 5 { columns: [ person_id, $expr1, $expr2, first_value(person_name order_by(person_name ASC)), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 4 { columns: [ person_id, $expr1, $expr2, internal_last_seen_value(person_name), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - Table 6 { columns: [ vnode, id, person_backfill_finished, person_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, id, person_backfill_finished, person_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 7 { columns: [ auction_seller, $expr3, $expr4, count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 6 { columns: [ auction_seller, $expr3, $expr4, count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 8 { columns: [ vnode, id, auction_backfill_finished, auction_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 7 { columns: [ vnode, id, auction_backfill_finished, auction_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ id, name, starttime, $expr2, auction.seller, $expr3, $expr4 ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ id, name, starttime, $expr2, auction.seller, $expr3, $expr4 ] + ├── primary key: [ $0 ASC, $2 ASC, $3 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 0, 2, 3 ] + └── read pk prefix len hint: 3 - id: nexmark_q9 before: @@ -1130,27 +1141,31 @@ └─StreamTableScan { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr1(hidden), side_input.key(hidden)], stream_key: [bid._row_id, $expr1], pk_columns: [bid._row_id, $expr1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } - ├─StreamExchange { dist: HashShard($expr1) } - │ └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] } - │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(side_input.key) } - └─StreamTableScan { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } + └─StreamExchange { dist: HashShard(bid._row_id, $expr1) } + └─StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } + ├─StreamExchange { dist: HashShard($expr1) } + │ └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] } + │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(side_input.key) } + └─StreamTableScan { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr1(hidden), side_input.key(hidden)], stream_key: [bid._row_id, $expr1], pk_columns: [bid._row_id, $expr1], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } - ├── StreamExchange Hash([4]) from 1 - └── StreamExchange NoShuffle from 2 + └── StreamExchange Hash([5, 6]) from 1 Fragment 1 + StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } + ├── StreamExchange Hash([4]) from 2 + └── StreamExchange NoShuffle from 3 + + Fragment 2 StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] } └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } { state table: 1 } ├── Upstream └── BatchPlanNode @@ -1163,7 +1178,7 @@ ├── columns: [ auction, bidder, price, date_time, value, bid._row_id, $expr1, side_input.key ] ├── primary key: [ $5 ASC, $6 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] - ├── distribution key: [ 6 ] + ├── distribution key: [ 5, 6 ] └── read pk prefix len hint: 2 - id: nexmark_q14 @@ -1792,30 +1807,34 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], stream_key: [bid._row_id, auction], pk_columns: [bid._row_id, auction], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] } - ├─StreamExchange { dist: HashShard(bid.auction) } - │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } - └─StreamExchange { dist: HashShard(auction.id) } - └─StreamFilter { predicate: (auction.category = 10:Int32) } - └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + └─StreamExchange { dist: HashShard(bid.auction, bid._row_id) } + └─StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] } + ├─StreamExchange { dist: HashShard(bid.auction) } + │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamExchange { dist: HashShard(auction.id) } + └─StreamFilter { predicate: (auction.category = 10:Int32) } + └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], stream_key: [bid._row_id, auction], pk_columns: [bid._row_id, auction], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0, 14]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 4 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 StreamFilter { predicate: (auction.category = 10:Int32) } └── Chain { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 5 } ├── Upstream @@ -1837,7 +1856,7 @@ ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id, auction.id ] ├── primary key: [ $14 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 14 ] └── read pk prefix len hint: 2 - id: nexmark_q21 @@ -1943,33 +1962,37 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.price], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] } - ├─StreamExchange { dist: HashShard(auction.id) } - │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } - └─StreamProject { exprs: [bid.auction, max(bid.price)] } - └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [max(bid.price), count] } - └─StreamExchange { dist: HashShard(bid.auction) } - └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamExchange { dist: HashShard(auction.id) } + └─StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] } + ├─StreamExchange { dist: HashShard(auction.id) } + │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + └─StreamProject { exprs: [bid.auction, max(bid.price)] } + └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [max(bid.price), count] } + └─StreamExchange { dist: HashShard(bid.auction) } + └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [bid.auction, max(bid.price)] } - └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [max(bid.price), count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [bid.auction, max(bid.price)] } + └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [max(bid.price), count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 4 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -2026,9 +2049,9 @@ SELECT COUNT(*) / COUNT(DISTINCT auction) FROM bid ) batch_plan: |- - BatchNestedLoopJoin { type: Inner, predicate: (count(bid.auction) >= $expr1), output: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } + BatchNestedLoopJoin { type: Inner, predicate: (count(bid.auction) >= $expr1), output: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } ├─BatchExchange { order: [], dist: Single } - │ └─BatchHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } + │ └─BatchHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction)] } │ └─BatchHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } │ ├─BatchExchange { order: [], dist: HashShard(auction.id) } │ │ └─BatchScan { table: auction, columns: [auction.id, auction.item_name], distribution: UpstreamHashShard(auction.id) } @@ -2043,9 +2066,9 @@ └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } - └─StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } - ├─StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } - │ └─StreamHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), count] } + └─StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } + ├─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } + │ └─StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } │ └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } │ ├─StreamExchange { dist: HashShard(auction.id) } │ │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } @@ -2063,50 +2086,50 @@ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } + └── StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } ├── left table: 0 ├── right table: 1 - ├── StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } - │ └── StreamHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), count] } - │ ├── intermediate state table: 3 - │ ├── state tables: [ 2 ] + ├── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } + │ └── StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } + │ ├── intermediate state table: 2 + │ ├── state tables: [] │ ├── distinct tables: [] │ └── StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } - │ ├── left table: 4 - │ ├── right table: 6 - │ ├── left degree table: 5 - │ ├── right degree table: 7 + │ ├── left table: 3 + │ ├── right table: 5 + │ ├── left degree table: 4 + │ ├── right degree table: 6 │ ├── StreamExchange Hash([0]) from 1 │ └── StreamExchange Hash([0]) from 2 └── StreamExchange Broadcast from 3 Fragment 1 - Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 8 } + Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 9 } + Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 8 } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(bid.auction))) as $expr1] } - └── StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(bid.auction)), count] } { intermediate state table: 10, state tables: [], distinct tables: [] } + └── StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(bid.auction)), count] } { intermediate state table: 9, state tables: [], distinct tables: [] } └── StreamExchange Single from 4 Fragment 4 StreamStatelessSimpleAgg { aggs: [sum0(count), count(bid.auction)] } - └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } { intermediate state table: 11, state tables: [], distinct tables: [] } + └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } { intermediate state table: 10, state tables: [], distinct tables: [] } └── StreamExchange Hash([0]) from 5 Fragment 5 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 12 } + Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 11 } ├── Upstream └── BatchPlanNode Table 0 - ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction) ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction) ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0 ] @@ -2115,28 +2138,36 @@ Table 1 { columns: [ $expr1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } Table 2 - ├── columns: [ auction_id, auction_item_name, bid__row_id ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 0, 1, 2 ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 3 - ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction), count ] + ├── columns: [ auction_id, auction_item_name ] ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2, 3 ] + ├── value indices: [ 0, 1 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 4 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 5 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ bid_auction, bid__row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 + ├── columns: [ bid_auction, bid__row_id ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 7 { columns: [ bid_auction, bid__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 + ├── columns: [ bid_auction, bid__row_id, _degree ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 8 + Table 7 ├── columns: [ vnode, id, auction_backfill_finished, auction_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] @@ -2144,7 +2175,7 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 9 + Table 8 ├── columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] @@ -2152,11 +2183,16 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 10 { columns: [ sum0(sum0(count)), sum0(count(bid_auction)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 9 + ├── columns: [ sum0(sum0(count)), sum0(count(bid_auction)), count ] + ├── primary key: [] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 - Table 11 { columns: [ bid_auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ bid_auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 + Table 11 ├── columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] @@ -2199,39 +2235,43 @@ └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all } - ├─StreamExchange { dist: HashShard(auction.id) } - │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } - └─StreamProject { exprs: [bid.auction] } - └─StreamFilter { predicate: (count >= 20:Int32) } - └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } - └─StreamExchange { dist: HashShard(bid.auction) } - └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamExchange { dist: HashShard(auction.id) } + └─StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all } + ├─StreamExchange { dist: HashShard(auction.id) } + │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + └─StreamProject { exprs: [bid.auction] } + └─StreamFilter { predicate: (count >= 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } + └─StreamExchange { dist: HashShard(bid.auction) } + └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [bid.auction] } - └── StreamFilter { predicate: (count >= 20:Int32) } - └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } - ├── intermediate state table: 5 - ├── state tables: [] - ├── distinct tables: [] - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [bid.auction] } + └── StreamFilter { predicate: (count >= 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } + ├── intermediate state table: 5 + ├── state tables: [] + ├── distinct tables: [] + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── state table: 4 ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -2317,39 +2357,43 @@ └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all } - ├─StreamExchange { dist: HashShard(auction.id) } - │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } - └─StreamProject { exprs: [bid.auction] } - └─StreamFilter { predicate: (count < 20:Int32) } - └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } - └─StreamExchange { dist: HashShard(bid.auction) } - └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamExchange { dist: HashShard(auction.id) } + └─StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all } + ├─StreamExchange { dist: HashShard(auction.id) } + │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + └─StreamProject { exprs: [bid.auction] } + └─StreamFilter { predicate: (count < 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } + └─StreamExchange { dist: HashShard(bid.auction) } + └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [bid.auction] } - └── StreamFilter { predicate: (count < 20:Int32) } - └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } - ├── intermediate state table: 5 - ├── state tables: [] - ├── distinct tables: [] - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [bid.auction] } + └── StreamFilter { predicate: (count < 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } + ├── intermediate state table: 5 + ├── state tables: [] + ├── distinct tables: [] + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── state table: 4 ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -2427,7 +2471,7 @@ BatchTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } └─BatchExchange { order: [], dist: Single } └─BatchTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } - └─BatchHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } + └─BatchHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction)] } └─BatchHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } ├─BatchExchange { order: [], dist: HashShard(auction.id) } │ └─BatchScan { table: auction, columns: [auction.id, auction.item_name], distribution: UpstreamHashShard(auction.id) } @@ -2435,12 +2479,12 @@ └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [bid_count, auction_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } + └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } └─StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), Vnode(auction.id) as $expr1] } - └─StreamHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), count] } + └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction), Vnode(auction.id) as $expr1] } + └─StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } ├─StreamExchange { dist: HashShard(auction.id) } │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } @@ -2450,44 +2494,44 @@ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [bid_count, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } + └── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } └── StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } { state table: 0 } └── StreamExchange Single from 1 Fragment 1 StreamGroupTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } { state table: 1 } - └── StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), Vnode(auction.id) as $expr1] } - └── StreamHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), count] } - ├── intermediate state table: 3 - ├── state tables: [ 2 ] + └── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction), Vnode(auction.id) as $expr1] } + └── StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } + ├── intermediate state table: 2 + ├── state tables: [] ├── distinct tables: [] └── StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } - ├── left table: 4 - ├── right table: 6 - ├── left degree table: 5 - ├── right degree table: 7 + ├── left table: 3 + ├── right table: 5 + ├── left degree table: 4 + ├── right degree table: 6 ├── StreamExchange Hash([0]) from 2 └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 8 } + Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 9 } + Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 8 } ├── Upstream └── BatchPlanNode Table 0 - ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction), $expr1 ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), $expr1 ] ├── primary key: [ $2 DESC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [] └── read pk prefix len hint: 0 Table 1 - ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction), $expr1 ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), $expr1 ] ├── primary key: [ $3 ASC, $2 DESC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -2495,38 +2539,31 @@ └── vnode column idx: 3 Table 2 - ├── columns: [ auction_id, auction_item_name, bid__row_id ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 - - Table 3 - ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction), count ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 4 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 + Table 5 ├── columns: [ bid_auction, bid__row_id ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 0, 1 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 7 + Table 6 ├── columns: [ bid_auction, bid__row_id, _degree ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 8 + Table 7 ├── columns: [ vnode, id, auction_backfill_finished, auction_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] @@ -2534,7 +2571,7 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 9 + Table 8 ├── columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml index 7c694fad1fa67..31be64b2c480a 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -158,29 +158,33 @@ └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, seller], pk_columns: [_row_id, _row_id#1, seller], pk_conflict: NoCheck } - └─StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } - ├─StreamExchange { dist: HashShard(seller) } - │ └─StreamFilter { predicate: (category = 10:Int32) } - │ └─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) } - └─StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } - └─StreamRowIdGen { row_id_index: 8 } - └─StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } + └─StreamExchange { dist: HashShard(_row_id, seller, _row_id) } + └─StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } + ├─StreamExchange { dist: HashShard(seller) } + │ └─StreamFilter { predicate: (category = 10:Int32) } + │ └─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) } + └─StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } + └─StreamRowIdGen { row_id_index: 8 } + └─StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, seller], pk_columns: [_row_id, _row_id#1, seller], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([7]) from 1 - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([4, 5, 6]) from 1 Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([7]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 StreamFilter { predicate: (category = 10:Int32) } └── StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { source state table: 4 } - Fragment 2 + Fragment 3 StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } └── StreamRowIdGen { row_id_index: 8 } └── StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } { source state table: 5 } @@ -211,7 +215,7 @@ ├── columns: [ name, city, state, id, _row_id, seller, _row_id#1 ] ├── primary key: [ $4 ASC, $6 ASC, $5 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] - ├── distribution key: [ 5 ] + ├── distribution key: [ 4, 5, 6 ] └── read pk prefix len hint: 3 - id: nexmark_q4 @@ -737,40 +741,44 @@ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, name, starttime, $expr2], pk_columns: [id, name, starttime, $expr2], pk_conflict: NoCheck } - └─StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } - ├─StreamExchange { dist: HashShard(id, $expr1, $expr2) } - │ └─StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } - │ └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) } - │ └─StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } - │ └─StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } - │ └─StreamRowIdGen { row_id_index: 8 } - │ └─StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } - └─StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } - └─StreamExchange { dist: HashShard(seller, $expr3, $expr4) } - └─StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } - └─StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } - └─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, name, $expr1, $expr2) } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } + ├─StreamExchange { dist: HashShard(id, $expr1, $expr2) } + │ └─StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } + │ └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) } + │ └─StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } + │ └─StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } + │ └─StreamRowIdGen { row_id_index: 8 } + │ └─StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } + └─StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } + └─StreamExchange { dist: HashShard(seller, $expr3, $expr4) } + └─StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } + └─StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, name, starttime, $expr2], pk_columns: [id, name, starttime, $expr2], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0, 2, 3]) from 1 - └── StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } { state table: 6 } - └── StreamExchange Hash([0, 1, 2]) from 3 + └── StreamExchange Hash([0, 1, 2, 3]) from 1 Fragment 1 - StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } { state table: 4 } - └── StreamExchange Hash([0, 1, 2, 3]) from 2 + StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0, 2, 3]) from 2 + └── StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } { state table: 6 } + └── StreamExchange Hash([0, 1, 2]) from 4 Fragment 2 + StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } { state table: 4 } + └── StreamExchange Hash([0, 1, 2, 3]) from 3 + + Fragment 3 StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } └── StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } └── StreamRowIdGen { row_id_index: 8 } └── StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } └── StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } └── StreamRowIdGen { row_id_index: 10 } @@ -796,7 +804,7 @@ ├── columns: [ id, name, starttime, $expr2, seller, $expr3, $expr4 ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] - ├── distribution key: [ 0, 2, 3 ] + ├── distribution key: [ 0, 1, 2, 3 ] └── read pk prefix len hint: 4 - id: nexmark_q9 @@ -1629,31 +1637,31 @@ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } - └─StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(auction) } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } - └─StreamExchange { dist: HashShard(id) } - └─StreamFilter { predicate: (category = 10:Int32) } - └─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(auction, _row_id, _row_id) } + └─StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(auction) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamExchange { dist: HashShard(id) } + └─StreamFilter { predicate: (category = 10:Int32) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0, 14, 15]) from 1 Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 4 } - Fragment 2 + Fragment 3 StreamFilter { predicate: (category = 10:Int32) } └── StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { source state table: 5 } @@ -1674,7 +1682,7 @@ ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ] ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 14, 15 ] └── read pk prefix len hint: 3 - id: nexmark_q21 @@ -1775,30 +1783,34 @@ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } - ├─StreamExchange { dist: HashShard(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] } - └─StreamProject { exprs: [auction, max(price)] } - └─StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamExchange { dist: HashShard(id, _row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } + ├─StreamExchange { dist: HashShard(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] } + └─StreamProject { exprs: [auction, max(price)] } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [auction, max(price)] } - └── StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0, 3]) from 1 Fragment 1 + StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [auction, max(price)] } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { source state table: 4 } - Fragment 2 + Fragment 3 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 6 } @@ -1825,7 +1837,7 @@ ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, auction ] ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 3 ] └── read pk prefix len hint: 2 - id: nexmark_q102 @@ -1992,37 +2004,41 @@ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] } - ├─StreamExchange { dist: HashShard(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] } - └─StreamProject { exprs: [auction] } - └─StreamFilter { predicate: (count >= 20:Int32) } - └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamExchange { dist: HashShard(id, _row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] } + ├─StreamExchange { dist: HashShard(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] } + └─StreamProject { exprs: [auction] } + └─StreamFilter { predicate: (count >= 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [auction] } - └── StreamFilter { predicate: (count >= 20:Int32) } - └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0, 2]) from 1 Fragment 1 + StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [auction] } + └── StreamFilter { predicate: (count >= 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └── source state table: 4 - Fragment 2 + Fragment 3 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 6 } @@ -2049,7 +2065,7 @@ ├── columns: [ auction_id, auction_item_name, _row_id ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 2 ] └── read pk prefix len hint: 2 - id: nexmark_q104 @@ -2080,37 +2096,41 @@ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] } - ├─StreamExchange { dist: HashShard(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] } - └─StreamProject { exprs: [auction] } - └─StreamFilter { predicate: (count < 20:Int32) } - └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamExchange { dist: HashShard(id, _row_id) } + └─StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] } + ├─StreamExchange { dist: HashShard(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] } + └─StreamProject { exprs: [auction] } + └─StreamFilter { predicate: (count < 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [auction] } - └── StreamFilter { predicate: (count < 20:Int32) } - └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0, 2]) from 1 Fragment 1 + StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [auction] } + └── StreamFilter { predicate: (count < 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └── source state table: 4 - Fragment 2 + Fragment 3 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 6 } @@ -2137,7 +2157,7 @@ ├── columns: [ auction_id, auction_item_name, _row_id ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 2 ] └── read pk prefix len hint: 2 - id: nexmark_q105 diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml index ccdde39e76764..c6c3ffd4f5ad6 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml @@ -717,65 +717,69 @@ AND P.endtime = A.endtime; stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr6(hidden), $expr8(hidden), $expr9(hidden), $expr10(hidden)], stream_key: [id, name, starttime, $expr6], pk_columns: [id, name, starttime, $expr6], pk_conflict: NoCheck } - └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all } - ├─StreamExchange { dist: HashShard($expr2, $expr5, $expr6) } - │ └─StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] } - │ └─StreamExchange { dist: HashShard($expr2, $expr3, $expr5, $expr6) } - │ └─StreamProject { exprs: [$expr2, $expr3, $expr5, ($expr5 + '00:00:10':Interval) as $expr6] } - │ └─StreamProject { exprs: [$expr2, $expr3, $expr4, TumbleStart($expr4, '00:00:10':Interval) as $expr5, _row_id] } - │ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr2, Field(person, 1:Int32) as $expr3, Field(person, 6:Int32) as $expr4, _row_id] } - │ └─StreamFilter { predicate: (event_type = 0:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, _row_id] } - │ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamAppendOnlyDedup { dedup_cols: [$expr8, $expr9, $expr10] } - └─StreamExchange { dist: HashShard($expr8, $expr9, $expr10) } - └─StreamProject { exprs: [$expr8, $expr9, ($expr9 + '00:00:10':Interval) as $expr10] } - └─StreamProject { exprs: [$expr7, $expr8, TumbleStart($expr7, '00:00:10':Interval) as $expr9, _row_id] } - └─StreamProject { exprs: [Field(auction, 5:Int32) as $expr7, Field(auction, 7:Int32) as $expr8, _row_id] } - └─StreamFilter { predicate: (event_type = 1:Int32) } - └─StreamShare { id: 5 } - └─StreamProject { exprs: [event_type, person, auction, _row_id] } - └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr2, $expr3, $expr5, $expr6) } + └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all } + ├─StreamExchange { dist: HashShard($expr2, $expr5, $expr6) } + │ └─StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] } + │ └─StreamExchange { dist: HashShard($expr2, $expr3, $expr5, $expr6) } + │ └─StreamProject { exprs: [$expr2, $expr3, $expr5, ($expr5 + '00:00:10':Interval) as $expr6] } + │ └─StreamProject { exprs: [$expr2, $expr3, $expr4, TumbleStart($expr4, '00:00:10':Interval) as $expr5, _row_id] } + │ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr2, Field(person, 1:Int32) as $expr3, Field(person, 6:Int32) as $expr4, _row_id] } + │ └─StreamFilter { predicate: (event_type = 0:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, _row_id] } + │ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamAppendOnlyDedup { dedup_cols: [$expr8, $expr9, $expr10] } + └─StreamExchange { dist: HashShard($expr8, $expr9, $expr10) } + └─StreamProject { exprs: [$expr8, $expr9, ($expr9 + '00:00:10':Interval) as $expr10] } + └─StreamProject { exprs: [$expr7, $expr8, TumbleStart($expr7, '00:00:10':Interval) as $expr9, _row_id] } + └─StreamProject { exprs: [Field(auction, 5:Int32) as $expr7, Field(auction, 7:Int32) as $expr8, _row_id] } + └─StreamFilter { predicate: (event_type = 1:Int32) } + └─StreamShare { id: 5 } + └─StreamProject { exprs: [event_type, person, auction, _row_id] } + └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [id, name, starttime, $expr6(hidden), $expr8(hidden), $expr9(hidden), $expr10(hidden)], stream_key: [id, name, starttime, $expr6], pk_columns: [id, name, starttime, $expr6], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0, 2, 3]) from 1 - └── StreamAppendOnlyDedup { dedup_cols: [$expr8, $expr9, $expr10] } { state table: 6 } - └── StreamExchange Hash([0, 1, 2]) from 4 + └── StreamExchange Hash([0, 1, 2, 3]) from 1 Fragment 1 - StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] } { state table: 4 } - └── StreamExchange Hash([0, 1, 2, 3]) from 2 + StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0, 2, 3]) from 2 + └── StreamAppendOnlyDedup { dedup_cols: [$expr8, $expr9, $expr10] } { state table: 6 } + └── StreamExchange Hash([0, 1, 2]) from 5 Fragment 2 + StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] } { state table: 4 } + └── StreamExchange Hash([0, 1, 2, 3]) from 3 + + Fragment 3 StreamProject { exprs: [$expr2, $expr3, $expr5, ($expr5 + '00:00:10':Interval) as $expr6] } └── StreamProject { exprs: [$expr2, $expr3, $expr4, TumbleStart($expr4, '00:00:10':Interval) as $expr5, _row_id] } └── StreamProject { exprs: [Field(person, 0:Int32) as $expr2, Field(person, 1:Int32) as $expr3, Field(person, 6:Int32) as $expr4, _row_id] } └── StreamFilter { predicate: (event_type = 0:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 - Fragment 3 + Fragment 4 StreamProject { exprs: [event_type, person, auction, _row_id] } └── StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } └── StreamRowIdGen { row_id_index: 5 } └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 4 + Fragment 5 StreamProject { exprs: [$expr8, $expr9, ($expr9 + '00:00:10':Interval) as $expr10] } └── StreamProject { exprs: [$expr7, $expr8, TumbleStart($expr7, '00:00:10':Interval) as $expr9, _row_id] } └── StreamProject { exprs: [Field(auction, 5:Int32) as $expr7, Field(auction, 7:Int32) as $expr8, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 Table 0 { columns: [ $expr2, $expr3, $expr5, $expr6 ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } @@ -795,7 +799,7 @@ ├── columns: [ id, name, starttime, $expr6, $expr8, $expr9, $expr10 ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] - ├── distribution key: [ 0, 2, 3 ] + ├── distribution key: [ 0, 1, 2, 3 ] └── read pk prefix len hint: 4 - id: nexmark_q9 @@ -1180,59 +1184,63 @@ WHERE A.category = 10; stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: $expr3 = $expr9, output: [$expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard($expr3) } - │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr3, Field(bid, 1:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, Field(bid, 3:Int32) as $expr6, Field(bid, 4:Int32) as $expr7, Field(bid, 5:Int32) as $expr8, _row_id] } - │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } - │ ├─StreamFilter { predicate: (event_type = 2:Int32) } - │ │ └─StreamShare { id: 5 } - │ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ │ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: HashShard($expr9) } - └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr9, Field(auction, 1:Int32) as $expr10, Field(auction, 2:Int32) as $expr11, Field(auction, 3:Int32) as $expr12, Field(auction, 4:Int32) as $expr13, Field(auction, 5:Int32) as $expr14, Field(auction, 6:Int32) as $expr15, Field(auction, 7:Int32) as $expr16, Field(auction, 8:Int32) as $expr17, _row_id] } - └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - └─StreamShare { id: 5 } - └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr3, _row_id, _row_id) } + └─StreamHashJoin { type: Inner, predicate: $expr3 = $expr9, output: [$expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard($expr3) } + │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr3, Field(bid, 1:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, Field(bid, 3:Int32) as $expr6, Field(bid, 4:Int32) as $expr7, Field(bid, 5:Int32) as $expr8, _row_id] } + │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } + │ ├─StreamFilter { predicate: (event_type = 2:Int32) } + │ │ └─StreamShare { id: 5 } + │ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ │ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard($expr9) } + └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr9, Field(auction, 1:Int32) as $expr10, Field(auction, 2:Int32) as $expr11, Field(auction, 3:Int32) as $expr12, Field(auction, 4:Int32) as $expr13, Field(auction, 5:Int32) as $expr14, Field(auction, 6:Int32) as $expr15, Field(auction, 7:Int32) as $expr16, Field(auction, 8:Int32) as $expr17, _row_id] } + └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } + └─StreamShare { id: 5 } + └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: $expr3 = $expr9, output: [$expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0, 14, 15]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: $expr3 = $expr9, output: [$expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 5 + + Fragment 2 StreamProject { exprs: [Field(bid, 0:Int32) as $expr3, Field(bid, 1:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, Field(bid, 3:Int32) as $expr6, Field(bid, 4:Int32) as $expr7, Field(bid, 5:Int32) as $expr8, _row_id] } └── StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } { left table: 4, right table: 5 } ├── StreamFilter { predicate: (event_type = 2:Int32) } - │ └── StreamExchange NoShuffle from 2 - └── StreamExchange Broadcast from 3 + │ └── StreamExchange NoShuffle from 3 + └── StreamExchange Broadcast from 4 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } └── StreamRowIdGen { row_id_index: 5 } └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 6 } - Fragment 3 + Fragment 4 StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └── StreamNow { output: [now] } { state table: 7 } - Fragment 4 + Fragment 5 StreamProject { exprs: [Field(auction, 0:Int32) as $expr9, Field(auction, 1:Int32) as $expr10, Field(auction, 2:Int32) as $expr11, Field(auction, 3:Int32) as $expr12, Field(auction, 4:Int32) as $expr13, Field(auction, 5:Int32) as $expr14, Field(auction, 6:Int32) as $expr15, Field(auction, 7:Int32) as $expr16, Field(auction, 8:Int32) as $expr17, _row_id] } └── StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, _row_id ], primary key: [ $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1250,7 +1258,12 @@ Table 7 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ], primary key: [ $14 ASC, $15 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ] + ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] + ├── distribution key: [ 0, 14, 15 ] + └── read pk prefix len hint: 3 - id: nexmark_q21 before: @@ -1375,61 +1388,65 @@ ) b ON a.id = b.auction; stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamProject { exprs: [$expr5, max($expr6)] } - └─StreamHashAgg { group_key: [$expr5], aggs: [max($expr6), count] } - └─StreamExchange { dist: HashShard($expr5) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6, _row_id] } - └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } - ├─StreamFilter { predicate: (event_type = 2:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } - └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard($expr2, _row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamProject { exprs: [$expr5, max($expr6)] } + └─StreamHashAgg { group_key: [$expr5], aggs: [max($expr6), count] } + └─StreamExchange { dist: HashShard($expr5) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6, _row_id] } + └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } + ├─StreamFilter { predicate: (event_type = 2:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } + └─StreamNow { output: [now] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [$expr5, max($expr6)] } - └── StreamHashAgg { group_key: [$expr5], aggs: [max($expr6), count] } { intermediate state table: 6, state tables: [ 5 ], distinct tables: [] } - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 3]) from 1 Fragment 1 + StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [$expr5, max($expr6)] } + └── StreamHashAgg { group_key: [$expr5], aggs: [max($expr6), count] } { intermediate state table: 6, state tables: [ 5 ], distinct tables: [] } + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 4 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6, _row_id] } └── StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } { left table: 7, right table: 8 } ├── StreamFilter { predicate: (event_type = 2:Int32) } - │ └── StreamExchange NoShuffle from 2 - └── StreamExchange Broadcast from 4 + │ └── StreamExchange NoShuffle from 3 + └── StreamExchange Broadcast from 5 - Fragment 4 + Fragment 5 StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } └── StreamNow { output: [now] } { state table: 9 } @@ -1457,7 +1474,7 @@ ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, $expr5 ] ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 3 ] └── read pk prefix len hint: 2 - id: nexmark_q102 @@ -1642,65 +1659,69 @@ ); stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr5, output: all } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamProject { exprs: [$expr5] } - └─StreamFilter { predicate: (count >= 20:Int32) } - └─StreamHashAgg { group_key: [$expr5], aggs: [count] } - └─StreamExchange { dist: HashShard($expr5) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } - └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } - ├─StreamFilter { predicate: (event_type = 2:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } - └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard($expr2, _row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr5, output: all } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamProject { exprs: [$expr5] } + └─StreamFilter { predicate: (count >= 20:Int32) } + └─StreamHashAgg { group_key: [$expr5], aggs: [count] } + └─StreamExchange { dist: HashShard($expr5) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } + └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } + ├─StreamFilter { predicate: (event_type = 2:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } + └─StreamNow { output: [now] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr5, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [$expr5] } - └── StreamFilter { predicate: (count >= 20:Int32) } - └── StreamHashAgg { group_key: [$expr5], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 2]) from 1 Fragment 1 + StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr5, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [$expr5] } + └── StreamFilter { predicate: (count >= 20:Int32) } + └── StreamHashAgg { group_key: [$expr5], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 4 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } └── StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } ├── left table: 6 ├── right table: 7 ├── StreamFilter { predicate: (event_type = 2:Int32) } - │ └── StreamExchange NoShuffle from 2 - └── StreamExchange Broadcast from 4 + │ └── StreamExchange NoShuffle from 3 + └── StreamExchange Broadcast from 5 - Fragment 4 + Fragment 5 StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } └── StreamNow { output: [now] } { state table: 8 } @@ -1731,7 +1752,7 @@ ├── columns: [ auction_id, auction_item_name, _row_id ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 2 ] └── read pk prefix len hint: 2 - id: nexmark_q104 @@ -1752,65 +1773,69 @@ ); stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr5, output: all } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamProject { exprs: [$expr5] } - └─StreamFilter { predicate: (count < 20:Int32) } - └─StreamHashAgg { group_key: [$expr5], aggs: [count] } - └─StreamExchange { dist: HashShard($expr5) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } - └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } - ├─StreamFilter { predicate: (event_type = 2:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } - └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard($expr2, _row_id) } + └─StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr5, output: all } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamProject { exprs: [$expr5] } + └─StreamFilter { predicate: (count < 20:Int32) } + └─StreamHashAgg { group_key: [$expr5], aggs: [count] } + └─StreamExchange { dist: HashShard($expr5) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } + └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } + ├─StreamFilter { predicate: (event_type = 2:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } + └─StreamNow { output: [now] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr5, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [$expr5] } - └── StreamFilter { predicate: (count < 20:Int32) } - └── StreamHashAgg { group_key: [$expr5], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 2]) from 1 Fragment 1 + StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr5, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [$expr5] } + └── StreamFilter { predicate: (count < 20:Int32) } + └── StreamHashAgg { group_key: [$expr5], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 4 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } └── StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } ├── left table: 6 ├── right table: 7 ├── StreamFilter { predicate: (event_type = 2:Int32) } - │ └── StreamExchange NoShuffle from 2 - └── StreamExchange Broadcast from 4 + │ └── StreamExchange NoShuffle from 3 + └── StreamExchange Broadcast from 5 - Fragment 4 + Fragment 5 StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } └── StreamNow { output: [now] } { state table: 8 } @@ -1841,7 +1866,7 @@ ├── columns: [ auction_id, auction_item_name, _row_id ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 2 ] └── read pk prefix len hint: 2 - id: nexmark_q105 diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml index 3554e31d281ec..39adc39a16653 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -131,45 +131,49 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), $expr3(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, $expr3], pk_columns: [_row_id, _row_id#1, $expr3], pk_conflict: NoCheck } - └─StreamHashJoin [append_only] { type: Inner, predicate: $expr3 = $expr4, output: [$expr5, $expr6, $expr7, $expr2, _row_id, $expr3, _row_id] } - ├─StreamExchange { dist: HashShard($expr3) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 7:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, person, auction, _row_id] } - │ └─StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32))) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: HashShard($expr4) } - └─StreamProject { exprs: [Field(person, 0:Int32) as $expr4, Field(person, 1:Int32) as $expr5, Field(person, 4:Int32) as $expr6, Field(person, 5:Int32) as $expr7, _row_id] } - └─StreamFilter { predicate: (((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, person, auction, _row_id] } - └─StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32))) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard(_row_id, $expr3, _row_id) } + └─StreamHashJoin [append_only] { type: Inner, predicate: $expr3 = $expr4, output: [$expr5, $expr6, $expr7, $expr2, _row_id, $expr3, _row_id] } + ├─StreamExchange { dist: HashShard($expr3) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 7:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, person, auction, _row_id] } + │ └─StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32))) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr4) } + └─StreamProject { exprs: [Field(person, 0:Int32) as $expr4, Field(person, 1:Int32) as $expr5, Field(person, 4:Int32) as $expr6, Field(person, 5:Int32) as $expr7, _row_id] } + └─StreamFilter { predicate: (((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, person, auction, _row_id] } + └─StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32))) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), $expr3(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, $expr3], pk_columns: [_row_id, _row_id#1, $expr3], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin [append_only] { type: Inner, predicate: $expr3 = $expr4, output: [$expr5, $expr6, $expr7, $expr2, _row_id, $expr3, _row_id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([1]) from 1 - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([4, 5, 6]) from 1 Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: $expr3 = $expr4, output: [$expr5, $expr6, $expr7, $expr2, _row_id, $expr3, _row_id] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([1]) from 2 + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 7:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, person, auction, _row_id] } └── StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32))) } └── StreamRowIdGen { row_id_index: 5 } @@ -177,10 +181,10 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(person, 0:Int32) as $expr4, Field(person, 1:Int32) as $expr5, Field(person, 4:Int32) as $expr6, Field(person, 5:Int32) as $expr7, _row_id] } └── StreamFilter { predicate: (((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } @@ -194,7 +198,7 @@ Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ name, city, state, id, _row_id, $expr3, _row_id#1 ], primary key: [ $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 5 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ name, city, state, id, _row_id, $expr3, _row_id#1 ], primary key: [ $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 4, 5, 6 ], read pk prefix len hint: 3 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -696,43 +700,48 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time, $expr5(hidden)] } - └─StreamHashJoin [interval] { type: Inner, predicate: $expr4 = max($expr4) AND ($expr1 >= $expr6) AND ($expr1 <= $expr5), conditions_to_clean_left_state_table: ($expr1 >= $expr6), conditions_to_clean_right_state_table: ($expr1 <= $expr5), output_watermarks: [$expr1, $expr5], output: [$expr2, $expr4, $expr3, $expr1, _row_id, $expr5] } - ├─StreamExchange { dist: HashShard($expr4) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: (event_type = 2:Int32) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: HashShard(max($expr4)) } - └─StreamProject { exprs: [$expr5, max($expr4), ($expr5 - '00:00:10':Interval) as $expr6], output_watermarks: [$expr5, $expr6] } - └─StreamHashAgg [append_only] { group_key: [$expr5], aggs: [max($expr4), count], output_watermarks: [$expr5] } - └─StreamExchange { dist: HashShard($expr5) } - └─StreamProject { exprs: [(TumbleStart($expr1, '00:00:10':Interval) + '00:00:10':Interval) as $expr5, $expr4, _row_id], output_watermarks: [$expr5] } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamFilter { predicate: (event_type = 2:Int32) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr4, _row_id, $expr5) } + └─StreamHashJoin [interval] { type: Inner, predicate: $expr4 = max($expr4) AND ($expr1 >= $expr6) AND ($expr1 <= $expr5), conditions_to_clean_left_state_table: ($expr1 >= $expr6), conditions_to_clean_right_state_table: ($expr1 <= $expr5), output_watermarks: [$expr1, $expr5], output: [$expr2, $expr4, $expr3, $expr1, _row_id, $expr5] } + ├─StreamExchange { dist: HashShard($expr4) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: (event_type = 2:Int32) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard(max($expr4)) } + └─StreamProject { exprs: [$expr5, max($expr4), ($expr5 - '00:00:10':Interval) as $expr6], output_watermarks: [$expr5, $expr6] } + └─StreamHashAgg [append_only] { group_key: [$expr5], aggs: [max($expr4), count], output_watermarks: [$expr5] } + └─StreamExchange { dist: HashShard($expr5) } + └─StreamProject { exprs: [(TumbleStart($expr1, '00:00:10':Interval) + '00:00:10':Interval) as $expr5, $expr4, _row_id], output_watermarks: [$expr5] } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamFilter { predicate: (event_type = 2:Int32) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time, $expr5(hidden)] } { materialized table: 4294967294 } - └── StreamHashJoin [interval] { type: Inner, predicate: $expr4 = max($expr4) AND ($expr1 >= $expr6) AND ($expr1 <= $expr5), conditions_to_clean_left_state_table: ($expr1 >= $expr6), conditions_to_clean_right_state_table: ($expr1 <= $expr5), output_watermarks: [$expr1, $expr5], output: [$expr2, $expr4, $expr3, $expr1, _row_id, $expr5] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([2]) from 1 - └── StreamExchange Hash([1]) from 3 + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time, $expr5(hidden)] } + ├── materialized table: 4294967294 + └── StreamExchange Hash([1, 4, 5]) from 1 Fragment 1 - StreamNoOp - └── StreamExchange NoShuffle from 2 + StreamHashJoin [interval] { type: Inner, predicate: $expr4 = max($expr4) AND ($expr1 >= $expr6) AND ($expr1 <= $expr5), conditions_to_clean_left_state_table: ($expr1 >= $expr6), conditions_to_clean_right_state_table: ($expr1 <= $expr5), output_watermarks: [$expr1, $expr5], output: [$expr2, $expr4, $expr3, $expr1, _row_id, $expr5] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([2]) from 2 + └── StreamExchange Hash([1]) from 4 Fragment 2 + StreamNoOp + └── StreamExchange NoShuffle from 3 + + Fragment 3 StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: (event_type = 2:Int32) } └── StreamRowIdGen { row_id_index: 5 } @@ -740,14 +749,14 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [$expr5, max($expr4), ($expr5 - '00:00:10':Interval) as $expr6], output_watermarks: [$expr5, $expr6] } └── StreamHashAgg [append_only] { group_key: [$expr5], aggs: [max($expr4), count], output_watermarks: [$expr5] } { intermediate state table: 6, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0]) from 5 - Fragment 4 + Fragment 5 StreamProject { exprs: [(TumbleStart($expr1, '00:00:10':Interval) + '00:00:10':Interval) as $expr5, $expr4, _row_id], output_watermarks: [$expr5] } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr2, $expr3, $expr4, $expr1, _row_id ], primary key: [ $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } @@ -763,7 +772,7 @@ Table 6 { columns: [ $expr5, max($expr4), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, price, bidder, date_time, _row_id, $expr5 ], primary key: [ $4 ASC, $5 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ auction, price, bidder, date_time, _row_id, $expr5 ], primary key: [ $4 ASC, $5 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1, 4, 5 ], read pk prefix len hint: 3 } eowc_stream_plan: |- StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time] } @@ -845,52 +854,56 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime, $expr5(hidden), $expr6(hidden), $expr8(hidden)] } - └─StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } - ├─StreamExchange { dist: HashShard($expr3, $expr2, $expr5) } - │ └─StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] } - │ └─StreamExchange { dist: HashShard($expr3, $expr4, $expr2, $expr5) } - │ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, $expr2, ($expr2 + '00:00:10':Interval) as $expr5], output_watermarks: [$expr2, $expr5] } - │ └─StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr2, _row_id], output_watermarks: [$expr1, $expr2] } - │ └─StreamFilter { predicate: (event_type = 0:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamAppendOnlyDedup { dedup_cols: [$expr7, $expr6, $expr8] } - └─StreamExchange { dist: HashShard($expr7, $expr6, $expr8) } - └─StreamProject { exprs: [Field(auction, 7:Int32) as $expr7, $expr6, ($expr6 + '00:00:10':Interval) as $expr8], output_watermarks: [$expr6, $expr8] } - └─StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr6, _row_id], output_watermarks: [$expr1, $expr6] } - └─StreamFilter { predicate: (event_type = 1:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr3, $expr4, $expr2, $expr5) } + └─StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } + ├─StreamExchange { dist: HashShard($expr3, $expr2, $expr5) } + │ └─StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] } + │ └─StreamExchange { dist: HashShard($expr3, $expr4, $expr2, $expr5) } + │ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, $expr2, ($expr2 + '00:00:10':Interval) as $expr5], output_watermarks: [$expr2, $expr5] } + │ └─StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr2, _row_id], output_watermarks: [$expr1, $expr2] } + │ └─StreamFilter { predicate: (event_type = 0:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamAppendOnlyDedup { dedup_cols: [$expr7, $expr6, $expr8] } + └─StreamExchange { dist: HashShard($expr7, $expr6, $expr8) } + └─StreamProject { exprs: [Field(auction, 7:Int32) as $expr7, $expr6, ($expr6 + '00:00:10':Interval) as $expr8], output_watermarks: [$expr6, $expr8] } + └─StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr6, _row_id], output_watermarks: [$expr1, $expr6] } + └─StreamFilter { predicate: (event_type = 1:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime, $expr5(hidden), $expr6(hidden), $expr8(hidden)] } ├── materialized table: 4294967294 - └── StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0, 2, 3]) from 1 - └── StreamAppendOnlyDedup { dedup_cols: [$expr7, $expr6, $expr8] } { state table: 7 } - └── StreamExchange Hash([0, 1, 2]) from 4 + └── StreamExchange Hash([0, 1, 2, 3]) from 1 Fragment 1 - StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] } { state table: 4 } - └── StreamExchange Hash([0, 1, 2, 3]) from 2 + StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0, 2, 3]) from 2 + └── StreamAppendOnlyDedup { dedup_cols: [$expr7, $expr6, $expr8] } { state table: 7 } + └── StreamExchange Hash([0, 1, 2]) from 5 Fragment 2 + StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] } { state table: 4 } + └── StreamExchange Hash([0, 1, 2, 3]) from 3 + + Fragment 3 StreamProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, $expr2, ($expr2 + '00:00:10':Interval) as $expr5], output_watermarks: [$expr2, $expr5] } └── StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr2, _row_id], output_watermarks: [$expr1, $expr2] } └── StreamFilter { predicate: (event_type = 0:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 - Fragment 3 + Fragment 4 StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } └── StreamRowIdGen { row_id_index: 5 } @@ -898,11 +911,11 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 6 } - Fragment 4 + Fragment 5 StreamProject { exprs: [Field(auction, 7:Int32) as $expr7, $expr6, ($expr6 + '00:00:10':Interval) as $expr8], output_watermarks: [$expr6, $expr8] } └── StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr6, _row_id], output_watermarks: [$expr1, $expr6] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 Table 0 { columns: [ $expr3, $expr4, $expr2, $expr5 ], primary key: [ $2 ASC, $3 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } @@ -920,7 +933,7 @@ Table 7 { columns: [ $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4294967294 { columns: [ id, name, starttime, $expr5, $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 4 } + Table 4294967294 { columns: [ id, name, starttime, $expr5, $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } eowc_stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime] } @@ -1715,41 +1728,45 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } - └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr7, output: [$expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: (event_type = 2:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: HashShard($expr7) } - └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr7, Field(auction, 1:Int32) as $expr8, Field(auction, 2:Int32) as $expr9, Field(auction, 3:Int32) as $expr10, Field(auction, 4:Int32) as $expr11, $expr1, Field(auction, 6:Int32) as $expr12, Field(auction, 7:Int32) as $expr13, Field(auction, 8:Int32) as $expr14, _row_id], output_watermarks: [$expr1] } - └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr2, _row_id, _row_id) } + └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr7, output: [$expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: (event_type = 2:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr7) } + └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr7, Field(auction, 1:Int32) as $expr8, Field(auction, 2:Int32) as $expr9, Field(auction, 3:Int32) as $expr10, Field(auction, 4:Int32) as $expr11, $expr1, Field(auction, 6:Int32) as $expr12, Field(auction, 7:Int32) as $expr13, Field(auction, 8:Int32) as $expr14, _row_id], output_watermarks: [$expr1] } + └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr7, output: [$expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 14, 15]) from 1 Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr7, output: [$expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: (event_type = 2:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } └── StreamRowIdGen { row_id_index: 5 } @@ -1757,10 +1774,10 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(auction, 0:Int32) as $expr7, Field(auction, 1:Int32) as $expr8, Field(auction, 2:Int32) as $expr9, Field(auction, 3:Int32) as $expr10, Field(auction, 4:Int32) as $expr11, $expr1, Field(auction, 6:Int32) as $expr12, Field(auction, 7:Int32) as $expr13, Field(auction, 8:Int32) as $expr14, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, _row_id ], primary key: [ $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1774,7 +1791,12 @@ Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ], primary key: [ $14 ASC, $15 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ] + ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] + ├── distribution key: [ 0, 14, 15 ] + └── read pk prefix len hint: 3 eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -1909,45 +1931,49 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr4(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamProject { exprs: [$expr4, max($expr5)] } - └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [max($expr5), count] } - └─StreamExchange { dist: HashShard($expr4) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, _row_id] } - └─StreamFilter { predicate: (event_type = 2:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr2, _row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamProject { exprs: [$expr4, max($expr5)] } + └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [max($expr5), count] } + └─StreamExchange { dist: HashShard($expr4) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, _row_id] } + └─StreamFilter { predicate: (event_type = 2:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr4(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [$expr4, max($expr5)] } - └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [max($expr5), count] } { intermediate state table: 6, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 3]) from 1 Fragment 1 + StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [$expr4, max($expr5)] } + └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [max($expr5), count] } { intermediate state table: 6, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, _row_id] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } @@ -1955,10 +1981,10 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, _row_id] } └── StreamFilter { predicate: (event_type = 2:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1978,7 +2004,7 @@ ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, $expr4 ] ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 3 ] └── read pk prefix len hint: 2 eowc_stream_error: |- @@ -2184,47 +2210,51 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr4, output: all } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamProject { exprs: [$expr4] } - └─StreamFilter { predicate: (count >= 20:Int32) } - └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } - └─StreamExchange { dist: HashShard($expr4) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } - └─StreamFilter { predicate: (event_type = 2:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr2, _row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr4, output: all } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamProject { exprs: [$expr4] } + └─StreamFilter { predicate: (count >= 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } + └─StreamExchange { dist: HashShard($expr4) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } + └─StreamFilter { predicate: (event_type = 2:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [$expr4] } - └── StreamFilter { predicate: (count >= 20:Int32) } - └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 2]) from 1 Fragment 1 + StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [$expr4] } + └── StreamFilter { predicate: (count >= 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, _row_id] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } @@ -2232,10 +2262,10 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } └── StreamFilter { predicate: (event_type = 2:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2251,7 +2281,7 @@ Table 6 { columns: [ $expr4, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 2 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -2290,47 +2320,51 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr4, output: all } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamProject { exprs: [$expr4] } - └─StreamFilter { predicate: (count < 20:Int32) } - └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } - └─StreamExchange { dist: HashShard($expr4) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } - └─StreamFilter { predicate: (event_type = 2:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr2, _row_id) } + └─StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr4, output: all } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamProject { exprs: [$expr4] } + └─StreamFilter { predicate: (count < 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } + └─StreamExchange { dist: HashShard($expr4) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } + └─StreamFilter { predicate: (event_type = 2:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [$expr4] } - └── StreamFilter { predicate: (count < 20:Int32) } - └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 2]) from 1 Fragment 1 + StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [$expr4] } + └── StreamFilter { predicate: (count < 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, _row_id] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } @@ -2338,10 +2372,10 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } └── StreamFilter { predicate: (event_type = 2:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2357,7 +2391,7 @@ Table 6 { columns: [ $expr4, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 2 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. diff --git a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml index 5cc81578f829c..733a19f4ba05c 100644 --- a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml @@ -185,15 +185,16 @@ └─BatchScan { table: t, columns: [t.x, t.y, t.w], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, y, sum, max, min, t._row_id(hidden), t.y(hidden)], stream_key: [t._row_id, y], pk_columns: [t._row_id, y], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.y = t.y, output: [t.x, t.y, sum(t.x), max(t.x), min(t.w), t._row_id, t.y] } - ├─StreamExchange { dist: HashShard(t.y) } - │ └─StreamShare { id: 1 } - │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.w, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamProject { exprs: [t.y, sum(t.x), max(t.x), min(t.w)] } - └─StreamHashAgg { group_key: [t.y], aggs: [sum(t.x), max(t.x), min(t.w), count] } - └─StreamExchange { dist: HashShard(t.y) } - └─StreamShare { id: 1 } - └─StreamTableScan { table: t, columns: [t.x, t.y, t.w, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.y, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.y = t.y, output: [t.x, t.y, sum(t.x), max(t.x), min(t.w), t._row_id, t.y] } + ├─StreamExchange { dist: HashShard(t.y) } + │ └─StreamShare { id: 1 } + │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.w, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProject { exprs: [t.y, sum(t.x), max(t.x), min(t.w)] } + └─StreamHashAgg { group_key: [t.y], aggs: [sum(t.x), max(t.x), min(t.w), count] } + └─StreamExchange { dist: HashShard(t.y) } + └─StreamShare { id: 1 } + └─StreamTableScan { table: t, columns: [t.x, t.y, t.w, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - id: aggregate with over clause, rows frame definition with implicit current row, without ORDER BY sql: | create table t(x int, y int); @@ -913,12 +914,13 @@ └─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [t1x, t2x, t1z, t2y, t2z, t._row_id(hidden)], stream_key: [t1x, t._row_id], pk_columns: [t1x, t._row_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.x = t.x, output: [t.x, t.x, t.z, t.y, t.z, t._row_id] } - ├─StreamGroupTopN { order: [t.y ASC], limit: 1, offset: 0, group_key: [t.x] } - │ └─StreamExchange { dist: HashShard(t.x) } - │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(t.x) } - └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.x, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.x = t.x, output: [t.x, t.x, t.z, t.y, t.z, t._row_id] } + ├─StreamGroupTopN { order: [t.y ASC], limit: 1, offset: 0, group_key: [t.x] } + │ └─StreamExchange { dist: HashShard(t.x) } + │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.x) } + └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - id: split calls with different ORDER BY or PARTITION BY sql: | create table t(x int, y int, z int); diff --git a/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml b/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml index 55131ed1614cd..65469e7754e6b 100644 --- a/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml @@ -21,15 +21,16 @@ Tone.id = Ttwo.id; stream_plan: |- StreamMaterialize { columns: [max_v1, max_v2, t1.id(hidden), t2.id(hidden)], stream_key: [t1.id], pk_columns: [t1.id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.id = t2.id, output: [max(t1.v1), max(t2.v2), t1.id, t2.id] } - ├─StreamProject { exprs: [t1.id, max(t1.v1)] } - │ └─StreamHashAgg { group_key: [t1.id], aggs: [max(t1.v1), count] } - │ └─StreamExchange { dist: HashShard(t1.id) } - │ └─StreamTableScan { table: t1, columns: [t1.id, t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamProject { exprs: [t2.id, max(t2.v2)] } - └─StreamHashAgg { group_key: [t2.id], aggs: [max(t2.v2), count] } - └─StreamExchange { dist: HashShard(t2.id) } - └─StreamTableScan { table: t2, columns: [t2.id, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.id) } + └─StreamHashJoin { type: Inner, predicate: t1.id = t2.id, output: [max(t1.v1), max(t2.v2), t1.id, t2.id] } + ├─StreamProject { exprs: [t1.id, max(t1.v1)] } + │ └─StreamHashAgg { group_key: [t1.id], aggs: [max(t1.v1), count] } + │ └─StreamExchange { dist: HashShard(t1.id) } + │ └─StreamTableScan { table: t1, columns: [t1.id, t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamProject { exprs: [t2.id, max(t2.v2)] } + └─StreamHashAgg { group_key: [t2.id], aggs: [max(t2.v2), count] } + └─StreamExchange { dist: HashShard(t2.id) } + └─StreamTableScan { table: t2, columns: [t2.id, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - sql: | create table t (id int, v int); SELECT Tone.max_v, Ttwo.min_v @@ -51,15 +52,16 @@ Tone.id = Ttwo.id; stream_plan: |- StreamMaterialize { columns: [max_v, min_v, t.id(hidden), t.id#1(hidden)], stream_key: [t.id], pk_columns: [t.id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.id = t.id, output: [max(t.v), min(t.v), t.id, t.id] } - ├─StreamProject { exprs: [t.id, max(t.v)] } - │ └─StreamHashAgg { group_key: [t.id], aggs: [max(t.v), count] } - │ └─StreamExchange { dist: HashShard(t.id) } - │ └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamProject { exprs: [t.id, min(t.v)] } - └─StreamHashAgg { group_key: [t.id], aggs: [min(t.v), count] } - └─StreamExchange { dist: HashShard(t.id) } - └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.id) } + └─StreamHashJoin { type: Inner, predicate: t.id = t.id, output: [max(t.v), min(t.v), t.id, t.id] } + ├─StreamProject { exprs: [t.id, max(t.v)] } + │ └─StreamHashAgg { group_key: [t.id], aggs: [max(t.v), count] } + │ └─StreamExchange { dist: HashShard(t.id) } + │ └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProject { exprs: [t.id, min(t.v)] } + └─StreamHashAgg { group_key: [t.id], aggs: [min(t.v), count] } + └─StreamExchange { dist: HashShard(t.id) } + └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 varchar, v2 varchar, v3 varchar); select diff --git a/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml index ae37459ef7bed..91dff73df0e6a 100644 --- a/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml @@ -260,15 +260,16 @@ └─LogicalScan { table: t2, columns: [t2.v2], predicate: (t2.v2 > ('2021-04-01 00:00:00+00:00':Timestamptz + '01:00:00':Interval)) } stream_plan: |- StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1], pk_columns: [t1._row_id, t2._row_id, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id], cleaned_by_watermark: true } - │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [AddWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: HashShard(t2.v2) } - └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id], cleaned_by_watermark: true } + │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [AddWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(t2.v2) } + └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: now() in a complex cmp expr does not get pushed down sql: | create table t1(v1 timestamp with time zone); @@ -343,14 +344,15 @@ └─LogicalScan { table: t2, columns: [t2.v2], predicate: (t2.v2 > '2021-04-01 00:00:00+00:00':Timestamptz) } stream_plan: |- StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1], pk_columns: [t1._row_id, t2._row_id, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamDynamicFilter { predicate: (t1.v1 > now), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id], cleaned_by_watermark: true } - │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: HashShard(t2.v2) } - └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamDynamicFilter { predicate: (t1.v1 > now), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id], cleaned_by_watermark: true } + │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(t2.v2) } + └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: eq-predicate derived condition is banned for mismatching types sql: | create table t1(v1 int, v2 int); diff --git a/src/frontend/planner_test/tests/testdata/output/project_set.yaml b/src/frontend/planner_test/tests/testdata/output/project_set.yaml index 23db668a070df..676772d99d72e 100644 --- a/src/frontend/planner_test/tests/testdata/output/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/output/project_set.yaml @@ -155,17 +155,18 @@ └─BatchScan { table: t, columns: [t.x], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [unnest, t._row_id(hidden), projected_row_id(hidden), t._row_id#1(hidden), projected_row_id#1(hidden)], stream_key: [t._row_id, projected_row_id, t._row_id#1, projected_row_id#1, unnest], pk_columns: [t._row_id, projected_row_id, t._row_id#1, projected_row_id#1, unnest], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: Unnest($0) = Unnest($0), output: [Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id] } - ├─StreamExchange { dist: HashShard(Unnest($0)) } - │ └─StreamShare { id: 3 } - │ └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } - │ └─StreamProjectSet { select_list: [Unnest($0), $1] } - │ └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(Unnest($0)) } - └─StreamShare { id: 3 } - └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } - └─StreamProjectSet { select_list: [Unnest($0), $1] } - └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id) } + └─StreamHashJoin { type: Inner, predicate: Unnest($0) = Unnest($0), output: [Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id] } + ├─StreamExchange { dist: HashShard(Unnest($0)) } + │ └─StreamShare { id: 3 } + │ └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } + │ └─StreamProjectSet { select_list: [Unnest($0), $1] } + │ └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(Unnest($0)) } + └─StreamShare { id: 3 } + └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } + └─StreamProjectSet { select_list: [Unnest($0), $1] } + └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: issue-10080 sql: | with cte as (SELECT 1 as v1, unnest(array[1,2,3,4,5]) AS v2) select v1 from cte; diff --git a/src/frontend/planner_test/tests/testdata/output/select_except.yaml b/src/frontend/planner_test/tests/testdata/output/select_except.yaml index 2193524b7076f..ffd6da30b90bc 100644 --- a/src/frontend/planner_test/tests/testdata/output/select_except.yaml +++ b/src/frontend/planner_test/tests/testdata/output/select_except.yaml @@ -34,11 +34,12 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v3, v2, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, v1], pk_columns: [t._row_id, t._row_id#1, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v3, t.v2, t._row_id, t._row_id] } - ├─StreamExchange { dist: HashShard(t.v1) } - │ └─StreamTableScan { table: t, columns: [t.v1, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(t.v1) } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1, t._row_id, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v3, t.v2, t._row_id, t._row_id] } + ├─StreamExchange { dist: HashShard(t.v1) } + │ └─StreamTableScan { table: t, columns: [t.v1, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1) } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: qualified wildcard sql: | create table t (v1 int, v2 int, v3 int); @@ -52,11 +53,12 @@ └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, v1], pk_columns: [t._row_id, t._row_id#1, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v2, t.v3, t._row_id, t._row_id] } - ├─StreamExchange { dist: HashShard(t.v1) } - │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(t.v1) } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1, t._row_id, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v2, t.v3, t._row_id, t._row_id] } + ├─StreamExchange { dist: HashShard(t.v1) } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1) } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: except with unknown column sql: | create table t (v1 int, v2 int, v3 int); diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index 2815b00784b1d..15404d6d863ab 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -343,24 +343,25 @@ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [a_id, b_id, a_ts, b_ts, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, a_id], pk_columns: [_row_id, _row_id#1, a_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: id = id, output: [id, id, date_time, date_time, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(id) } - │ └─StreamProject { exprs: [id, date_time, _row_id] } - │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [id, date_time, $expr1, _row_id], cleaned_by_watermark: true } - │ ├─StreamProject { exprs: [id, date_time, AtTimeZone(date_time, 'UTC':Varchar) as $expr1, _row_id] } - │ │ └─StreamFilter { predicate: (initial_bid = 1:Int32) } - │ │ └─StreamShare { id: 4 } - │ │ └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } - │ │ └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } - │ │ └─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: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:00:01':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: HashShard(id) } - └─StreamFilter { predicate: (initial_bid = 2:Int32) } - └─StreamShare { id: 4 } - └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } - └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } - └─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, _row_id, _row_id) } + └─StreamHashJoin { type: Inner, predicate: id = id, output: [id, id, date_time, date_time, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamProject { exprs: [id, date_time, _row_id] } + │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [id, date_time, $expr1, _row_id], cleaned_by_watermark: true } + │ ├─StreamProject { exprs: [id, date_time, AtTimeZone(date_time, 'UTC':Varchar) as $expr1, _row_id] } + │ │ └─StreamFilter { predicate: (initial_bid = 1:Int32) } + │ │ └─StreamShare { id: 4 } + │ │ └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } + │ │ └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } + │ │ └─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: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:00:01':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(id) } + └─StreamFilter { predicate: (initial_bid = 2:Int32) } + └─StreamShare { id: 4 } + └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } + └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } + └─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/output/shared_views.yaml b/src/frontend/planner_test/tests/testdata/output/shared_views.yaml index 775812f77b59c..3777705c97ced 100644 --- a/src/frontend/planner_test/tests/testdata/output/shared_views.yaml +++ b/src/frontend/planner_test/tests/testdata/output/shared_views.yaml @@ -23,19 +23,20 @@ └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } stream_plan: |- StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden)], stream_key: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, z], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, z], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x] } - ├─StreamExchange { dist: HashShard($expr1) } - │ └─StreamShare { id: 3 } - │ └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } - │ └─StreamFilter { predicate: (t1.y > 0:Int32) } - │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard($expr2) } - └─StreamProject { exprs: [(t1.x * $expr1) as $expr2, (t1.y * $expr1) as $expr3, t1._row_id, t1._row_id, t1.x] } - └─StreamHashJoin { type: Inner, predicate: t1.x = $expr1, output: [t1.x, t1.y, $expr1, t1._row_id, t1._row_id] } - ├─StreamExchange { dist: HashShard(t1.x) } - │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamShare { id: 3 } - └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } - └─StreamFilter { predicate: (t1.y > 0:Int32) } - └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard($expr1, t1._row_id, t1._row_id, t1._row_id, t1.x) } + └─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x] } + ├─StreamExchange { dist: HashShard($expr1) } + │ └─StreamShare { id: 3 } + │ └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } + │ └─StreamFilter { predicate: (t1.y > 0:Int32) } + │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [(t1.x * $expr1) as $expr2, (t1.y * $expr1) as $expr3, t1._row_id, t1._row_id, t1.x] } + └─StreamHashJoin { type: Inner, predicate: t1.x = $expr1, output: [t1.x, t1.y, $expr1, t1._row_id, t1._row_id] } + ├─StreamExchange { dist: HashShard(t1.x) } + │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamShare { id: 3 } + └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } + └─StreamFilter { predicate: (t1.y > 0:Int32) } + └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 1ad1d9f92c418..e07e84e040929 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -346,21 +346,22 @@ └─BatchScan { table: auction, columns: [auction.date_time], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [date_time, window_start, window_end, auction._row_id(hidden)], stream_key: [auction._row_id, window_start, window_end, date_time], pk_columns: [auction._row_id, window_start, window_end, date_time], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: auction.date_time IS NOT DISTINCT FROM auction.date_time, output: all } - ├─StreamExchange { dist: HashShard(auction.date_time) } - │ └─StreamShare { id: 3 } - │ └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] } - │ └─StreamFilter { predicate: IsNotNull(auction.date_time) } - │ └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], pk: [auction._row_id], dist: UpstreamHashShard(auction._row_id) } - └─StreamProject { exprs: [auction.date_time] } - └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] } - └─StreamProject { exprs: [auction.date_time] } - └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] } - └─StreamExchange { dist: HashShard(auction.date_time) } - └─StreamShare { id: 3 } - └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] } - └─StreamFilter { predicate: IsNotNull(auction.date_time) } - └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], pk: [auction._row_id], dist: UpstreamHashShard(auction._row_id) } + └─StreamExchange { dist: HashShard(auction.date_time, window_start, window_end, auction._row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: auction.date_time IS NOT DISTINCT FROM auction.date_time, output: all } + ├─StreamExchange { dist: HashShard(auction.date_time) } + │ └─StreamShare { id: 3 } + │ └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] } + │ └─StreamFilter { predicate: IsNotNull(auction.date_time) } + │ └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], pk: [auction._row_id], dist: UpstreamHashShard(auction._row_id) } + └─StreamProject { exprs: [auction.date_time] } + └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] } + └─StreamProject { exprs: [auction.date_time] } + └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] } + └─StreamExchange { dist: HashShard(auction.date_time) } + └─StreamShare { id: 3 } + └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] } + └─StreamFilter { predicate: IsNotNull(auction.date_time) } + └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], pk: [auction._row_id], dist: UpstreamHashShard(auction._row_id) } - sql: | CREATE TABLE t (v int); SELECT 1 FROM t AS t_inner WHERE EXISTS ( SELECT 1 HAVING t_inner.v > 1); @@ -535,22 +536,23 @@ └─BatchScan { table: t, columns: [t.x], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, y, k, sum_x, t.x(hidden)], stream_key: [k, x], pk_columns: [k, x], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, t.y, t.k, sum(Unnest($0)), t.x] } - ├─StreamExchange { dist: HashShard(t.x) } - │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } - └─StreamProject { exprs: [t.x, sum(Unnest($0))] } - └─StreamHashAgg { group_key: [t.x], aggs: [sum(Unnest($0)), count] } - └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, Unnest($0), t.x, projected_row_id] } - ├─StreamProject { exprs: [t.x] } - │ └─StreamHashAgg { group_key: [t.x], aggs: [count] } - │ └─StreamExchange { dist: HashShard(t.x) } - │ └─StreamTableScan { table: t, columns: [t.x, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } - └─StreamProject { exprs: [t.x, Unnest($0), projected_row_id] } - └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.x] } - └─StreamHashAgg { group_key: [t.x], aggs: [count] } - └─StreamExchange { dist: HashShard(t.x) } - └─StreamTableScan { table: t, columns: [t.x, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } + └─StreamExchange { dist: HashShard(t.x, t.k) } + └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, t.y, t.k, sum(Unnest($0)), t.x] } + ├─StreamExchange { dist: HashShard(t.x) } + │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } + └─StreamProject { exprs: [t.x, sum(Unnest($0))] } + └─StreamHashAgg { group_key: [t.x], aggs: [sum(Unnest($0)), count] } + └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, Unnest($0), t.x, projected_row_id] } + ├─StreamProject { exprs: [t.x] } + │ └─StreamHashAgg { group_key: [t.x], aggs: [count] } + │ └─StreamExchange { dist: HashShard(t.x) } + │ └─StreamTableScan { table: t, columns: [t.x, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } + └─StreamProject { exprs: [t.x, Unnest($0), projected_row_id] } + └─StreamProjectSet { select_list: [$0, Unnest($0)] } + └─StreamProject { exprs: [t.x] } + └─StreamHashAgg { group_key: [t.x], aggs: [count] } + └─StreamExchange { dist: HashShard(t.x) } + └─StreamTableScan { table: t, columns: [t.x, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } - name: CorrelatedInputRef in ProjectSet and apply on condition is true. sql: | create table t(x int[], y int[], k int primary key); @@ -582,29 +584,29 @@ create table t(x int[], y int[], k int primary key); select *, (select sum(i) from (select unnest(x) i, 1 c) Q where k = c ) as sum_x from t; optimized_logical_plan_for_batch: |- - LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t.x, first_value(t.x order_by(t.x ASC))) AND IsNotDistinctFrom(t.k, t.k), output: [t.x, t.y, t.k, sum(Unnest($0))] } + LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t.x, internal_last_seen_value(t.x)) AND IsNotDistinctFrom(t.k, t.k), output: [t.x, t.y, t.k, sum(Unnest($0))] } ├─LogicalScan { table: t, columns: [t.x, t.y, t.k] } - └─LogicalAgg { group_key: [first_value(t.x order_by(t.x ASC)), t.k], aggs: [sum(Unnest($0))] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(first_value(t.x order_by(t.x ASC)), first_value(t.x order_by(t.x ASC))) AND IsNotDistinctFrom(t.k, t.k), output: [first_value(t.x order_by(t.x ASC)), t.k, Unnest($0)] } - ├─LogicalAgg { group_key: [t.k], aggs: [first_value(t.x order_by(t.x ASC))] } + └─LogicalAgg { group_key: [internal_last_seen_value(t.x), t.k], aggs: [sum(Unnest($0))] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(internal_last_seen_value(t.x), internal_last_seen_value(t.x)) AND IsNotDistinctFrom(t.k, t.k), output: [internal_last_seen_value(t.x), t.k, Unnest($0)] } + ├─LogicalAgg { group_key: [t.k], aggs: [internal_last_seen_value(t.x)] } │ └─LogicalScan { table: t, columns: [t.x, t.k] } - └─LogicalProject { exprs: [first_value(t.x order_by(t.x ASC)), t.k, Unnest($0)] } + └─LogicalProject { exprs: [internal_last_seen_value(t.x), t.k, Unnest($0)] } └─LogicalProjectSet { select_list: [$0, $1, Unnest($0)] } - └─LogicalJoin { type: Inner, on: true, output: [first_value(t.x order_by(t.x ASC)), t.k] } - ├─LogicalAgg { group_key: [t.k], aggs: [first_value(t.x order_by(t.x ASC))] } + └─LogicalJoin { type: Inner, on: true, output: [internal_last_seen_value(t.x), t.k] } + ├─LogicalAgg { group_key: [t.k], aggs: [internal_last_seen_value(t.x)] } │ └─LogicalScan { table: t, columns: [t.x, t.k], predicate: (t.k = 1:Int32) } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } optimized_logical_plan_for_stream: |- - LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t.x, first_value(t.x order_by(t.x ASC))) AND IsNotDistinctFrom(t.k, t.k), output: [t.x, t.y, t.k, sum(Unnest($0))] } + LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t.x, internal_last_seen_value(t.x)) AND IsNotDistinctFrom(t.k, t.k), output: [t.x, t.y, t.k, sum(Unnest($0))] } ├─LogicalScan { table: t, columns: [t.x, t.y, t.k] } - └─LogicalAgg { group_key: [first_value(t.x order_by(t.x ASC)), t.k], aggs: [sum(Unnest($0))] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(first_value(t.x order_by(t.x ASC)), first_value(t.x order_by(t.x ASC))) AND IsNotDistinctFrom(t.k, t.k), output: [first_value(t.x order_by(t.x ASC)), t.k, Unnest($0)] } - ├─LogicalAgg { group_key: [t.k], aggs: [first_value(t.x order_by(t.x ASC))] } + └─LogicalAgg { group_key: [internal_last_seen_value(t.x), t.k], aggs: [sum(Unnest($0))] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(internal_last_seen_value(t.x), internal_last_seen_value(t.x)) AND IsNotDistinctFrom(t.k, t.k), output: [internal_last_seen_value(t.x), t.k, Unnest($0)] } + ├─LogicalAgg { group_key: [t.k], aggs: [internal_last_seen_value(t.x)] } │ └─LogicalScan { table: t, columns: [t.x, t.k] } - └─LogicalProject { exprs: [first_value(t.x order_by(t.x ASC)), t.k, Unnest($0)] } + └─LogicalProject { exprs: [internal_last_seen_value(t.x), t.k, Unnest($0)] } └─LogicalProjectSet { select_list: [$0, $1, Unnest($0)] } - └─LogicalJoin { type: Inner, on: true, output: [first_value(t.x order_by(t.x ASC)), t.k] } - ├─LogicalAgg { group_key: [t.k], aggs: [first_value(t.x order_by(t.x ASC))] } + └─LogicalJoin { type: Inner, on: true, output: [internal_last_seen_value(t.x), t.k] } + ├─LogicalAgg { group_key: [t.k], aggs: [internal_last_seen_value(t.x)] } │ └─LogicalScan { table: t, columns: [t.x, t.k], predicate: (t.k = 1:Int32) } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: CorrelatedInputRef in ProjectSet and apply on condition refers to table function. @@ -632,16 +634,17 @@ └─BatchScan { table: integers, columns: [integers.i], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, col, integers._row_id(hidden), integers.i(hidden)], stream_key: [integers._row_id, i], pk_columns: [i, integers._row_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, row_number, integers._row_id, integers.i] } - ├─StreamExchange { dist: HashShard(integers.i) } - │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } - └─StreamProject { exprs: [integers.i, row_number, integers._row_id] } - └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─StreamExchange { dist: HashShard(integers.i) } - └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } - └─StreamFilter { predicate: IsNotNull(integers.i) } - └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamExchange { dist: HashShard(integers.i, integers._row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, row_number, integers._row_id, integers.i] } + ├─StreamExchange { dist: HashShard(integers.i) } + │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } + └─StreamProject { exprs: [integers.i, row_number, integers._row_id] } + └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(integers.i) } + └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } + └─StreamFilter { predicate: IsNotNull(integers.i) } + └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - name: test over window subquery 2 (with nested loop join so cannot be transformed into a stream plan) sql: | CREATE TABLE integers(i INTEGER); @@ -690,16 +693,17 @@ └─BatchScan { table: integers, columns: [integers.i], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, col, integers._row_id(hidden), integers.i(hidden)], stream_key: [integers._row_id, i], pk_columns: [i, integers._row_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, sum, integers._row_id, integers.i] } - ├─StreamExchange { dist: HashShard(integers.i) } - │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } - └─StreamProject { exprs: [integers.i, sum, integers._row_id] } - └─StreamOverWindow { window_functions: [sum(integers.i) OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─StreamExchange { dist: HashShard(integers.i) } - └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } - └─StreamFilter { predicate: IsNotNull(integers.i) } - └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamExchange { dist: HashShard(integers.i, integers._row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, sum, integers._row_id, integers.i] } + ├─StreamExchange { dist: HashShard(integers.i) } + │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } + └─StreamProject { exprs: [integers.i, sum, integers._row_id] } + └─StreamOverWindow { window_functions: [sum(integers.i) OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(integers.i) } + └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } + └─StreamFilter { predicate: IsNotNull(integers.i) } + └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - name: test over window subquery 4 (with nested loop join so cannot be transformed into a stream plan) sql: | CREATE TABLE integers(i INTEGER); @@ -747,17 +751,18 @@ └─BatchScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, integers._row_id(hidden), $expr1(hidden), integers.correlated_col(hidden)], stream_key: [integers._row_id, $expr1, integers.correlated_col], pk_columns: [integers._row_id, $expr1, integers.correlated_col], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: $expr1 = sum AND integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.i, integers._row_id, $expr1, integers.correlated_col] } - ├─StreamExchange { dist: HashShard(integers.correlated_col, $expr1) } - │ └─StreamProject { exprs: [integers.i, integers.correlated_col, integers.i::Int64 as $expr1, integers._row_id] } - │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamExchange { dist: HashShard(rows.correlated_col, sum) } - └─StreamProject { exprs: [rows.correlated_col, sum, rows._row_id, rows.k] } - └─StreamOverWindow { window_functions: [sum(rows.v) OVER(PARTITION BY rows.correlated_col, rows.k ORDER BY rows.v ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─StreamExchange { dist: HashShard(rows.correlated_col, rows.k) } - └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] } - └─StreamFilter { predicate: IsNotNull(rows.correlated_col) } - └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) } + └─StreamExchange { dist: HashShard(integers._row_id, $expr1, integers.correlated_col) } + └─StreamHashJoin { type: LeftSemi, predicate: $expr1 = sum AND integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.i, integers._row_id, $expr1, integers.correlated_col] } + ├─StreamExchange { dist: HashShard(integers.correlated_col, $expr1) } + │ └─StreamProject { exprs: [integers.i, integers.correlated_col, integers.i::Int64 as $expr1, integers._row_id] } + │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamExchange { dist: HashShard(rows.correlated_col, sum) } + └─StreamProject { exprs: [rows.correlated_col, sum, rows._row_id, rows.k] } + └─StreamOverWindow { window_functions: [sum(rows.v) OVER(PARTITION BY rows.correlated_col, rows.k ORDER BY rows.v ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(rows.correlated_col, rows.k) } + └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] } + └─StreamFilter { predicate: IsNotNull(rows.correlated_col) } + └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) } - name: test cardinality visitor with correlated filter sql: | CREATE TABLE t1(i INT); @@ -818,21 +823,22 @@ └─BatchScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, correlated_col, integers._row_id(hidden), 2:Int64(hidden)], stream_key: [integers._row_id, correlated_col, 2:Int64], pk_columns: [integers._row_id, correlated_col, 2:Int64], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: integers.correlated_col IS NOT DISTINCT FROM integers.correlated_col AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col, integers._row_id, 2:Int64] } - ├─StreamExchange { dist: HashShard(integers.correlated_col) } - │ └─StreamProject { exprs: [integers.i, integers.correlated_col, 2:Int64, integers._row_id] } - │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamProject { exprs: [integers.correlated_col, (count(distinct rows.k) + count(distinct rows.v)) as $expr1] } - └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count(distinct rows.k), count(distinct rows.v), count] } - └─StreamHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, rows._row_id] } - ├─StreamProject { exprs: [integers.correlated_col] } - │ └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count] } - │ └─StreamExchange { dist: HashShard(integers.correlated_col) } - │ └─StreamTableScan { table: integers, columns: [integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamExchange { dist: HashShard(rows.correlated_col) } - └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] } - └─StreamFilter { predicate: IsNotNull(rows.correlated_col) } - └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) } + └─StreamExchange { dist: HashShard(integers.correlated_col, integers._row_id, 2:Int64) } + └─StreamHashJoin { type: LeftSemi, predicate: integers.correlated_col IS NOT DISTINCT FROM integers.correlated_col AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col, integers._row_id, 2:Int64] } + ├─StreamExchange { dist: HashShard(integers.correlated_col) } + │ └─StreamProject { exprs: [integers.i, integers.correlated_col, 2:Int64, integers._row_id] } + │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamProject { exprs: [integers.correlated_col, (count(distinct rows.k) + count(distinct rows.v)) as $expr1] } + └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count(distinct rows.k), count(distinct rows.v), count] } + └─StreamHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, rows._row_id] } + ├─StreamProject { exprs: [integers.correlated_col] } + │ └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count] } + │ └─StreamExchange { dist: HashShard(integers.correlated_col) } + │ └─StreamTableScan { table: integers, columns: [integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamExchange { dist: HashShard(rows.correlated_col) } + └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] } + └─StreamFilter { predicate: IsNotNull(rows.correlated_col) } + └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) } - name: test hop window subquery 1 sql: | create table t1 (k int primary key, ts timestamp); @@ -848,12 +854,13 @@ └─BatchValues { rows: [[1:Int32], [2:Int32]] } stream_plan: |- StreamMaterialize { columns: [col, k, ts, window_start, window_end], stream_key: [col, window_start, window_end], pk_columns: [col, window_start, window_end], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: 1:Int32 = t1.k, output: all } - ├─StreamAppendOnlyDedup { dedup_cols: [1:Int32] } - │ └─StreamExchange { dist: HashShard(1:Int32) } - │ └─StreamProject { exprs: [1:Int32] } - │ └─StreamValues { rows: [[1:Int32, 0:Int64], [2:Int32, 1:Int64]] } - └─StreamExchange { dist: HashShard(t1.k) } - └─StreamHopWindow { time_col: t1.ts, slide: 00:10:00, size: 00:30:00, output: all } - └─StreamFilter { predicate: IsNotNull(t1.ts) } - └─StreamTableScan { table: t1, columns: [t1.k, t1.ts], pk: [t1.k], dist: UpstreamHashShard(t1.k) } + └─StreamExchange { dist: HashShard(1:Int32, window_start, window_end) } + └─StreamHashJoin { type: Inner, predicate: 1:Int32 = t1.k, output: all } + ├─StreamAppendOnlyDedup { dedup_cols: [1:Int32] } + │ └─StreamExchange { dist: HashShard(1:Int32) } + │ └─StreamProject { exprs: [1:Int32] } + │ └─StreamValues { rows: [[1:Int32, 0:Int64], [2:Int32, 1:Int64]] } + └─StreamExchange { dist: HashShard(t1.k) } + └─StreamHopWindow { time_col: t1.ts, slide: 00:10:00, size: 00:30:00, output: all } + └─StreamFilter { predicate: IsNotNull(t1.ts) } + └─StreamTableScan { table: t1, columns: [t1.k, t1.ts], pk: [t1.k], dist: UpstreamHashShard(t1.k) } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml index 6d216ad9c81c4..0d393c378ff85 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml @@ -466,14 +466,14 @@ └─LogicalScan { table: c, columns: [c.c1, c.c2, c.c3, c._row_id] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a3, first_value(a.a3 order_by(a.a3 ASC))) AND IsNotDistinctFrom(b.b2, b.b2), output: [] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a3, internal_last_seen_value(a.a3)) AND IsNotDistinctFrom(b.b2, b.b2), output: [] } ├─LogicalJoin { type: Inner, on: (a.a3 = b.b2), output: all } │ ├─LogicalScan { table: a, columns: [a.a3] } │ └─LogicalScan { table: b, columns: [b.b2] } └─LogicalFilter { predicate: (3:Int32 = count(1:Int32)) } - └─LogicalAgg { group_key: [first_value(a.a3 order_by(a.a3 ASC)), b.b2], aggs: [count(1:Int32)] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(first_value(a.a3 order_by(a.a3 ASC)), c.c3) AND IsNotDistinctFrom(b.b2, c.c2), output: [first_value(a.a3 order_by(a.a3 ASC)), b.b2, 1:Int32] } - ├─LogicalAgg { group_key: [b.b2], aggs: [first_value(a.a3 order_by(a.a3 ASC))] } + └─LogicalAgg { group_key: [internal_last_seen_value(a.a3), b.b2], aggs: [count(1:Int32)] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(internal_last_seen_value(a.a3), c.c3) AND IsNotDistinctFrom(b.b2, c.c2), output: [internal_last_seen_value(a.a3), b.b2, 1:Int32] } + ├─LogicalAgg { group_key: [b.b2], aggs: [internal_last_seen_value(a.a3)] } │ └─LogicalJoin { type: Inner, on: (a.a3 = b.b2), output: all } │ ├─LogicalScan { table: a, columns: [a.a3] } │ └─LogicalScan { table: b, columns: [b.b2] } @@ -717,15 +717,16 @@ └─BatchScan { table: t2, columns: [t2.x], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, y, t1._row_id(hidden)], stream_key: [t1._row_id, x], pk_columns: [t1._row_id, x], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: t1.x IS NOT DISTINCT FROM t2.x, output: all } - ├─StreamExchange { dist: HashShard(t1.x) } - │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamProject { exprs: [t2.x] } - └─StreamGroupTopN { order: [t2.x ASC], limit: 1, offset: 0, group_key: [t2.x] } - └─StreamExchange { dist: HashShard(t2.x) } - └─StreamProject { exprs: [t2.x, t2.x, t2._row_id] } - └─StreamFilter { predicate: IsNotNull(t2.x) } - └─StreamTableScan { table: t2, columns: [t2.x, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.x, t1._row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: t1.x IS NOT DISTINCT FROM t2.x, output: all } + ├─StreamExchange { dist: HashShard(t1.x) } + │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamProject { exprs: [t2.x] } + └─StreamGroupTopN { order: [t2.x ASC], limit: 1, offset: 0, group_key: [t2.x] } + └─StreamExchange { dist: HashShard(t2.x) } + └─StreamProject { exprs: [t2.x, t2.x, t2._row_id] } + └─StreamFilter { predicate: IsNotNull(t2.x) } + └─StreamTableScan { table: t2, columns: [t2.x, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -883,14 +884,15 @@ └─BatchScan { table: t2, columns: [t2.v2, t2.k2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, k1, t1._row_id(hidden)], stream_key: [t1._row_id, v1, k1], pk_columns: [t1._row_id, v1, k1], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2 AND t1.k1 IS NOT DISTINCT FROM t2.k2, output: all } - ├─StreamExchange { dist: HashShard(t1.k1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamGroupTopN { order: [t2.v2 ASC], limit: 1, offset: 0, group_key: [t2.k2] } - └─StreamExchange { dist: HashShard(t2.k2) } - └─StreamProject { exprs: [t2.k2, t2.v2, t2._row_id] } - └─StreamFilter { predicate: IsNotNull(t2.k2) } - └─StreamTableScan { table: t2, columns: [t2.v2, t2.k2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1.k1, t1._row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2 AND t1.k1 IS NOT DISTINCT FROM t2.k2, output: all } + ├─StreamExchange { dist: HashShard(t1.k1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamGroupTopN { order: [t2.v2 ASC], limit: 1, offset: 0, group_key: [t2.k2] } + └─StreamExchange { dist: HashShard(t2.k2) } + └─StreamProject { exprs: [t2.k2, t2.v2, t2._row_id] } + └─StreamFilter { predicate: IsNotNull(t2.k2) } + └─StreamTableScan { table: t2, columns: [t2.v2, t2.k2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: test ApplyTopNTransposeRule case 2 sql: | create table t1 (v1 int, k1 int); @@ -908,16 +910,17 @@ └─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, k1, t1._row_id(hidden)], stream_key: [t1._row_id, v1], pk_columns: [t1._row_id, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2, output: all } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.v2) } - └─StreamProject { exprs: [t2.v2, t2._row_id] } - └─StreamTopN { order: [t2.v2 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [t2.v2 ASC], limit: 1, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } - └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2, output: all } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v2) } + └─StreamProject { exprs: [t2.v2, t2._row_id] } + └─StreamTopN { order: [t2.v2 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [t2.v2 ASC], limit: 1, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } + └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: test ApplyLimitTransposeRule case 1 sql: | create table t1 (v1 int, k1 int); @@ -935,11 +938,12 @@ └─BatchScan { table: t2, columns: [t2.v2, t2.k2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, k1, t1._row_id(hidden)], stream_key: [t1._row_id, v1, k1], pk_columns: [t1._row_id, v1, k1], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2 AND t1.k1 IS NOT DISTINCT FROM t2.k2, output: all } - ├─StreamExchange { dist: HashShard(t1.k1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamGroupTopN { order: [t2.k2 ASC], limit: 1, offset: 0, group_key: [t2.k2] } - └─StreamExchange { dist: HashShard(t2.k2) } - └─StreamProject { exprs: [t2.k2, t2.v2, t2._row_id] } - └─StreamFilter { predicate: IsNotNull(t2.k2) } - └─StreamTableScan { table: t2, columns: [t2.v2, t2.k2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1.k1, t1._row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2 AND t1.k1 IS NOT DISTINCT FROM t2.k2, output: all } + ├─StreamExchange { dist: HashShard(t1.k1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamGroupTopN { order: [t2.k2 ASC], limit: 1, offset: 0, group_key: [t2.k2] } + └─StreamExchange { dist: HashShard(t2.k2) } + └─StreamProject { exprs: [t2.k2, t2.v2, t2._row_id] } + └─StreamFilter { predicate: IsNotNull(t2.k2) } + └─StreamTableScan { table: t2, columns: [t2.v2, t2.k2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml index 6673d86fd9745..29e391853cf8a 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml @@ -122,19 +122,20 @@ select * from t1 join t2 on a = b AND ta < now() - interval '1 hour' and ta >= now() - interval '2 hour'; stream_plan: |- StreamMaterialize { columns: [a, ta, b, tb, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, a], pk_columns: [t1._row_id, t2._row_id, a], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.a) } - │ └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id] } - │ ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true } - │ │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ │ └─StreamExchange { dist: Broadcast } - │ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } - │ │ └─StreamNow { output: [now] } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: HashShard(t2.b) } - └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.a, t1._row_id, t2._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.a) } + │ └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id] } + │ ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true } + │ │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ └─StreamExchange { dist: Broadcast } + │ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ │ └─StreamNow { output: [now] } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(t2.b) } + └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Temporal filter in on clause for left join's left side sql: | create table t1 (a int, ta timestamp with time zone); @@ -150,19 +151,20 @@ select * from t1 right join t2 on a = b AND ta < now() - interval '1 hour' and ta >= now() - interval '2 hour'; stream_plan: |- StreamMaterialize { columns: [a, ta, b, tb, t2._row_id(hidden), t1._row_id(hidden)], stream_key: [t2._row_id, t1._row_id, b], pk_columns: [t2._row_id, t1._row_id, b], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: t2.b = t1.a, output: [t1.a, t1.ta, t2.b, t2.tb, t2._row_id, t1._row_id] } - ├─StreamExchange { dist: HashShard(t2.b) } - │ └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - └─StreamExchange { dist: HashShard(t1.a) } - └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id] } - ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true } - │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } - └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(t2.b, t2._row_id, t1._row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: t2.b = t1.a, output: [t1.a, t1.ta, t2.b, t2.tb, t2._row_id, t1._row_id] } + ├─StreamExchange { dist: HashShard(t2.b) } + │ └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.a) } + └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id] } + ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true } + │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + └─StreamNow { output: [now] } - name: Temporal filter in on clause for full join's left side sql: | create table t1 (a int, ta timestamp with time zone); @@ -178,19 +180,20 @@ select * from t1 left join t2 on a = b AND tb < now() - interval '1 hour' and tb >= now() - interval '2 hour'; stream_plan: |- StreamMaterialize { columns: [a, ta, b, tb, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, a], pk_columns: [t1._row_id, t2._row_id, a], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.a) } - │ └─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.b) } - └─StreamDynamicFilter { predicate: (t2.tb < $expr2), output: [t2.b, t2.tb, t2._row_id] } - ├─StreamDynamicFilter { predicate: (t2.tb >= $expr1), output_watermarks: [t2.tb], output: [t2.b, t2.tb, t2._row_id], cleaned_by_watermark: true } - │ ├─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } - └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(t1.a, t1._row_id, t2._row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.a) } + │ └─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.b) } + └─StreamDynamicFilter { predicate: (t2.tb < $expr2), output: [t2.b, t2.tb, t2._row_id] } + ├─StreamDynamicFilter { predicate: (t2.tb >= $expr1), output_watermarks: [t2.tb], output: [t2.b, t2.tb, t2._row_id], cleaned_by_watermark: true } + │ ├─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + └─StreamNow { output: [now] } - name: Temporal filter in on clause for right join's right side sql: | create table t1 (a int, ta timestamp with time zone); diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml index a1020b8d16ee5..f49a82be2dd78 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml @@ -6,11 +6,12 @@ select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on id1= id2 stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } - ├─StreamExchange { dist: HashShard(stream.id1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } - └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } + └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + ├─StreamExchange { dist: HashShard(stream.id1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } + └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } batch_error: |- Not supported: do not support temporal join for batch queries HINT: please use temporal join in streaming queries @@ -21,11 +22,12 @@ select id1, a1, id2, a2 from stream join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 where a2 < 10; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } - ├─StreamExchange { dist: HashShard(stream.id1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } - └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } + └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) } + └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + ├─StreamExchange { dist: HashShard(stream.id1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } + └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } - name: implicit join with temporal tables sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -33,11 +35,12 @@ select id1, a1, id2, a2 from stream, version FOR SYSTEM_TIME AS OF PROCTIME() where id1 = id2 AND a2 < 10; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } - ├─StreamExchange { dist: HashShard(stream.id1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } - └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } + └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) } + └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + ├─StreamExchange { dist: HashShard(stream.id1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } + └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } - name: Multi join key for temporal join sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -45,11 +48,12 @@ select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and id1 = id2 where b2 != a2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1, a1], pk_columns: [stream._row_id, id1, a1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND stream.a1 = version.a2 AND (version.b2 <> version.a2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } - ├─StreamExchange { dist: HashShard(stream.id1, stream.a1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2, version.a2) } - └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2], pk: [version.id2, version.a2], dist: UpstreamHashShard(version.id2, version.a2) } + └─StreamExchange { dist: HashShard(stream.id1, stream.a1, stream._row_id) } + └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND stream.a1 = version.a2 AND (version.b2 <> version.a2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + ├─StreamExchange { dist: HashShard(stream.id1, stream.a1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2, version.a2) } + └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2], pk: [version.id2, version.a2], dist: UpstreamHashShard(version.id2, version.a2) } - name: Temporal join with Aggregation sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -101,15 +105,16 @@ join version2 FOR SYSTEM_TIME AS OF PROCTIME() on stream.k = version2.k where a1 < 10; stream_plan: |- StreamMaterialize { columns: [k, x1, x2, a1, b1, stream._row_id(hidden), version2.k(hidden)], stream_key: [stream._row_id, k], pk_columns: [stream._row_id, k], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.k = version2.k, output: [stream.k, version1.x1, version2.x2, stream.a1, stream.b1, stream._row_id, version2.k] } - ├─StreamTemporalJoin { type: Inner, predicate: stream.k = version1.k, output: [stream.k, stream.a1, stream.b1, version1.x1, stream._row_id, version1.k] } - │ ├─StreamExchange { dist: HashShard(stream.k) } - │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } - │ │ └─StreamTableScan { table: stream, columns: [stream.k, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.k) } - │ └─StreamTableScan { table: version1, columns: [version1.k, version1.x1], pk: [version1.k], dist: UpstreamHashShard(version1.k) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.k) } - └─StreamTableScan { table: version2, columns: [version2.k, version2.x2], pk: [version2.k], dist: UpstreamHashShard(version2.k) } + └─StreamExchange { dist: HashShard(stream.k, stream._row_id) } + └─StreamTemporalJoin { type: Inner, predicate: stream.k = version2.k, output: [stream.k, version1.x1, version2.x2, stream.a1, stream.b1, stream._row_id, version2.k] } + ├─StreamTemporalJoin { type: Inner, predicate: stream.k = version1.k, output: [stream.k, stream.a1, stream.b1, version1.x1, stream._row_id, version1.k] } + │ ├─StreamExchange { dist: HashShard(stream.k) } + │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } + │ │ └─StreamTableScan { table: stream, columns: [stream.k, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.k) } + │ └─StreamTableScan { table: version1, columns: [version1.k, version1.x1], pk: [version1.k], dist: UpstreamHashShard(version1.k) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.k) } + └─StreamTableScan { table: version2, columns: [version2.k, version2.x2], pk: [version2.k], dist: UpstreamHashShard(version2.k) } - name: multi-way temporal join with different keys sql: | create table stream(id1 int, id2 int, a1 int, b1 int) APPEND ONLY; @@ -121,16 +126,17 @@ join version2 FOR SYSTEM_TIME AS OF PROCTIME() on stream.id2 = version2.id2 where a1 < 10; stream_plan: |- StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version2.id2(hidden)], stream_key: [stream._row_id, id1, id2], pk_columns: [stream._row_id, id1, id2], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } - ├─StreamExchange { dist: HashShard(stream.id2) } - │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } - │ ├─StreamExchange { dist: HashShard(stream.id1) } - │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } - │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.id1) } - │ └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.id2) } - └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) } + └─StreamExchange { dist: HashShard(stream.id1, stream.id2, stream._row_id) } + └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } + ├─StreamExchange { dist: HashShard(stream.id2) } + │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } + │ ├─StreamExchange { dist: HashShard(stream.id1) } + │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } + │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.id1) } + │ └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.id2) } + └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) } - name: multi-way temporal join with different keys sql: | create table stream(id1 int, id2 int, a1 int, b1 int) APPEND ONLY; @@ -142,16 +148,17 @@ join version2 FOR SYSTEM_TIME AS OF PROCTIME() on stream.id2 = version2.id2 where a1 < 10; stream_plan: |- StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version2.id2(hidden)], stream_key: [stream._row_id, id1, id2], pk_columns: [stream._row_id, id1, id2], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } - ├─StreamExchange { dist: HashShard(stream.id2) } - │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } - │ ├─StreamExchange { dist: HashShard(stream.id1) } - │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } - │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.id1) } - │ └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.id2) } - └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) } + └─StreamExchange { dist: HashShard(stream.id1, stream.id2, stream._row_id) } + └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } + ├─StreamExchange { dist: HashShard(stream.id2) } + │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } + │ ├─StreamExchange { dist: HashShard(stream.id1) } + │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } + │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.id1) } + │ └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.id2) } + └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) } - name: temporal join with an index (distribution key size = 1) sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -160,11 +167,12 @@ select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } - ├─StreamExchange { dist: HashShard(stream.a1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } - └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) } + └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } + ├─StreamExchange { dist: HashShard(stream.a1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } + └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) } - name: temporal join with an index (distribution key size = 2) sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -173,11 +181,12 @@ select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } - ├─StreamExchange { dist: HashShard(stream.a1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } - └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) } + └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } + ├─StreamExchange { dist: HashShard(stream.a1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } + └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) } - name: temporal join with an index (index column size = 1) sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -186,11 +195,12 @@ select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, stream.b1, a1], pk_columns: [stream._row_id, id2, stream.b1, a1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.b1 = idx2.b2 AND (stream.a1 = idx2.a2), output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } - ├─StreamExchange { dist: HashShard(stream.b1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.b2) } - └─StreamTableScan { table: idx2, columns: [idx2.b2, idx2.id2, idx2.a2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.b2) } + └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.b1 = idx2.b2 AND (stream.a1 = idx2.a2), output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } + ├─StreamExchange { dist: HashShard(stream.b1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.b2) } + └─StreamTableScan { table: idx2, columns: [idx2.b2, idx2.id2, idx2.a2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.b2) } - name: temporal join with singleton table sql: | create table t (a int) append only; @@ -212,11 +222,12 @@ select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx.a2 AND stream.b1 = idx.b2, output: [stream.id1, stream.a1, idx.id2, idx.a2, stream._row_id, stream.b1] } - ├─StreamExchange { dist: HashShard(stream.a1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx.a2) } - └─StreamTableScan { table: idx, columns: [idx.id2, idx.a2, idx.b2], pk: [idx.id2], dist: UpstreamHashShard(idx.a2) } + └─StreamExchange { dist: HashShard(stream.a1, idx.id2, stream._row_id, stream.b1) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx.a2 AND stream.b1 = idx.b2, output: [stream.id1, stream.a1, idx.id2, idx.a2, stream._row_id, stream.b1] } + ├─StreamExchange { dist: HashShard(stream.a1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx.a2) } + └─StreamTableScan { table: idx, columns: [idx.id2, idx.a2, idx.b2], pk: [idx.id2], dist: UpstreamHashShard(idx.a2) } - name: index selection for temporal join (with two indexes) and should choose the index with a longer prefix.. sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -226,11 +237,12 @@ select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } - ├─StreamExchange { dist: HashShard(stream.a1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } - └─StreamTableScan { table: idx2, columns: [idx2.id2, idx2.a2, idx2.b2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) } + └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } + ├─StreamExchange { dist: HashShard(stream.a1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } + └─StreamTableScan { table: idx2, columns: [idx2.id2, idx2.a2, idx2.b2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) } - name: index selection for temporal join (with three indexes) and should choose primary table. sql: | create table stream(id1 int, a1 int, b1 int, c1 int) APPEND ONLY; @@ -241,11 +253,12 @@ select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2 and c1 = c2 and id1 = id2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden), stream.c1(hidden)], stream_key: [stream._row_id, id1, a1, stream.b1, stream.c1], pk_columns: [stream._row_id, id1, a1, stream.b1, stream.c1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2 AND (stream.a1 = version.a2) AND (stream.b1 = version.b2) AND (stream.c1 = version.c2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id, stream.b1, stream.c1] } - ├─StreamExchange { dist: HashShard(stream.id1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream.c1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } - └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2, version.c2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } + └─StreamExchange { dist: HashShard(stream.id1, stream.a1, stream._row_id, stream.b1, stream.c1) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2 AND (stream.a1 = version.a2) AND (stream.b1 = version.b2) AND (stream.c1 = version.c2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id, stream.b1, stream.c1] } + ├─StreamExchange { dist: HashShard(stream.id1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream.c1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } + └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2, version.c2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } - name: index selection for temporal join (two index) and no one matches. sql: | create table stream(id1 int, a1 int, b1 int, c1 int) APPEND ONLY; diff --git a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml index eaaa1f8e5c8d5..fdf928a0c9c84 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml @@ -243,214 +243,218 @@ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), r_regionkey(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_suppkey(hidden), n_nationkey(hidden), ps_supplycost(hidden), p_partkey#1(hidden)], stream_key: [_row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, p_partkey, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } - ├─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamShare { id: 26 } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } - │ │ │ └─StreamShare { id: 3 } - │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] } - │ │ │ └─StreamRowIdGen { row_id_index: 3 } - │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ │ └─StreamShare { id: 7 } - │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 4 } - │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(p_partkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 9 } - │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamShare { id: 15 } - │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamShare { id: 21 } - │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - └─StreamProject { exprs: [p_partkey, min(ps_supplycost)] } - └─StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] } - └─StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] } - ├─StreamAppendOnlyDedup { dedup_cols: [p_partkey] } - │ └─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamProject { exprs: [p_partkey] } - │ └─StreamShare { id: 26 } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } - │ │ │ └─StreamShare { id: 3 } - │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] } - │ │ │ └─StreamRowIdGen { row_id_index: 3 } - │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ │ └─StreamShare { id: 7 } - │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 4 } - │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(p_partkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 9 } - │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamShare { id: 15 } - │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamShare { id: 21 } - │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - └─StreamExchange { dist: HashShard(ps_partkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } - ├─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ └─StreamFilter { predicate: IsNotNull(ps_partkey) } - │ │ └─StreamShare { id: 15 } - │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamShare { id: 21 } - │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - └─StreamExchange { dist: HashShard(n_nationkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } - ├─StreamExchange { dist: HashShard(r_regionkey) } - │ └─StreamShare { id: 3 } - │ └─StreamProject { exprs: [r_regionkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 3 } - │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } - └─StreamExchange { dist: HashShard(n_regionkey) } - └─StreamShare { id: 7 } - └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - └─StreamRowIdGen { row_id_index: 4 } - └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─StreamExchange { dist: HashShard(p_partkey, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost) } + └─StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } + ├─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamShare { id: 26 } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ │ └─StreamShare { id: 3 } + │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 3 } + │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 7 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 4 } + │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 9 } + │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamShare { id: 15 } + │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamShare { id: 21 } + │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + └─StreamProject { exprs: [p_partkey, min(ps_supplycost)] } + └─StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] } + └─StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] } + ├─StreamAppendOnlyDedup { dedup_cols: [p_partkey] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamProject { exprs: [p_partkey] } + │ └─StreamShare { id: 26 } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ │ └─StreamShare { id: 3 } + │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 3 } + │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 7 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 4 } + │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 9 } + │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamShare { id: 15 } + │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamShare { id: 21 } + │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + └─StreamExchange { dist: HashShard(ps_partkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } + ├─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ └─StreamFilter { predicate: IsNotNull(ps_partkey) } + │ │ └─StreamShare { id: 15 } + │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamShare { id: 21 } + │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + └─StreamExchange { dist: HashShard(n_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } + ├─StreamExchange { dist: HashShard(r_regionkey) } + │ └─StreamShare { id: 3 } + │ └─StreamProject { exprs: [r_regionkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 3 } + │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } + └─StreamExchange { dist: HashShard(n_regionkey) } + └─StreamShare { id: 7 } + └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + └─StreamRowIdGen { row_id_index: 4 } + └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), r_regionkey(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_suppkey(hidden), n_nationkey(hidden), ps_supplycost(hidden), p_partkey#1(hidden)], stream_key: [_row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, p_partkey, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [p_partkey, min(ps_supplycost)] } - └── StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] } { intermediate state table: 26, state tables: [ 25 ], distinct tables: [] } - └── StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] } { left table: 27, right table: 29, left degree table: 28, right degree table: 30 } - ├── StreamAppendOnlyDedup { dedup_cols: [p_partkey] } { state table: 31 } - │ └── StreamExchange Hash([0]) from 15 - └── StreamExchange Hash([0]) from 16 + └── StreamExchange Hash([3, 8, 9, 10, 11, 12, 13, 14, 15, 16]) from 1 Fragment 1 - StreamNoOp - └── StreamExchange NoShuffle from 2 + StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [p_partkey, min(ps_supplycost)] } + └── StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] } { intermediate state table: 26, state tables: [ 25 ], distinct tables: [] } + └── StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] } { left table: 27, right table: 29, left degree table: 28, right degree table: 30 } + ├── StreamAppendOnlyDedup { dedup_cols: [p_partkey] } { state table: 31 } + │ └── StreamExchange Hash([0]) from 16 + └── StreamExchange Hash([0]) from 17 Fragment 2 - StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([5]) from 8 + StreamNoOp + └── StreamExchange NoShuffle from 3 Fragment 3 - StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } { left table: 8, right table: 10, left degree table: 9, right degree table: 11 } + StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([2]) from 6 + └── StreamExchange Hash([5]) from 9 Fragment 4 - StreamNoOp - └── StreamExchange NoShuffle from 5 + StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } { left table: 8, right table: 10, left degree table: 9, right degree table: 11 } + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([2]) from 7 Fragment 5 + StreamNoOp + └── StreamExchange NoShuffle from 6 + + Fragment 6 StreamProject { exprs: [r_regionkey, _row_id] } └── StreamRowIdGen { row_id_index: 3 } └── StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } { source state table: 12 } - Fragment 6 + Fragment 7 StreamNoOp - └── StreamExchange NoShuffle from 7 + └── StreamExchange NoShuffle from 8 - Fragment 7 + Fragment 8 StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } └── StreamRowIdGen { row_id_index: 4 } └── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { source state table: 13 } - Fragment 8 + Fragment 9 StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 } - ├── StreamExchange Hash([2]) from 9 - └── StreamExchange Hash([0]) from 13 + ├── StreamExchange Hash([2]) from 10 + └── StreamExchange Hash([0]) from 14 - Fragment 9 + Fragment 10 StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } { left table: 18, right table: 20, left degree table: 19, right degree table: 21 } - ├── StreamExchange Hash([0]) from 10 - └── StreamExchange Hash([0]) from 11 + ├── StreamExchange Hash([0]) from 11 + └── StreamExchange Hash([0]) from 12 - Fragment 10 + Fragment 11 StreamRowIdGen { row_id_index: 9 } └── StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } { source state table: 22 } - Fragment 11 + Fragment 12 StreamNoOp - └── StreamExchange NoShuffle from 12 + └── StreamExchange NoShuffle from 13 - Fragment 12 + Fragment 13 StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } └── StreamRowIdGen { row_id_index: 5 } └── StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } { source state table: 23 } - Fragment 13 + Fragment 14 StreamNoOp - └── StreamExchange NoShuffle from 14 + └── StreamExchange NoShuffle from 15 - Fragment 14 + Fragment 15 StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } └── StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } { source state table: 24 } - Fragment 15 + Fragment 16 StreamProject { exprs: [p_partkey] } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 16 + Fragment 17 StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } { left table: 32, right table: 34, left degree table: 33, right degree table: 35 } - ├── StreamExchange Hash([2]) from 17 - └── StreamExchange Hash([0]) from 20 + ├── StreamExchange Hash([2]) from 18 + └── StreamExchange Hash([0]) from 21 - Fragment 17 + Fragment 18 StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } { left table: 36, right table: 38, left degree table: 37, right degree table: 39 } - ├── StreamExchange Hash([1]) from 18 - └── StreamExchange Hash([0]) from 19 + ├── StreamExchange Hash([1]) from 19 + └── StreamExchange Hash([0]) from 20 - Fragment 18 + Fragment 19 StreamFilter { predicate: IsNotNull(ps_partkey) } - └── StreamExchange NoShuffle from 12 + └── StreamExchange NoShuffle from 13 - Fragment 19 + Fragment 20 StreamNoOp - └── StreamExchange NoShuffle from 14 + └── StreamExchange NoShuffle from 15 - Fragment 20 + Fragment 21 StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } { left table: 40, right table: 42, left degree table: 41, right degree table: 43 } - ├── StreamExchange Hash([0]) from 21 - └── StreamExchange Hash([2]) from 22 + ├── StreamExchange Hash([0]) from 22 + └── StreamExchange Hash([2]) from 23 - Fragment 21 + Fragment 22 StreamNoOp - └── StreamExchange NoShuffle from 5 + └── StreamExchange NoShuffle from 6 - Fragment 22 + Fragment 23 StreamNoOp - └── StreamExchange NoShuffle from 7 + └── StreamExchange NoShuffle from 8 Table 0 { columns: [ p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id_0, r_regionkey, n_nationkey, _row_id_1, _row_id_2, _row_id_3, ps_suppkey ], primary key: [ $0 ASC, $7 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $12 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16 ], distribution key: [ 0 ], read pk prefix len hint: 2 } @@ -540,7 +544,7 @@ Table 43 { columns: [ n_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost, p_partkey#1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3 ], read pk prefix len hint: 13 } + Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost, p_partkey#1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3, 8, 9, 10, 11, 12, 13, 14, 15, 16 ], read pk prefix len hint: 13 } - id: tpch_q5 before: @@ -1797,76 +1801,80 @@ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [_row_id, _row_id#1, s_nationkey, s_suppkey], pk_columns: [s_name, _row_id, _row_id#1, s_nationkey, s_suppkey], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } - ├─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(s_nationkey) } - │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(n_nationkey) } - │ └─StreamRowIdGen { row_id_index: 4 } - │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } - └─StreamExchange { dist: HashShard(ps_suppkey) } - └─StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, ps_partkey, ps_suppkey] } - └─StreamFilter { predicate: ($expr1 > $expr2) } - └─StreamHashJoin { type: Inner, predicate: ps_partkey IS NOT DISTINCT FROM ps_partkey AND ps_suppkey IS NOT DISTINCT FROM ps_suppkey, output: all } - ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } - │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] } - │ └─StreamShare { id: 13 } - │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamProject { exprs: [p_partkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 9 } - │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } - └─StreamProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } - └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity), count] } - └─StreamHashJoin { type: LeftOuter, predicate: ps_partkey IS NOT DISTINCT FROM l_partkey AND ps_suppkey IS NOT DISTINCT FROM l_suppkey, output: [ps_partkey, ps_suppkey, l_quantity, _row_id] } - ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } - │ └─StreamProject { exprs: [ps_partkey, ps_suppkey] } - │ └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [count] } - │ └─StreamShare { id: 13 } - │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamProject { exprs: [p_partkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 9 } - │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } - └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) } - └─StreamProject { exprs: [l_partkey, l_suppkey, l_quantity, _row_id] } - └─StreamFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) } - └─StreamRowIdGen { row_id_index: 16 } - └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + └─StreamExchange { dist: HashShard(_row_id, _row_id, s_nationkey, s_suppkey) } + └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamRowIdGen { row_id_index: 4 } + │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─StreamExchange { dist: HashShard(ps_suppkey) } + └─StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, ps_partkey, ps_suppkey] } + └─StreamFilter { predicate: ($expr1 > $expr2) } + └─StreamHashJoin { type: Inner, predicate: ps_partkey IS NOT DISTINCT FROM ps_partkey AND ps_suppkey IS NOT DISTINCT FROM ps_suppkey, output: all } + ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } + │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] } + │ └─StreamShare { id: 13 } + │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamProject { exprs: [p_partkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 9 } + │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + └─StreamProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } + └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity), count] } + └─StreamHashJoin { type: LeftOuter, predicate: ps_partkey IS NOT DISTINCT FROM l_partkey AND ps_suppkey IS NOT DISTINCT FROM l_suppkey, output: [ps_partkey, ps_suppkey, l_quantity, _row_id] } + ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } + │ └─StreamProject { exprs: [ps_partkey, ps_suppkey] } + │ └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [count] } + │ └─StreamShare { id: 13 } + │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamProject { exprs: [p_partkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 9 } + │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) } + └─StreamProject { exprs: [l_partkey, l_suppkey, l_quantity, _row_id] } + └─StreamFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) } + └─StreamRowIdGen { row_id_index: 16 } + └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [_row_id, _row_id#1, s_nationkey, s_suppkey], pk_columns: [s_name, _row_id, _row_id#1, s_nationkey, s_suppkey], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([2, 3, 4, 5]) from 1 Fragment 1 - StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } - ├── StreamExchange Hash([3]) from 2 - └── StreamExchange Hash([0]) from 3 + StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 5 Fragment 2 + StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } + ├── StreamExchange Hash([3]) from 3 + └── StreamExchange Hash([0]) from 4 + + Fragment 3 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } { source state table: 8 } - Fragment 3 + Fragment 4 StreamRowIdGen { row_id_index: 4 } └── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { source state table: 9 } - Fragment 4 + Fragment 5 StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, ps_partkey, ps_suppkey] } └── StreamFilter { predicate: ($expr1 > $expr2) } └── StreamHashJoin { type: Inner, predicate: ps_partkey IS NOT DISTINCT FROM ps_partkey AND ps_suppkey IS NOT DISTINCT FROM ps_suppkey, output: all } { left table: 10, right table: 12, left degree table: 11, right degree table: 13 } - ├── StreamExchange Hash([0, 1]) from 5 + ├── StreamExchange Hash([0, 1]) from 6 └── StreamProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } └── StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity), count] } { intermediate state table: 20, state tables: [], distinct tables: [] } └── StreamHashJoin { type: LeftOuter, predicate: ps_partkey IS NOT DISTINCT FROM l_partkey AND ps_suppkey IS NOT DISTINCT FROM l_suppkey, output: [ps_partkey, ps_suppkey, l_quantity, _row_id] } @@ -1874,33 +1882,33 @@ ├── right table: 23 ├── left degree table: 22 ├── right degree table: 24 - ├── StreamExchange Hash([0, 1]) from 9 - └── StreamExchange Hash([0, 1]) from 10 + ├── StreamExchange Hash([0, 1]) from 10 + └── StreamExchange Hash([0, 1]) from 11 - Fragment 5 + Fragment 6 StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] } - └── StreamExchange NoShuffle from 6 + └── StreamExchange NoShuffle from 7 - Fragment 6 + Fragment 7 StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 } - ├── StreamExchange Hash([0]) from 7 - └── StreamExchange Hash([0]) from 8 + ├── StreamExchange Hash([0]) from 8 + └── StreamExchange Hash([0]) from 9 - Fragment 7 + Fragment 8 StreamRowIdGen { row_id_index: 5 } └── StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } { source state table: 18 } - Fragment 8 + Fragment 9 StreamProject { exprs: [p_partkey, _row_id] } └── StreamRowIdGen { row_id_index: 9 } └── StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } { source state table: 19 } - Fragment 9 + Fragment 10 StreamProject { exprs: [ps_partkey, ps_suppkey] } └── StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [count] } { intermediate state table: 25, state tables: [], distinct tables: [] } - └── StreamExchange NoShuffle from 6 + └── StreamExchange NoShuffle from 7 - Fragment 10 + Fragment 11 StreamProject { exprs: [l_partkey, l_suppkey, l_quantity, _row_id] } └── StreamFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) } └── StreamRowIdGen { row_id_index: 16 } @@ -1961,7 +1969,7 @@ Table 26 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ s_name, s_address, _row_id, _row_id#1, s_nationkey, s_suppkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 5 } + Table 4294967294 { columns: [ s_name, s_address, _row_id, _row_id#1, s_nationkey, s_suppkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2, 3, 4, 5 ], read pk prefix len hint: 5 } - id: tpch_q21 before: diff --git a/src/frontend/planner_test/tests/testdata/output/watermark.yaml b/src/frontend/planner_test/tests/testdata/output/watermark.yaml index e4ef42b121528..d57d41fa76bc3 100644 --- a/src/frontend/planner_test/tests/testdata/output/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/watermark.yaml @@ -79,11 +79,12 @@ 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)], stream_key: [t1._row_id, t2._row_id, t1_ts], pk_columns: [t1._row_id, t2._row_id, t1_ts], pk_conflict: NoCheck, watermark_columns: [t1_ts, ts2] } - └─StreamHashJoin [window, append_only] { 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) } - └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.ts, t1._row_id, t2._row_id) } + └─StreamHashJoin [window, append_only] { 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) } + └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: left semi window join sql: | create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; @@ -91,11 +92,12 @@ select t1.ts as t1_ts, t1.v1 as t1_v1, t1.v2 as t1_v2 from t1 where exists (select * from t2 where t1.ts = t2.ts); stream_plan: |- StreamMaterialize { columns: [t1_ts, t1_v1, t1_v2, t1._row_id(hidden)], stream_key: [t1._row_id, t1_ts], pk_columns: [t1._row_id, t1_ts], pk_conflict: NoCheck, watermark_columns: [t1_ts] } - └─StreamHashJoin [window] { type: LeftSemi, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts], output: all } - ├─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) } - └─StreamTableScan { table: t2, columns: [t2.ts, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.ts, t1._row_id) } + └─StreamHashJoin [window] { type: LeftSemi, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts], output: all } + ├─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) } + └─StreamTableScan { table: t2, columns: [t2.ts, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: interval join(left outer join) sql: | create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; @@ -108,13 +110,14 @@ └─LogicalScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id] } stream_plan: |- StreamMaterialize { columns: [t1_ts, t1_v1, t1_v2, t2_ts, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1], pk_columns: [t1._row_id, t2._row_id, t1_v1], pk_conflict: NoCheck, watermark_columns: [t1_ts, t2_ts] } - └─StreamHashJoin [interval] { type: LeftOuter, predicate: t1.v1 = t2.v1 AND (t1.ts >= $expr2) AND ($expr1 <= t2.ts), conditions_to_clean_left_state_table: (t1.ts >= $expr2), conditions_to_clean_right_state_table: ($expr1 <= t2.ts), output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, AddWithTimeZone(t1.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr1, t1._row_id], output_watermarks: [t1.ts, $expr1] } - │ └─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.v1) } - └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, AddWithTimeZone(t2.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr2, t2._row_id], output_watermarks: [t2.ts, $expr2] } - └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) } + └─StreamHashJoin [interval] { type: LeftOuter, predicate: t1.v1 = t2.v1 AND (t1.ts >= $expr2) AND ($expr1 <= t2.ts), conditions_to_clean_left_state_table: (t1.ts >= $expr2), conditions_to_clean_right_state_table: ($expr1 <= t2.ts), output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, AddWithTimeZone(t1.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr1, t1._row_id], output_watermarks: [t1.ts, $expr1] } + │ └─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.v1) } + └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, AddWithTimeZone(t2.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr2, t2._row_id], output_watermarks: [t2.ts, $expr2] } + └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: interval join (inner join) sql: | create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; @@ -127,13 +130,14 @@ └─LogicalScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id] } stream_plan: |- StreamMaterialize { columns: [t1_ts, t1_v1, t1_v2, t2_ts, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1], pk_columns: [t1._row_id, t2._row_id, t1_v1], pk_conflict: NoCheck, watermark_columns: [t1_ts, t2_ts] } - └─StreamHashJoin [interval, append_only] { type: Inner, predicate: t1.v1 = t2.v1 AND (t1.ts >= $expr2) AND ($expr1 <= t2.ts), conditions_to_clean_left_state_table: (t1.ts >= $expr2), conditions_to_clean_right_state_table: ($expr1 <= t2.ts), output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, AddWithTimeZone(t1.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr1, t1._row_id], output_watermarks: [t1.ts, $expr1] } - │ └─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.v1) } - └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, AddWithTimeZone(t2.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr2, t2._row_id], output_watermarks: [t2.ts, $expr2] } - └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) } + └─StreamHashJoin [interval, append_only] { type: Inner, predicate: t1.v1 = t2.v1 AND (t1.ts >= $expr2) AND ($expr1 <= t2.ts), conditions_to_clean_left_state_table: (t1.ts >= $expr2), conditions_to_clean_right_state_table: ($expr1 <= t2.ts), output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, AddWithTimeZone(t1.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr1, t1._row_id], output_watermarks: [t1.ts, $expr1] } + │ └─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.v1) } + └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, AddWithTimeZone(t2.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr2, t2._row_id], output_watermarks: [t2.ts, $expr2] } + └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: union all sql: | create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; diff --git a/src/frontend/planner_test/tests/testdata/output/window_join.yaml b/src/frontend/planner_test/tests/testdata/output/window_join.yaml index 4113a6021e866..17c5e76f6e806 100644 --- a/src/frontend/planner_test/tests/testdata/output/window_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/window_join.yaml @@ -12,15 +12,16 @@ select * from t1, t2 where ts1 = ts2 and a1 = a2; stream_plan: |- StreamMaterialize { columns: [ts1, a1, b1, ts2, a2, b2, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, ts1, a1], pk_columns: [_row_id, _row_id#1, ts1, a1], pk_conflict: NoCheck, watermark_columns: [ts1, ts2] } - └─StreamHashJoin [window, append_only] { type: Inner, predicate: ts1 = ts2 AND a1 = a2, output_watermarks: [ts1, ts2], output: [ts1, a1, b1, ts2, a2, b2, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(ts1, a1) } - │ └─StreamRowIdGen { row_id_index: 3 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts1, expr: (ts1 - '00:00:01':Interval) }], output_watermarks: [ts1] } - │ └─StreamSource { source: t1, columns: [ts1, a1, b1, _row_id] } - └─StreamExchange { dist: HashShard(ts2, a2) } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts2, expr: (ts2 - '00:00:01':Interval) }], output_watermarks: [ts2] } - └─StreamSource { source: t2, columns: [ts2, a2, b2, _row_id] } + └─StreamExchange { dist: HashShard(ts1, a1, _row_id, _row_id) } + └─StreamHashJoin [window, append_only] { type: Inner, predicate: ts1 = ts2 AND a1 = a2, output_watermarks: [ts1, ts2], output: [ts1, a1, b1, ts2, a2, b2, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(ts1, a1) } + │ └─StreamRowIdGen { row_id_index: 3 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts1, expr: (ts1 - '00:00:01':Interval) }], output_watermarks: [ts1] } + │ └─StreamSource { source: t1, columns: [ts1, a1, b1, _row_id] } + └─StreamExchange { dist: HashShard(ts2, a2) } + └─StreamRowIdGen { row_id_index: 3 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts2, expr: (ts2 - '00:00:01':Interval) }], output_watermarks: [ts2] } + └─StreamSource { source: t2, columns: [ts2, a2, b2, _row_id] } - name: Window join expression reorder sql: | create source t1 (ts1 timestamp with time zone, a1 int, b1 int, watermark for ts1 as ts1 - INTERVAL '1' SECOND) with ( @@ -34,12 +35,13 @@ select * from t1, t2 where a1 = a2 and ts1 = ts2; stream_plan: |- StreamMaterialize { columns: [ts1, a1, b1, ts2, a2, b2, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, a1, ts1], pk_columns: [_row_id, _row_id#1, a1, ts1], pk_conflict: NoCheck, watermark_columns: [ts1, ts2] } - └─StreamHashJoin [window, append_only] { type: Inner, predicate: ts1 = ts2 AND a1 = a2, output_watermarks: [ts1, ts2], output: [ts1, a1, b1, ts2, a2, b2, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(ts1, a1) } - │ └─StreamRowIdGen { row_id_index: 3 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts1, expr: (ts1 - '00:00:01':Interval) }], output_watermarks: [ts1] } - │ └─StreamSource { source: t1, columns: [ts1, a1, b1, _row_id] } - └─StreamExchange { dist: HashShard(ts2, a2) } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts2, expr: (ts2 - '00:00:01':Interval) }], output_watermarks: [ts2] } - └─StreamSource { source: t2, columns: [ts2, a2, b2, _row_id] } + └─StreamExchange { dist: HashShard(ts1, a1, _row_id, _row_id) } + └─StreamHashJoin [window, append_only] { type: Inner, predicate: ts1 = ts2 AND a1 = a2, output_watermarks: [ts1, ts2], output: [ts1, a1, b1, ts2, a2, b2, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(ts1, a1) } + │ └─StreamRowIdGen { row_id_index: 3 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts1, expr: (ts1 - '00:00:01':Interval) }], output_watermarks: [ts1] } + │ └─StreamSource { source: t1, columns: [ts1, a1, b1, _row_id] } + └─StreamExchange { dist: HashShard(ts2, a2) } + └─StreamRowIdGen { row_id_index: 3 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts2, expr: (ts2 - '00:00:01':Interval) }], output_watermarks: [ts2] } + └─StreamSource { source: t2, columns: [ts2, a2, b2, _row_id] } diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 32279dd4e70eb..ddb1d697b856d 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -244,6 +244,7 @@ fn bind_sink_format_desc(value: SinkSchema) -> Result { E::Json => SinkEncode::Json, E::Protobuf => SinkEncode::Protobuf, E::Avro => SinkEncode::Avro, + E::Template => SinkEncode::Template, e @ (E::Native | E::Csv | E::Bytes) => { return Err(ErrorCode::BindError(format!("sink encode unsupported: {e}")).into()) } @@ -262,6 +263,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock vec![Encode::Json], Format::Debezium => vec![Encode::Json], ), + RedisSink::SINK_NAME => hashmap!( + Format::Plain => vec![Encode::Json,Encode::Template], + Format::Upsert => vec![Encode::Json,Encode::Template], + ), )) }); pub fn validate_compatibility(connector: &str, format_desc: &SinkSchema) -> Result<()> { diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 2fb251ca89aa6..e0c7e339ee6a6 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -516,11 +516,8 @@ impl Agg { // we use materialized input state for non-retractable aggregate function. // for backward compatibility, the state type is same as the return type. // its values in the intermediate state table are always null. - } else { - field.data_type = sig - .state_type - .clone() - .unwrap_or(sig.ret_type.as_exact().clone()); + } else if let Some(state_type) = &sig.state_type { + field.data_type = state_type.clone(); } } let in_dist_key = self.input.distribution().dist_column_indices().to_vec(); diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index d8972436d5c78..9c87f1a34abbd 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -151,7 +151,22 @@ impl StreamMaterialize { TableType::MaterializedView => { assert_matches!(user_distributed_by, RequiredDist::Any); // ensure the same pk will not shuffle to different node - RequiredDist::shard_by_key(input.schema().len(), input.expect_stream_key()) + let required_dist = + RequiredDist::shard_by_key(input.schema().len(), input.expect_stream_key()); + + // If the input is a stream join, enforce the stream key as the materialized + // view distribution key to avoid slow backfilling caused by + // data skew of the dimension table join key. + // See for more information. + let is_stream_join = matches!(input.as_stream_hash_join(), Some(_join)) + || matches!(input.as_stream_temporal_join(), Some(_join)) + || matches!(input.as_stream_delta_join(), Some(_join)); + + if is_stream_join { + return Ok(required_dist.enforce(input, &Order::any())); + } + + required_dist } TableType::Index => { assert_matches!( diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs index b6e7715dd155f..2df1d7ae00bc3 100644 --- a/src/frontend/src/optimizer/property/distribution.rs +++ b/src/frontend/src/optimizer/property/distribution.rs @@ -331,7 +331,7 @@ impl RequiredDist { } } - fn enforce(&self, plan: PlanRef, required_order: &Order) -> PlanRef { + pub fn enforce(&self, plan: PlanRef, required_order: &Order) -> PlanRef { let dist = self.to_dist(); match plan.convention() { Convention::Batch => BatchExchange::new(plan, required_order.clone(), dist).into(), diff --git a/src/frontend/src/optimizer/rule/agg_group_by_simplify_rule.rs b/src/frontend/src/optimizer/rule/agg_group_by_simplify_rule.rs index 34025eca43032..3e22348e27b49 100644 --- a/src/frontend/src/optimizer/rule/agg_group_by_simplify_rule.rs +++ b/src/frontend/src/optimizer/rule/agg_group_by_simplify_rule.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::aggregate::AggKind; use super::super::plan_node::*; @@ -48,11 +47,11 @@ impl Rule for AggGroupBySimplifyRule { if !new_group_key.contains(i) { let data_type = agg_input.schema().fields[i].data_type(); new_agg_calls.push(PlanAggCall { - agg_kind: AggKind::FirstValue, + agg_kind: AggKind::InternalLastSeenValue, return_type: data_type.clone(), inputs: vec![InputRef::new(i, data_type)], distinct: false, - order_by: vec![ColumnOrder::new(i, OrderType::ascending())], + order_by: vec![], filter: Condition::true_cond(), direct_args: vec![], }); diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index ed6ad289a5a68..d39dde51399d8 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -626,7 +626,7 @@ impl GlobalBarrierManager { let paused = self.take_pause_on_bootstrap().await.unwrap_or(false); let paused_reason = paused.then_some(PausedReason::Manual); - self.recovery(prev_epoch, paused_reason, true) + self.recovery(prev_epoch, paused_reason) .instrument(span) .await }; @@ -981,10 +981,7 @@ impl GlobalBarrierManager { // No need to clean dirty tables for barrier recovery, // The foreground stream job should cleanup their own tables. - *state = self - .recovery(prev_epoch, None, false) - .instrument(span) - .await; + *state = self.recovery(prev_epoch, None).instrument(span).await; self.set_status(BarrierManagerStatus::Running).await; } else { panic!("failed to execute barrier: {:?}", err); diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index 21197a8df98d4..3e319f0e69a52 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -219,7 +219,6 @@ impl GlobalBarrierManager { &self, prev_epoch: TracedEpoch, paused_reason: Option, - bootstrap_recovery: bool, ) -> BarrierManagerState { // Mark blocked and abort buffered schedules, they might be dirty already. self.scheduled_barriers @@ -227,11 +226,9 @@ impl GlobalBarrierManager { .await; tracing::info!("recovery start!"); - if bootstrap_recovery { - self.clean_dirty_tables() - .await - .expect("clean dirty tables should not fail"); - } + self.clean_dirty_tables() + .await + .expect("clean dirty tables should not fail"); self.clean_dirty_fragments() .await .expect("clean dirty fragments"); diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index bcac32922d180..f988646428aac 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -853,14 +853,18 @@ impl CatalogManager { database_core.clear_creating_stream_jobs(); let user_core = &mut core.user; for table in &tables_to_clean { - // Recovered when init database manager. - for relation_id in &table.dependent_relations { - database_core.decrease_ref_count(*relation_id); + // If table type is internal, no need to update the ref count OR + // user ref count. + if table.table_type != TableType::Internal as i32 { + // Recovered when init database manager. + for relation_id in &table.dependent_relations { + database_core.decrease_ref_count(*relation_id); + } + // Recovered when init user manager. + tracing::debug!("decrease ref for {}", table.id); + user_core.decrease_ref(table.owner); } - // Recovered when init user manager. - user_core.decrease_ref(table.owner); } - Ok(()) } @@ -919,10 +923,11 @@ impl CatalogManager { let database_core = &mut core.database; let tables = &mut database_core.tables; let Some(table) = tables.get(&table_id).cloned() else { - bail!( - "table_id {} missing when attempting to cancel job", + tracing::warn!( + "table_id {} missing when attempting to cancel job, could be cleaned on recovery", table_id - ) + ); + return Ok(()); }; table }; @@ -938,7 +943,8 @@ impl CatalogManager { let tables = &mut database_core.tables; let mut tables = BTreeMapTransaction::new(tables); for table_id in table_ids { - tables.remove(table_id); + let res = tables.remove(table_id); + assert!(res.is_some()); } commit_meta!(self, tables)?; } @@ -2032,8 +2038,7 @@ impl CatalogManager { let user_core = &mut core.user; let key = (index.database_id, index.schema_id, index.name.clone()); assert!( - !database_core.indexes.contains_key(&index.id) - && database_core.has_in_progress_creation(&key), + !database_core.indexes.contains_key(&index.id), "index must be in creating procedure" ); @@ -2188,8 +2193,7 @@ impl CatalogManager { let user_core = &mut core.user; let key = (sink.database_id, sink.schema_id, sink.name.clone()); assert!( - !database_core.sinks.contains_key(&sink.id) - && database_core.has_in_progress_creation(&key), + !database_core.sinks.contains_key(&sink.id), "sink must be in creating procedure" ); diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index 76de970a919a9..58fb2d50c6287 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -294,6 +294,7 @@ pub enum Encode { Json, // Keyword::JSON Bytes, // Keyword::BYTES Native, + Template, } // TODO: unify with `from_keyword` @@ -309,6 +310,7 @@ impl fmt::Display for Encode { Encode::Json => "JSON", Encode::Bytes => "BYTES", Encode::Native => "NATIVE", + Encode::Template => "TEMPLATE", } ) } @@ -322,13 +324,12 @@ impl Encode { "CSV" => Encode::Csv, "PROTOBUF" => Encode::Protobuf, "JSON" => Encode::Json, + "TEMPLATE" => Encode::Template, "NATIVE" => Encode::Native, // used internally for schema change - _ => { - return Err(ParserError::ParserError( - "expected AVRO | BYTES | CSV | PROTOBUF | JSON | NATIVE after Encode" - .to_string(), - )) - } + _ => return Err(ParserError::ParserError( + "expected AVRO | BYTES | CSV | PROTOBUF | JSON | NATIVE | TEMPLATE after Encode" + .to_string(), + )), }) } } diff --git a/src/storage/src/row_serde/value_serde.rs b/src/storage/src/row_serde/value_serde.rs index 5d56cdba2d96d..9048b90c23a53 100644 --- a/src/storage/src/row_serde/value_serde.rs +++ b/src/storage/src/row_serde/value_serde.rs @@ -114,9 +114,10 @@ impl ValueRowSerdeNew for ColumnAwareSerde { // It's okay since we previously banned impure expressions in default columns. build_from_prost(&expr.expect("expr should not be none")) .expect("build_from_prost error") - .eval_row_infallible(&OwnedRow::empty()) + .eval_row(&OwnedRow::empty()) .now_or_never() .expect("constant expression should not be async") + .expect("eval_row failed") }; Some((i, value)) } else { diff --git a/src/stream/clippy.toml b/src/stream/clippy.toml index a6969d5bd607b..b7257c4acb98c 100644 --- a/src/stream/clippy.toml +++ b/src/stream/clippy.toml @@ -3,8 +3,8 @@ disallowed-methods = [ { path = "risingwave_expr::expr::build_from_prost", reason = "Expressions in streaming must be in non-strict mode. Please use `build_non_strict_from_prost` instead." }, { path = "risingwave_expr::expr::build_func", reason = "Expressions in streaming must be in non-strict mode. Please use `build_func_non_strict` instead." }, - { path = "risingwave_expr::expr::Expression::eval", reason = "Please use `Expression::eval_infallible` instead." }, - { path = "risingwave_expr::expr::Expression::eval_row", reason = "Please use `Expression::eval_row_infallible` instead." }, + { path = "risingwave_expr::expr::Expression::eval", reason = "Please use `NonStrictExpression::eval_infallible` instead." }, + { path = "risingwave_expr::expr::Expression::eval_row", reason = "Please use `NonStrictExpression::eval_row_infallible` instead." }, { path = "risingwave_common::error::internal_err", reason = "Please use per-crate error type instead." }, { path = "risingwave_common::error::internal_error", reason = "Please use per-crate error type instead." }, diff --git a/src/stream/src/executor/aggregation/mod.rs b/src/stream/src/executor/aggregation/mod.rs index dd0ce9d01c544..9bb1113152962 100644 --- a/src/stream/src/executor/aggregation/mod.rs +++ b/src/stream/src/executor/aggregation/mod.rs @@ -21,6 +21,7 @@ use risingwave_common::bail; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{Field, Schema}; use risingwave_expr::aggregate::{AggCall, AggKind}; +use risingwave_expr::expr::{LogReport, NonStrictExpression}; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; @@ -74,7 +75,12 @@ pub async fn agg_call_filter_res( } if let Some(ref filter) = agg_call.filter { - if let Bool(filter_res) = filter.eval_infallible(chunk).await.as_ref() { + // TODO: should we build `filter` in non-strict mode? + if let Bool(filter_res) = NonStrictExpression::new_topmost(&**filter, LogReport) + .eval_infallible(chunk) + .await + .as_ref() + { vis &= filter_res.to_bitmap(); } else { bail!("Filter can only receive bool array"); diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs index e8eb4da545f2e..ccb55b75c24fc 100644 --- a/src/stream/src/executor/dynamic_filter.rs +++ b/src/stream/src/executor/dynamic_filter.rs @@ -26,7 +26,7 @@ use risingwave_common::row::{self, once, OwnedRow, OwnedRow as RowData, Row}; use risingwave_common::types::{DataType, Datum, DefaultOrd, ScalarImpl, ToDatumRef, ToOwnedDatum}; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_expr::expr::{ - build_func_non_strict, BoxedExpression, InputRefExpression, LiteralExpression, + build_func_non_strict, InputRefExpression, LiteralExpression, NonStrictExpression, }; use risingwave_pb::expr::expr_node::Type as ExprNodeType; use risingwave_pb::expr::expr_node::Type::{ @@ -97,7 +97,7 @@ impl DynamicFilterExecutor, + condition: Option, ) -> Result<(Vec, Bitmap), StreamExecutorError> { let mut new_ops = Vec::with_capacity(chunk.capacity()); let mut new_visibility = BitmapBuilder::with_capacity(chunk.capacity()); @@ -265,7 +265,7 @@ impl DynamicFilterExecutor, - expr: BoxedExpression, + expr: NonStrictExpression, executor_id: u64, ) -> Self { let input_info = input.info(); @@ -190,8 +190,8 @@ mod tests { use risingwave_common::array::StreamChunk; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::DataType; - use risingwave_expr::expr::build_from_pretty; + use super::super::test_utils::expr::build_from_pretty; use super::super::test_utils::MockSource; use super::super::*; use super::*; diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index f15ccfb69dc09..75414fe24a379 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -28,7 +28,7 @@ use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, DefaultOrd, ToOwnedDatum}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_expr::expr::BoxedExpression; +use risingwave_expr::expr::NonStrictExpression; use risingwave_expr::ExprError; use risingwave_storage::StateStore; use tokio::time::Instant; @@ -242,9 +242,9 @@ pub struct HashJoinExecutor, /// Optional non-equi join conditions - cond: Option, + cond: Option, /// Column indices of watermark output and offset expression of each inequality, respectively. - inequality_pairs: Vec<(Vec, Option)>, + inequality_pairs: Vec<(Vec, Option)>, /// The output watermark of each inequality condition and its value is the minimum of the /// calculation result of both side. It will be used to generate watermark into downstream /// and do state cleaning if `clean_state` field of that inequality is `true`. @@ -313,7 +313,7 @@ struct EqJoinArgs<'a, K: HashKey, S: StateStore> { side_l: &'a mut JoinSide, side_r: &'a mut JoinSide, actual_output_data_types: &'a [DataType], - cond: &'a mut Option, + cond: &'a mut Option, inequality_watermarks: &'a [Option], chunk: StreamChunk, append_only_optimize: bool, @@ -448,8 +448,8 @@ impl HashJoinExecutor, executor_id: u64, - cond: Option, - inequality_pairs: Vec<(usize, usize, bool, Option)>, + cond: Option, + inequality_pairs: Vec<(usize, usize, bool, Option)>, op_info: String, state_table_l: StateTable, degree_state_table_l: StateTable, @@ -912,7 +912,7 @@ impl HashJoinExecutor input_watermark.val = value.unwrap(), @@ -1275,11 +1275,11 @@ mod tests { use risingwave_common::hash::{Key128, Key64}; use risingwave_common::types::ScalarImpl; use risingwave_common::util::sort_util::OrderType; - use risingwave_expr::expr::build_from_pretty; use risingwave_storage::memory::MemoryStateStore; use super::*; use crate::common::table::state_table::StateTable; + use crate::executor::test_utils::expr::build_from_pretty; use crate::executor::test_utils::{MessageSender, MockSource, StreamExecutorTestExt}; use crate::executor::{ActorContext, Barrier, EpochPair}; @@ -1327,7 +1327,7 @@ mod tests { (state_table, degree_state_table) } - fn create_cond(condition_text: Option) -> BoxedExpression { + fn create_cond(condition_text: Option) -> NonStrictExpression { build_from_pretty( condition_text .as_deref() @@ -1339,7 +1339,7 @@ mod tests { with_condition: bool, null_safe: bool, condition_text: Option, - inequality_pairs: Vec<(usize, usize, bool, Option)>, + inequality_pairs: Vec<(usize, usize, bool, Option)>, ) -> (MessageSender, MessageSender, BoxedMessageStream) { let schema = Schema { fields: vec![ diff --git a/src/stream/src/executor/hop_window.rs b/src/stream/src/executor/hop_window.rs index c6fffcd94896d..42d13d790da88 100644 --- a/src/stream/src/executor/hop_window.rs +++ b/src/stream/src/executor/hop_window.rs @@ -19,7 +19,7 @@ use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::{DataChunk, Op}; use risingwave_common::types::Interval; -use risingwave_expr::expr::BoxedExpression; +use risingwave_expr::expr::NonStrictExpression; use risingwave_expr::ExprError; use super::error::StreamExecutorError; @@ -33,8 +33,8 @@ pub struct HopWindowExecutor { pub time_col_idx: usize, pub window_slide: Interval, pub window_size: Interval, - window_start_exprs: Vec, - window_end_exprs: Vec, + window_start_exprs: Vec, + window_end_exprs: Vec, pub output_indices: Vec, chunk_size: usize, } @@ -48,8 +48,8 @@ impl HopWindowExecutor { time_col_idx: usize, window_slide: Interval, window_size: Interval, - window_start_exprs: Vec, - window_end_exprs: Vec, + window_start_exprs: Vec, + window_end_exprs: Vec, output_indices: Vec, chunk_size: usize, ) -> Self { @@ -251,6 +251,7 @@ mod tests { use risingwave_common::types::test_utils::IntervalTestExt; use risingwave_common::types::{DataType, Interval}; use risingwave_expr::expr::test_utils::make_hop_window_expression; + use risingwave_expr::expr::NonStrictExpression; use crate::executor::test_utils::MockSource; use crate::executor::{ActorContext, Executor, ExecutorInfo, StreamChunk}; @@ -302,8 +303,14 @@ mod tests { 2, window_slide, window_size, - window_start_exprs, - window_end_exprs, + window_start_exprs + .into_iter() + .map(NonStrictExpression::for_test) + .collect(), + window_end_exprs + .into_iter() + .map(NonStrictExpression::for_test) + .collect(), output_indices, CHUNK_SIZE, ) diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index a9c219a25641f..cd505093294f1 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -152,7 +152,7 @@ async fn test_merger_sum_aggr() { vec![], vec![ // TODO: use the new streaming_if_null expression here, and add `None` tests - Box::new(InputRefExpression::new(DataType::Int64, 1)), + NonStrictExpression::for_test(InputRefExpression::new(DataType::Int64, 1)), ], 3, MultiMap::new(), diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index 99b090e21a240..c28d6ec8564d9 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -31,7 +31,7 @@ use risingwave_common::util::epoch::{Epoch, EpochPair}; use risingwave_common::util::tracing::TracingContext; use risingwave_common::util::value_encoding::{DatumFromProtoExt, DatumToProtoExt}; use risingwave_connector::source::SplitImpl; -use risingwave_expr::expr::BoxedExpression; +use risingwave_expr::expr::{Expression, NonStrictExpression}; use risingwave_pb::data::PbEpoch; use risingwave_pb::expr::PbInputRef; use risingwave_pb::stream_plan::barrier::{BarrierKind, PbMutation}; @@ -641,7 +641,7 @@ impl Watermark { pub async fn transform_with_expr( self, - expr: &BoxedExpression, + expr: &NonStrictExpression, new_col_idx: usize, ) -> Option { let Self { col_idx, val, .. } = self; @@ -651,7 +651,7 @@ impl Watermark { OwnedRow::new(row) }; let val = expr.eval_row_infallible(&row).await?; - Some(Self::new(new_col_idx, expr.return_type(), val)) + Some(Self::new(new_col_idx, expr.inner().return_type(), val)) } /// Transform the watermark with the given output indices. If this watermark is not in the diff --git a/src/stream/src/executor/project.rs b/src/stream/src/executor/project.rs index 56a31bde901b9..8cfebfecd3f33 100644 --- a/src/stream/src/executor/project.rs +++ b/src/stream/src/executor/project.rs @@ -21,7 +21,7 @@ use risingwave_common::catalog::{Field, Schema}; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::ToOwnedDatum; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_expr::expr::BoxedExpression; +use risingwave_expr::expr::NonStrictExpression; use super::*; @@ -38,7 +38,7 @@ struct Inner { info: ExecutorInfo, /// Expressions of the current projection. - exprs: Vec, + exprs: Vec, /// All the watermark derivations, (input_column_index, output_column_index). And the /// derivation expression is the project's expression itself. watermark_derivations: MultiMap, @@ -58,7 +58,7 @@ impl ProjectExecutor { ctx: ActorContextRef, input: Box, pk_indices: PkIndices, - exprs: Vec, + exprs: Vec, executor_id: u64, watermark_derivations: MultiMap, nondecreasing_expr_indices: Vec, @@ -233,11 +233,12 @@ mod tests { use risingwave_common::array::{DataChunk, StreamChunk}; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::{DataType, Datum}; - use risingwave_expr::expr::{self, build_from_pretty, Expression, ValueImpl}; + use risingwave_expr::expr::{self, Expression, ValueImpl}; use super::super::test_utils::MockSource; use super::super::*; use super::*; + use crate::executor::test_utils::expr::build_from_pretty; use crate::executor::test_utils::StreamExecutorTestExt; #[tokio::test] @@ -345,7 +346,7 @@ mod tests { let a_expr = build_from_pretty("(add:int8 $0:int8 1:int8)"); let b_expr = build_from_pretty("(subtract:int8 $0:int8 1:int8)"); - let c_expr = DummyNondecreasingExpr.boxed(); + let c_expr = NonStrictExpression::for_test(DummyNondecreasingExpr); let project = Box::new(ProjectExecutor::new( ActorContext::create(123), diff --git a/src/stream/src/executor/project_set.rs b/src/stream/src/executor/project_set.rs index 6867e3d55bfde..ff3214db88eaa 100644 --- a/src/stream/src/executor/project_set.rs +++ b/src/stream/src/executor/project_set.rs @@ -24,6 +24,7 @@ use risingwave_common::catalog::{Field, Schema}; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::{DataType, Datum, DatumRef, ToOwnedDatum}; use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_expr::expr::{LogReport, NonStrictExpression}; use risingwave_expr::table_function::ProjectSetSelectItem; use super::error::StreamExecutorError; @@ -260,7 +261,11 @@ impl Inner { ProjectSetSelectItem::Expr(expr) => { watermark .clone() - .transform_with_expr(expr, expr_idx + PROJ_ROW_ID_OFFSET) + .transform_with_expr( + // TODO: should we build `expr` in non-strict mode? + &NonStrictExpression::new_topmost(expr, LogReport), + expr_idx + PROJ_ROW_ID_OFFSET, + ) .await } ProjectSetSelectItem::TableFunction(_) => { diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs index 3c8cde63c4ca9..82c1e56649672 100644 --- a/src/stream/src/executor/temporal_join.rs +++ b/src/stream/src/executor/temporal_join.rs @@ -32,7 +32,7 @@ use risingwave_common::hash::{HashKey, NullBitmap}; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_expr::expr::BoxedExpression; +use risingwave_expr::expr::NonStrictExpression; use risingwave_hummock_sdk::{HummockEpoch, HummockReadEpoch}; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -57,7 +57,7 @@ pub struct TemporalJoinExecutor, right_join_keys: Vec, null_safe: Vec, - condition: Option, + condition: Option, output_indices: Vec, pk_indices: PkIndices, schema: Schema, @@ -338,7 +338,7 @@ impl TemporalJoinExecutor left_join_keys: Vec, right_join_keys: Vec, null_safe: Vec, - condition: Option, + condition: Option, pk_indices: PkIndices, output_indices: Vec, table_output_indices: Vec, diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index bb4864ac04ef8..13a9237cf0159 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -34,11 +34,11 @@ pub mod prelude { pub use risingwave_common::test_prelude::StreamChunkTestExt; pub use risingwave_common::types::DataType; pub use risingwave_common::util::sort_util::OrderType; - pub use risingwave_expr::expr::build_from_pretty; pub use risingwave_storage::memory::MemoryStateStore; pub use risingwave_storage::StateStore; pub use crate::common::table::state_table::StateTable; + pub use crate::executor::test_utils::expr::build_from_pretty; pub use crate::executor::test_utils::{MessageSender, MockSource, StreamExecutorTestExt}; pub use crate::executor::{ActorContext, BoxedMessageStream, Executor, PkIndices}; } @@ -263,6 +263,14 @@ pub trait StreamExecutorTestExt: MessageStream + Unpin { // FIXME: implement on any `impl MessageStream` if the analyzer works well. impl StreamExecutorTestExt for BoxedMessageStream {} +pub mod expr { + use risingwave_expr::expr::NonStrictExpression; + + pub fn build_from_pretty(s: impl AsRef) -> NonStrictExpression { + NonStrictExpression::for_test(risingwave_expr::expr::build_from_pretty(s)) + } +} + pub mod agg_executor { use std::sync::atomic::AtomicU64; use std::sync::Arc; diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 624b2531bf7bd..8c09b56aa3551 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -21,7 +21,7 @@ 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 risingwave_expr::expr::NonStrictExpression; use tokio::sync::mpsc::UnboundedReceiver; use super::{ @@ -40,7 +40,7 @@ pub struct ValuesExecutor { barrier_receiver: UnboundedReceiver, progress: CreateMviewProgress, - rows: vec::IntoIter>, + rows: vec::IntoIter>, pk_indices: PkIndices, identity: String, schema: Schema, @@ -51,7 +51,7 @@ impl ValuesExecutor { pub fn new( ctx: ActorContextRef, progress: CreateMviewProgress, - rows: Vec>, + rows: Vec>, schema: Schema, barrier_receiver: UnboundedReceiver, executor_id: u64, @@ -167,7 +167,7 @@ mod tests { }; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::{DataType, ScalarImpl, StructType}; - use risingwave_expr::expr::{BoxedExpression, LiteralExpression}; + use risingwave_expr::expr::{BoxedExpression, LiteralExpression, NonStrictExpression}; use tokio::sync::mpsc::unbounded_channel; use super::ValuesExecutor; @@ -202,11 +202,11 @@ mod tests { 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 @@ -215,7 +215,10 @@ mod tests { let values_executor_struct = ValuesExecutor::new( ActorContext::create(actor_id), progress, - vec![exprs], + vec![exprs + .into_iter() + .map(NonStrictExpression::for_test) + .collect()], Schema { fields }, barrier_receiver, 10005, diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index ad332112ef269..5e5454cecff93 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -23,7 +23,8 @@ use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, DefaultOrd, ScalarImpl}; use risingwave_common::{bail, row}; use risingwave_expr::expr::{ - build_func_non_strict, BoxedExpression, Expression, InputRefExpression, LiteralExpression, + build_func_non_strict, ExpressionBoxExt, InputRefExpression, LiteralExpression, + NonStrictExpression, }; use risingwave_expr::Result as ExprResult; use risingwave_pb::expr::expr_node::Type; @@ -44,7 +45,7 @@ use crate::task::ActorEvalErrorReport; pub struct WatermarkFilterExecutor { input: BoxedExecutor, /// The expression used to calculate the watermark value. - watermark_expr: BoxedExpression, + watermark_expr: NonStrictExpression, /// The column we should generate watermark and filter on. event_time_col_idx: usize, ctx: ActorContextRef, @@ -55,7 +56,7 @@ pub struct WatermarkFilterExecutor { impl WatermarkFilterExecutor { pub fn new( input: BoxedExecutor, - watermark_expr: BoxedExpression, + watermark_expr: NonStrictExpression, event_time_col_idx: usize, ctx: ActorContextRef, table: StateTable, @@ -298,7 +299,7 @@ impl WatermarkFilterExecutor { event_time_col_idx: usize, watermark: ScalarImpl, eval_error_report: ActorEvalErrorReport, - ) -> ExprResult { + ) -> ExprResult { build_func_non_strict( Type::GreaterThanOrEqual, DataType::Boolean, @@ -350,11 +351,11 @@ mod tests { use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::types::Date; use risingwave_common::util::sort_util::OrderType; - use risingwave_expr::expr::build_from_pretty; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::table::Distribution; use super::*; + use crate::executor::test_utils::expr::build_from_pretty; use crate::executor::test_utils::{MessageSender, MockSource}; use crate::executor::ActorContext; diff --git a/src/stream/src/from_proto/hash_join.rs b/src/stream/src/from_proto/hash_join.rs index 44799af9405c2..87174282e517a 100644 --- a/src/stream/src/from_proto/hash_join.rs +++ b/src/stream/src/from_proto/hash_join.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use risingwave_common::hash::{HashKey, HashKeyDispatcher}; use risingwave_common::types::DataType; use risingwave_expr::expr::{ - build_func_non_strict, build_non_strict_from_prost, BoxedExpression, InputRefExpression, + build_func_non_strict, build_non_strict_from_prost, InputRefExpression, NonStrictExpression, }; pub use risingwave_pb::expr::expr_node::Type as ExprType; use risingwave_pb::plan_common::JoinType as JoinTypeProto; @@ -109,7 +109,8 @@ impl ExecutorBuilder for HashJoinExecutorBuilder { build_non_strict_from_prost( delta_expression.delta.as_ref().unwrap(), params.eval_error_report.clone(), - )?, + )? + .into_inner(), ], params.eval_error_report.clone(), )?) @@ -175,8 +176,8 @@ struct HashJoinExecutorDispatcherArgs { pk_indices: PkIndices, output_indices: Vec, executor_id: u64, - cond: Option, - inequality_pairs: Vec<(usize, usize, bool, Option)>, + cond: Option, + inequality_pairs: Vec<(usize, usize, bool, Option)>, op_info: String, state_table_l: StateTable, degree_state_table_l: StateTable, diff --git a/src/stream/src/from_proto/temporal_join.rs b/src/stream/src/from_proto/temporal_join.rs index 8b7b3b6af1335..58699089e8c27 100644 --- a/src/stream/src/from_proto/temporal_join.rs +++ b/src/stream/src/from_proto/temporal_join.rs @@ -18,7 +18,7 @@ use risingwave_common::catalog::{ColumnDesc, TableId, TableOption}; use risingwave_common::hash::{HashKey, HashKeyDispatcher}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; -use risingwave_expr::expr::{build_non_strict_from_prost, BoxedExpression}; +use risingwave_expr::expr::{build_non_strict_from_prost, NonStrictExpression}; use risingwave_pb::plan_common::{JoinType as JoinTypeProto, StorageTableDesc}; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::Distribution; @@ -190,7 +190,7 @@ struct TemporalJoinExecutorDispatcherArgs { left_join_keys: Vec, right_join_keys: Vec, null_safe: Vec, - condition: Option, + condition: Option, pk_indices: PkIndices, output_indices: Vec, table_output_indices: Vec, diff --git a/src/stream/tests/integration_tests/hop_window.rs b/src/stream/tests/integration_tests/hop_window.rs index 167857cc7d9fc..9d6d879240fc0 100644 --- a/src/stream/tests/integration_tests/hop_window.rs +++ b/src/stream/tests/integration_tests/hop_window.rs @@ -15,6 +15,7 @@ use risingwave_common::types::test_utils::IntervalTestExt; use risingwave_common::types::{Interval, Timestamp}; use risingwave_expr::expr::test_utils::make_hop_window_expression; +use risingwave_expr::expr::NonStrictExpression; use risingwave_stream::executor::{ExecutorInfo, HopWindowExecutor}; use crate::prelude::*; @@ -55,8 +56,14 @@ fn create_executor(output_indices: Vec) -> (MessageSender, BoxedMessageSt TIME_COL_IDX, window_slide, window_size, - window_start_exprs, - window_end_exprs, + window_start_exprs + .into_iter() + .map(NonStrictExpression::for_test) + .collect(), + window_end_exprs + .into_iter() + .map(NonStrictExpression::for_test) + .collect(), output_indices, CHUNK_SIZE, ) diff --git a/src/stream/tests/integration_tests/project_set.rs b/src/stream/tests/integration_tests/project_set.rs index bf1354c25b83b..61a879256108d 100644 --- a/src/stream/tests/integration_tests/project_set.rs +++ b/src/stream/tests/integration_tests/project_set.rs @@ -29,10 +29,10 @@ fn create_executor() -> (MessageSender, BoxedMessageStream) { }; let (tx, source) = MockSource::channel(schema, PkIndices::new()); - let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)"); - let test_expr_watermark = build_from_pretty("(add:int8 $0:int8 1:int8)"); - let tf1 = repeat(build_from_pretty("1:int4"), 1); - let tf2 = repeat(build_from_pretty("2:int4"), 2); + let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)").into_inner(); + let test_expr_watermark = build_from_pretty("(add:int8 $0:int8 1:int8)").into_inner(); + let tf1 = repeat(build_from_pretty("1:int4").into_inner(), 1); + let tf2 = repeat(build_from_pretty("2:int4").into_inner(), 2); let project_set = Box::new(ProjectSetExecutor::new( ActorContext::create(123), diff --git a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs index 1fd5c90e59e4b..89df82d4c21a0 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs @@ -61,6 +61,9 @@ async fn test_background_mv_barrier_recovery() -> Result<()> { .run("create materialized view m1 as select * from t1;") .await?; + // If the CN is killed before first barrier pass for the MV, the MV will be dropped. + // This is because it's table fragments will NOT be committed until first barrier pass. + sleep(Duration::from_secs(5)).await; kill_cn_and_wait_recover(&cluster).await; // Send some upstream updates. diff --git a/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs b/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs index c05e52c927424..776692b2fab90 100644 --- a/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs +++ b/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs @@ -25,7 +25,7 @@ const ROOT_TABLE_CREATE: &str = "create table t1 (v1 int);"; const MV1: &str = "create materialized view m1 as select * from t1 where v1 > 5;"; const MV2: &str = "create materialized view m2 as select * from t1 where v1 > 10;"; const MV3: &str = "create materialized view m3 as select * from m2 where v1 < 15;"; -const MV4: &str = "create materialized view m4 as select m1.v1 as m1v, m3.v1 as m3v from m1 join m3 on m1.v1 = m3.v1;"; +const MV4: &str = "create materialized view m4 as select m1.v1 as m1v, m3.v1 as m3v from m1 join m3 on m1.v1 = m3.v1 limit 100;"; const MV5: &str = "create materialized view m5 as select * from m4;"; #[tokio::test] @@ -40,6 +40,7 @@ async fn test_simple_cascade_materialized_view() -> Result<()> { .locate_one_fragment([ identity_contains("materialize"), no_identity_contains("chain"), + no_identity_contains("topn"), no_identity_contains("hashjoin"), ]) .await?; @@ -129,6 +130,7 @@ async fn test_diamond_cascade_materialized_view() -> Result<()> { .locate_one_fragment([ identity_contains("materialize"), no_identity_contains("chain"), + no_identity_contains("topn"), no_identity_contains("hashjoin"), ]) .await?; diff --git a/src/tests/simulation/tests/integration_tests/scale/plan.rs b/src/tests/simulation/tests/integration_tests/scale/plan.rs index c7244dc826b42..8b62a58998a3f 100644 --- a/src/tests/simulation/tests/integration_tests/scale/plan.rs +++ b/src/tests/simulation/tests/integration_tests/scale/plan.rs @@ -39,10 +39,7 @@ async fn test_resize_normal() -> Result<()> { .await?; let join_fragment = cluster - .locate_one_fragment([ - identity_contains("hashJoin"), - identity_contains("materialize"), - ]) + .locate_one_fragment([identity_contains("hashJoin")]) .await?; let join_fragment_id = join_fragment.inner.fragment_id; @@ -270,7 +267,7 @@ async fn test_resize_no_shuffle() -> Result<()> { session .run( "create materialized view mv7 as select mv1.v as mv1v, mv5.v as mv5v from mv1 -join mv5 on mv1.v = mv5.v;", +join mv5 on mv1.v = mv5.v limit 1;", ) .await?; @@ -316,6 +313,7 @@ join mv5 on mv1.v = mv5.v;", let top_materialize_fragment = cluster .locate_one_fragment([ identity_contains("materialize"), + no_identity_contains("topn"), no_identity_contains("chain"), no_identity_contains("hashJoin"), ])