diff --git a/dashboard/lib/api/streaming.ts b/dashboard/lib/api/streaming.ts index 948cd567d3f2b..95ad89fdf5c58 100644 --- a/dashboard/lib/api/streaming.ts +++ b/dashboard/lib/api/streaming.ts @@ -28,12 +28,30 @@ import { } from "../../proto/gen/catalog" import { ListObjectDependenciesResponse_ObjectDependencies as ObjectDependencies, + RelationIdInfos, TableFragments, } from "../../proto/gen/meta" import { ColumnCatalog, Field } from "../../proto/gen/plan_common" import { UserInfo } from "../../proto/gen/user" import api from "./api" +// NOTE(kwannoel): This can be optimized further, instead of fetching the entire TableFragments struct, +// We can fetch the fields we need from TableFragments, in a truncated struct. +export async function getFragmentsByJobId( + jobId: number +): Promise { + let route = "/fragments/job_id/" + jobId.toString() + let tableFragments: TableFragments = TableFragments.fromJSON( + await api.get(route) + ) + return tableFragments +} + +export async function getRelationIdInfos(): Promise { + let fragmentIds: RelationIdInfos = await api.get("/relation_id_infos") + return fragmentIds +} + export async function getFragments(): Promise { let fragmentList: TableFragments[] = (await api.get("/fragments2")).map( TableFragments.fromJSON diff --git a/dashboard/pages/fragment_graph.tsx b/dashboard/pages/fragment_graph.tsx index 1c5f6d7746535..0cf4ccec94f1b 100644 --- a/dashboard/pages/fragment_graph.tsx +++ b/dashboard/pages/fragment_graph.tsx @@ -45,7 +45,11 @@ import { fetchEmbeddedBackPressure, fetchPrometheusBackPressure, } from "../lib/api/metric" -import { getFragments, getStreamingJobs } from "../lib/api/streaming" +import { + getFragmentsByJobId, + getRelationIdInfos, + getStreamingJobs, +} from "../lib/api/streaming" import { FragmentBox } from "../lib/layout" import { TableFragments, TableFragments_Fragment } from "../proto/gen/meta" import { Dispatcher, MergeNode, StreamNode } from "../proto/gen/stream_plan" @@ -194,28 +198,33 @@ interface EmbeddedBackPressureInfo { export default function Streaming() { const { response: relationList } = useFetch(getStreamingJobs) - const { response: fragmentList } = useFetch(getFragments) + const { response: relationIdInfos } = useFetch(getRelationIdInfos) const [relationId, setRelationId] = useQueryState("id", parseAsInteger) const [selectedFragmentId, setSelectedFragmentId] = useState() + const [tableFragments, setTableFragments] = useState() const toast = useErrorToast() + useEffect(() => { + if (relationId) { + setTableFragments(undefined) + getFragmentsByJobId(relationId).then((tf) => { + setTableFragments(tf) + }) + } + }, [relationId]) + const fragmentDependencyCallback = useCallback(() => { - if (fragmentList) { - if (relationId) { - const fragments = fragmentList.find((x) => x.tableId === relationId) - if (fragments) { - const fragmentDep = buildFragmentDependencyAsEdges(fragments) - return { - fragments, - fragmentDep, - fragmentDepDag: dagStratify()(fragmentDep), - } - } + if (tableFragments) { + const fragmentDep = buildFragmentDependencyAsEdges(tableFragments) + return { + fragments: tableFragments, + fragmentDep, + fragmentDepDag: dagStratify()(fragmentDep), } } - }, [fragmentList, relationId]) + }, [tableFragments]) useEffect(() => { if (relationList) { @@ -255,38 +264,38 @@ export default function Streaming() { const handleSearchFragment = () => { const searchFragIdInt = parseInt(searchFragId) - if (fragmentList) { - for (const tf of fragmentList) { - for (const fragmentId in tf.fragments) { - if (tf.fragments[fragmentId].fragmentId == searchFragIdInt) { - setRelationId(tf.tableId) + if (relationIdInfos) { + let map = relationIdInfos.map + for (const relationId in map) { + const fragmentIdToRelationId = map[relationId].map + for (const fragmentId in fragmentIdToRelationId) { + if (parseInt(fragmentId) == searchFragIdInt) { + setRelationId(parseInt(relationId)) setSelectedFragmentId(searchFragIdInt) return } } } } - toast(new Error(`Fragment ${searchFragIdInt} not found`)) } const handleSearchActor = () => { const searchActorIdInt = parseInt(searchActorId) - if (fragmentList) { - for (const tf of fragmentList) { - for (const fragmentId in tf.fragments) { - const fragment = tf.fragments[fragmentId] - for (const actor of fragment.actors) { - if (actor.actorId == searchActorIdInt) { - setRelationId(tf.tableId) - setSelectedFragmentId(fragment.fragmentId) - return - } + if (relationIdInfos) { + let map = relationIdInfos.map + for (const relationId in map) { + const fragmentIdToRelationId = map[relationId].map + for (const fragmentId in fragmentIdToRelationId) { + let actorIds = fragmentIdToRelationId[fragmentId].ids + if (actorIds.includes(searchActorIdInt)) { + setRelationId(parseInt(relationId)) + setSelectedFragmentId(parseInt(fragmentId)) + return } } } } - toast(new Error(`Actor ${searchActorIdInt} not found`)) } diff --git a/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt b/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt index 31bb9d1b0421b..3c386a2718479 100644 --- a/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt +++ b/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt @@ -42,6 +42,7 @@ distribution key id NULL NULL table description rw_customers NULL NULL +# add column system ok mysql -e " USE mytest; @@ -64,6 +65,57 @@ primary key id NULL NULL distribution key id NULL NULL table description rw_customers NULL NULL +# rename column on upstream will not be replicated, since we do not support rename column +system ok +mysql -e " + USE mytest; + ALTER TABLE customers RENAME COLUMN v1 TO v11; + ALTER TABLE customers CHANGE COLUMN v2 v22 decimal(5,2); +" + +sleep 3s + +# table schema unchanges, since we reject rename column +query TTTT +describe rw_customers; +---- +id bigint false NULL +modified timestamp without time zone false NULL +custinfo jsonb false NULL +v1 character varying false NULL +v2 double precision false NULL +primary key id NULL NULL +distribution key id NULL NULL +table description rw_customers NULL NULL + +# revert column rename on upstream +system ok +mysql -e " + USE mytest; + ALTER TABLE customers RENAME COLUMN v11 TO v1; + ALTER TABLE customers CHANGE COLUMN v22 v2 double(5,2); +" + +# drop columns +system ok +mysql -e " + USE mytest; + ALTER TABLE customers DROP COLUMN modified; + ALTER TABLE customers DROP COLUMN v1; + ALTER TABLE customers DROP COLUMN v2; +" + +sleep 3s + +# modified column should be dropped +query TTTT +describe rw_customers; +---- +id bigint false NULL +custinfo jsonb false NULL +primary key id NULL NULL +distribution key id NULL NULL +table description rw_customers NULL NULL statement ok drop source mysql_source cascade; diff --git a/e2e_test/source_inline/kafka/protobuf/basic.slt b/e2e_test/source_inline/kafka/protobuf/basic.slt index 44153949e79e8..920f415adb592 100644 --- a/e2e_test/source_inline/kafka/protobuf/basic.slt +++ b/e2e_test/source_inline/kafka/protobuf/basic.slt @@ -44,7 +44,7 @@ with ( ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, topic = 'sr_pb_test', scan.startup.mode = 'earliest') -FORMAT plain ENCODE protobuf( +FORMAT upsert ENCODE protobuf( schema.registry = '${RISEDEV_SCHEMA_REGISTRY_URL}', message = 'test.User' ); diff --git a/proto/meta.proto b/proto/meta.proto index 8a1b55e9d2c32..d75494625edd4 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -772,3 +772,20 @@ service EventLogService { rpc ListEventLog(ListEventLogRequest) returns (ListEventLogResponse); rpc AddEventLog(AddEventLogRequest) returns (AddEventLogResponse); } + +message ActorIds { + repeated uint32 ids = 1; +} + +message FragmentIdToActorIdMap { + map map = 1; +} + +/// Provides all the ids: relation_id, fragment_id, actor_id +/// in an hierarchical format. +/// relation_id -> [fragment_id] +/// fragment_id -> [actor_id] +message RelationIdInfos { + // relation_id -> FragmentIdToActorIdMap + map map = 1; +} diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 8b6e8fa9775f0..d87e89c1cf65d 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -93,6 +93,7 @@ opendal = { workspace = true, features = [ "services-gcs", "services-memory", "services-s3", + "services-webhdfs", ] } openssl = "0.10" parking_lot = { workspace = true } diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index b422a029fa761..1dc304c651a85 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -39,6 +39,7 @@ macro_rules! for_all_classified_sources { { Gcs, $crate::source::filesystem::opendal_source::GcsProperties , $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalGcs> }, { OpendalS3, $crate::source::filesystem::opendal_source::OpendalS3Properties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalS3> }, { PosixFs, $crate::source::filesystem::opendal_source::PosixFsProperties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalPosixFs> }, + { Azblob, $crate::source::filesystem::opendal_source::AzblobProperties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalAzblob> }, { Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit}, { Iceberg, $crate::source::iceberg::IcebergProperties, $crate::source::iceberg::IcebergSplit} } diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index 54b1120d06438..c30f5f74ba390 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -31,8 +31,8 @@ use risingwave_pb::plan_common::{ use crate::error::ConnectorResult; use crate::source::cdc::MONGODB_CDC_CONNECTOR; use crate::source::{ - GCS_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, OPENDAL_S3_CONNECTOR, PULSAR_CONNECTOR, - S3_CONNECTOR, + AZBLOB_CONNECTOR, GCS_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, OPENDAL_S3_CONNECTOR, + POSIX_FS_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, }; // Hidden additional columns connectors which do not support `include` syntax. @@ -57,6 +57,8 @@ pub static COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock AccessResult { let mut schema_changes = vec![]; - let upstream_ddl = accessor + let upstream_ddl: String = accessor .access(&[UPSTREAM_DDL], &DataType::Varchar)? .to_owned_datum() .unwrap() diff --git a/src/connector/src/sink/file_sink/mod.rs b/src/connector/src/sink/file_sink/mod.rs index fe25df4a5d1eb..f32c6812f3c00 100644 --- a/src/connector/src/sink/file_sink/mod.rs +++ b/src/connector/src/sink/file_sink/mod.rs @@ -17,3 +17,4 @@ pub mod fs; pub mod gcs; pub mod opendal_sink; pub mod s3; +pub mod webhdfs; diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 1f6ec2b635fe9..1fd461015b4ba 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -87,6 +87,7 @@ pub enum EngineType { S3, Fs, Azblob, + Webhdfs, } impl Sink for FileSink { diff --git a/src/connector/src/sink/file_sink/webhdfs.rs b/src/connector/src/sink/file_sink/webhdfs.rs new file mode 100644 index 0000000000000..b41a27e12db1f --- /dev/null +++ b/src/connector/src/sink/file_sink/webhdfs.rs @@ -0,0 +1,107 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +use std::collections::{BTreeMap, HashMap}; + +use anyhow::anyhow; +use opendal::layers::LoggingLayer; +use opendal::services::Webhdfs; +use opendal::Operator; +use serde::Deserialize; +use serde_with::serde_as; +use with_options::WithOptions; + +use super::opendal_sink::FileSink; +use crate::sink::file_sink::opendal_sink::OpendalSinkBackend; +use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; +use crate::source::UnknownFields; +#[derive(Deserialize, Debug, Clone, WithOptions)] +pub struct WebhdfsCommon { + #[serde(rename = "webhdfs.endpoint")] + pub endpoint: String, + /// The directory where the sink file is located. + #[serde(rename = "webhdfs.path")] + pub path: String, +} + +#[serde_as] +#[derive(Clone, Debug, Deserialize, WithOptions)] +pub struct WebhdfsConfig { + #[serde(flatten)] + pub common: WebhdfsCommon, + + pub r#type: String, // accept "append-only" + + #[serde(flatten)] + pub unknown_fields: HashMap, +} + +pub const WEBHDFS_SINK: &str = "webhdfs"; + +impl FileSink { + pub fn new_webhdfs_sink(config: WebhdfsConfig) -> Result { + // Create webhdfs backend builder. + let mut builder = Webhdfs::default(); + // Set the name node for hdfs. + builder.endpoint(&config.common.endpoint); + builder.root(&config.common.path); + + let operator: Operator = Operator::new(builder)? + .layer(LoggingLayer::default()) + .finish(); + + Ok(operator) + } +} + +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct WebhdfsSink; + +impl UnknownFields for WebhdfsConfig { + fn unknown_fields(&self) -> HashMap { + self.unknown_fields.clone() + } +} + +impl OpendalSinkBackend for WebhdfsSink { + type Properties = WebhdfsConfig; + + const SINK_NAME: &'static str = WEBHDFS_SINK; + + fn from_btreemap(btree_map: BTreeMap) -> Result { + let config = + serde_json::from_value::(serde_json::to_value(btree_map).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 + ))); + } + Ok(config) + } + + fn new_operator(properties: WebhdfsConfig) -> Result { + FileSink::::new_webhdfs_sink(properties) + } + + fn get_path(properties: Self::Properties) -> String { + properties.common.path + } + + fn get_engine_type() -> super::opendal_sink::EngineType { + super::opendal_sink::EngineType::Webhdfs + } +} diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index b734551080296..dafbc856207a9 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -105,6 +105,7 @@ macro_rules! for_all_sinks { { Gcs, $crate::sink::file_sink::opendal_sink::FileSink<$crate::sink::file_sink::gcs::GcsSink> }, { Azblob, $crate::sink::file_sink::opendal_sink::FileSink<$crate::sink::file_sink::azblob::AzblobSink>}, + { Webhdfs, $crate::sink::file_sink::opendal_sink::FileSink<$crate::sink::file_sink::webhdfs::WebhdfsSink>}, { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, { Snowflake, $crate::sink::snowflake::SnowflakeSink }, diff --git a/src/connector/src/sink/trivial.rs b/src/connector/src/sink/trivial.rs index 0cfa82c5c4d19..5c5e093c8e0f0 100644 --- a/src/connector/src/sink/trivial.rs +++ b/src/connector/src/sink/trivial.rs @@ -15,7 +15,9 @@ use std::marker::PhantomData; use async_trait::async_trait; +use risingwave_common::session_config::sink_decouple::SinkDecouple; +use super::catalog::desc::SinkDesc; use crate::sink::log_store::{LogStoreReadItem, TruncateOffset}; use crate::sink::{ DummySinkCommitCoordinator, LogSinker, Result, Sink, SinkError, SinkLogReader, SinkParam, @@ -64,6 +66,11 @@ impl Sink for TrivialSink { const SINK_NAME: &'static str = T::SINK_NAME; + // Disable sink decoupling for all trivial sinks because it introduces overhead without any benefit + fn is_sink_decouple(_desc: &SinkDesc, _user_specified: &SinkDecouple) -> Result { + Ok(false) + } + async fn new_log_sinker(&self, _writer_env: SinkWriterParam) -> Result { Ok(Self(PhantomData)) } diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index fd3fad8275154..38c2f25eb0336 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -41,7 +41,7 @@ use super::kafka::KafkaMeta; use super::kinesis::KinesisMeta; use super::monitor::SourceMetrics; use super::nexmark::source::message::NexmarkMeta; -use super::{GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR}; +use super::{AZBLOB_CONNECTOR, GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR}; use crate::error::ConnectorResult as Result; use crate::parser::schema_change::SchemaChangeEnvelope; use crate::parser::ParserConfig; @@ -323,6 +323,9 @@ pub fn extract_source_struct(info: &PbStreamSourceInfo) -> Result (SourceFormat::DebeziumMongo, SourceEncode::Json) } (PbFormatType::Plain, PbEncodeType::Bytes) => (SourceFormat::Plain, SourceEncode::Bytes), + (PbFormatType::Upsert, PbEncodeType::Protobuf) => { + (SourceFormat::Upsert, SourceEncode::Protobuf) + } (format, encode) => { bail!( "Unsupported combination of format {:?} and encode {:?}", @@ -385,6 +388,7 @@ impl ConnectorProperties { s.eq_ignore_ascii_case(OPENDAL_S3_CONNECTOR) || s.eq_ignore_ascii_case(POSIX_FS_CONNECTOR) || s.eq_ignore_ascii_case(GCS_CONNECTOR) + || s.eq_ignore_ascii_case(AZBLOB_CONNECTOR) }) .unwrap_or(false) } @@ -435,6 +439,7 @@ impl ConnectorProperties { matches!(self, ConnectorProperties::Kafka(_)) || matches!(self, ConnectorProperties::OpendalS3(_)) || matches!(self, ConnectorProperties::Gcs(_)) + || matches!(self, ConnectorProperties::Azblob(_)) } } diff --git a/src/connector/src/source/filesystem/opendal_source/azblob_source.rs b/src/connector/src/source/filesystem/opendal_source/azblob_source.rs new file mode 100644 index 0000000000000..2ee050f21f812 --- /dev/null +++ b/src/connector/src/source/filesystem/opendal_source/azblob_source.rs @@ -0,0 +1,78 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::marker::PhantomData; + +use anyhow::Context; +use opendal::layers::{LoggingLayer, RetryLayer}; +use opendal::services::Azblob; +use opendal::Operator; + +use super::opendal_enumerator::OpendalEnumerator; +use super::{AzblobProperties, OpendalSource}; +use crate::error::ConnectorResult; +use crate::source::filesystem::s3::enumerator::get_prefix; + +impl OpendalEnumerator { + /// create opendal azblob source. + pub fn new_azblob_source(azblob_properties: AzblobProperties) -> ConnectorResult { + // Create azblob builder. + let mut builder = Azblob::default(); + + builder.container(&azblob_properties.container_name); + + builder.endpoint(&azblob_properties.endpoint_url); + + if let Some(account_name) = azblob_properties.account_name { + builder.account_name(&account_name); + } else { + tracing::warn!( + "account_name azblob is not set, container {}", + azblob_properties.container_name + ); + } + + if let Some(account_key) = azblob_properties.account_key { + builder.account_key(&account_key); + } else { + tracing::warn!( + "account_key azblob is not set, container {}", + azblob_properties.container_name + ); + } + let op: Operator = Operator::new(builder)? + .layer(LoggingLayer::default()) + .layer(RetryLayer::default()) + .finish(); + + let (prefix, matcher) = if let Some(pattern) = azblob_properties.match_pattern.as_ref() { + let prefix = get_prefix(pattern); + let matcher = glob::Pattern::new(pattern) + .with_context(|| format!("Invalid match_pattern: {}", pattern))?; + (Some(prefix), Some(matcher)) + } else { + (None, None) + }; + + let compression_format = azblob_properties.compression_format; + + Ok(Self { + op, + prefix, + matcher, + marker: PhantomData, + compression_format, + }) + } +} diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index 78c6ebf4cd8c7..1efeee0908e70 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -14,6 +14,7 @@ use std::collections::HashMap; +pub mod azblob_source; pub mod gcs_source; pub mod posix_fs_source; pub mod s3_source; @@ -31,6 +32,7 @@ use super::OpendalFsSplit; use crate::error::ConnectorResult; use crate::source::{SourceProperties, UnknownFields}; +pub const AZBLOB_CONNECTOR: &str = "azblob"; pub const GCS_CONNECTOR: &str = "gcs"; // The new s3_v2 will use opendal. pub const OPENDAL_S3_CONNECTOR: &str = "s3_v2"; @@ -168,3 +170,50 @@ impl SourceProperties for PosixFsProperties { const SOURCE_NAME: &'static str = POSIX_FS_CONNECTOR; } + +#[derive(Clone, Debug, Deserialize, PartialEq, WithOptions)] +pub struct AzblobProperties { + #[serde(rename = "azblob.container_name")] + pub container_name: String, + + #[serde(rename = "azblob.credentials.account_name", default)] + pub account_name: Option, + #[serde(rename = "azblob.credentials.account_key", default)] + pub account_key: Option, + #[serde(rename = "azblob.endpoint_url")] + pub endpoint_url: String, + + #[serde(rename = "match_pattern", default)] + pub match_pattern: Option, + + #[serde(flatten)] + pub unknown_fields: HashMap, + + #[serde(rename = "compression_format", default = "Default::default")] + pub compression_format: CompressionFormat, +} + +impl UnknownFields for AzblobProperties { + fn unknown_fields(&self) -> HashMap { + self.unknown_fields.clone() + } +} + +impl SourceProperties for AzblobProperties { + type Split = OpendalFsSplit; + type SplitEnumerator = OpendalEnumerator; + type SplitReader = OpendalReader; + + const SOURCE_NAME: &'static str = AZBLOB_CONNECTOR; +} + +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct OpendalAzblob; + +impl OpendalSource for OpendalAzblob { + type Properties = AzblobProperties; + + fn new_enumerator(properties: Self::Properties) -> ConnectorResult> { + OpendalEnumerator::new_azblob_source(properties) + } +} diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index cffeb5dfe5f65..c6d0fe4af0c2c 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -72,7 +72,7 @@ impl OpendalEnumerator { let object_lister = self .op .lister_with(prefix) - .recursive(true) + .recursive(false) .metakey(Metakey::ContentLength | Metakey::LastModified) .await?; let stream = stream::unfold(object_lister, |mut object_lister| async move { diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index ed8842e70825f..dc965c9274ff1 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -45,7 +45,7 @@ use risingwave_common::array::{Array, ArrayRef}; use thiserror_ext::AsReport; pub use crate::source::filesystem::opendal_source::{ - GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, + AZBLOB_CONNECTOR, GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, }; pub use crate::source::filesystem::S3_CONNECTOR; pub use crate::source::nexmark::NEXMARK_CONNECTOR; diff --git a/src/connector/src/source/reader/reader.rs b/src/connector/src/source/reader/reader.rs index 10d549df0c49a..d27e14253dd42 100644 --- a/src/connector/src/source/reader/reader.rs +++ b/src/connector/src/source/reader/reader.rs @@ -29,7 +29,7 @@ use crate::error::ConnectorResult; use crate::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use crate::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; use crate::source::filesystem::opendal_source::{ - OpendalGcs, OpendalPosixFs, OpendalS3, OpendalSource, + OpendalAzblob, OpendalGcs, OpendalPosixFs, OpendalS3, OpendalSource, }; use crate::source::filesystem::{FsPageItem, OpendalFsSplit}; use crate::source::{ @@ -95,6 +95,11 @@ impl SourceReader { OpendalEnumerator::new_s3_source(prop.s3_properties, prop.assume_role)?; Ok(build_opendal_fs_list_stream(lister)) } + ConnectorProperties::Azblob(prop) => { + let lister: OpendalEnumerator = + OpendalEnumerator::new_azblob_source(*prop)?; + Ok(build_opendal_fs_list_stream(lister)) + } ConnectorProperties::PosixFs(prop) => { let lister: OpendalEnumerator = OpendalEnumerator::new_posix_fs_source(*prop)?; @@ -192,6 +197,7 @@ async fn build_opendal_fs_list_stream(lister: OpendalEnumera .map(|m| m.matches(&res.name)) .unwrap_or(true) { + println!("这里 res{:?}", res.name); yield res } else { // Currrntly due to the lack of prefix list, we just skip the unmatched files. diff --git a/src/connector/src/with_options.rs b/src/connector/src/with_options.rs index 90dcfc5b1d88f..ae2d432fdfd74 100644 --- a/src/connector/src/with_options.rs +++ b/src/connector/src/with_options.rs @@ -20,7 +20,8 @@ use crate::sink::catalog::SinkFormatDesc; use crate::source::cdc::external::CdcTableType; use crate::source::iceberg::ICEBERG_CONNECTOR; use crate::source::{ - GCS_CONNECTOR, KAFKA_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, UPSTREAM_SOURCE_KEY, + AZBLOB_CONNECTOR, GCS_CONNECTOR, KAFKA_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, + UPSTREAM_SOURCE_KEY, }; /// Marker trait for `WITH` options. Only for `#[derive(WithOptions)]`, should not be used manually. @@ -144,6 +145,7 @@ pub trait WithPropertiesExt: Get + Sized { s.eq_ignore_ascii_case(OPENDAL_S3_CONNECTOR) || s.eq_ignore_ascii_case(POSIX_FS_CONNECTOR) || s.eq_ignore_ascii_case(GCS_CONNECTOR) + || s.eq_ignore_ascii_case(AZBLOB_CONNECTOR) }) .unwrap_or(false) } diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index 6b661139e13e7..9d704d31942e9 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -1022,3 +1022,15 @@ StarrocksConfig: - name: r#type field_type: String required: true +WebhdfsConfig: + fields: + - name: webhdfs.endpoint + field_type: String + required: true + - name: webhdfs.path + field_type: String + comments: The directory where the sink file is located. + required: true + - name: r#type + field_type: String + required: true diff --git a/src/connector/with_options_source.yaml b/src/connector/with_options_source.yaml index 5f9be2070635f..4787963455015 100644 --- a/src/connector/with_options_source.yaml +++ b/src/connector/with_options_source.yaml @@ -1,6 +1,30 @@ # THIS FILE IS AUTO_GENERATED. DO NOT EDIT # UPDATE WITH: ./risedev generate-with-options +AzblobProperties: + fields: + - name: azblob.container_name + field_type: String + required: true + - name: azblob.credentials.account_name + field_type: String + required: false + default: Default::default + - name: azblob.credentials.account_key + field_type: String + required: false + default: Default::default + - name: azblob.endpoint_url + field_type: String + required: true + - name: match_pattern + field_type: String + required: false + default: Default::default + - name: compression_format + field_type: CompressionFormat + required: false + default: Default::default DatagenProperties: fields: - name: datagen.split.num diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 3c6f0d6133616..e44426caa3a49 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -71,7 +71,7 @@ └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [agg], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v2)) * sum0(count(t.v3)))) as $expr1] } + └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v2)) * sum0(count(t.v3)))) as $expr1], noop_update_hint: true } └─StreamSimpleAgg { aggs: [min(min(t.v1)), max(max(t.v2)), sum0(count(t.v3)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [min(t.v1), max(t.v2), count(t.v3), count] } @@ -273,7 +273,7 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [cnt, sum], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum0(count($expr1)), sum(sum($expr1))] } + └─StreamProject { exprs: [sum0(count($expr1)), sum(sum($expr1))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(count($expr1)), sum(sum($expr1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count($expr1), sum($expr1)] } @@ -571,7 +571,7 @@ └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [agg], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v3)) * sum0(count(t.v2)))) as $expr1] } + └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v3)) * sum0(count(t.v2)))) as $expr1], noop_update_hint: true } └─StreamSimpleAgg { aggs: [min(min(t.v1)), max(max(t.v3)), sum0(count(t.v2)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [min(t.v1), max(t.v3), count(t.v2), count] } @@ -628,7 +628,7 @@ └─LogicalScan { table: t, columns: [t.v1] } stream_plan: |- StreamMaterialize { columns: [s1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(t.v1))] } + └─StreamProject { exprs: [sum(sum(t.v1))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } @@ -647,7 +647,7 @@ └─LogicalScan { table: t, columns: [t.v1] } stream_plan: |- StreamMaterialize { columns: [s1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(t.v1))] } + └─StreamProject { exprs: [sum(sum(t.v1))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } @@ -666,7 +666,7 @@ └─LogicalScan { table: t, columns: [t.v1] } stream_plan: |- StreamMaterialize { columns: [s1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(t.v1))] } + └─StreamProject { exprs: [sum(sum(t.v1))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } @@ -685,7 +685,7 @@ └─LogicalScan { table: t, columns: [t.v1] } stream_plan: |- StreamMaterialize { columns: [sa], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(t.v1) filter((t.v1 > 0:Int32)))] } + └─StreamProject { exprs: [sum(sum(t.v1) filter((t.v1 > 0:Int32)))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(t.v1) filter((t.v1 > 0:Int32))), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(t.v1) filter((t.v1 > 0:Int32))] } @@ -720,7 +720,7 @@ └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: |- StreamMaterialize { columns: [sab], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32))))] } + └─StreamProject { exprs: [max(max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32))))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32)))), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [$expr2], aggs: [max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32))), count] } @@ -759,7 +759,7 @@ └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: |- StreamMaterialize { columns: [cnt_agb], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum0(count filter((t.a > t.b)))] } + └─StreamProject { exprs: [sum0(count filter((t.a > t.b)))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(count filter((t.a > t.b))), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count filter((t.a > t.b))] } @@ -813,7 +813,7 @@ └─BatchScan { table: t, columns: [t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [b], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(t.v2) filter((t.v2 < 5:Int32)))] } + └─StreamProject { exprs: [sum(sum(t.v2) filter((t.v2 < 5:Int32)))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(t.v2) filter((t.v2 < 5:Int32))), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(t.v2) filter((t.v2 < 5:Int32))] } @@ -896,7 +896,7 @@ └─BatchScan { table: t, columns: [t.x, t.y], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [string_agg, count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [string_agg(t.y, ',':Varchar), count(distinct t.x)] } + └─StreamProject { exprs: [string_agg(t.y, ',':Varchar), count(distinct t.x)], noop_update_hint: true } └─StreamSimpleAgg { aggs: [string_agg(t.y, ',':Varchar), count(distinct t.x), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.y, ',':Varchar, t.x, t._row_id] } @@ -917,7 +917,7 @@ └─BatchScan { table: t, columns: [t.x, t.y], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [string_agg, count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [string_agg(t.y, ',':Varchar order_by(t.y ASC)), count(distinct t.x)] } + └─StreamProject { exprs: [string_agg(t.y, ',':Varchar order_by(t.y ASC)), count(distinct t.x)], noop_update_hint: true } └─StreamSimpleAgg { aggs: [string_agg(t.y, ',':Varchar order_by(t.y ASC)), count(distinct t.x), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.y, ',':Varchar, t.x, t._row_id] } @@ -938,7 +938,7 @@ └─BatchScan { table: t, columns: [t.x, t.y], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [string_agg, count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [string_agg(distinct t.y, ',':Varchar order_by(t.y ASC)), count(distinct t.x)] } + └─StreamProject { exprs: [string_agg(distinct t.y, ',':Varchar order_by(t.y ASC)), count(distinct t.x)], noop_update_hint: true } └─StreamSimpleAgg { aggs: [string_agg(distinct t.y, ',':Varchar order_by(t.y ASC)), count(distinct t.x), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.y, ',':Varchar, t.x, t._row_id] } @@ -1006,7 +1006,7 @@ └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: |- StreamMaterialize { columns: [s1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32)))] } + └─StreamProject { exprs: [sum(sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32)))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))] } @@ -1313,7 +1313,7 @@ stream_plan: |- StreamMaterialize { columns: [stddev_samp, stddev_pop], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Decimal, Sqrt(((sum(sum($expr1))::Decimal - (($expr2 * $expr2) / $expr3)) / (sum0(count(t.v1)) - 1:Int64)::Decimal))) as $expr4, Sqrt(((sum(sum($expr1))::Decimal - (($expr2 * $expr2) / $expr3)) / $expr3)) as $expr5] } - └─StreamProject { exprs: [sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1)), sum(sum(t.v1))::Decimal as $expr2, sum0(count(t.v1))::Decimal as $expr3] } + └─StreamProject { exprs: [sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1)), sum(sum(t.v1))::Decimal as $expr2, sum0(count(t.v1))::Decimal as $expr3], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum($expr1), sum(t.v1), count(t.v1)] } @@ -1370,7 +1370,7 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [min, sum], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [min(min(t.v1)), sum(sum(t.v2))] } + └─StreamProject { exprs: [min(min(t.v1)), sum(sum(t.v2))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [min(min(t.v1)), sum(sum(t.v2)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [min(t.v1), sum(t.v2), count] } @@ -1388,7 +1388,7 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [min, sum], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [min(t.v1), sum(t.v2)] } + └─StreamProject { exprs: [min(t.v1), sum(t.v2)], noop_update_hint: true } └─StreamSimpleAgg { aggs: [min(t.v1), sum(t.v2), count] } └─StreamExchange { dist: Single } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -1677,7 +1677,7 @@ └─BatchScan { table: t, columns: [t.x, t.y], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [first_value], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [first_value(t.x order_by(t.y ASC))] } + └─StreamProject { exprs: [first_value(t.x order_by(t.y ASC))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [first_value(t.x order_by(t.y ASC)), count] } └─StreamExchange { dist: Single } └─StreamTableScan { table: t, columns: [t.x, t.y, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -1685,7 +1685,7 @@ Fragment 0 StreamMaterialize { columns: [first_value], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [first_value(t.x order_by(t.y ASC))] } + └── StreamProject { exprs: [first_value(t.x order_by(t.y ASC))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [first_value(t.x order_by(t.y ASC)), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] └── StreamExchange Single from 1 @@ -1717,7 +1717,7 @@ Fragment 0 StreamMaterialize { columns: [first_value], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [first_value(distinct t.x order_by(t.x ASC))] } + └── StreamProject { exprs: [first_value(distinct t.x order_by(t.x ASC))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [first_value(distinct t.x order_by(t.x ASC)), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0, SimpleAggDedupForCol0: 2 ] └── StreamExchange Single from 1 @@ -1753,7 +1753,7 @@ └─BatchScan { table: t, columns: [t.x, t.y], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [last_value], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [last_value(t.x order_by(t.y DESC NULLS LAST))] } + └─StreamProject { exprs: [last_value(t.x order_by(t.y DESC NULLS LAST))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [last_value(t.x order_by(t.y DESC NULLS LAST)), count] } └─StreamExchange { dist: Single } └─StreamTableScan { table: t, columns: [t.x, t.y, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -1874,7 +1874,7 @@ └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: |- StreamMaterialize { columns: [x, y, z, w], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(t.v1)), sum0(count(t.v1)), sum(sum(t.v1)), sum0(count(t.v1))] } + └─StreamProject { exprs: [sum(sum(t.v1)), sum0(count(t.v1)), sum(sum(t.v1)), sum0(count(t.v1))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), sum0(count(t.v1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(t.v1), count(t.v1)] } @@ -1895,11 +1895,12 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - └─StreamExchange { dist: Single } - └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProject { exprs: [approx_percentile], noop_update_hint: true } + └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } + └─StreamExchange { dist: Single } + └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test simple approx_percentile with other simple aggs sql: | CREATE TABLE t (v1 int); @@ -1916,19 +1917,20 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [approx_percentile, sum], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamRowMerge { output: [approx_percentile:Float64, sum(sum(t.v1)):Int64] } - ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamExchange { dist: Single } - │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamShare { id: 2 } - │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } - │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } - └─StreamShare { id: 2 } - └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProject { exprs: [approx_percentile, sum(sum(t.v1))], noop_update_hint: true } + └─StreamRowMerge { output: [approx_percentile:Float64, sum(sum(t.v1)):Int64] } + ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ └─StreamExchange { dist: Single } + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } + └─StreamShare { id: 2 } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test simple approx_percentile with other simple aggs (sum, count) sql: | CREATE TABLE t (v1 int); @@ -1946,7 +1948,7 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s1, approx_percentile, s2, count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum(sum(t.v1)), sum0(count(t.v1))] } + └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum(sum(t.v1)), sum0(count(t.v1))], noop_update_hint: true } └─StreamRowMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count(t.v1)):Int64] } ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamExchange { dist: Single } @@ -1971,7 +1973,7 @@ └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: |- StreamMaterialize { columns: [x, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [approx_percentile, approx_percentile] } + └─StreamProject { exprs: [approx_percentile, approx_percentile], noop_update_hint: true } └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } └─StreamExchange { dist: Single } └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } @@ -1993,19 +1995,20 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamRowMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } - ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamExchange { dist: Single } - │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamShare { id: 2 } - │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } - │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - └─StreamExchange { dist: Single } - └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - └─StreamShare { id: 2 } - └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProject { exprs: [approx_percentile, approx_percentile], noop_update_hint: true } + └─StreamRowMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } + ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ └─StreamExchange { dist: Single } + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } + └─StreamExchange { dist: Single } + └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + └─StreamShare { id: 2 } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test simple approx_percentile with different approx_percentile interleaved with stateless simple aggs sql: | CREATE TABLE t (v1 int, v2 int); @@ -2023,7 +2026,7 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s1, x, count, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum0(count), (sum(sum(t.v2))::Float64 + approx_percentile) as $expr3] } + └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum0(count), (sum(sum(t.v2))::Float64 + approx_percentile) as $expr3], noop_update_hint: true } └─StreamRowMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count):Int64, sum(sum(t.v2)):Int64, approx_percentile:Float64] } ├─StreamRowMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } │ ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } @@ -2061,7 +2064,7 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s1, x, count, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum0(count), (sum(sum(t.v2))::Float64 + approx_percentile) as $expr3] } + └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum0(count), (sum(sum(t.v2))::Float64 + approx_percentile) as $expr3], noop_update_hint: true } └─StreamRowMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count):Int64, sum(sum(t.v2)):Int64, approx_percentile:Float64] } ├─StreamRowMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } │ ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } @@ -2098,19 +2101,20 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s1, approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamRowMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64] } - ├─StreamGlobalApproxPercentile { quantile: 0.8:Float64, relative_error: 0.01:Float64 } - │ └─StreamExchange { dist: Single } - │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.8:Float64, relative_error: 0.01:Float64 } - │ └─StreamShare { id: 2 } - │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } - │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } - └─StreamShare { id: 2 } - └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile], noop_update_hint: true } + └─StreamRowMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64] } + ├─StreamGlobalApproxPercentile { quantile: 0.8:Float64, relative_error: 0.01:Float64 } + │ └─StreamExchange { dist: Single } + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.8:Float64, relative_error: 0.01:Float64 } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } + └─StreamShare { id: 2 } + └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test simple approx_percentile with different approx_percentile interleaved with stateless + stateful simple aggs sql: | CREATE TABLE t (v1 int, v2 int); @@ -2127,25 +2131,26 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s1, x, count, m2, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamRowMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count):Int64, max(max(t.v2)):Int32, approx_percentile:Float64] } - ├─StreamRowMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } - │ ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ │ └─StreamExchange { dist: Single } - │ │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ │ └─StreamShare { id: 2 } - │ │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } - │ │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - │ └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamExchange { dist: Single } - │ └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamShare { id: 2 } - │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } - │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), sum0(count), max(max(t.v2)), count] } - └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr5], aggs: [sum(t.v1), count, max(t.v2)] } - └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr3, t.v2, t.v2::Float64 as $expr4, t._row_id, Vnode(t._row_id) as $expr5] } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum0(count), max(max(t.v2)), approx_percentile], noop_update_hint: true } + └─StreamRowMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count):Int64, max(max(t.v2)):Int32, approx_percentile:Float64] } + ├─StreamRowMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } + │ ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ │ └─StreamExchange { dist: Single } + │ │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ │ └─StreamShare { id: 2 } + │ │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } + │ │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ └─StreamExchange { dist: Single } + │ └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), sum0(count), max(max(t.v2)), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [$expr5], aggs: [sum(t.v1), count, max(t.v2)] } + └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr3, t.v2, t.v2::Float64 as $expr4, t._row_id, Vnode(t._row_id) as $expr5] } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test hash approx_percentile sql: | CREATE TABLE t (v1 int, v2 int); @@ -2193,8 +2198,9 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } - └─StreamExchange { dist: Single } - └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProject { exprs: [approx_percentile], noop_update_hint: true } + └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } + └─StreamExchange { dist: Single } + └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } 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 d0701675c3617..e76813e05f759 100644 --- a/src/frontend/planner_test/tests/testdata/output/append_only.yaml +++ b/src/frontend/planner_test/tests/testdata/output/append_only.yaml @@ -33,7 +33,7 @@ select max(v1) as max_v1 from t1; stream_plan: |- StreamMaterialize { columns: [max_v1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(max(t1.v1))] } + └─StreamProject { exprs: [max(max(t1.v1))], noop_update_hint: true } └─StreamSimpleAgg [append_only] { aggs: [max(max(t1.v1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [max(t1.v1)] } diff --git a/src/frontend/planner_test/tests/testdata/output/bushy_join.yaml b/src/frontend/planner_test/tests/testdata/output/bushy_join.yaml index 9d042f1e60c8b..a785ac443901a 100644 --- a/src/frontend/planner_test/tests/testdata/output/bushy_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/bushy_join.yaml @@ -8,7 +8,7 @@ sql: select count(*) from t t1 join t t2 on t1.id = t2.id join t t3 on t1.id = t3.id join t t4 on t1.id = t4.id join t t5 on t1.id = t5.id join t t6 on t1.id = t6.id join t t7 on t1.id = t7.id join t t8 on t1.id = t8.id; stream_plan: |- StreamMaterialize { columns: [count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum0(count)] } + └─StreamProject { exprs: [sum0(count)], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count] } 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 ce98b8bea75c9..3b96806aabc7a 100644 --- a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml @@ -869,7 +869,7 @@ └─BatchScan { table: order_line, columns: [order_line.ol_amount, order_line.ol_delivery_d, order_line.ol_quantity], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(order_line.ol_amount))] } + └─StreamProject { exprs: [sum(sum(order_line.ol_amount))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(order_line.ol_amount)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(order_line.ol_amount)] } @@ -880,7 +880,7 @@ Fragment 0 StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [sum(sum(order_line.ol_amount))] } + └── StreamProject { exprs: [sum(sum(order_line.ol_amount))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum(sum(order_line.ol_amount)), count] } ├── tables: [ SimpleAggState: 0 ] └── StreamExchange Single from 1 @@ -1940,7 +1940,7 @@ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum(stock.s_order_cnt))::Decimal * 0.005:Decimal) as $expr3] } + └─StreamProject { exprs: [(sum(sum(stock.s_order_cnt))::Decimal * 0.005:Decimal) as $expr3], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(stock.s_order_cnt)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(stock.s_order_cnt)] } @@ -2008,7 +2008,7 @@ └── BatchPlanNode Fragment 7 - StreamProject { exprs: [(sum(sum(stock.s_order_cnt))::Decimal * 0.005:Decimal) as $expr3] } + StreamProject { exprs: [(sum(sum(stock.s_order_cnt))::Decimal * 0.005:Decimal) as $expr3], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum(sum(stock.s_order_cnt)), count] } { tables: [ SimpleAggState: 14 ] } └── StreamExchange Single from 8 @@ -2265,7 +2265,7 @@ └─BatchScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_delivery_d], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [promo_revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / (1:Decimal + sum(sum(order_line.ol_amount)))) as $expr2] } + └─StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / (1:Decimal + sum(sum(order_line.ol_amount)))) as $expr2], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum($expr1)), sum(sum(order_line.ol_amount)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum($expr1), sum(order_line.ol_amount)] } @@ -2279,11 +2279,9 @@ └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], stream_scan_type: ArrangementBackfill, stream_key: [item.i_id], pk: [i_id], dist: UpstreamHashShard(item.i_id) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [promo_revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / (1:Decimal + sum(sum(order_line.ol_amount)))) as $expr2] } - └── StreamSimpleAgg { aggs: [sum(sum($expr1)), sum(sum(order_line.ol_amount)), count] } - ├── tables: [ SimpleAggState: 0 ] + StreamMaterialize { columns: [promo_revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / (1:Decimal + sum(sum(order_line.ol_amount)))) as $expr2], noop_update_hint: true } + └── StreamSimpleAgg { aggs: [sum(sum($expr1)), sum(sum(order_line.ol_amount)), count] } { tables: [ SimpleAggState: 0 ] } └── StreamExchange Single from 1 Fragment 1 @@ -2358,7 +2356,7 @@ │ └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no::Int64 as $expr1, revenue1.supplier_no] } │ └─StreamTableScan { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue], stream_scan_type: ArrangementBackfill, stream_key: [revenue1.supplier_no], pk: [supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } └─StreamExchange { dist: HashShard(max(max(revenue1.total_revenue))) } - └─StreamProject { exprs: [max(max(revenue1.total_revenue))] } + └─StreamProject { exprs: [max(max(revenue1.total_revenue))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(max(revenue1.total_revenue)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [max(revenue1.total_revenue), count] } @@ -2396,7 +2394,7 @@ └── BatchPlanNode Fragment 5 - StreamProject { exprs: [max(max(revenue1.total_revenue))] } + StreamProject { exprs: [max(max(revenue1.total_revenue))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [max(max(revenue1.total_revenue)), count] } { tables: [ SimpleAggState: 11, SimpleAggCall0: 10 ] } └── StreamExchange Single from 6 @@ -2628,7 +2626,7 @@ └─BatchScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [avg_yearly], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [(sum(sum(order_line.ol_amount)) / 2.0:Decimal) as $expr3] } + └─StreamProject { exprs: [(sum(sum(order_line.ol_amount)) / 2.0:Decimal) as $expr3], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(order_line.ol_amount)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(order_line.ol_amount)] } @@ -2651,9 +2649,8 @@ Fragment 0 StreamMaterialize { columns: [avg_yearly], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [(sum(sum(order_line.ol_amount)) / 2.0:Decimal) as $expr3] } - └── StreamSimpleAgg { aggs: [sum(sum(order_line.ol_amount)), count] } - ├── tables: [ SimpleAggState: 0 ] + └── StreamProject { exprs: [(sum(sum(order_line.ol_amount)) / 2.0:Decimal) as $expr3], noop_update_hint: true } + └── StreamSimpleAgg { aggs: [sum(sum(order_line.ol_amount)), count] } { tables: [ SimpleAggState: 0 ] } └── StreamExchange Single from 1 Fragment 1 @@ -2861,7 +2858,7 @@ └─BatchScan { table: order_line, columns: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_quantity], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(order_line.ol_amount))] } + └─StreamProject { exprs: [sum(sum(order_line.ol_amount))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(order_line.ol_amount)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(order_line.ol_amount)] } @@ -2880,7 +2877,7 @@ Fragment 0 StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [sum(sum(order_line.ol_amount))] } + └── StreamProject { exprs: [sum(sum(order_line.ol_amount))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum(sum(order_line.ol_amount)), count] } ├── tables: [ SimpleAggState: 0 ] └── StreamExchange Single from 1 @@ -3410,7 +3407,7 @@ │ └─StreamProject { exprs: [orders.o_c_id, orders.o_w_id, orders.o_d_id, orders.o_id] } │ └─StreamTableScan { table: orders, columns: [orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_id], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum(customer.c_balance)) / sum0(count(customer.c_balance))::Decimal) as $expr1] } + └─StreamProject { exprs: [(sum(sum(customer.c_balance)) / sum0(count(customer.c_balance))::Decimal) as $expr1], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(customer.c_balance)), sum0(count(customer.c_balance)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(customer.c_balance), count(customer.c_balance)] } diff --git a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml index abbc0aae184e0..ce97db3c0d33e 100644 --- a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml @@ -74,7 +74,7 @@ └─StreamProject { exprs: [Sqrt($expr5) as $expr6, Case((sum0(count(t.v)) <= 1:Int64), null:Decimal, Sqrt(($expr4 / (sum0(count(t.v)) - 1:Int64)::Decimal))) as $expr7, $expr5, Case((sum0(count(t.v)) <= 1:Int64), null:Decimal, ($expr4 / (sum0(count(t.v)) - 1:Int64)::Decimal)) as $expr8] } └─StreamProject { exprs: [sum(sum($expr1)), sum(sum(t.v)), sum0(count(t.v)), ($expr4 / $expr3) as $expr5, $expr4] } └─StreamProject { exprs: [sum(sum($expr1)), sum(sum(t.v)), sum0(count(t.v)), (sum(sum($expr1))::Decimal - (($expr2 * $expr2) / $expr3)) as $expr4, $expr3] } - └─StreamProject { exprs: [sum(sum($expr1)), sum(sum(t.v)), sum0(count(t.v)), sum(sum(t.v))::Decimal as $expr2, sum0(count(t.v))::Decimal as $expr3] } + └─StreamProject { exprs: [sum(sum($expr1)), sum(sum(t.v)), sum0(count(t.v)), sum(sum(t.v))::Decimal as $expr2, sum0(count(t.v))::Decimal as $expr3], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum($expr1)), sum(sum(t.v)), sum0(count(t.v)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum($expr1), sum(t.v), count(t.v)] } 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 89aea24f2bd80..922723851944b 100644 --- a/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml @@ -18,7 +18,7 @@ └─StreamDynamicFilter { predicate: (t1.v1 > max(max(t2.v2))), output: [t1.v1, t1._row_id] } ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [max(max(t2.v2))] } + └─StreamProject { exprs: [max(max(t2.v2))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [max(t2.v2), count] } @@ -77,7 +77,7 @@ ├─StreamProject { exprs: [t1.v1, (t1.v1 + t1.v1) as $expr1, t1._row_id] } │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [max(max(t2.v2))] } + └─StreamProject { exprs: [max(max(t2.v2))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [max(t2.v2), count] } @@ -129,7 +129,7 @@ ├─StreamExchange { dist: HashShard(t1.v1) } │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard(max(max(t2.v2))) } - └─StreamProject { exprs: [max(max(t2.v2))] } + └─StreamProject { exprs: [max(max(t2.v2))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [max(t2.v2), count] } @@ -153,7 +153,7 @@ ├─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] } │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [max(max(t2.v2))] } + └─StreamProject { exprs: [max(max(t2.v2))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [max(t2.v2), count] } @@ -169,7 +169,7 @@ └─StreamDynamicFilter { predicate: (t1.v1 > max(max(t2.v2))), output: [t1.v1, t1._row_id] } ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [max(max(t2.v2))] } + └─StreamProject { exprs: [max(max(t2.v2))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [max(t2.v2), count] } @@ -191,7 +191,7 @@ └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output: [t1.v1, t1._row_id] } ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(2:Int32 * max(max(t2.v2))) as $expr1] } + └─StreamProject { exprs: [(2:Int32 * max(max(t2.v2))) as $expr1], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [max(t2.v2), count] } @@ -220,7 +220,7 @@ │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } │ └─StreamExchange { dist: Broadcast } │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [max(max(t2.v2))] } + │ └─StreamProject { exprs: [max(max(t2.v2))], noop_update_hint: true } │ └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] } │ └─StreamExchange { dist: Single } │ └─StreamHashAgg { group_key: [_vnode], aggs: [max(t2.v2), count] } @@ -229,7 +229,7 @@ └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [(max(max(t2.v2)) + 5:Int32) as $expr1] } └─StreamShare { id: 6 } - └─StreamProject { exprs: [max(max(t2.v2))] } + └─StreamProject { exprs: [max(max(t2.v2))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [max(t2.v2), count] } 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 815890d6a73b8..e7a1951ffde54 100644 --- a/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml @@ -180,7 +180,7 @@ where path_val = t1.id; stream_plan: |- StreamMaterialize { columns: [array_agg], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [array_agg(t1.n order_by($expr1 ASC))] } + └─StreamProject { exprs: [array_agg(t1.n order_by($expr1 ASC))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [array_agg(t1.n order_by($expr1 ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t1.n, (projected_row_id + 1:Int64) as $expr1, t1._row_id, t2.p, t2.p, t2.d, t2.d, projected_row_id, t1.id, t2._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/limit.yaml b/src/frontend/planner_test/tests/testdata/output/limit.yaml index 22fb2add9d30c..500dbe1dd5824 100644 --- a/src/frontend/planner_test/tests/testdata/output/limit.yaml +++ b/src/frontend/planner_test/tests/testdata/output/limit.yaml @@ -131,7 +131,7 @@ stream_plan: |- StreamMaterialize { columns: [count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamTopN { order: [sum0(count) ASC], limit: 1, offset: 0 } - └─StreamProject { exprs: [sum0(count)] } + └─StreamProject { exprs: [sum0(count)], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count] } @@ -154,7 +154,7 @@ stream_plan: |- StreamMaterialize { columns: [count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamTopN { order: [sum0(count) ASC], limit: 1, offset: 0 } - └─StreamProject { exprs: [sum0(count)] } + └─StreamProject { exprs: [sum0(count)], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count] } diff --git a/src/frontend/planner_test/tests/testdata/output/mv_column_name.yaml b/src/frontend/planner_test/tests/testdata/output/mv_column_name.yaml index 3db4034336315..91352992bb17a 100644 --- a/src/frontend/planner_test/tests/testdata/output/mv_column_name.yaml +++ b/src/frontend/planner_test/tests/testdata/output/mv_column_name.yaml @@ -63,7 +63,7 @@ select count(*), max(a) from t; stream_plan: |- StreamMaterialize { columns: [count, max], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum0(count), max(max(t.a))] } + └─StreamProject { exprs: [sum0(count), max(max(t.a))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(count), max(max(t.a)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [count, max(t.a)] } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index d6b90da0a8c1a..1ea7349b24769 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -1891,7 +1891,7 @@ │ └─StreamExchange { dist: HashShard(bid.auction) } │ └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], stream_scan_type: ArrangementBackfill, stream_key: [bid._row_id], pk: [_row_id], dist: UpstreamHashShard(bid._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(bid.auction))) as $expr1] } + └─StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(bid.auction))) as $expr1], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(bid.auction)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum0(count), count(bid.auction)] } @@ -1926,7 +1926,7 @@ └── BatchPlanNode Fragment 3 - StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(bid.auction))) as $expr1] } + StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(bid.auction))) as $expr1], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(bid.auction)), count] } { tables: [ SimpleAggState: 9 ] } └── StreamExchange Single from 4 @@ -2331,7 +2331,7 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.price, bid.date_time], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [min(min(max(bid.price)))] } + └─StreamProject { exprs: [min(min(max(bid.price)))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [min(min(max(bid.price))), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [$expr1], aggs: [min(max(bid.price)), count] } @@ -2348,7 +2348,7 @@ Fragment 0 StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [min(min(max(bid.price)))] } + └── StreamProject { exprs: [min(min(max(bid.price)))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [min(min(max(bid.price))), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] └── StreamExchange Single from 1 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 35713c9682a35..15e1647721d53 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -1878,7 +1878,7 @@ │ └─StreamRowIdGen { row_id_index: 7 } │ └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(auction))) as $expr1] } + └─StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(auction))) as $expr1], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(auction)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum0(count), count(auction)] } @@ -1915,7 +1915,7 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 8 ] } Fragment 4 - StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(auction))) as $expr1] } + StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(auction))) as $expr1], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(auction)), count] } { tables: [ SimpleAggState: 9 ] } └── StreamExchange Single from 5 @@ -2277,7 +2277,7 @@ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [min(min(max(price)))] } + └─StreamProject { exprs: [min(min(max(price)))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [min(min(max(price))), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [$expr1], aggs: [min(max(price)), count] } @@ -2296,7 +2296,7 @@ Fragment 0 StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [min(min(max(price)))] } + └── StreamProject { exprs: [min(min(max(price)))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [min(min(max(price))), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] └── StreamExchange Single from 1 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 d5d948e5b507c..0658030573dd1 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 @@ -1517,7 +1517,7 @@ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } │ └─StreamNow { output: [now] } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum0(sum0(count)) / sum0(count($expr5))) as $expr6] } + └─StreamProject { exprs: [(sum0(sum0(count)) / sum0(count($expr5))) as $expr6], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count($expr5)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum0(count), count($expr5)] } @@ -1578,7 +1578,7 @@ └── StreamNow { output: [now] } { tables: [ Now: 10 ] } Fragment 6 - StreamProject { exprs: [(sum0(sum0(count)) / sum0(count($expr5))) as $expr6] } + StreamProject { exprs: [(sum0(sum0(count)) / sum0(count($expr5))) as $expr6], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count($expr5)), count] } { tables: [ SimpleAggState: 11 ] } └── StreamExchange Single from 7 @@ -2000,7 +2000,7 @@ ) stream_plan: |- StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [min(min(max($expr7)))] } + └─StreamProject { exprs: [min(min(max($expr7)))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [min(min(max($expr7))), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [$expr9], aggs: [min(max($expr7)), count] } @@ -2035,7 +2035,7 @@ Fragment 0 StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [min(min(max($expr7)))] } + └── StreamProject { exprs: [min(min(max($expr7)))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [min(min(max($expr7))), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] └── StreamExchange Single from 1 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 f065ba33c252d..c577b72eaafd6 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -2059,7 +2059,7 @@ │ └─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: Broadcast } - └─StreamProject { exprs: [(sum0(sum0(count)) / sum0(count($expr4))) as $expr5] } + └─StreamProject { exprs: [(sum0(sum0(count)) / sum0(count($expr4))) as $expr5], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count($expr4)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum0(count), count($expr4)] } @@ -2111,7 +2111,7 @@ └── StreamExchange NoShuffle from 2 Fragment 5 - StreamProject { exprs: [(sum0(sum0(count)) / sum0(count($expr4))) as $expr5] } + StreamProject { exprs: [(sum0(sum0(count)) / sum0(count($expr4))) as $expr5], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count($expr4)), count] } { tables: [ SimpleAggState: 9 ] } └── StreamExchange Single from 6 @@ -2533,7 +2533,7 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [min(min(max($expr5)))] } + └─StreamProject { exprs: [min(min(max($expr5)))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [min(min(max($expr5))), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [$expr6], aggs: [min(max($expr5)), count] } @@ -2564,7 +2564,7 @@ Fragment 0 StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [min(min(max($expr5)))] } + └── StreamProject { exprs: [min(min(max($expr5)))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [min(min(max($expr5))), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] └── StreamExchange Single from 1 diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index 2cf3aee9fe043..7962e4724f347 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -33,7 +33,7 @@ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [cnt], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum0(count)] } + └─StreamProject { exprs: [sum0(count)], noop_update_hint: true } └─StreamSimpleAgg [append_only] { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count] } @@ -155,7 +155,7 @@ ├─StreamExchange { dist: HashShard(0:Int32) } │ └─StreamProject { exprs: [sum0(count), 0:Int32] } │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [sum0(count)] } + │ └─StreamProject { exprs: [sum0(count)], noop_update_hint: true } │ └─StreamSimpleAgg { aggs: [sum0(count), count] } │ └─StreamExchange { dist: Single } │ └─StreamStatelessSimpleAgg { aggs: [count] } @@ -163,7 +163,7 @@ └─StreamExchange { dist: HashShard(1:Int32) } └─StreamProject { exprs: [sum0(count), 1:Int32] } └─StreamShare { id: 5 } - └─StreamProject { exprs: [sum0(count)] } + └─StreamProject { exprs: [sum0(count)], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count] } @@ -176,13 +176,13 @@ StreamMaterialize { columns: [count, $src(hidden)], stream_key: [$src], pk_columns: [$src], pk_conflict: NoCheck } └─StreamUnion { all: true } ├─StreamExchange { dist: HashShard(0:Int32) } - │ └─StreamProject { exprs: [sum0(count), 0:Int32] } + │ └─StreamProject { exprs: [sum0(count), 0:Int32], noop_update_hint: true } │ └─StreamSimpleAgg { aggs: [sum0(count), count] } │ └─StreamExchange { dist: Single } │ └─StreamStatelessSimpleAgg { aggs: [count] } │ └─StreamTableScan { table: t, columns: [t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: HashShard(1:Int32) } - └─StreamProject { exprs: [sum0(count), 1:Int32] } + └─StreamProject { exprs: [sum0(count), 1:Int32], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count] } @@ -195,7 +195,7 @@ select count(*) cnt from auction A join auction B on A.id = B.id; stream_plan: |- StreamMaterialize { columns: [cnt], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum0(count)] } + └─StreamProject { exprs: [sum0(count)], noop_update_hint: true } └─StreamSimpleAgg [append_only] { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count] } @@ -216,7 +216,7 @@ with cte as (select a, sum(b) sum from t group by a) select count(*) from cte c1 join cte c2 on c1.a = c2.a; stream_plan: |- StreamMaterialize { columns: [count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum0(count)] } + └─StreamProject { exprs: [sum0(count)], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count] } @@ -235,7 +235,7 @@ Fragment 0 StreamMaterialize { columns: [count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [sum0(count)] } + └── StreamProject { exprs: [sum0(count)], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum0(count), count] } { tables: [ SimpleAggState: 0 ] } └── StreamExchange Single from 1 diff --git a/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml index 48caec86bd940..0b7d7d7f2f2bf 100644 --- a/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml @@ -17,13 +17,13 @@ └─BatchScan { table: s, columns: [s.v], distribution: Single } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(s.v)] } + └─StreamProject { exprs: [max(s.v)], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(s.v), count] } └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } - └── StreamProject { exprs: [max(s.v)] } + └── StreamProject { exprs: [max(s.v)], noop_update_hint: true } └── StreamSimpleAgg { aggs: [max(s.v), count] } { tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] } └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } ├── tables: [ StreamScan: 2 ] @@ -55,13 +55,13 @@ └─BatchScan { table: s, columns: [s.v], distribution: Single } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(s.v)] } + └─StreamProject { exprs: [sum(s.v)], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(s.v), count] } └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } - └── StreamProject { exprs: [sum(s.v)] } + └── StreamProject { exprs: [sum(s.v)], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum(s.v), count] } { tables: [ SimpleAggState: 0 ] } └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } ├── tables: [ StreamScan: 1 ] @@ -91,13 +91,13 @@ └─BatchScan { table: s, columns: [s.v], distribution: Single } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [count(s.v)] } + └─StreamProject { exprs: [count(s.v)], noop_update_hint: true } └─StreamSimpleAgg { aggs: [count(s.v), count] } └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } - └── StreamProject { exprs: [count(s.v)] } + └── StreamProject { exprs: [count(s.v)], noop_update_hint: true } └── StreamSimpleAgg { aggs: [count(s.v), count] } { tables: [ SimpleAggState: 0 ] } └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } ├── tables: [ StreamScan: 1 ] @@ -128,14 +128,14 @@ └─BatchScan { table: s, columns: [s.v, s.s], distribution: Single } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } + └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC)), count] } └─StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } └─StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } - └── StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } + └── StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC)), count] } { tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] } └── StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } └── StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } @@ -169,7 +169,7 @@ └─BatchScan { table: t, columns: [t.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(max(t.v))] } + └─StreamProject { exprs: [max(max(t.v))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(max(t.v)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [max(t.v), count] } @@ -179,7 +179,7 @@ Fragment 0 StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [max(max(t.v))] } + └── StreamProject { exprs: [max(max(t.v))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [max(max(t.v)), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] └── StreamExchange Single from 1 @@ -223,7 +223,7 @@ select max(v) as a1 from AO; stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(max(ao.v))] } + └─StreamProject { exprs: [max(max(ao.v))], noop_update_hint: true } └─StreamSimpleAgg [append_only] { aggs: [max(max(ao.v)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [max(ao.v)] } @@ -232,7 +232,7 @@ Fragment 0 StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [max(max(ao.v))] } + └── StreamProject { exprs: [max(max(ao.v))], noop_update_hint: true } └── StreamSimpleAgg [append_only] { aggs: [max(max(ao.v)), count] } ├── tables: [ SimpleAggState: 0 ] └── StreamExchange Single from 1 @@ -268,7 +268,7 @@ └─BatchScan { table: t, columns: [t.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(t.v))] } + └─StreamProject { exprs: [sum(sum(t.v))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(t.v)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(t.v)] } @@ -277,7 +277,7 @@ Fragment 0 StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [sum(sum(t.v))] } + └── StreamProject { exprs: [sum(sum(t.v))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum(sum(t.v)), count] } ├── tables: [ SimpleAggState: 0 ] └── StreamExchange Single from 1 @@ -308,7 +308,7 @@ select sum(v) as a1 from AO; stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum(ao.v))] } + └─StreamProject { exprs: [sum(sum(ao.v))], noop_update_hint: true } └─StreamSimpleAgg [append_only] { aggs: [sum(sum(ao.v)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(ao.v)] } @@ -317,7 +317,7 @@ Fragment 0 StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [sum(sum(ao.v))] } + └── StreamProject { exprs: [sum(sum(ao.v))], noop_update_hint: true } └── StreamSimpleAgg [append_only] { aggs: [sum(sum(ao.v)), count] } ├── tables: [ SimpleAggState: 0 ] └── StreamExchange Single from 1 @@ -353,7 +353,7 @@ └─BatchScan { table: t, columns: [t.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum0(count(t.v))] } + └─StreamProject { exprs: [sum0(count(t.v))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum0(count(t.v)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count(t.v)] } @@ -362,7 +362,7 @@ Fragment 0 StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [sum0(count(t.v))] } + └── StreamProject { exprs: [sum0(count(t.v))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum0(count(t.v)), count] } ├── tables: [ SimpleAggState: 0 ] └── StreamExchange Single from 1 @@ -393,7 +393,7 @@ select count(v) as a1 from AO; stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum0(count(ao.v))] } + └─StreamProject { exprs: [sum0(count(ao.v))], noop_update_hint: true } └─StreamSimpleAgg [append_only] { aggs: [sum0(count(ao.v)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count(ao.v)] } @@ -402,7 +402,7 @@ Fragment 0 StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [sum0(count(ao.v))] } + └── StreamProject { exprs: [sum0(count(ao.v))], noop_update_hint: true } └── StreamSimpleAgg [append_only] { aggs: [sum0(count(ao.v)), count] } ├── tables: [ SimpleAggState: 0 ] └── StreamExchange Single from 1 @@ -438,7 +438,7 @@ └─BatchScan { table: t, columns: [t.o, t.s], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └─StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.s, ',':Varchar, t.o, t._row_id] } @@ -447,7 +447,7 @@ Fragment 0 StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └── StreamProject { exprs: [string_agg(t.s, ',':Varchar order_by(t.o ASC))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] └── StreamExchange Single from 1 @@ -480,7 +480,7 @@ select string_agg(s, ',' order by o) as a1 from AO; stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + └─StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC))], noop_update_hint: true } └─StreamSimpleAgg [append_only] { aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [ao.s, ',':Varchar, ao.o, ao._row_id] } @@ -489,7 +489,7 @@ Fragment 0 StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + └── StreamProject { exprs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC))], noop_update_hint: true } └── StreamSimpleAgg [append_only] { aggs: [string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] └── StreamExchange Single from 1 @@ -527,7 +527,7 @@ └─BatchScan { table: t, columns: [t.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1, a2], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(max(t.v)), sum0(count(t.v))] } + └─StreamProject { exprs: [max(max(t.v)), sum0(count(t.v))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(max(t.v)), sum0(count(t.v)), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [max(t.v), count(t.v), count] } @@ -537,7 +537,7 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [max(max(t.v)), sum0(count(t.v))] } + └── StreamProject { exprs: [max(max(t.v)), sum0(count(t.v))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [max(max(t.v)), sum0(count(t.v)), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] └── StreamExchange Single from 1 @@ -587,7 +587,7 @@ select max(v) as a1, count(v) as a2 from AO; stream_plan: |- StreamMaterialize { columns: [a1, a2], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(max(ao.v)), sum0(count(ao.v))] } + └─StreamProject { exprs: [max(max(ao.v)), sum0(count(ao.v))], noop_update_hint: true } └─StreamSimpleAgg [append_only] { aggs: [max(max(ao.v)), sum0(count(ao.v)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [max(ao.v), count(ao.v)] } @@ -596,7 +596,7 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [max(max(ao.v)), sum0(count(ao.v))] } + └── StreamProject { exprs: [max(max(ao.v)), sum0(count(ao.v))], noop_update_hint: true } └── StreamSimpleAgg [append_only] { aggs: [max(max(ao.v)), sum0(count(ao.v)), count] } ├── tables: [ SimpleAggState: 0 ] └── StreamExchange Single from 1 @@ -632,7 +632,7 @@ └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1, a2], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └─StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.v, t.s, ',':Varchar, t.o, t._row_id] } @@ -641,7 +641,7 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └── StreamProject { exprs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [count(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall1: 0 ] └── StreamExchange Single from 1 @@ -679,7 +679,7 @@ select count(v) as a1, string_agg(s, ',' order by o) as a2 from AO; stream_plan: |- StreamMaterialize { columns: [a1, a2], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + └─StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))], noop_update_hint: true } └─StreamSimpleAgg [append_only] { aggs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [ao.v, ao.s, ',':Varchar, ao.o, ao._row_id] } @@ -688,7 +688,7 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + └── StreamProject { exprs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))], noop_update_hint: true } └── StreamSimpleAgg [append_only] { aggs: [count(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall1: 0 ] └── StreamExchange Single from 1 @@ -726,7 +726,7 @@ └─BatchScan { table: t, columns: [t.v, t.o, t.s], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1, a2], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └─StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [t.v, t.s, ',':Varchar, t.o, t._row_id] } @@ -735,7 +735,7 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))] } + └── StreamProject { exprs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [max(t.v), string_agg(t.s, ',':Varchar order_by(t.o ASC)), count] } ├── tables: [ SimpleAggState: 2, SimpleAggCall0: 0, SimpleAggCall1: 1 ] └── StreamExchange Single from 1 @@ -770,7 +770,7 @@ select max(v) as a1, string_agg(s, ',' order by o) as a2 from AO; stream_plan: |- StreamMaterialize { columns: [a1, a2], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + └─StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))], noop_update_hint: true } └─StreamSimpleAgg [append_only] { aggs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [ao.v, ao.s, ',':Varchar, ao.o, ao._row_id] } @@ -779,7 +779,7 @@ Fragment 0 StreamMaterialize { columns: [a1, a2], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))] } + └── StreamProject { exprs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC))], noop_update_hint: true } └── StreamSimpleAgg [append_only] { aggs: [max(ao.v), string_agg(ao.s, ',':Varchar order_by(ao.o ASC)), count] } ├── tables: [ SimpleAggState: 1, SimpleAggCall1: 0 ] └── StreamExchange Single from 1 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 5cdfdf6cf45ea..49d14526af640 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml @@ -61,7 +61,7 @@ select count(*) from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 where a2 < 10; stream_plan: |- StreamMaterialize { columns: [count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum0(count)] } + └─StreamProject { exprs: [sum0(count)], noop_update_hint: true } └─StreamSimpleAgg [append_only] { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count] } diff --git a/src/frontend/planner_test/tests/testdata/output/tpch.yaml b/src/frontend/planner_test/tests/testdata/output/tpch.yaml index 3c43faa8d2494..dbb7a5c08a62a 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch.yaml @@ -1160,7 +1160,7 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum($expr1))] } + └─StreamProject { exprs: [sum(sum($expr1))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum($expr1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum($expr1)] } @@ -1171,7 +1171,7 @@ Fragment 0 StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [sum(sum($expr1))] } + └── StreamProject { exprs: [sum(sum($expr1))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum(sum($expr1)), count] } { tables: [ SimpleAggState: 0 ] } └── StreamExchange Single from 1 @@ -2389,7 +2389,7 @@ │ └─StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } + └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum($expr2)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum($expr2)] } @@ -2461,7 +2461,7 @@ └── BatchPlanNode Fragment 8 - StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } + StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum(sum($expr2)), count] } { tables: [ SimpleAggState: 16 ] } └── StreamExchange Single from 9 @@ -2818,7 +2818,7 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [promo_revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / sum(sum($expr2))) as $expr3] } + └─StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / sum(sum($expr2))) as $expr3], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum($expr1)), sum(sum($expr2)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum($expr1), sum($expr2)] } @@ -2834,9 +2834,8 @@ Fragment 0 StreamMaterialize { columns: [promo_revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / sum(sum($expr2))) as $expr3] } - └── StreamSimpleAgg { aggs: [sum(sum($expr1)), sum(sum($expr2)), count] } - ├── tables: [ SimpleAggState: 0 ] + └── StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / sum(sum($expr2))) as $expr3], noop_update_hint: true } + └── StreamSimpleAgg { aggs: [sum(sum($expr1)), sum(sum($expr2)), count] } { tables: [ SimpleAggState: 0 ] } └── StreamExchange Single from 1 Fragment 1 @@ -2966,7 +2965,7 @@ │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } - └─StreamProject { exprs: [max(max(sum($expr1)))] } + └─StreamProject { exprs: [max(max(sum($expr1)))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } └─StreamExchange { dist: Single } └─StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } @@ -3020,7 +3019,7 @@ └── BatchPlanNode Fragment 6 - StreamProject { exprs: [max(max(sum($expr1)))] } + StreamProject { exprs: [max(max(sum($expr1)))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } { tables: [ SimpleAggState: 14, SimpleAggCall0: 13 ] } └── StreamExchange Single from 7 @@ -3296,7 +3295,7 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [avg_yearly], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [(sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal) as $expr2] } + └─StreamProject { exprs: [(sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal) as $expr2], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } @@ -3319,9 +3318,8 @@ Fragment 0 StreamMaterialize { columns: [avg_yearly], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [(sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal) as $expr2] } - └── StreamSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice)), count] } - ├── tables: [ SimpleAggState: 0 ] + └── StreamProject { exprs: [(sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal) as $expr2], noop_update_hint: true } + └── StreamSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice)), count] } { tables: [ SimpleAggState: 0 ] } └── StreamExchange Single from 1 Fragment 1 @@ -3672,7 +3670,7 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipinstruct, lineitem.l_shipmode], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(sum($expr1))] } + └─StreamProject { exprs: [sum(sum($expr1))], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum($expr1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum($expr1)] } @@ -3690,7 +3688,7 @@ Fragment 0 StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [sum(sum($expr1))] } + └── StreamProject { exprs: [sum(sum($expr1))], noop_update_hint: true } └── StreamSimpleAgg { aggs: [sum(sum($expr1)), count] } { tables: [ SimpleAggState: 0 ] } └── StreamExchange Single from 1 @@ -4342,7 +4340,7 @@ │ └─StreamExchange { dist: HashShard(orders.o_custkey) } │ └─StreamTableScan { table: orders, columns: [orders.o_custkey, orders.o_orderkey], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))::Decimal) as $expr1] } + └─StreamProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))::Decimal) as $expr1], noop_update_hint: true } └─StreamSimpleAgg { aggs: [sum(sum(customer.c_acctbal)), sum0(count(customer.c_acctbal)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(customer.c_acctbal), count(customer.c_acctbal)] } diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index 1e11e390edfd8..f00ff35992b43 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -69,6 +69,7 @@ pub async fn replace_table_with_definition( Ok(()) } +/// Used in auto schema change process pub async fn get_new_table_definition_for_cdc_table( session: &Arc, table_name: ObjectName, diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index f0c1ed074ed1e..7ef118891865e 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -874,6 +874,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock::SINK_NAME => hashmap!( Format::Plain => vec![Encode::Parquet], ), + FileSink::::SINK_NAME => hashmap!( + Format::Plain => vec![Encode::Parquet], + ), FileSink::::SINK_NAME => hashmap!( Format::Plain => vec![Encode::Parquet], ), diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 0be418aecd9d6..3f83c25fb3efa 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -54,7 +54,7 @@ use risingwave_connector::source::nexmark::source::{get_event_data_types_with_na use risingwave_connector::source::test_source::TEST_CONNECTOR; pub use risingwave_connector::source::UPSTREAM_SOURCE_KEY; use risingwave_connector::source::{ - ConnectorProperties, GCS_CONNECTOR, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, + ConnectorProperties, AZBLOB_CONNECTOR, GCS_CONNECTOR, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, MQTT_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, }; @@ -362,7 +362,7 @@ pub(crate) async fn bind_columns_from_source( (Format::Native, Encode::Native) | (Format::Plain, Encode::Bytes) | (Format::DebeziumMongo, Encode::Json) => None, - (Format::Plain, Encode::Protobuf) => { + (Format::Plain, Encode::Protobuf) | (Format::Upsert, Encode::Protobuf) => { let (row_schema_location, use_schema_registry) = get_schema_location(&mut format_encode_options_to_consume)?; let protobuf_schema = ProtobufSchema { @@ -1064,7 +1064,10 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock vec![Encode::Csv, Encode::Json, Encode::Parquet], ), GCS_CONNECTOR => hashmap!( - Format::Plain => vec![Encode::Csv, Encode::Json], + Format::Plain => vec![Encode::Csv, Encode::Json, Encode::Parquet], + ), + AZBLOB_CONNECTOR => hashmap!( + Format::Plain => vec![Encode::Csv, Encode::Json, Encode::Parquet], ), POSIX_FS_CONNECTOR => hashmap!( Format::Plain => vec![Encode::Csv], diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 380212e1a92dd..825ee595b4b29 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -1015,6 +1015,7 @@ pub(super) async fn handle_create_table_plan( &constraints, connect_properties.clone(), wildcard_idx.is_some(), + None, ) .await?; @@ -1123,6 +1124,8 @@ async fn derive_schema_for_cdc_table( constraints: &Vec, connect_properties: WithOptionsSecResolved, need_auto_schema_map: bool, + // original table catalog available in auto schema change process + original_catalog: Option>, ) -> Result<(Vec, Vec)> { // read cdc table schema from external db or parsing the schema from SQL definitions if need_auto_schema_map { @@ -1154,10 +1157,20 @@ async fn derive_schema_for_cdc_table( table.pk_names().clone(), )) } else { - Ok(( - bind_sql_columns(column_defs)?, - bind_sql_pk_names(column_defs, constraints)?, - )) + let columns = bind_sql_columns(column_defs)?; + // For table created by `create table t (*)` the constraint is empty, we need to + // retrieve primary key names from original table catalog if available + let pk_names = if let Some(original_catalog) = original_catalog { + original_catalog + .pk + .iter() + .map(|x| original_catalog.columns[x.column_index].name().to_string()) + .collect() + } else { + bind_sql_pk_names(column_defs, constraints)? + }; + + Ok((columns, pk_names)) } } @@ -1328,6 +1341,7 @@ pub async fn generate_stream_graph_for_table( &constraints, connect_properties.clone(), false, + Some(original_catalog.clone()), ) .await?; diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index cf7025be90872..9e774628fc262 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -1386,12 +1386,19 @@ impl ToStream for LogicalAgg { panic!("the root PlanNode must be StreamHashAgg, StreamSimpleAgg, StreamGlobalApproxPercentile, or StreamRowMerge"); }; - if self.agg_calls().len() == n_final_agg_calls { + let is_hash_agg = !self.group_key().is_empty(); + // "Simple Agg" includes normal simple agg, as well as approx percentile simple 2 phase agg. + let is_simple_agg = !is_hash_agg; + if self.agg_calls().len() == n_final_agg_calls && is_hash_agg { // an existing `count(*)` is used as row count column in `StreamXxxAgg` Ok(plan) } else { - // a `count(*)` is appended, should project the output - assert_eq!(self.agg_calls().len() + 1, n_final_agg_calls); + // For hash agg, a `count(*)` is appended, should project the output. + // For simple agg, we output every epoch, so we will always add a project + // to filter out no-op updates, and we don't need the following assert. + if is_hash_agg { + assert_eq!(self.agg_calls().len() + 1, n_final_agg_calls); + } Ok(StreamProject::new(generic::Project::with_out_col_idx( plan, 0..self.schema().len(), @@ -1400,7 +1407,9 @@ impl ToStream for LogicalAgg { // Since it'll be pruned immediately in `StreamProject`, the update records are likely to be // no-op. So we set the hint to instruct the executor to eliminate them. // See https://github.com/risingwavelabs/risingwave/issues/17030. - .with_noop_update_hint(self.agg_calls().is_empty()) + // Further for simple agg, we also have to set the hint to eliminate no-op updates. + // Since we will output every epoch. + .with_noop_update_hint(self.agg_calls().is_empty() || is_simple_agg) .into()) } } diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index 65bfbe09c54b0..09e4cbc0bfa03 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -33,7 +33,9 @@ use risingwave_common::hash::table_distribution::TableDistribution; use risingwave_common::hash::{VirtualNode, WorkerSlotId, WorkerSlotMapping}; use risingwave_common::util::scan_range::ScanRange; use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; -use risingwave_connector::source::filesystem::opendal_source::{OpendalGcs, OpendalS3}; +use risingwave_connector::source::filesystem::opendal_source::{ + OpendalAzblob, OpendalGcs, OpendalS3, +}; use risingwave_connector::source::iceberg::{IcebergSplitEnumerator, IcebergTimeTravelInfo}; use risingwave_connector::source::kafka::KafkaSplitEnumerator; use risingwave_connector::source::reader::reader::build_opendal_fs_list_for_batch; @@ -330,6 +332,15 @@ impl SourceScanInfo { Ok(SourceScanInfo::Complete(res)) } + ConnectorProperties::Azblob(prop) => { + let lister: OpendalEnumerator = + OpendalEnumerator::new_azblob_source(*prop)?; + let stream = build_opendal_fs_list_for_batch(lister); + let batch_res: Vec<_> = stream.try_collect().await?; + let res = batch_res.into_iter().map(SplitImpl::Azblob).collect_vec(); + + Ok(SourceScanInfo::Complete(res)) + } ConnectorProperties::Iceberg(prop) => { let iceberg_enumerator = IcebergSplitEnumerator::new(*prop, SourceEnumeratorContext::dummy().into()) @@ -740,7 +751,9 @@ impl StageGraph { let task_parallelism = match &stage.source_info { Some(SourceScanInfo::Incomplete(source_fetch_info)) => { match source_fetch_info.connector { - ConnectorProperties::Gcs(_) | ConnectorProperties::OpendalS3(_) => (min( + ConnectorProperties::Gcs(_) + | ConnectorProperties::OpendalS3(_) + | ConnectorProperties::Azblob(_) => (min( complete_source_info.split_info().unwrap().len() as u32, (self.batch_parallelism / 2) as u32, )) diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 782ac8bfe1cd9..1e8e1c9641d8a 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -12,12 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::sync::Arc; use anyhow::anyhow; use rand::seq::SliceRandom; use rand::thread_rng; +use risingwave_common::catalog::ColumnCatalog; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_connector::sink::catalog::SinkId; use risingwave_meta::manager::{EventLogManagerRef, MetadataManager}; @@ -966,6 +967,46 @@ impl DdlService for DdlServiceImpl { .await?; for table in tables { + // Since we only support `ADD` and `DROP` column, we check whether the new columns and the original columns + // is a subset of the other. + let original_column_names: HashSet = HashSet::from_iter( + table + .columns + .iter() + .map(|col| ColumnCatalog::from(col.clone()).column_desc.name), + ); + let new_column_names: HashSet = HashSet::from_iter( + table_change + .columns + .iter() + .map(|col| ColumnCatalog::from(col.clone()).column_desc.name), + ); + if !(original_column_names.is_subset(&new_column_names) + || original_column_names.is_superset(&new_column_names)) + { + tracing::warn!(target: "auto_schema_change", + table_id = table.id, + cdc_table_id = table.cdc_table_id, + upstraem_ddl = table_change.upstream_ddl, + original_columns = ?original_column_names, + new_columns = ?new_column_names, + "New columns should be a subset or superset of the original columns, since only `ADD COLUMN` and `DROP COLUMN` is supported"); + return Err(Status::invalid_argument( + "New columns should be a subset or superset of the original columns", + )); + } + // skip the schema change if there is no change to original columns + if original_column_names == new_column_names { + tracing::warn!(target: "auto_schema_change", + table_id = table.id, + cdc_table_id = table.cdc_table_id, + upstraem_ddl = table_change.upstream_ddl, + original_columns = ?original_column_names, + new_columns = ?new_column_names, + "No change to columns, skipping the schema change"); + continue; + } + let latency_timer = self .meta_metrics .auto_schema_change_latency diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index 1229554032614..cfdf2bea0f322 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -49,16 +49,22 @@ pub struct DashboardService { pub type Service = Arc; pub(super) mod handlers { + use std::collections::HashMap; + use anyhow::Context; use axum::Json; use futures::future::join_all; use itertools::Itertools; + use risingwave_common::bail; + use risingwave_common::catalog::TableId; use risingwave_common_heap_profiling::COLLAPSED_SUFFIX; use risingwave_pb::catalog::table::TableType; use risingwave_pb::catalog::{PbDatabase, PbSchema, Sink, Source, Subscription, Table, View}; use risingwave_pb::common::{WorkerNode, WorkerType}; use risingwave_pb::meta::list_object_dependencies_response::PbObjectDependencies; - use risingwave_pb::meta::PbTableFragments; + use risingwave_pb::meta::{ + ActorIds, FragmentIdToActorIdMap, PbTableFragments, RelationIdInfos, + }; use risingwave_pb::monitor_service::{ GetBackPressureResponse, HeapProfilingResponse, ListHeapProfilingResponse, StackTraceResponse, @@ -211,6 +217,92 @@ pub(super) mod handlers { Ok(Json(table_fragments)) } + pub async fn get_relation_id_infos( + Extension(srv): Extension, + ) -> Result> { + let map = match &srv.metadata_manager { + MetadataManager::V1(mgr) => { + let core = mgr.fragment_manager.get_fragment_read_guard().await; + let table_fragments = core.table_fragments(); + let mut map = HashMap::new(); + for (id, tf) in table_fragments { + let mut fragment_id_to_actor_ids = HashMap::new(); + for (fragment_id, fragment) in &tf.fragments { + let actor_ids = fragment.actors.iter().map(|a| a.actor_id).collect_vec(); + fragment_id_to_actor_ids.insert(*fragment_id, ActorIds { ids: actor_ids }); + } + map.insert( + id.table_id, + FragmentIdToActorIdMap { + map: fragment_id_to_actor_ids, + }, + ); + } + map + } + MetadataManager::V2(mgr) => { + let table_fragments = mgr + .catalog_controller + .table_fragments() + .await + .map_err(err)?; + let mut map = HashMap::new(); + for (id, tf) in table_fragments { + let mut fragment_id_to_actor_ids = HashMap::new(); + for (fragment_id, fragment) in &tf.fragments { + let actor_ids = fragment.actors.iter().map(|a| a.actor_id).collect_vec(); + fragment_id_to_actor_ids.insert(*fragment_id, ActorIds { ids: actor_ids }); + } + map.insert( + id as u32, + FragmentIdToActorIdMap { + map: fragment_id_to_actor_ids, + }, + ); + } + map + } + }; + let relation_id_infos = RelationIdInfos { map }; + + Ok(Json(relation_id_infos)) + } + + pub async fn list_fragments_by_job_id( + Extension(srv): Extension, + Path(job_id): Path, + ) -> Result> { + let table_fragments = match &srv.metadata_manager { + MetadataManager::V1(mgr) => { + if let Some(tf) = mgr + .fragment_manager + .get_fragment_read_guard() + .await + .table_fragments() + .get(&TableId::new(job_id)) + { + tf.to_protobuf() + } else { + bail!("job_id {} not found", job_id) + } + } + MetadataManager::V2(mgr) => { + let mut table_fragments = mgr + .catalog_controller + .table_fragments() + .await + .map_err(err)?; + if let Some(tf) = table_fragments.remove(&(job_id as i32)) { + tf + } else { + bail!("job_id {} not found", job_id) + } + } + }; + + Ok(Json(table_fragments)) + } + pub async fn list_users(Extension(srv): Extension) -> Result>> { let users = match &srv.metadata_manager { MetadataManager::V1(mgr) => mgr.catalog_manager.list_users().await, @@ -428,6 +520,8 @@ impl DashboardService { let api_router = Router::new() .route("/clusters/:ty", get(list_clusters)) .route("/fragments2", get(list_fragments)) + .route("/fragments/job_id/:job_id", get(list_fragments_by_job_id)) + .route("/relation_id_infos", get(get_relation_id_infos)) .route("/views", get(list_views)) .route("/materialized_views", get(list_materialized_views)) .route("/tables", get(list_tables)) diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 13580ca49e001..1576d65487d09 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -21,7 +21,7 @@ use risingwave_common::catalog::{ColumnId, TableId}; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::types::ScalarRef; use risingwave_connector::source::filesystem::opendal_source::{ - OpendalGcs, OpendalPosixFs, OpendalS3, OpendalSource, + OpendalAzblob, OpendalGcs, OpendalPosixFs, OpendalS3, OpendalSource, }; use risingwave_connector::source::filesystem::OpendalFsSplit; use risingwave_connector::source::reader::desc::SourceDesc; @@ -110,6 +110,11 @@ impl FsFetchExecutor { OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?; SplitImpl::from(split) } + risingwave_connector::source::ConnectorProperties::Azblob(_) => { + let split: OpendalFsSplit = + OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?; + SplitImpl::from(split) + } risingwave_connector::source::ConnectorProperties::PosixFs(_) => { let split: OpendalFsSplit = OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?; diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index 8351e48023a05..39a458b28ff47 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -41,7 +41,7 @@ use crate::executor::prelude::*; use crate::executor::source::source_executor::WAIT_BARRIER_MULTIPLE_TIMES; use crate::executor::{AddMutation, UpdateMutation}; -#[derive(Clone, Debug, Deserialize, Serialize)] +#[derive(Clone, Debug, Deserialize, Serialize, PartialEq)] pub enum BackfillState { /// `None` means not started yet. It's the initial state. Backfilling(Option), @@ -138,30 +138,32 @@ pub struct SourceBackfillExecutorInner { } /// Local variables used in the backfill stage. +#[derive(Debug)] struct BackfillStage { - // stream: Option>, states: BackfillStates, - /// Note: the offsets are not updated. Should use `state`'s offset to update before using it. - unfinished_splits: Vec, + /// A copy of all splits (incl unfinished and finished ones) assigned to the actor. + /// + /// Note: the offsets are not updated. Should use `state`'s offset to update before using it (`get_latest_unfinished_splits`). + splits: Vec, } impl BackfillStage { /// Get unfinished splits with latest offsets according to the backfill states. - fn get_latest_unfinished_splits(&mut self) -> StreamExecutorResult<&Vec> { + fn get_latest_unfinished_splits(&self) -> StreamExecutorResult> { let mut unfinished_splits = Vec::new(); - for split in &mut self.unfinished_splits { + for split in &self.splits { let state = self.states.get(split.id().as_ref()).unwrap(); match state { BackfillState::Backfilling(Some(offset)) => { - split.update_in_place(offset.clone())?; - unfinished_splits.push(split.clone()); + let mut updated_split = split.clone(); + updated_split.update_in_place(offset.clone())?; + unfinished_splits.push(updated_split); } BackfillState::Backfilling(None) => unfinished_splits.push(split.clone()), BackfillState::SourceCachingUp(_) | BackfillState::Finished => {} } } - self.unfinished_splits = unfinished_splits; - Ok(&self.unfinished_splits) + Ok(unfinished_splits) } } @@ -262,39 +264,30 @@ impl SourceBackfillExecutorInner { self.backfill_state_store.init_epoch(barrier.epoch); let mut backfill_states: BackfillStates = HashMap::new(); - let mut unfinished_splits = Vec::new(); - for mut split in owned_splits { + + for split in &owned_splits { let split_id = split.id(); let backfill_state = self .backfill_state_store .try_recover_from_state_store(&split_id) - .await?; - match backfill_state { - None => { - backfill_states.insert(split_id, BackfillState::Backfilling(None)); - unfinished_splits.push(split); - } - Some(backfill_state) => { - match backfill_state { - BackfillState::Backfilling(ref offset) => { - if let Some(offset) = offset { - split.update_in_place(offset.clone())?; - } - unfinished_splits.push(split); - } - BackfillState::SourceCachingUp(_) | BackfillState::Finished => {} - } - backfill_states.insert(split_id, backfill_state); - } - } + .await? + .unwrap_or(BackfillState::Backfilling(None)); + backfill_states.insert(split_id, backfill_state); } - tracing::debug!(?backfill_states, "source backfill started"); + let mut backfill_stage = BackfillStage { + states: backfill_states, + splits: owned_splits, + }; + tracing::debug!(?backfill_stage, "source backfill started"); // Return the ownership of `stream_source_core` to the source executor. self.stream_source_core = core; let source_chunk_reader = self - .build_stream_source_reader(&source_desc, unfinished_splits.clone()) + .build_stream_source_reader( + &source_desc, + backfill_stage.get_latest_unfinished_splits()?, + ) .instrument_await("source_build_reader") .await?; @@ -336,11 +329,6 @@ impl SourceBackfillExecutorInner { yield Message::Barrier(barrier); - let mut backfill_stage = BackfillStage { - states: backfill_states, - unfinished_splits, - }; - if !self.backfill_finished(&backfill_stage.states).await? { let source_backfill_row_count = self .metrics @@ -382,7 +370,7 @@ impl SourceBackfillExecutorInner { let reader = self .build_stream_source_reader( &source_desc, - backfill_stage.get_latest_unfinished_splits()?.clone(), + backfill_stage.get_latest_unfinished_splits()?, ) .await?; @@ -464,7 +452,7 @@ impl SourceBackfillExecutorInner { let reader = self .build_stream_source_reader( &source_desc, - latest_unfinished_splits.clone(), + latest_unfinished_splits, ) .await?; @@ -681,19 +669,18 @@ impl SourceBackfillExecutorInner { stage: &mut BackfillStage, should_trim_state: bool, ) -> StreamExecutorResult { - let target_splits: HashMap<_, _> = target_splits - .into_iter() - .map(|split| (split.id(), split)) - .collect(); - - let mut target_state: HashMap = - HashMap::with_capacity(target_splits.len()); + let mut target_state: BackfillStates = HashMap::with_capacity(target_splits.len()); let mut split_changed = false; - - // Checks added splits - for (split_id, mut split) in target_splits { - if let Some(s) = stage.states.get(&split_id) { + // Take out old states (immutable, only used to build target_state and check for added/dropped splits). + // Will be set to target_state in the end. + let old_states = std::mem::take(&mut stage.states); + // Iterate over the target (assigned) splits + // - check if any new splits are added + // - build target_state + for split in target_splits { + let split_id = split.id(); + if let Some(s) = old_states.get(&split_id) { target_state.insert(split_id, s.clone()); } else { split_changed = true; @@ -705,19 +692,12 @@ impl SourceBackfillExecutorInner { match backfill_state { None => { // Newly added split. We don't need to backfill. + // Note that this branch is different from the initial barrier (BackfillState::Backfilling(None) there). target_state.insert(split_id, BackfillState::Finished); } Some(backfill_state) => { // Migrated split. Backfill if unfinished. - match backfill_state { - BackfillState::Backfilling(ref offset) => { - if let Some(offset) = offset { - split.update_in_place(offset.clone())?; - } - stage.unfinished_splits.push(split); - } - BackfillState::SourceCachingUp(_) | BackfillState::Finished => {} - } + // TODO: disallow online scaling during backfilling. target_state.insert(split_id, backfill_state); } } @@ -725,7 +705,7 @@ impl SourceBackfillExecutorInner { } // Checks dropped splits - for existing_split_id in stage.states.keys() { + for existing_split_id in old_states.keys() { if !target_state.contains_key(existing_split_id) { tracing::info!("split dropping detected: {}", existing_split_id); split_changed = true; @@ -733,10 +713,6 @@ impl SourceBackfillExecutorInner { } if split_changed { - stage - .unfinished_splits - .retain(|split| target_state.contains_key(split.id().as_ref())); - let dropped_splits = stage .states .extract_if(|split_id, _| !target_state.contains_key(split_id)) @@ -746,9 +722,11 @@ impl SourceBackfillExecutorInner { // trim dropped splits' state self.backfill_state_store.trim_state(dropped_splits).await?; } - tracing::info!(old_state=?stage.states, new_state=?target_state, "finish split change"); - stage.states = target_state; + tracing::info!(old_state=?old_states, new_state=?target_state, "finish split change"); + } else { + debug_assert_eq!(old_states, target_state); } + stage.states = target_state; Ok(split_changed) } diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index 7a08c2d2f512e..a8f6d588904be 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -16,7 +16,7 @@ use std::sync::Arc; use risingwave_common::catalog::TableId; use risingwave_connector::source::filesystem::opendal_source::{ - OpendalGcs, OpendalPosixFs, OpendalS3, + OpendalAzblob, OpendalGcs, OpendalPosixFs, OpendalS3, }; use risingwave_connector::source::reader::desc::SourceDescBuilder; use risingwave_connector::source::ConnectorProperties; @@ -104,6 +104,15 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { ) .boxed() } + risingwave_connector::source::ConnectorProperties::Azblob(_) => { + FsFetchExecutor::<_, OpendalAzblob>::new( + params.actor_context.clone(), + stream_source_core, + upstream, + source.rate_limit, + ) + .boxed() + } risingwave_connector::source::ConnectorProperties::PosixFs(_) => { FsFetchExecutor::<_, OpendalPosixFs>::new( params.actor_context.clone(),