diff --git a/e2e_test/sql_migration/check.slt b/e2e_test/sql_migration/check.slt index b4c97bba50bf1..c79721f1145e8 100644 --- a/e2e_test/sql_migration/check.slt +++ b/e2e_test/sql_migration/check.slt @@ -64,6 +64,14 @@ create materialized view mv3 as select * from mv2; statement ok REVOKE ALL PRIVILEGES ON ALL TABLES IN SCHEMA schema1 FROM user1; +query T +show secrets; +---- +secret_1 + +statement ok +drop secret secret_1; + statement error Permission denied drop source src; diff --git a/e2e_test/sql_migration/prepare.slt b/e2e_test/sql_migration/prepare.slt index f0669a4c6b297..ecb295cf6c534 100644 --- a/e2e_test/sql_migration/prepare.slt +++ b/e2e_test/sql_migration/prepare.slt @@ -66,3 +66,8 @@ statement ok create function int_42() returns int language javascript as $$ return 42; $$; + +statement ok +create secret secret_1 with ( + backend = 'meta' +) as 'demo_secret'; diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index c4b3e949fe4f5..223a23813f68d 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -278,6 +278,9 @@ message ProjectNode { repeated uint32 watermark_input_cols = 2; repeated uint32 watermark_output_cols = 3; repeated uint32 nondecreasing_exprs = 4; + // Whether there are likely no-op updates in the output chunks, so that eliminating them with + // `StreamChunk::eliminate_adjacent_noop_update` could be beneficial. + bool noop_update_hint = 5; } message FilterNode { diff --git a/src/connector/benches/json_vs_plain_parser.rs b/src/connector/benches/json_vs_plain_parser.rs index a176e3b2b0203..b9a1139dcb03b 100644 --- a/src/connector/benches/json_vs_plain_parser.rs +++ b/src/connector/benches/json_vs_plain_parser.rs @@ -83,8 +83,7 @@ mod old_json_parser { let mut errors = Vec::new(); for value in values { let accessor = JsonAccess::new(value); - match writer - .insert(|column| accessor.access(&[&column.name], Some(&column.data_type))) + match writer.do_insert(|column| accessor.access(&[&column.name], &column.data_type)) { Ok(_) => {} Err(err) => errors.push(err), diff --git a/src/connector/src/parser/csv_parser.rs b/src/connector/src/parser/csv_parser.rs index eaf09ab839891..25948c273d3e1 100644 --- a/src/connector/src/parser/csv_parser.rs +++ b/src/connector/src/parser/csv_parser.rs @@ -112,7 +112,7 @@ impl CsvParser { // The header row does not output a row, so we return early. return Ok(()); } - writer.insert(|desc| { + writer.do_insert(|desc| { if let Some(i) = headers.iter().position(|name| name == &desc.name) { let value = fields.get_mut(i).map(std::mem::take).unwrap_or_default(); if value.is_empty() { @@ -125,7 +125,7 @@ impl CsvParser { })?; } else { fields.reverse(); - writer.insert(|desc| { + writer.do_insert(|desc| { if let Some(value) = fields.pop() { if value.is_empty() { return Ok(None); diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 5c511af9efb40..8d8925d4768cd 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -47,6 +47,7 @@ pub struct JsonAccessBuilder { impl AccessBuilder for JsonAccessBuilder { #[allow(clippy::unused_async)] async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { + // XXX: When will we enter this branch? if payload.is_empty() { self.value = Some("{}".into()); } else { diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index be697d990a39a..4e0549ef99a98 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -490,7 +490,7 @@ impl SourceStreamChunkRowWriter<'_> { /// produces one [`Datum`] by corresponding [`SourceColumnDesc`]. /// /// See the [struct-level documentation](SourceStreamChunkRowWriter) for more details. - pub fn insert( + pub fn do_insert( &mut self, f: impl FnMut(&SourceColumnDesc) -> AccessResult, ) -> AccessResult<()> { @@ -501,7 +501,7 @@ impl SourceStreamChunkRowWriter<'_> { /// produces one [`Datum`] by corresponding [`SourceColumnDesc`]. /// /// See the [struct-level documentation](SourceStreamChunkRowWriter) for more details. - pub fn delete( + pub fn do_delete( &mut self, f: impl FnMut(&SourceColumnDesc) -> AccessResult, ) -> AccessResult<()> { @@ -512,7 +512,7 @@ impl SourceStreamChunkRowWriter<'_> { /// produces two [`Datum`]s as old and new value by corresponding [`SourceColumnDesc`]. /// /// See the [struct-level documentation](SourceStreamChunkRowWriter) for more details. - pub fn update( + pub fn do_update( &mut self, f: impl FnMut(&SourceColumnDesc) -> AccessResult<(Datum, Datum)>, ) -> AccessResult<()> { @@ -590,7 +590,7 @@ pub trait ByteStreamSourceParser: Send + Debug + Sized + 'static { } fn emit_empty_row<'a>(&'a mut self, mut writer: SourceStreamChunkRowWriter<'a>) { - _ = writer.insert(|_column| Ok(None)); + _ = writer.do_insert(|_column| Ok(None)); } } diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 2241f786cfdd5..a0c5bee568687 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -15,7 +15,7 @@ use risingwave_common::bail; use super::unified::json::TimestamptzHandling; -use super::unified::ChangeEvent; +use super::unified::kv_event::KvEvent; use super::{ AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, SourceStreamChunkRowWriter, SpecificParserConfig, @@ -24,7 +24,6 @@ use crate::error::ConnectorResult; use crate::parser::bytes_parser::BytesAccessBuilder; use crate::parser::simd_json_parser::DebeziumJsonAccessBuilder; use crate::parser::unified::debezium::parse_transaction_meta; -use crate::parser::unified::upsert::UpsertChangeEvent; use crate::parser::unified::AccessImpl; use crate::parser::upsert_parser::get_key_column_name; use crate::parser::{BytesProperties, ParseResult, ParserFormat}; @@ -103,22 +102,20 @@ impl PlainParser { }; } - // reuse upsert component but always insert - let mut row_op: UpsertChangeEvent, AccessImpl<'_, '_>> = - UpsertChangeEvent::default(); + let mut row_op: KvEvent, AccessImpl<'_, '_>> = KvEvent::default(); if let Some(data) = key && let Some(key_builder) = self.key_builder.as_mut() { // key is optional in format plain - row_op = row_op.with_key(key_builder.generate_accessor(data).await?); + row_op.with_key(key_builder.generate_accessor(data).await?); } if let Some(data) = payload { // the data part also can be an empty vec - row_op = row_op.with_value(self.payload_builder.generate_accessor(data).await?); + row_op.with_value(self.payload_builder.generate_accessor(data).await?); } - writer.insert(|column: &SourceColumnDesc| row_op.access_field(column))?; + writer.do_insert(|column: &SourceColumnDesc| row_op.access_field(column))?; Ok(ParseResult::Rows) } diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index 7dcb502b1f674..7638215c0b27a 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -896,7 +896,8 @@ mod test { } fn pb_eq(a: &ProtobufAccess, field_name: &str, value: ScalarImpl) { - let d = a.access(&[field_name], None).unwrap().unwrap(); + let dummy_type = DataType::Varchar; + let d = a.access(&[field_name], &dummy_type).unwrap().unwrap(); assert_eq!(d, value, "field: {} value: {:?}", field_name, d); } diff --git a/src/connector/src/parser/unified/avro.rs b/src/connector/src/parser/unified/avro.rs index 2c94eb47ccfd1..491dba46f53f8 100644 --- a/src/connector/src/parser/unified/avro.rs +++ b/src/connector/src/parser/unified/avro.rs @@ -24,7 +24,7 @@ use risingwave_common::array::{ListValue, StructValue}; use risingwave_common::bail; use risingwave_common::log::LogSuppresser; use risingwave_common::types::{ - DataType, Date, Datum, Interval, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, + DataType, Date, Interval, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, }; use risingwave_common::util::iter_util::ZipEqFast; @@ -82,7 +82,7 @@ impl<'a> AvroParseOptions<'a> { pub fn convert_to_datum<'b>( &self, value: &'b Value, - type_expected: Option<&'b DataType>, + type_expected: &'b DataType, ) -> AccessResult where 'b: 'a, @@ -104,25 +104,25 @@ impl<'a> AvroParseOptions<'a> { .convert_to_datum(v, type_expected); } // ---- Boolean ----- - (Some(DataType::Boolean) | None, Value::Boolean(b)) => (*b).into(), + (DataType::Boolean, Value::Boolean(b)) => (*b).into(), // ---- Int16 ----- - (Some(DataType::Int16), Value::Int(i)) if self.relax_numeric => (*i as i16).into(), - (Some(DataType::Int16), Value::Long(i)) if self.relax_numeric => (*i as i16).into(), + (DataType::Int16, Value::Int(i)) if self.relax_numeric => (*i as i16).into(), + (DataType::Int16, Value::Long(i)) if self.relax_numeric => (*i as i16).into(), // ---- Int32 ----- - (Some(DataType::Int32) | None, Value::Int(i)) => (*i).into(), - (Some(DataType::Int32), Value::Long(i)) if self.relax_numeric => (*i as i32).into(), + (DataType::Int32, Value::Int(i)) => (*i).into(), + (DataType::Int32, Value::Long(i)) if self.relax_numeric => (*i as i32).into(), // ---- Int64 ----- - (Some(DataType::Int64) | None, Value::Long(i)) => (*i).into(), - (Some(DataType::Int64), Value::Int(i)) if self.relax_numeric => (*i as i64).into(), + (DataType::Int64, Value::Long(i)) => (*i).into(), + (DataType::Int64, Value::Int(i)) if self.relax_numeric => (*i as i64).into(), // ---- Float32 ----- - (Some(DataType::Float32) | None, Value::Float(i)) => (*i).into(), - (Some(DataType::Float32), Value::Double(i)) => (*i as f32).into(), + (DataType::Float32, Value::Float(i)) => (*i).into(), + (DataType::Float32, Value::Double(i)) => (*i as f32).into(), // ---- Float64 ----- - (Some(DataType::Float64) | None, Value::Double(i)) => (*i).into(), - (Some(DataType::Float64), Value::Float(i)) => (*i as f64).into(), + (DataType::Float64, Value::Double(i)) => (*i).into(), + (DataType::Float64, Value::Float(i)) => (*i as f64).into(), // ---- Decimal ----- - (Some(DataType::Decimal) | None, Value::Decimal(avro_decimal)) => { + (DataType::Decimal, Value::Decimal(avro_decimal)) => { let (precision, scale) = match self.schema { Some(Schema::Decimal(DecimalSchema { precision, scale, .. @@ -133,7 +133,7 @@ impl<'a> AvroParseOptions<'a> { .map_err(|_| create_error())?; ScalarImpl::Decimal(risingwave_common::types::Decimal::Normalized(decimal)) } - (Some(DataType::Decimal), Value::Record(fields)) => { + (DataType::Decimal, Value::Record(fields)) => { // VariableScaleDecimal has fixed fields, scale(int) and value(bytes) let find_in_records = |field_name: &str| { fields @@ -167,56 +167,46 @@ impl<'a> AvroParseOptions<'a> { ScalarImpl::Decimal(risingwave_common::types::Decimal::Normalized(decimal)) } // ---- Time ----- - (Some(DataType::Time), Value::TimeMillis(ms)) => Time::with_milli(*ms as u32) + (DataType::Time, Value::TimeMillis(ms)) => Time::with_milli(*ms as u32) .map_err(|_| create_error())? .into(), - (Some(DataType::Time), Value::TimeMicros(us)) => Time::with_micro(*us as u64) + (DataType::Time, Value::TimeMicros(us)) => Time::with_micro(*us as u64) .map_err(|_| create_error())? .into(), // ---- Date ----- - (Some(DataType::Date) | None, Value::Date(days)) => { - Date::with_days(days + unix_epoch_days()) - .map_err(|_| create_error())? - .into() - } + (DataType::Date, Value::Date(days)) => Date::with_days(days + unix_epoch_days()) + .map_err(|_| create_error())? + .into(), // ---- Varchar ----- - (Some(DataType::Varchar) | None, Value::Enum(_, symbol)) => { - symbol.clone().into_boxed_str().into() - } - (Some(DataType::Varchar) | None, Value::String(s)) => s.clone().into_boxed_str().into(), + (DataType::Varchar, Value::Enum(_, symbol)) => symbol.clone().into_boxed_str().into(), + (DataType::Varchar, Value::String(s)) => s.clone().into_boxed_str().into(), // ---- Timestamp ----- - (Some(DataType::Timestamp) | None, Value::LocalTimestampMillis(ms)) => { - Timestamp::with_millis(*ms) - .map_err(|_| create_error())? - .into() - } - (Some(DataType::Timestamp) | None, Value::LocalTimestampMicros(us)) => { - Timestamp::with_micros(*us) - .map_err(|_| create_error())? - .into() - } + (DataType::Timestamp, Value::LocalTimestampMillis(ms)) => Timestamp::with_millis(*ms) + .map_err(|_| create_error())? + .into(), + (DataType::Timestamp, Value::LocalTimestampMicros(us)) => Timestamp::with_micros(*us) + .map_err(|_| create_error())? + .into(), // ---- TimestampTz ----- - (Some(DataType::Timestamptz) | None, Value::TimestampMillis(ms)) => { - Timestamptz::from_millis(*ms) - .ok_or_else(|| { - uncategorized!("timestamptz with milliseconds {ms} * 1000 is out of range") - })? - .into() - } - (Some(DataType::Timestamptz) | None, Value::TimestampMicros(us)) => { + (DataType::Timestamptz, Value::TimestampMillis(ms)) => Timestamptz::from_millis(*ms) + .ok_or_else(|| { + uncategorized!("timestamptz with milliseconds {ms} * 1000 is out of range") + })? + .into(), + (DataType::Timestamptz, Value::TimestampMicros(us)) => { Timestamptz::from_micros(*us).into() } // ---- Interval ----- - (Some(DataType::Interval) | None, Value::Duration(duration)) => { + (DataType::Interval, Value::Duration(duration)) => { let months = u32::from(duration.months()) as i32; let days = u32::from(duration.days()) as i32; let usecs = (u32::from(duration.millis()) as i64) * 1000; // never overflows ScalarImpl::Interval(Interval::from_month_day_usec(months, days, usecs)) } // ---- Struct ----- - (Some(DataType::Struct(struct_type_info)), Value::Record(descs)) => StructValue::new( + (DataType::Struct(struct_type_info), Value::Record(descs)) => StructValue::new( struct_type_info .names() .zip_eq_fast(struct_type_info.types()) @@ -228,7 +218,7 @@ impl<'a> AvroParseOptions<'a> { schema, relax_numeric: self.relax_numeric, } - .convert_to_datum(value, Some(field_type))?) + .convert_to_datum(value, field_type)?) } else { Ok(None) } @@ -236,22 +226,8 @@ impl<'a> AvroParseOptions<'a> { .collect::>()?, ) .into(), - (None, Value::Record(descs)) => { - let rw_values = descs - .iter() - .map(|(field_name, field_value)| { - let schema = self.extract_inner_schema(Some(field_name)); - Self { - schema, - relax_numeric: self.relax_numeric, - } - .convert_to_datum(field_value, None) - }) - .collect::, AccessError>>()?; - ScalarImpl::Struct(StructValue::new(rw_values)) - } // ---- List ----- - (Some(DataType::List(item_type)), Value::Array(array)) => ListValue::new({ + (DataType::List(item_type), Value::Array(array)) => ListValue::new({ let schema = self.extract_inner_schema(None); let mut builder = item_type.create_array_builder(array.len()); for v in array { @@ -259,18 +235,16 @@ impl<'a> AvroParseOptions<'a> { schema, relax_numeric: self.relax_numeric, } - .convert_to_datum(v, Some(item_type))?; + .convert_to_datum(v, item_type)?; builder.append(value); } builder.finish() }) .into(), // ---- Bytea ----- - (Some(DataType::Bytea) | None, Value::Bytes(value)) => { - value.clone().into_boxed_slice().into() - } + (DataType::Bytea, Value::Bytes(value)) => value.clone().into_boxed_slice().into(), // ---- Jsonb ----- - (Some(DataType::Jsonb), v @ Value::Map(_)) => { + (DataType::Jsonb, v @ Value::Map(_)) => { let mut builder = jsonbb::Builder::default(); avro_to_jsonb(v, &mut builder)?; let jsonb = builder.finish(); @@ -299,7 +273,7 @@ impl<'a, 'b> Access for AvroAccess<'a, 'b> where 'a: 'b, { - fn access(&self, path: &[&str], type_expected: Option<&DataType>) -> AccessResult { + fn access(&self, path: &[&str], type_expected: &DataType) -> AccessResult { let mut value = self.value; let mut options: AvroParseOptions<'_> = self.options.clone(); @@ -436,7 +410,7 @@ mod tests { use std::str::FromStr; use apache_avro::Decimal as AvroDecimal; - use risingwave_common::types::Decimal; + use risingwave_common::types::{Datum, Decimal}; use super::*; @@ -489,7 +463,7 @@ mod tests { shape: &DataType, ) -> crate::error::ConnectorResult { AvroParseOptions::create(value_schema) - .convert_to_datum(&value, Some(shape)) + .convert_to_datum(&value, shape) .map_err(Into::into) } @@ -532,7 +506,7 @@ mod tests { let value = Value::Decimal(AvroDecimal::from(bytes)); let options = AvroParseOptions::create(&schema); let resp = options - .convert_to_datum(&value, Some(&DataType::Decimal)) + .convert_to_datum(&value, &DataType::Decimal) .unwrap(); assert_eq!( resp, @@ -571,7 +545,7 @@ mod tests { let options = AvroParseOptions::create(&schema); let resp = options - .convert_to_datum(&value, Some(&DataType::Decimal)) + .convert_to_datum(&value, &DataType::Decimal) .unwrap(); assert_eq!(resp, Some(ScalarImpl::Decimal(Decimal::from(66051)))); } diff --git a/src/connector/src/parser/unified/bytes.rs b/src/connector/src/parser/unified/bytes.rs index ff47424d60acf..f9064c3ec3079 100644 --- a/src/connector/src/parser/unified/bytes.rs +++ b/src/connector/src/parser/unified/bytes.rs @@ -31,8 +31,8 @@ impl<'a> BytesAccess<'a> { impl<'a> Access for BytesAccess<'a> { /// path is empty currently, `type_expected` should be `Bytea` - fn access(&self, path: &[&str], type_expected: Option<&DataType>) -> AccessResult { - if let DataType::Bytea = type_expected.unwrap() { + fn access(&self, path: &[&str], type_expected: &DataType) -> AccessResult { + if let DataType::Bytea = type_expected { if self.column_name.is_none() || (path.len() == 1 && self.column_name.as_ref().unwrap() == path[0]) { diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index 966c5f167474c..3c415ad96678b 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -79,8 +79,8 @@ pub fn parse_transaction_meta( connector_props: &ConnectorProperties, ) -> AccessResult { if let (Some(ScalarImpl::Utf8(status)), Some(ScalarImpl::Utf8(id))) = ( - accessor.access(&[TRANSACTION_STATUS], Some(&DataType::Varchar))?, - accessor.access(&[TRANSACTION_ID], Some(&DataType::Varchar))?, + accessor.access(&[TRANSACTION_STATUS], &DataType::Varchar)?, + accessor.access(&[TRANSACTION_ID], &DataType::Varchar)?, ) { // The id field has different meanings for different databases: // PG: txID:LSN @@ -172,16 +172,16 @@ where .key_accessor .as_ref() .expect("key_accessor must be provided for delete operation") - .access(&[&desc.name], Some(&desc.data_type)); + .access(&[&desc.name], &desc.data_type); } if let Some(va) = self.value_accessor.as_ref() { - va.access(&[BEFORE, &desc.name], Some(&desc.data_type)) + va.access(&[BEFORE, &desc.name], &desc.data_type) } else { self.key_accessor .as_ref() .unwrap() - .access(&[&desc.name], Some(&desc.data_type)) + .access(&[&desc.name], &desc.data_type) } } @@ -193,7 +193,7 @@ where self.value_accessor .as_ref() .expect("value_accessor must be provided for upsert operation") - .access(&[AFTER, &desc.name], Some(&desc.data_type)) + .access(&[AFTER, &desc.name], &desc.data_type) }, |additional_column_type| { match additional_column_type { @@ -203,7 +203,7 @@ where .value_accessor .as_ref() .expect("value_accessor must be provided for upsert operation") - .access(&[SOURCE, SOURCE_TS_MS], Some(&DataType::Int64))?; + .access(&[SOURCE, SOURCE_TS_MS], &DataType::Int64)?; Ok(ts_ms.map(|scalar| { Timestamptz::from_millis(scalar.into_int64()) .expect("source.ts_ms must in millisecond") @@ -222,7 +222,7 @@ where fn op(&self) -> Result { if let Some(accessor) = &self.value_accessor { - if let Some(ScalarImpl::Utf8(op)) = accessor.access(&[OP], Some(&DataType::Varchar))? { + if let Some(ScalarImpl::Utf8(op)) = accessor.access(&[OP], &DataType::Varchar)? { match op.as_ref() { DEBEZIUM_READ_OP | DEBEZIUM_CREATE_OP | DEBEZIUM_UPDATE_OP => { return Ok(ChangeEventOperation::Upsert) @@ -309,15 +309,12 @@ impl Access for MongoJsonAccess where A: Access, { - fn access(&self, path: &[&str], type_expected: Option<&DataType>) -> super::AccessResult { + fn access(&self, path: &[&str], type_expected: &DataType) -> super::AccessResult { match path { ["after" | "before", "_id"] => { - let payload = self.access(&[path[0]], Some(&DataType::Jsonb))?; + let payload = self.access(&[path[0]], &DataType::Jsonb)?; if let Some(ScalarImpl::Jsonb(bson_doc)) = payload { - Ok(extract_bson_id( - type_expected.unwrap_or(&DataType::Jsonb), - &bson_doc.take(), - )?) + Ok(extract_bson_id(type_expected, &bson_doc.take())?) } else { // fail to extract the "_id" field from the message payload Err(AccessError::Undefined { @@ -326,19 +323,16 @@ where })? } } - ["after" | "before", "payload"] => self.access(&[path[0]], Some(&DataType::Jsonb)), + ["after" | "before", "payload"] => self.access(&[path[0]], &DataType::Jsonb), // To handle a DELETE message, we need to extract the "_id" field from the message key, because it is not in the payload. // In addition, the "_id" field is named as "id" in the key. An example of message key: // {"schema":null,"payload":{"id":"{\"$oid\": \"65bc9fb6c485f419a7a877fe\"}"}} ["_id"] => { let ret = self.accessor.access(path, type_expected); if matches!(ret, Err(AccessError::Undefined { .. })) { - let id_bson = self.accessor.access(&["id"], Some(&DataType::Jsonb))?; + let id_bson = self.accessor.access(&["id"], &DataType::Jsonb)?; if let Some(ScalarImpl::Jsonb(bson_doc)) = id_bson { - Ok(extract_bson_id( - type_expected.unwrap_or(&DataType::Jsonb), - &bson_doc.take(), - )?) + Ok(extract_bson_id(type_expected, &bson_doc.take())?) } else { // fail to extract the "_id" field from the message key Err(AccessError::Undefined { diff --git a/src/connector/src/parser/unified/json.rs b/src/connector/src/parser/unified/json.rs index 11c569832268e..a40b1153d5215 100644 --- a/src/connector/src/parser/unified/json.rs +++ b/src/connector/src/parser/unified/json.rs @@ -199,11 +199,7 @@ impl JsonParseOptions { } } - pub fn parse( - &self, - value: &BorrowedValue<'_>, - type_expected: Option<&DataType>, - ) -> AccessResult { + pub fn parse(&self, value: &BorrowedValue<'_>, type_expected: &DataType) -> AccessResult { let create_error = || AccessError::TypeError { expected: format!("{:?}", type_expected), got: value.value_type().to_string(), @@ -213,10 +209,10 @@ impl JsonParseOptions { let v: ScalarImpl = match (type_expected, value.value_type()) { (_, ValueType::Null) => return Ok(None), // ---- Boolean ----- - (Some(DataType::Boolean) | None, ValueType::Bool) => value.as_bool().unwrap().into(), + (DataType::Boolean , ValueType::Bool) => value.as_bool().unwrap().into(), ( - Some(DataType::Boolean), + DataType::Boolean, ValueType::I64 | ValueType::I128 | ValueType::U64 | ValueType::U128, ) if matches!(self.boolean_handling, BooleanHandling::Relax { .. }) && matches!(value.as_i64(), Some(0i64) | Some(1i64)) => @@ -224,7 +220,7 @@ impl JsonParseOptions { (value.as_i64() == Some(1i64)).into() } - (Some(DataType::Boolean), ValueType::String) + (DataType::Boolean, ValueType::String) if matches!( self.boolean_handling, BooleanHandling::Relax { @@ -256,11 +252,11 @@ impl JsonParseOptions { } // ---- Int16 ----- ( - Some(DataType::Int16), + DataType::Int16, ValueType::I64 | ValueType::I128 | ValueType::U64 | ValueType::U128, ) => value.try_as_i16().map_err(|_| create_error())?.into(), - (Some(DataType::Int16), ValueType::String) + (DataType::Int16, ValueType::String) if matches!( self.numeric_handling, NumericHandling::Relax { @@ -277,11 +273,11 @@ impl JsonParseOptions { } // ---- Int32 ----- ( - Some(DataType::Int32), + DataType::Int32, ValueType::I64 | ValueType::I128 | ValueType::U64 | ValueType::U128, ) => value.try_as_i32().map_err(|_| create_error())?.into(), - (Some(DataType::Int32), ValueType::String) + (DataType::Int32, ValueType::String) if matches!( self.numeric_handling, NumericHandling::Relax { @@ -297,15 +293,12 @@ impl JsonParseOptions { .into() } // ---- Int64 ----- - (None, ValueType::I64 | ValueType::U64) => { - value.try_as_i64().map_err(|_| create_error())?.into() - } ( - Some(DataType::Int64), + DataType::Int64, ValueType::I64 | ValueType::I128 | ValueType::U64 | ValueType::U128, ) => value.try_as_i64().map_err(|_| create_error())?.into(), - (Some(DataType::Int64), ValueType::String) + (DataType::Int64, ValueType::String) if matches!( self.numeric_handling, NumericHandling::Relax { @@ -322,12 +315,12 @@ impl JsonParseOptions { } // ---- Float32 ----- ( - Some(DataType::Float32), + DataType::Float32, ValueType::I64 | ValueType::I128 | ValueType::U64 | ValueType::U128, ) if matches!(self.numeric_handling, NumericHandling::Relax { .. }) => { (value.try_as_i64().map_err(|_| create_error())? as f32).into() } - (Some(DataType::Float32), ValueType::String) + (DataType::Float32, ValueType::String) if matches!( self.numeric_handling, NumericHandling::Relax { @@ -342,17 +335,17 @@ impl JsonParseOptions { .map_err(|_| create_error())? .into() } - (Some(DataType::Float32), ValueType::F64) => { + (DataType::Float32, ValueType::F64) => { value.try_as_f32().map_err(|_| create_error())?.into() } // ---- Float64 ----- ( - Some(DataType::Float64), + DataType::Float64, ValueType::I64 | ValueType::I128 | ValueType::U64 | ValueType::U128, ) if matches!(self.numeric_handling, NumericHandling::Relax { .. }) => { (value.try_as_i64().map_err(|_| create_error())? as f64).into() } - (Some(DataType::Float64), ValueType::String) + (DataType::Float64, ValueType::String) if matches!( self.numeric_handling, NumericHandling::Relax { @@ -367,25 +360,25 @@ impl JsonParseOptions { .map_err(|_| create_error())? .into() } - (Some(DataType::Float64) | None, ValueType::F64) => { + (DataType::Float64 , ValueType::F64) => { value.try_as_f64().map_err(|_| create_error())?.into() } // ---- Decimal ----- - (Some(DataType::Decimal) | None, ValueType::I128 | ValueType::U128) => { + (DataType::Decimal , ValueType::I128 | ValueType::U128) => { Decimal::from_str(&value.try_as_i128().map_err(|_| create_error())?.to_string()) .map_err(|_| create_error())? .into() } - (Some(DataType::Decimal), ValueType::I64 | ValueType::U64) => { + (DataType::Decimal, ValueType::I64 | ValueType::U64) => { Decimal::from(value.try_as_i64().map_err(|_| create_error())?).into() } - (Some(DataType::Decimal), ValueType::F64) => { + (DataType::Decimal, ValueType::F64) => { Decimal::try_from(value.try_as_f64().map_err(|_| create_error())?) .map_err(|_| create_error())? .into() } - (Some(DataType::Decimal), ValueType::String) => { + (DataType::Decimal, ValueType::String) => { let str = value.as_str().unwrap(); // the following values are special string generated by Debezium and should be handled separately match str { @@ -395,7 +388,7 @@ impl JsonParseOptions { _ => ScalarImpl::Decimal(Decimal::from_str(str).map_err(|_err| create_error())?), } } - (Some(DataType::Decimal), ValueType::Object) => { + (DataType::Decimal, ValueType::Object) => { // ref https://github.com/risingwavelabs/risingwave/issues/10628 // handle debezium json (variable scale): {"scale": int, "value": bytes} let scale = value @@ -418,21 +411,21 @@ impl JsonParseOptions { } // ---- Date ----- ( - Some(DataType::Date), + DataType::Date, ValueType::I64 | ValueType::I128 | ValueType::U64 | ValueType::U128, ) => Date::with_days_since_unix_epoch(value.try_as_i32().map_err(|_| create_error())?) .map_err(|_| create_error())? .into(), - (Some(DataType::Date), ValueType::String) => value + (DataType::Date, ValueType::String) => value .as_str() .unwrap() .parse::() .map_err(|_| create_error())? .into(), // ---- Varchar ----- - (Some(DataType::Varchar) | None, ValueType::String) => value.as_str().unwrap().into(), + (DataType::Varchar , ValueType::String) => value.as_str().unwrap().into(), ( - Some(DataType::Varchar), + DataType::Varchar, ValueType::Bool | ValueType::I64 | ValueType::I128 @@ -443,7 +436,7 @@ impl JsonParseOptions { value.to_string().into() } ( - Some(DataType::Varchar), + DataType::Varchar, ValueType::Bool | ValueType::I64 | ValueType::I128 @@ -456,14 +449,14 @@ impl JsonParseOptions { value.to_string().into() } // ---- Time ----- - (Some(DataType::Time), ValueType::String) => value + (DataType::Time, ValueType::String) => value .as_str() .unwrap() .parse:: ChangeEvent for (ChangeEventOperation, A) where A: Access, { - fn op(&self) -> std::result::Result { + fn op(&self) -> AccessResult { Ok(self.0) } fn access_field(&self, desc: &SourceColumnDesc) -> AccessResult { - self.1.access(&[desc.name.as_str()], Some(&desc.data_type)) + self.1.access(&[desc.name.as_str()], &desc.data_type) } } diff --git a/src/connector/src/parser/unified/protobuf.rs b/src/connector/src/parser/unified/protobuf.rs index cd9178c7dd08d..4455dea6a905d 100644 --- a/src/connector/src/parser/unified/protobuf.rs +++ b/src/connector/src/parser/unified/protobuf.rs @@ -38,7 +38,7 @@ impl ProtobufAccess { } impl Access for ProtobufAccess { - fn access(&self, path: &[&str], _type_expected: Option<&DataType>) -> AccessResult { + fn access(&self, path: &[&str], _type_expected: &DataType) -> AccessResult { debug_assert_eq!(1, path.len()); let field_desc = self .message diff --git a/src/connector/src/parser/unified/util.rs b/src/connector/src/parser/unified/util.rs index 7641e5f0b2432..2ed7e1fbefd4e 100644 --- a/src/connector/src/parser/unified/util.rs +++ b/src/connector/src/parser/unified/util.rs @@ -24,8 +24,8 @@ pub fn apply_row_operation_on_stream_chunk_writer_with_op( ) -> AccessResult<()> { let f = |column: &SourceColumnDesc| row_op.access_field(column); match op { - ChangeEventOperation::Upsert => writer.insert(f), - ChangeEventOperation::Delete => writer.delete(f), + ChangeEventOperation::Upsert => writer.do_insert(f), + ChangeEventOperation::Delete => writer.do_delete(f), } } diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index 048fd0beca3ff..604b6a840a7ff 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -16,14 +16,13 @@ use risingwave_common::bail; use risingwave_pb::plan_common::additional_column::ColumnType as AdditionalColumnType; use super::bytes_parser::BytesAccessBuilder; -use super::unified::upsert::UpsertChangeEvent; -use super::unified::util::apply_row_operation_on_stream_chunk_writer_with_op; use super::unified::{AccessImpl, ChangeEventOperation}; use super::{ AccessBuilderImpl, ByteStreamSourceParser, BytesProperties, EncodingProperties, EncodingType, SourceStreamChunkRowWriter, SpecificParserConfig, }; use crate::error::ConnectorResult; +use crate::parser::unified::kv_event::KvEvent; use crate::parser::ParserFormat; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; @@ -97,22 +96,26 @@ impl UpsertParser { payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, ) -> ConnectorResult<()> { - let mut row_op: UpsertChangeEvent, AccessImpl<'_, '_>> = - UpsertChangeEvent::default(); - let mut change_event_op = ChangeEventOperation::Delete; + let mut row_op: KvEvent, AccessImpl<'_, '_>> = KvEvent::default(); if let Some(data) = key { - row_op = row_op.with_key(self.key_builder.generate_accessor(data).await?); + row_op.with_key(self.key_builder.generate_accessor(data).await?); } // Empty payload of kafka is Some(vec![]) + let change_event_op; if let Some(data) = payload && !data.is_empty() { - row_op = row_op.with_value(self.payload_builder.generate_accessor(data).await?); + row_op.with_value(self.payload_builder.generate_accessor(data).await?); change_event_op = ChangeEventOperation::Upsert; + } else { + change_event_op = ChangeEventOperation::Delete; } - - apply_row_operation_on_stream_chunk_writer_with_op(row_op, &mut writer, change_event_op) - .map_err(Into::into) + let f = |column: &SourceColumnDesc| row_op.access_field(column); + match change_event_op { + ChangeEventOperation::Upsert => writer.do_insert(f)?, + ChangeEventOperation::Delete => writer.do_delete(f)?, + } + Ok(()) } } diff --git a/src/ctl/src/cmd_impl/meta/migration.rs b/src/ctl/src/cmd_impl/meta/migration.rs index 391d6c41a2985..b9aa81b677519 100644 --- a/src/ctl/src/cmd_impl/meta/migration.rs +++ b/src/ctl/src/cmd_impl/meta/migration.rs @@ -45,20 +45,20 @@ use risingwave_meta_model_v2::hummock_version_stats::TableStats; use risingwave_meta_model_v2::object::ObjectType; use risingwave_meta_model_v2::prelude::{ Actor, ActorDispatcher, CatalogVersion, Cluster, Connection, Database, Fragment, Function, - Index, Object, ObjectDependency, Schema, Sink, Source, StreamingJob, Subscription, + Index, Object, ObjectDependency, Schema, Secret, Sink, Source, StreamingJob, Subscription, SystemParameter, Table, User, UserPrivilege, View, Worker, WorkerProperty, }; use risingwave_meta_model_v2::{ catalog_version, cluster, compaction_config, compaction_status, compaction_task, connection, database, function, hummock_pinned_snapshot, hummock_pinned_version, hummock_sequence, - hummock_version_delta, hummock_version_stats, index, object, object_dependency, schema, sink, - source, streaming_job, subscription, table, user, user_privilege, view, worker, + hummock_version_delta, hummock_version_stats, index, object, object_dependency, schema, secret, + sink, source, streaming_job, subscription, table, user, user_privilege, view, worker, worker_property, CreateType, JobStatus, ObjectId, StreamingParallelism, }; use risingwave_pb::catalog::table::PbTableType; use risingwave_pb::catalog::{ - PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbSubscription, - PbTable, PbView, + PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSecret, PbSink, PbSource, + PbSubscription, PbTable, PbView, }; use risingwave_pb::common::WorkerType; use risingwave_pb::hummock::{ @@ -188,6 +188,7 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an let functions = PbFunction::list(&meta_store).await?; let connections = PbConnection::list(&meta_store).await?; let subscriptions = PbSubscription::list(&meta_store).await?; + let secrets = PbSecret::list(&meta_store).await?; // inuse object ids. let mut inuse_obj_ids = tables @@ -319,6 +320,28 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an } println!("connections migrated"); + // secret mapping + let mut secret_rewrite = HashMap::new(); + for mut secret in secrets { + let id = next_available_id(); + secret_rewrite.insert(secret.id, id); + secret.id = id as _; + + let obj = object::ActiveModel { + oid: Set(id as _), + obj_type: Set(ObjectType::Secret), + owner_id: Set(secret.owner as _), + database_id: Set(Some(*db_rewrite.get(&secret.database_id).unwrap() as _)), + schema_id: Set(Some(*schema_rewrite.get(&secret.schema_id).unwrap() as _)), + ..Default::default() + }; + Object::insert(obj).exec(&meta_store_sql.conn).await?; + Secret::insert(secret::ActiveModel::from(secret)) + .exec(&meta_store_sql.conn) + .await?; + } + println!("secrets migrated"); + // add object: table, source, sink, index, view, subscription. macro_rules! insert_objects { ($objects:expr, $object_type:expr) => { @@ -525,6 +548,16 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an if let Some(id) = s.connection_id.as_mut() { *id = *connection_rewrite.get(id).unwrap(); } + for secret_id in s.secret_ref.values_mut() { + *secret_id = *secret_rewrite.get(secret_id).unwrap(); + } + object_dependencies.extend(s.secret_ref.values().map(|id| { + object_dependency::ActiveModel { + id: NotSet, + oid: Set(*id as _), + used_by: Set(s.id as _), + } + })); s.into() }) .collect(); diff --git a/src/expr/impl/src/window_function/buffer.rs b/src/expr/impl/src/window_function/buffer.rs index 1fd44151f89b1..6c6277c20ee63 100644 --- a/src/expr/impl/src/window_function/buffer.rs +++ b/src/expr/impl/src/window_function/buffer.rs @@ -89,7 +89,7 @@ impl WindowBuffer { /// Get the current window info. pub fn curr_window(&self) -> CurrWindow<'_, W::Key> { - let buffer_ref = BufferRef { + let window = BufferRef { buffer: &self.buffer, curr_idx: self.curr_idx, left_idx: self.left_idx, @@ -97,8 +97,8 @@ impl WindowBuffer { }; CurrWindow { key: self.curr_key(), - preceding_saturated: self.window_impl.preceding_saturated(buffer_ref), - following_saturated: self.window_impl.following_saturated(buffer_ref), + preceding_saturated: self.window_impl.preceding_saturated(window), + following_saturated: self.window_impl.following_saturated(window), } } @@ -190,13 +190,13 @@ impl WindowBuffer { } fn recalculate_left_right(&mut self) { - let buffer_ref = BufferRefMut { + let window = BufferRefMut { buffer: &self.buffer, curr_idx: &mut self.curr_idx, left_idx: &mut self.left_idx, right_excl_idx: &mut self.right_excl_idx, }; - self.window_impl.recalculate_left_right(buffer_ref); + self.window_impl.recalculate_left_right(window); } } @@ -227,14 +227,14 @@ pub(super) trait WindowImpl { /// Whether the preceding half of the current window is saturated. /// By "saturated" we mean that every row that is possible to be in the preceding half of the /// current window is already in the buffer. - fn preceding_saturated(&self, buffer_ref: BufferRef<'_, Self::Key, Self::Value>) -> bool; + fn preceding_saturated(&self, window: BufferRef<'_, Self::Key, Self::Value>) -> bool; /// Whether the following half of the current window is saturated. - fn following_saturated(&self, buffer_ref: BufferRef<'_, Self::Key, Self::Value>) -> bool; + fn following_saturated(&self, window: BufferRef<'_, Self::Key, Self::Value>) -> bool; /// Recalculate the left and right indices of the current window, according to the latest /// `curr_idx`. The indices are indices in the buffer vector. - fn recalculate_left_right(&self, buffer_ref: BufferRefMut<'_, Self::Key, Self::Value>); + fn recalculate_left_right(&self, window: BufferRefMut<'_, Self::Key, Self::Value>); } /// The sliding window implementation for `ROWS` frames. @@ -258,8 +258,8 @@ impl WindowImpl for RowsWindow { type Key = K; type Value = V; - fn preceding_saturated(&self, buffer_ref: BufferRef<'_, Self::Key, Self::Value>) -> bool { - buffer_ref.curr_idx < buffer_ref.buffer.len() && { + fn preceding_saturated(&self, window: BufferRef<'_, Self::Key, Self::Value>) -> bool { + window.curr_idx < window.buffer.len() && { let start_off = self.frame_bounds.start.to_offset(); if let Some(start_off) = start_off { if start_off >= 0 { @@ -269,9 +269,9 @@ impl WindowImpl for RowsWindow { // the following can be simplified. #[allow(clippy::nonminimal_bool)] { - assert!(buffer_ref.curr_idx >= buffer_ref.left_idx); + assert!(window.curr_idx >= window.left_idx); } - buffer_ref.curr_idx - buffer_ref.left_idx >= start_off.unsigned_abs() + window.curr_idx - window.left_idx >= start_off.unsigned_abs() } } else { false // unbounded frame start, never preceding-saturated @@ -279,8 +279,8 @@ impl WindowImpl for RowsWindow { } } - fn following_saturated(&self, buffer_ref: BufferRef<'_, Self::Key, Self::Value>) -> bool { - buffer_ref.curr_idx < buffer_ref.buffer.len() && { + fn following_saturated(&self, window: BufferRef<'_, Self::Key, Self::Value>) -> bool { + window.curr_idx < window.buffer.len() && { let end_off = self.frame_bounds.end.to_offset(); if let Some(end_off) = end_off { if end_off <= 0 { @@ -289,11 +289,11 @@ impl WindowImpl for RowsWindow { // FIXME(rc): Ditto. #[allow(clippy::nonminimal_bool)] { - assert!(buffer_ref.right_excl_idx > 0); - assert!(buffer_ref.right_excl_idx > buffer_ref.curr_idx); - assert!(buffer_ref.right_excl_idx <= buffer_ref.buffer.len()); + assert!(window.right_excl_idx > 0); + assert!(window.right_excl_idx > window.curr_idx); + assert!(window.right_excl_idx <= window.buffer.len()); } - buffer_ref.right_excl_idx - 1 - buffer_ref.curr_idx >= end_off as usize + window.right_excl_idx - 1 - window.curr_idx >= end_off as usize } } else { false // unbounded frame end, never following-saturated @@ -301,37 +301,36 @@ impl WindowImpl for RowsWindow { } } - fn recalculate_left_right(&self, buffer_ref: BufferRefMut<'_, Self::Key, Self::Value>) { - if buffer_ref.buffer.is_empty() { - *buffer_ref.left_idx = 0; - *buffer_ref.right_excl_idx = 0; + fn recalculate_left_right(&self, window: BufferRefMut<'_, Self::Key, Self::Value>) { + if window.buffer.is_empty() { + *window.left_idx = 0; + *window.right_excl_idx = 0; } let start_off = self.frame_bounds.start.to_offset(); let end_off = self.frame_bounds.end.to_offset(); if let Some(start_off) = start_off { - let logical_left_idx = *buffer_ref.curr_idx as isize + start_off; + let logical_left_idx = *window.curr_idx as isize + start_off; if logical_left_idx >= 0 { - *buffer_ref.left_idx = - std::cmp::min(logical_left_idx as usize, buffer_ref.buffer.len()); + *window.left_idx = std::cmp::min(logical_left_idx as usize, window.buffer.len()); } else { - *buffer_ref.left_idx = 0; + *window.left_idx = 0; } } else { // unbounded start - *buffer_ref.left_idx = 0; + *window.left_idx = 0; } if let Some(end_off) = end_off { - let logical_right_excl_idx = *buffer_ref.curr_idx as isize + end_off + 1; + let logical_right_excl_idx = *window.curr_idx as isize + end_off + 1; if logical_right_excl_idx >= 0 { - *buffer_ref.right_excl_idx = - std::cmp::min(logical_right_excl_idx as usize, buffer_ref.buffer.len()); + *window.right_excl_idx = + std::cmp::min(logical_right_excl_idx as usize, window.buffer.len()); } else { - *buffer_ref.right_excl_idx = 0; + *window.right_excl_idx = 0; } } else { // unbounded end - *buffer_ref.right_excl_idx = buffer_ref.buffer.len(); + *window.right_excl_idx = window.buffer.len(); } } } @@ -355,36 +354,36 @@ impl WindowImpl for RangeWindow { type Key = StateKey; type Value = V; - fn preceding_saturated(&self, buffer_ref: BufferRef<'_, Self::Key, Self::Value>) -> bool { - buffer_ref.curr_idx < buffer_ref.buffer.len() && { + fn preceding_saturated(&self, window: BufferRef<'_, Self::Key, Self::Value>) -> bool { + window.curr_idx < window.buffer.len() && { // XXX(rc): It seems that preceding saturation is not important, may remove later. true } } - fn following_saturated(&self, buffer_ref: BufferRef<'_, Self::Key, Self::Value>) -> bool { - buffer_ref.curr_idx < buffer_ref.buffer.len() + fn following_saturated(&self, window: BufferRef<'_, Self::Key, Self::Value>) -> bool { + window.curr_idx < window.buffer.len() && { // Left OK? (note that `left_idx` can be greater than `right_idx`) // The following line checks whether the left value is the last one in the buffer. // Here we adopt a conservative approach, which means we assume the next future value // is likely to be the same as the last value in the current window, in which case // we can't say the current window is saturated. - buffer_ref.left_idx < buffer_ref.buffer.len() /* non-zero */ - 1 + window.left_idx < window.buffer.len() /* non-zero */ - 1 } && { // Right OK? Ditto. - buffer_ref.right_excl_idx < buffer_ref.buffer.len() + window.right_excl_idx < window.buffer.len() } } - fn recalculate_left_right(&self, buffer_ref: BufferRefMut<'_, Self::Key, Self::Value>) { - if buffer_ref.buffer.is_empty() { - *buffer_ref.left_idx = 0; - *buffer_ref.right_excl_idx = 0; + fn recalculate_left_right(&self, window: BufferRefMut<'_, Self::Key, Self::Value>) { + if window.buffer.is_empty() { + *window.left_idx = 0; + *window.right_excl_idx = 0; } - let Some(entry) = buffer_ref.buffer.get(*buffer_ref.curr_idx) else { + let Some(entry) = window.buffer.get(*window.curr_idx) else { // If the current index has been moved to a future position, we can't touch anything // because the next coming key may equal to the previous one which means the left and // right indices will be the same. @@ -403,7 +402,7 @@ impl WindowImpl for RangeWindow { Sentinelled::Smallest => { // unbounded frame start assert_eq!( - *buffer_ref.left_idx, 0, + *window.left_idx, 0, "for unbounded start, left index should always be 0" ); } @@ -411,7 +410,7 @@ impl WindowImpl for RangeWindow { // bounded, find the start position let value_enc = memcmp_encoding::encode_value(value, self.frame_bounds.order_type) .expect("no reason to fail here"); - *buffer_ref.left_idx = buffer_ref + *window.left_idx = window .buffer .partition_point(|elem| elem.key.order_key < value_enc); } @@ -421,13 +420,13 @@ impl WindowImpl for RangeWindow { match self.frame_bounds.frame_end_of(curr_order_value) { Sentinelled::Largest => { // unbounded frame end - *buffer_ref.right_excl_idx = buffer_ref.buffer.len(); + *window.right_excl_idx = window.buffer.len(); } Sentinelled::Normal(value) => { // bounded, find the end position let value_enc = memcmp_encoding::encode_value(value, self.frame_bounds.order_type) .expect("no reason to fail here"); - *buffer_ref.right_excl_idx = buffer_ref + *window.right_excl_idx = window .buffer .partition_point(|elem| elem.key.order_key <= value_enc); } diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 154680c8acbc8..70f83549ff18c 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -281,12 +281,14 @@ select distinct v1 from t; expected_outputs: - logical_plan + - stream_plan - name: distinct with agg sql: | create table t (v1 int, v2 int); select distinct sum(v1) from t group by v2; expected_outputs: - logical_plan + - stream_plan - name: distinct on sql: | create table t (v1 int, v2 int, v3 int); diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 9a07df7558d96..4c75b83187741 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -451,6 +451,12 @@ LogicalAgg { group_key: [t.v1], aggs: [] } └─LogicalProject { exprs: [t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + stream_plan: |- + StreamMaterialize { columns: [v1], stream_key: [v1], pk_columns: [v1], pk_conflict: NoCheck } + └─StreamProject { exprs: [t.v1], noop_update_hint: true } + └─StreamHashAgg { group_key: [t.v1], aggs: [count] } + └─StreamExchange { dist: HashShard(t.v1) } + └─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: distinct with agg sql: | create table t (v1 int, v2 int); @@ -461,6 +467,15 @@ └─LogicalAgg { group_key: [t.v2], aggs: [sum(t.v1)] } └─LogicalProject { exprs: [t.v2, t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + stream_plan: |- + StreamMaterialize { columns: [sum], stream_key: [sum], pk_columns: [sum], pk_conflict: NoCheck } + └─StreamProject { exprs: [sum(t.v1)], noop_update_hint: true } + └─StreamHashAgg { group_key: [sum(t.v1)], aggs: [count] } + └─StreamExchange { dist: HashShard(sum(t.v1)) } + └─StreamProject { exprs: [t.v2, sum(t.v1)] } + └─StreamHashAgg { group_key: [t.v2], aggs: [sum(t.v1), count] } + └─StreamExchange { dist: HashShard(t.v2) } + └─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: distinct on sql: | create table t (v1 int, v2 int, v3 int); @@ -576,7 +591,7 @@ └─LogicalScan { table: t, columns: [t.v1] } stream_plan: |- StreamMaterialize { columns: [v1], stream_key: [v1], pk_columns: [v1], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.v1] } + └─StreamProject { exprs: [t.v1], noop_update_hint: true } └─StreamHashAgg { group_key: [t.v1], aggs: [count] } └─StreamExchange { dist: HashShard(t.v1) } └─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) } @@ -1797,7 +1812,7 @@ stream_plan: |- StreamMaterialize { columns: [a, row_number], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY t.a ORDER BY t.a DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─StreamProject { exprs: [t.a] } + └─StreamProject { exprs: [t.a], noop_update_hint: true } └─StreamHashAgg { group_key: [t.a], aggs: [count] } └─StreamExchange { dist: HashShard(t.a) } └─StreamTableScan { table: t, columns: [t.a, 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/except.yaml b/src/frontend/planner_test/tests/testdata/output/except.yaml index 58b4f757955c3..d19ec9882c42e 100644 --- a/src/frontend/planner_test/tests/testdata/output/except.yaml +++ b/src/frontend/planner_test/tests/testdata/output/except.yaml @@ -25,7 +25,7 @@ └─BatchScan { table: t2, columns: [t2.a, t2.b, t2.c], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a, b, c], stream_key: [a, b, c], pk_columns: [a, b, c], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.a, t1.b, t1.c] } + └─StreamProject { exprs: [t1.a, t1.b, t1.c], noop_update_hint: true } └─StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } └─StreamHashJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } ├─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) } @@ -35,7 +35,7 @@ stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a, b, c], stream_key: [a, b, c], pk_columns: [a, b, c], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } - └── StreamProject { exprs: [t1.a, t1.b, t1.c] } + └── StreamProject { exprs: [t1.a, t1.b, t1.c], noop_update_hint: true } └── StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } { tables: [ HashAggState: 0 ] } └── StreamHashJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] diff --git a/src/frontend/planner_test/tests/testdata/output/intersect.yaml b/src/frontend/planner_test/tests/testdata/output/intersect.yaml index 87834b2ca2d2e..678091bbb2e47 100644 --- a/src/frontend/planner_test/tests/testdata/output/intersect.yaml +++ b/src/frontend/planner_test/tests/testdata/output/intersect.yaml @@ -25,7 +25,7 @@ └─BatchScan { table: t2, columns: [t2.a, t2.b, t2.c], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a, b, c], stream_key: [a, b, c], pk_columns: [a, b, c], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.a, t1.b, t1.c] } + └─StreamProject { exprs: [t1.a, t1.b, t1.c], noop_update_hint: true } └─StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } └─StreamHashJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } ├─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) } @@ -35,7 +35,7 @@ stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a, b, c], stream_key: [a, b, c], pk_columns: [a, b, c], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } - └── StreamProject { exprs: [t1.a, t1.b, t1.c] } + └── StreamProject { exprs: [t1.a, t1.b, t1.c], noop_update_hint: true } └── StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } { tables: [ HashAggState: 0 ] } └── StreamHashJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] 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 acac3cc236994..39639b3ebb647 100644 --- a/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml @@ -164,7 +164,7 @@ ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.arr] } + └─StreamProject { exprs: [t.arr], noop_update_hint: true } └─StreamHashAgg { group_key: [t.arr], aggs: [count] } └─StreamExchange { dist: HashShard(t.arr) } └─StreamTableScan { table: t, columns: [t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -193,7 +193,7 @@ │ │ └─StreamTableScan { table: t1, columns: [t1.n, t1.id, t1._row_id, t1.c], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } │ └─StreamExchange { dist: HashShard(Unnest(Case(($1 <> '':Varchar), ArrayAppend(StringToArray(Trim($1, ',':Varchar), ',':Varchar), $3), Array($3)))) } │ └─StreamProjectSet { select_list: [$0, $1, $2, $3, Unnest(Case(($1 <> '':Varchar), ArrayAppend(StringToArray(Trim($1, ',':Varchar), ',':Varchar), $3), Array($3)))] } - │ └─StreamProject { exprs: [t2.p, t2.p, t2.d, t2.d] } + │ └─StreamProject { exprs: [t2.p, t2.p, t2.d, t2.d], noop_update_hint: true } │ └─StreamHashAgg { group_key: [t2.p, t2.p, t2.d, t2.d], aggs: [count] } │ └─StreamExchange { dist: HashShard(t2.p, t2.p, t2.d, t2.d) } │ └─StreamProject { exprs: [t2.p, t2.p, t2.d, t2.d, t2._row_id] } @@ -226,7 +226,7 @@ └─StreamHashAgg { group_key: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5], aggs: [sum($expr4) filter(((t2.c7 * t1.c3) <= (t2.c7 * t2.c8))), sum(t1.c2) filter(((t2.c7 * t1.c3) <= (t2.c7 * t2.c8))), count] } └─StreamHashJoin { type: LeftOuter, predicate: t2.c7 IS NOT DISTINCT FROM t2.c7 AND t2.c7 IS NOT DISTINCT FROM t2.c7 AND t2.c8 IS NOT DISTINCT FROM t2.c8 AND t2.c7 IS NOT DISTINCT FROM t2.c7 AND t2.c7 IS NOT DISTINCT FROM t2.c7 AND t2.c8 IS NOT DISTINCT FROM t2.c8 AND t2.c1 IS NOT DISTINCT FROM t2.c1 AND t2.c5 IS NOT DISTINCT FROM t2.c5 AND t2.c5 IS NOT DISTINCT FROM t2.c5, output: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5, t1.c3, $expr4, t1.c2, t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5, t1._row_id, $expr1] } ├─StreamExchange { dist: HashShard(t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5) } - │ └─StreamProject { exprs: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5] } + │ └─StreamProject { exprs: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5], noop_update_hint: true } │ └─StreamHashAgg { group_key: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5], aggs: [count] } │ └─StreamExchange { dist: HashShard(t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5) } │ └─StreamTableScan { table: t2, columns: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) } @@ -235,7 +235,7 @@ └─StreamFilter { predicate: (t2.c5 <= $expr2) } └─StreamHashJoin { type: Inner, predicate: t2.c1 = t1.c1 AND $expr1 = $expr3, output: all } ├─StreamExchange { dist: HashShard(t2.c1, $expr1) } - │ └─StreamProject { exprs: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5, AtTimeZone(t2.c5, 'UTC':Varchar)::Date as $expr1] } + │ └─StreamProject { exprs: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5, AtTimeZone(t2.c5, 'UTC':Varchar)::Date as $expr1], noop_update_hint: true } │ └─StreamHashAgg { group_key: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5], aggs: [count] } │ └─StreamExchange { dist: HashShard(t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5) } │ └─StreamTableScan { table: t2, columns: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index 41007d8de1181..298653450f659 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -748,7 +748,7 @@ │ └─StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } │ └─StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] } │ └─StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], stream_scan_type: ArrangementBackfill, stream_key: [person.id], pk: [id], dist: UpstreamHashShard(person.id) } - └─StreamProject { exprs: [auction.seller, $expr3, $expr4] } + └─StreamProject { exprs: [auction.seller, $expr3, $expr4], noop_update_hint: true } └─StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } └─StreamExchange { dist: HashShard(auction.seller, $expr3, $expr4) } └─StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] } @@ -764,7 +764,7 @@ StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] } ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] ├── StreamExchange Hash([0, 1, 2]) from 2 - └── StreamProject { exprs: [auction.seller, $expr3, $expr4] } + └── StreamProject { exprs: [auction.seller, $expr3, $expr4], noop_update_hint: true } └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { tables: [ HashAggState: 6 ] } └── StreamExchange Hash([0, 1, 2]) from 3 diff --git a/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml b/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml index 5c8e87aea559f..7392c88bde3cc 100644 --- a/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml @@ -77,7 +77,7 @@ └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3] } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3], stream_key: [v1, v2, v3], pk_columns: [v1, v2, v3], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.v1, t.v2, t.v3] } + └─StreamProject { exprs: [t.v1, t.v2, t.v3], noop_update_hint: true } └─StreamHashAgg { group_key: [t.v1, t.v2, t.v3], aggs: [count] } └─StreamExchange { dist: HashShard(t.v1, t.v2, t.v3) } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, 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/project_set.yaml b/src/frontend/planner_test/tests/testdata/output/project_set.yaml index 890480493af92..8373ea5187fca 100644 --- a/src/frontend/planner_test/tests/testdata/output/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/output/project_set.yaml @@ -118,7 +118,7 @@ └─BatchScan { table: t, columns: [t.x], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [unnest], stream_key: [unnest], pk_columns: [unnest], pk_conflict: NoCheck } - └─StreamProject { exprs: [Unnest($0)] } + └─StreamProject { exprs: [Unnest($0)], noop_update_hint: true } └─StreamHashAgg { group_key: [Unnest($0)], aggs: [count] } └─StreamExchange { dist: HashShard(Unnest($0)) } └─StreamProjectSet { select_list: [Unnest($0), $1] } diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index cfe5d841cb2c1..2cf3aee9fe043 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -222,12 +222,12 @@ └─StreamStatelessSimpleAgg { aggs: [count] } └─StreamHashJoin { type: Inner, predicate: t.a = t.a, output: all } ├─StreamShare { id: 4 } - │ └─StreamProject { exprs: [t.a] } + │ └─StreamProject { exprs: [t.a], noop_update_hint: true } │ └─StreamHashAgg { group_key: [t.a], aggs: [count] } │ └─StreamExchange { dist: HashShard(t.a) } │ └─StreamTableScan { table: t, columns: [t.a, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamShare { id: 4 } - └─StreamProject { exprs: [t.a] } + └─StreamProject { exprs: [t.a], noop_update_hint: true } └─StreamHashAgg { group_key: [t.a], aggs: [count] } └─StreamExchange { dist: HashShard(t.a) } └─StreamTableScan { table: t, columns: [t.a, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -251,7 +251,7 @@ └── StreamExchange NoShuffle from 4 Fragment 2 - StreamProject { exprs: [t.a] } + StreamProject { exprs: [t.a], noop_update_hint: true } └── StreamHashAgg { group_key: [t.a], aggs: [count] } { tables: [ HashAggState: 5 ] } └── StreamExchange Hash([0]) from 3 diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index b981f8f968ed8..f9ae7cdee3f0d 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -331,9 +331,9 @@ │ └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] } │ └─StreamFilter { predicate: IsNotNull(auction.date_time) } │ └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], stream_scan_type: ArrangementBackfill, stream_key: [auction._row_id], pk: [_row_id], dist: UpstreamHashShard(auction._row_id) } - └─StreamProject { exprs: [auction.date_time] } + └─StreamProject { exprs: [auction.date_time], noop_update_hint: true } └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] } - └─StreamProject { exprs: [auction.date_time] } + └─StreamProject { exprs: [auction.date_time], noop_update_hint: true } └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] } └─StreamExchange { dist: HashShard(auction.date_time) } └─StreamShare { id: 3 } @@ -529,13 +529,13 @@ └─StreamProject { exprs: [t.x, sum(Unnest($0))] } └─StreamHashAgg { group_key: [t.x], aggs: [sum(Unnest($0)), count] } └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, Unnest($0), t.x, projected_row_id] } - ├─StreamProject { exprs: [t.x] } + ├─StreamProject { exprs: [t.x], noop_update_hint: true } │ └─StreamHashAgg { group_key: [t.x], aggs: [count] } │ └─StreamExchange { dist: HashShard(t.x) } │ └─StreamTableScan { table: t, columns: [t.x, t.k], stream_scan_type: ArrangementBackfill, stream_key: [t.k], pk: [k], dist: UpstreamHashShard(t.k) } └─StreamProject { exprs: [t.x, Unnest($0), projected_row_id] } └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.x] } + └─StreamProject { exprs: [t.x], noop_update_hint: true } └─StreamHashAgg { group_key: [t.x], aggs: [count] } └─StreamExchange { dist: HashShard(t.x) } └─StreamTableScan { table: t, columns: [t.x, t.k], stream_scan_type: ArrangementBackfill, stream_key: [t.k], pk: [k], dist: UpstreamHashShard(t.k) } @@ -817,7 +817,7 @@ └─StreamProject { exprs: [integers.correlated_col, (count(distinct rows.k) + count(distinct rows.v)) as $expr1] } └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count(distinct rows.k), count(distinct rows.v), count] } └─StreamHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, rows._row_id] } - ├─StreamProject { exprs: [integers.correlated_col] } + ├─StreamProject { exprs: [integers.correlated_col], noop_update_hint: true } │ └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count] } │ └─StreamExchange { dist: HashShard(integers.correlated_col) } │ └─StreamTableScan { table: integers, columns: [integers.correlated_col, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml index 3bb2639f12938..09905835f2009 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml @@ -1096,7 +1096,7 @@ └─StreamProject { exprs: [t.b, count(1:Int32)] } └─StreamHashAgg { group_key: [t.b], aggs: [count(1:Int32), count] } └─StreamHashJoin { type: LeftOuter, predicate: t.b IS NOT DISTINCT FROM t2.d, output: [t.b, 1:Int32, t2._row_id] } - ├─StreamProject { exprs: [t.b] } + ├─StreamProject { exprs: [t.b], noop_update_hint: true } │ └─StreamHashAgg { group_key: [t.b], aggs: [count] } │ └─StreamExchange { dist: HashShard(t.b) } │ └─StreamTableScan { table: t, columns: [t.b, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -1331,7 +1331,7 @@ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1340,7 +1340,7 @@ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1349,7 +1349,7 @@ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1358,7 +1358,7 @@ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1367,7 +1367,7 @@ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1376,7 +1376,7 @@ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1385,7 +1385,7 @@ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1394,7 +1394,7 @@ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1403,7 +1403,7 @@ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1412,7 +1412,7 @@ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1421,7 +1421,7 @@ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1430,7 +1430,7 @@ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1439,7 +1439,7 @@ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1448,7 +1448,7 @@ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1457,7 +1457,7 @@ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1466,7 +1466,7 @@ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1475,7 +1475,7 @@ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1484,7 +1484,7 @@ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } ├─StreamExchange { dist: HashShard(t1.a) } - │ └─StreamProject { exprs: [t1.a, t1.b] } + │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1802,7 +1802,7 @@ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d), count] } │ │ │ │ │ └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t1.a, t1.b, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } - │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1814,7 +1814,7 @@ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1822,7 +1822,7 @@ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c, t2.d], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f, t3._row_id] } │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1830,7 +1830,7 @@ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t3, columns: [t3.e, t3.f, t3._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t3._row_id], pk: [_row_id], dist: UpstreamHashShard(t3._row_id) } │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h, t4._row_id] } │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1838,7 +1838,7 @@ │ │ │ │ │ │ │ └─StreamTableScan { table: t4, columns: [t4.g, t4.h, t4._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t4._row_id], pk: [_row_id], dist: UpstreamHashShard(t4._row_id) } │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j, t5._row_id] } │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1846,7 +1846,7 @@ │ │ │ │ │ │ └─StreamTableScan { table: t5, columns: [t5.i, t5.j, t5._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t5._row_id], pk: [_row_id], dist: UpstreamHashShard(t5._row_id) } │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k, t6._row_id] } │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } - │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1856,7 +1856,7 @@ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d), count] } │ │ │ │ └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t1.a, t1.b, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } │ │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } - │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1868,7 +1868,7 @@ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1876,7 +1876,7 @@ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c, t2.d], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f, t3._row_id] } │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1884,7 +1884,7 @@ │ │ │ │ │ │ │ └─StreamTableScan { table: t3, columns: [t3.e, t3.f, t3._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t3._row_id], pk: [_row_id], dist: UpstreamHashShard(t3._row_id) } │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h, t4._row_id] } │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1892,7 +1892,7 @@ │ │ │ │ │ │ └─StreamTableScan { table: t4, columns: [t4.g, t4.h, t4._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t4._row_id], pk: [_row_id], dist: UpstreamHashShard(t4._row_id) } │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j, t5._row_id] } │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1900,7 +1900,7 @@ │ │ │ │ │ └─StreamTableScan { table: t5, columns: [t5.i, t5.j, t5._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t5._row_id], pk: [_row_id], dist: UpstreamHashShard(t5._row_id) } │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k, t6._row_id] } │ │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } - │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1910,7 +1910,7 @@ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d), count] } │ │ │ └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t1.a, t1.b, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } - │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1922,7 +1922,7 @@ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1930,7 +1930,7 @@ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c, t2.d], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f, t3._row_id] } │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1938,7 +1938,7 @@ │ │ │ │ │ │ └─StreamTableScan { table: t3, columns: [t3.e, t3.f, t3._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t3._row_id], pk: [_row_id], dist: UpstreamHashShard(t3._row_id) } │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h, t4._row_id] } │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1946,7 +1946,7 @@ │ │ │ │ │ └─StreamTableScan { table: t4, columns: [t4.g, t4.h, t4._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t4._row_id], pk: [_row_id], dist: UpstreamHashShard(t4._row_id) } │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j, t5._row_id] } │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1954,7 +1954,7 @@ │ │ │ │ └─StreamTableScan { table: t5, columns: [t5.i, t5.j, t5._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t5._row_id], pk: [_row_id], dist: UpstreamHashShard(t5._row_id) } │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k, t6._row_id] } │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } - │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1964,7 +1964,7 @@ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d), count] } │ │ └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t1.a, t1.b, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } │ │ ├─StreamExchange { dist: HashShard(t1.b) } - │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1976,7 +1976,7 @@ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1984,7 +1984,7 @@ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c, t2.d], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f, t3._row_id] } │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -1992,7 +1992,7 @@ │ │ │ │ │ └─StreamTableScan { table: t3, columns: [t3.e, t3.f, t3._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t3._row_id], pk: [_row_id], dist: UpstreamHashShard(t3._row_id) } │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h, t4._row_id] } │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2000,7 +2000,7 @@ │ │ │ │ └─StreamTableScan { table: t4, columns: [t4.g, t4.h, t4._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t4._row_id], pk: [_row_id], dist: UpstreamHashShard(t4._row_id) } │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j, t5._row_id] } │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2008,7 +2008,7 @@ │ │ │ └─StreamTableScan { table: t5, columns: [t5.i, t5.j, t5._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t5._row_id], pk: [_row_id], dist: UpstreamHashShard(t5._row_id) } │ │ └─StreamHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k, t6._row_id] } │ │ ├─StreamExchange { dist: HashShard(t1.b) } - │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2018,7 +2018,7 @@ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d), count] } │ └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t1.a, t1.b, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } │ ├─StreamExchange { dist: HashShard(t1.b) } - │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2030,7 +2030,7 @@ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2038,7 +2038,7 @@ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c, t2.d], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f, t3._row_id] } │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2046,7 +2046,7 @@ │ │ │ │ └─StreamTableScan { table: t3, columns: [t3.e, t3.f, t3._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t3._row_id], pk: [_row_id], dist: UpstreamHashShard(t3._row_id) } │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h, t4._row_id] } │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2054,7 +2054,7 @@ │ │ │ └─StreamTableScan { table: t4, columns: [t4.g, t4.h, t4._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t4._row_id], pk: [_row_id], dist: UpstreamHashShard(t4._row_id) } │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j, t5._row_id] } │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2062,7 +2062,7 @@ │ │ └─StreamTableScan { table: t5, columns: [t5.i, t5.j, t5._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t5._row_id], pk: [_row_id], dist: UpstreamHashShard(t5._row_id) } │ └─StreamHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k, t6._row_id] } │ ├─StreamExchange { dist: HashShard(t1.b) } - │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2072,7 +2072,7 @@ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d), count] } └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t1.a, t1.b, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } ├─StreamExchange { dist: HashShard(t1.b) } - │ └─StreamProject { exprs: [t1.a, t1.b] } + │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2084,7 +2084,7 @@ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } - │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2092,7 +2092,7 @@ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c, t2.d], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f, t3._row_id] } │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2100,7 +2100,7 @@ │ │ │ └─StreamTableScan { table: t3, columns: [t3.e, t3.f, t3._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t3._row_id], pk: [_row_id], dist: UpstreamHashShard(t3._row_id) } │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h, t4._row_id] } │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2108,7 +2108,7 @@ │ │ └─StreamTableScan { table: t4, columns: [t4.g, t4.h, t4._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t4._row_id], pk: [_row_id], dist: UpstreamHashShard(t4._row_id) } │ └─StreamHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j, t5._row_id] } │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } - │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -2116,7 +2116,7 @@ │ └─StreamTableScan { table: t5, columns: [t5.i, t5.j, t5._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t5._row_id], pk: [_row_id], dist: UpstreamHashShard(t5._row_id) } └─StreamHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k, t6._row_id] } ├─StreamExchange { dist: HashShard(t1.b) } - │ └─StreamProject { exprs: [t1.a, t1.b] } + │ └─StreamProject { exprs: [t1.a, t1.b], noop_update_hint: true } │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/union.yaml b/src/frontend/planner_test/tests/testdata/output/union.yaml index c9591f2dee2d4..ffd31dec73da5 100644 --- a/src/frontend/planner_test/tests/testdata/output/union.yaml +++ b/src/frontend/planner_test/tests/testdata/output/union.yaml @@ -78,7 +78,7 @@ └─BatchScan { table: t2, columns: [t2.a, t2.b, t2.c], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a, b, c], stream_key: [a, b, c], pk_columns: [a, b, c], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.a, t1.b, t1.c] } + └─StreamProject { exprs: [t1.a, t1.b, t1.c], noop_update_hint: true } └─StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } └─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) } └─StreamUnion { all: true } @@ -92,7 +92,7 @@ Fragment 0 StreamMaterialize { columns: [a, b, c], stream_key: [a, b, c], pk_columns: [a, b, c], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [t1.a, t1.b, t1.c] } + └── StreamProject { exprs: [t1.a, t1.b, t1.c], noop_update_hint: true } └── StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } { tables: [ HashAggState: 0 ] } └── StreamExchange Hash([0, 1, 2]) from 1 @@ -155,7 +155,7 @@ └─BatchScan { table: t2, columns: [t2.a, t2.b, t2.c], distribution: UpstreamHashShard(t2.a) } stream_plan: |- StreamMaterialize { columns: [a, b, c], stream_key: [a, b, c], pk_columns: [a, b, c], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.a, t1.b, t1.c] } + └─StreamProject { exprs: [t1.a, t1.b, t1.c], noop_update_hint: true } └─StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } └─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) } └─StreamUnion { all: true } @@ -169,7 +169,7 @@ Fragment 0 StreamMaterialize { columns: [a, b, c], stream_key: [a, b, c], pk_columns: [a, b, c], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [t1.a, t1.b, t1.c] } + └── StreamProject { exprs: [t1.a, t1.b, t1.c], noop_update_hint: true } └── StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } { tables: [ HashAggState: 0 ] } └── StreamExchange Hash([0, 1, 2]) from 1 diff --git a/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml b/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml index be7c3a19a5664..1b811a4871993 100644 --- a/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml +++ b/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml @@ -30,7 +30,7 @@ ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.arr] } + └─StreamProject { exprs: [t.arr], noop_update_hint: true } └─StreamHashAgg { group_key: [t.arr], aggs: [count] } └─StreamExchange { dist: HashShard(t.arr) } └─StreamTableScan { table: t, columns: [t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -54,7 +54,7 @@ ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.arr] } + └─StreamProject { exprs: [t.arr], noop_update_hint: true } └─StreamHashAgg { group_key: [t.arr], aggs: [count] } └─StreamExchange { dist: HashShard(t.arr) } └─StreamTableScan { table: t, columns: [t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -78,7 +78,7 @@ ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.arr] } + └─StreamProject { exprs: [t.arr], noop_update_hint: true } └─StreamHashAgg { group_key: [t.arr], aggs: [count] } └─StreamExchange { dist: HashShard(t.arr) } └─StreamTableScan { table: t, columns: [t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -102,7 +102,7 @@ ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.arr] } + └─StreamProject { exprs: [t.arr], noop_update_hint: true } └─StreamHashAgg { group_key: [t.arr], aggs: [count] } └─StreamExchange { dist: HashShard(t.arr) } └─StreamTableScan { table: t, columns: [t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -127,7 +127,7 @@ ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.arr] } + └─StreamProject { exprs: [t.arr], noop_update_hint: true } └─StreamHashAgg { group_key: [t.arr], aggs: [count] } └─StreamExchange { dist: HashShard(t.arr) } └─StreamTableScan { table: t, columns: [t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -161,12 +161,12 @@ │ ├─StreamExchange { dist: HashShard(t.arr) } │ │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } │ └─StreamProjectSet { select_list: [$0, Unnest($0)] } - │ └─StreamProject { exprs: [t.arr] } + │ └─StreamProject { exprs: [t.arr], noop_update_hint: true } │ └─StreamHashAgg { group_key: [t.arr], aggs: [count] } │ └─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.arr] } + └─StreamProject { exprs: [t.arr], noop_update_hint: true } └─StreamHashAgg { group_key: [t.arr], aggs: [count] } └─StreamExchange { dist: HashShard(t.arr) } └─StreamTableScan { table: t, columns: [t.arr, 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/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 6d49098027907..f32b37d250b5a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -1140,10 +1140,16 @@ impl ToStream for LogicalAgg { Ok(plan) } else { // a `count(*)` is appended, should project the output + 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(), )) + // If there's no agg call, then `count(*)` will be the only column in the output besides keys. + // 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()) .into()) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index c077af9241aa1..e8ff1df6e82db 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -38,10 +38,15 @@ pub struct StreamProject { watermark_derivations: Vec<(usize, usize)>, /// Nondecreasing expression indices. `Project` can produce watermarks for these expressions. nondecreasing_exprs: Vec, + /// Whether there are likely no-op updates in the output chunks, so that eliminating them with + /// `StreamChunk::eliminate_adjacent_noop_update` could be beneficial. + noop_update_hint: bool, } impl Distill for StreamProject { fn distill<'a>(&self) -> XmlNode<'a> { + let verbose = self.base.ctx().is_explain_verbose(); + let schema = self.schema(); let mut vec = self.core.fields_pretty(schema); if let Some(display_output_watermarks) = @@ -49,12 +54,27 @@ impl Distill for StreamProject { { vec.push(("output_watermarks", display_output_watermarks)); } + if verbose && self.noop_update_hint { + vec.push(("noop_update_hint", "true".into())); + } childless_record("StreamProject", vec) } } impl StreamProject { pub fn new(core: generic::Project) -> Self { + Self::new_inner(core, false) + } + + /// Set the `noop_update_hint` flag to the given value. + pub fn with_noop_update_hint(self, noop_update_hint: bool) -> Self { + Self { + noop_update_hint, + ..self + } + } + + fn new_inner(core: generic::Project, noop_update_hint: bool) -> Self { let input = core.input.clone(); let distribution = core .i2o_col_mapping() @@ -90,11 +110,13 @@ impl StreamProject { input.emit_on_window_close(), watermark_columns, ); + StreamProject { base, core, watermark_derivations, nondecreasing_exprs, + noop_update_hint, } } @@ -105,6 +127,10 @@ impl StreamProject { pub fn exprs(&self) -> &Vec { &self.core.exprs } + + pub fn noop_update_hint(&self) -> bool { + self.noop_update_hint + } } impl PlanTreeNodeUnary for StreamProject { @@ -115,7 +141,7 @@ impl PlanTreeNodeUnary for StreamProject { fn clone_with_input(&self, input: PlanRef) -> Self { let mut core = self.core.clone(); core.input = input; - Self::new(core) + Self::new_inner(core, self.noop_update_hint) } } impl_plan_tree_node_for_unary! {StreamProject} @@ -132,6 +158,7 @@ impl StreamNode for StreamProject { watermark_input_cols, watermark_output_cols, nondecreasing_exprs: self.nondecreasing_exprs.iter().map(|i| *i as _).collect(), + noop_update_hint: self.noop_update_hint, }) } } @@ -144,7 +171,7 @@ impl ExprRewritable for StreamProject { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut core = self.core.clone(); core.rewrite_exprs(r); - Self::new(core).into() + Self::new_inner(core, self.noop_update_hint).into() } } diff --git a/src/frontend/src/optimizer/rule/stream/stream_project_merge_rule.rs b/src/frontend/src/optimizer/rule/stream/stream_project_merge_rule.rs index 0eba52e193d22..91ab942e3a7fc 100644 --- a/src/frontend/src/optimizer/rule/stream/stream_project_merge_rule.rs +++ b/src/frontend/src/optimizer/rule/stream/stream_project_merge_rule.rs @@ -47,7 +47,15 @@ impl Rule for StreamProjectMergeRule { .map(|expr| subst.rewrite_expr(expr)) .collect(); let logical_project = generic::Project::new(exprs, inner_project.input()); - Some(StreamProject::new(logical_project).into()) + + // If either of the projects has the hint, we should keep it. + let noop_update_hint = outer_project.noop_update_hint() || inner_project.noop_update_hint(); + + Some( + StreamProject::new(logical_project) + .with_noop_update_hint(noop_update_hint) + .into(), + ) } } diff --git a/src/meta/src/hummock/manager/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction_group_manager.rs index 1ae6d207e8b2e..2aaf1a2192826 100644 --- a/src/meta/src/hummock/manager/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction_group_manager.rs @@ -337,12 +337,19 @@ impl HummockManager { .filter_map(|(group_id, member_count)| { if member_count == 0 && group_id > StaticCompactionGroupId::End as CompactionGroupId { - return Some(group_id); + return Some(( + group_id, + new_version_delta + .latest_version() + .get_compaction_group_levels(group_id) + .get_levels() + .len(), + )); } None }) .collect_vec(); - for group_id in &groups_to_remove { + for (group_id, _) in &groups_to_remove { let group_deltas = &mut new_version_delta .group_deltas .entry(*group_id) @@ -353,13 +360,8 @@ impl HummockManager { }); } - for group_id in &groups_to_remove { - let max_level = new_version_delta - .latest_version() - .get_compaction_group_levels(*group_id) - .get_levels() - .len(); - remove_compaction_group_in_sst_stat(&self.metrics, *group_id, max_level); + for (group_id, max_level) in groups_to_remove { + remove_compaction_group_in_sst_stat(&self.metrics, group_id, max_level); } new_version_delta.pre_apply(); diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 88513da3323cf..a75e9fcdacf93 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -2273,3 +2273,126 @@ async fn test_partition_level() { } } } + +#[tokio::test] +async fn test_unregister_moved_table() { + let (_env, hummock_manager, _, worker_node) = setup_compute_env(80).await; + let context_id = worker_node.id; + let original_groups = hummock_manager + .get_current_version() + .await + .levels + .keys() + .cloned() + .sorted() + .collect_vec(); + assert_eq!( + original_groups, + vec![ + StaticCompactionGroupId::StateDefault as u64, + StaticCompactionGroupId::MaterializedView as u64 + ] + ); + + hummock_manager + .register_table_ids(&[(100, 2)]) + .await + .unwrap(); + hummock_manager + .register_table_ids(&[(101, 2)]) + .await + .unwrap(); + let sst_1 = ExtendedSstableInfo { + compaction_group_id: 2, + sst_info: SstableInfo { + object_id: 10, + sst_id: 10, + key_range: Some(KeyRange { + left: iterator_test_key_of_epoch(100, 1, 20), + right: iterator_test_key_of_epoch(100, 100, 20), + right_exclusive: false, + }), + table_ids: vec![100], + min_epoch: 20, + max_epoch: 20, + ..Default::default() + }, + table_stats: Default::default(), + }; + let sst_2 = ExtendedSstableInfo { + compaction_group_id: 2, + sst_info: SstableInfo { + object_id: 11, + sst_id: 11, + key_range: Some(KeyRange { + left: iterator_test_key_of_epoch(100, 101, 20), + right: iterator_test_key_of_epoch(101, 100, 20), + right_exclusive: false, + }), + table_ids: vec![100, 101], + min_epoch: 20, + max_epoch: 20, + ..Default::default() + }, + table_stats: Default::default(), + }; + hummock_manager + .commit_epoch( + 30, + CommitEpochInfo::for_test( + vec![sst_1, sst_2], + HashMap::from([(10, context_id), (11, context_id)]), + ), + ) + .await + .unwrap(); + + let new_group_id = hummock_manager + .split_compaction_group(2, &[100]) + .await + .unwrap(); + assert_ne!(new_group_id, 2); + assert!(new_group_id > StaticCompactionGroupId::End as u64); + + let current_version = hummock_manager.get_current_version().await; + assert_eq!( + new_group_id, + current_version.levels.keys().max().cloned().unwrap() + ); + assert_eq!(current_version.levels.len(), 3); + assert_eq!( + get_compaction_group_object_ids(¤t_version, 2), + vec![10, 11] + ); + assert_eq!( + get_compaction_group_object_ids(¤t_version, new_group_id), + vec![10, 11] + ); + assert_eq!( + current_version + .get_compaction_group_levels(2) + .member_table_ids, + vec![101] + ); + assert_eq!( + current_version + .get_compaction_group_levels(new_group_id) + .member_table_ids, + vec![100] + ); + + hummock_manager.unregister_table_ids(&[100]).await.unwrap(); + let current_version = hummock_manager.get_current_version().await; + assert_eq!(current_version.levels.len(), 2); + assert!(!current_version.levels.contains_key(&new_group_id)); + assert_eq!( + get_compaction_group_object_ids(¤t_version, 2), + vec![10, 11] + ); + assert_eq!( + current_version + .get_compaction_group_levels(2) + .member_table_ids, + vec![101] + ); +} diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index 1cea1e0c393d8..5911188e2bfb5 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -81,11 +81,12 @@ pub struct DatabaseManager { pub(super) secrets: BTreeMap, /// Relation reference count mapping. - // TODO(zehua): avoid key conflicts after distinguishing table's and source's id generator. pub(super) relation_ref_count: HashMap, /// Secret reference count mapping pub(super) secret_ref_count: HashMap, + /// Connection reference count mapping. + pub(super) connection_ref_count: HashMap, // In-progress creation tracker. pub(super) in_progress_creation_tracker: HashSet, // In-progress creating streaming job tracker: this is a temporary workaround to avoid clean up @@ -110,6 +111,7 @@ impl DatabaseManager { let secrets = Secret::list(env.meta_store().as_kv()).await?; let mut relation_ref_count = HashMap::new(); + let mut connection_ref_count = HashMap::new(); let mut _secret_ref_count = HashMap::new(); let databases = BTreeMap::from_iter( @@ -119,9 +121,8 @@ impl DatabaseManager { ); let schemas = BTreeMap::from_iter(schemas.into_iter().map(|schema| (schema.id, schema))); let sources = BTreeMap::from_iter(sources.into_iter().map(|source| { - // TODO(weili): wait for yezizp to refactor ref cnt if let Some(connection_id) = source.connection_id { - *relation_ref_count.entry(connection_id).or_default() += 1; + *connection_ref_count.entry(connection_id).or_default() += 1; } (source.id, source) })); @@ -129,6 +130,9 @@ impl DatabaseManager { for depend_relation_id in &sink.dependent_relations { *relation_ref_count.entry(*depend_relation_id).or_default() += 1; } + if let Some(connection_id) = sink.connection_id { + *connection_ref_count.entry(connection_id).or_default() += 1; + } (sink.id, sink) })); let subscriptions = BTreeMap::from_iter(subscriptions.into_iter().map(|subscription| { @@ -168,6 +172,7 @@ impl DatabaseManager { functions, connections, relation_ref_count, + connection_ref_count, secrets, secret_ref_count: _secret_ref_count, in_progress_creation_tracker: HashSet::default(), @@ -475,11 +480,11 @@ impl DatabaseManager { && self.views.values().all(|v| v.schema_id != schema_id) } - pub fn increase_ref_count(&mut self, relation_id: RelationId) { + pub fn increase_relation_ref_count(&mut self, relation_id: RelationId) { *self.relation_ref_count.entry(relation_id).or_insert(0) += 1; } - pub fn decrease_ref_count(&mut self, relation_id: RelationId) { + pub fn decrease_relation_ref_count(&mut self, relation_id: RelationId) { match self.relation_ref_count.entry(relation_id) { Entry::Occupied(mut o) => { *o.get_mut() -= 1; @@ -507,6 +512,22 @@ impl DatabaseManager { } } + pub fn increase_connection_ref_count(&mut self, connection_id: ConnectionId) { + *self.connection_ref_count.entry(connection_id).or_insert(0) += 1; + } + + pub fn decrease_connection_ref_count(&mut self, connection_id: ConnectionId) { + match self.connection_ref_count.entry(connection_id) { + Entry::Occupied(mut o) => { + *o.get_mut() -= 1; + if *o.get() == 0 { + o.remove_entry(); + } + } + Entry::Vacant(_) => unreachable!(), + } + } + pub fn has_creation_in_database(&self, database_id: DatabaseId) -> bool { self.in_progress_creation_tracker .iter() @@ -659,7 +680,6 @@ impl DatabaseManager { } } - // TODO(zehua): refactor when using SourceId. pub fn ensure_table_view_or_source_id(&self, table_id: &TableId) -> MetaResult<()> { if self.tables.contains_key(table_id) || self.sources.contains_key(table_id) diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 824d656e506c0..de9c637df9dda 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -437,9 +437,8 @@ impl CatalogManager { for view in &views_to_drop { database_core.relation_ref_count.remove(&view.id); } - // TODO(weili): wait for yezizp to refactor ref cnt for connection in &connections_to_drop { - database_core.relation_ref_count.remove(&connection.id); + database_core.connection_ref_count.remove(&connection.id); } for user in users_need_update { self.notify_frontend(Operation::Update, Info::User(user)) @@ -576,12 +575,11 @@ impl CatalogManager { let user_core = &mut core.user; let mut connections = BTreeMapTransaction::new(&mut database_core.connections); - // TODO(weili): wait for yezizp to refactor ref cnt - match database_core.relation_ref_count.get(&conn_id) { + match database_core.connection_ref_count.get(&conn_id) { Some(ref_count) => { let connection_name = connections .get(&conn_id) - .ok_or_else(|| anyhow!("connection not found"))? + .ok_or_else(|| MetaError::catalog_id_not_found("connection", conn_id))? .name .clone(); Err(MetaError::permission_denied(format!( @@ -592,7 +590,7 @@ impl CatalogManager { None => { let connection = connections .remove(conn_id) - .ok_or_else(|| anyhow!("connection not found"))?; + .ok_or_else(|| MetaError::catalog_id_not_found("connection", conn_id))?; commit_meta!(self, connections)?; user_core.decrease_ref(connection.owner); @@ -688,7 +686,7 @@ impl CatalogManager { user_core.increase_ref(view.owner); for &dependent_relation_id in &view.dependent_relations { - database_core.increase_ref_count(dependent_relation_id); + database_core.increase_relation_ref_count(dependent_relation_id); } let version = self @@ -737,7 +735,7 @@ impl CatalogManager { let function = functions .remove(function_id) - .ok_or_else(|| anyhow!("function not found"))?; + .ok_or_else(|| MetaError::catalog_id_not_found("function", function_id))?; let objects = &[Object::FunctionId(function_id)]; let users_need_update = Self::update_user_privileges(&mut users, objects); @@ -845,7 +843,6 @@ impl CatalogManager { database_core.ensure_database_id(table.database_id)?; database_core.ensure_schema_id(table.schema_id)?; for dependent_id in &table.dependent_relations { - // TODO(zehua): refactor when using SourceId. database_core.ensure_table_view_or_source_id(dependent_id)?; } #[cfg(not(test))] @@ -866,7 +863,7 @@ impl CatalogManager { commit_meta!(self, tables)?; for &dependent_relation_id in &table.dependent_relations { - database_core.increase_ref_count(dependent_relation_id); + database_core.increase_relation_ref_count(dependent_relation_id); } user_core.increase_ref(table.owner); Ok(()) @@ -1064,7 +1061,7 @@ impl CatalogManager { if table.table_type != TableType::Internal as i32 { // Recovered when init database manager. for relation_id in &table.dependent_relations { - database_core.decrease_ref_count(*relation_id); + database_core.decrease_relation_ref_count(*relation_id); } // Recovered when init user manager. tracing::debug!("decrease ref for {}", table.id); @@ -1175,7 +1172,7 @@ impl CatalogManager { { let database_core = &mut core.database; for &dependent_relation_id in &table.dependent_relations { - database_core.decrease_ref_count(dependent_relation_id); + database_core.decrease_relation_ref_count(dependent_relation_id); } } } @@ -1762,28 +1759,25 @@ impl CatalogManager { // decrease dependent relations for table in &tables_removed { for dependent_relation_id in &table.dependent_relations { - database_core.decrease_ref_count(*dependent_relation_id); + database_core.decrease_relation_ref_count(*dependent_relation_id); } } for view in &views_removed { for dependent_relation_id in &view.dependent_relations { - database_core.decrease_ref_count(*dependent_relation_id); + database_core.decrease_relation_ref_count(*dependent_relation_id); } } for sink in &sinks_removed { - if let Some(connection_id) = sink.connection_id { - // TODO(siyuan): wait for yezizp to refactor ref cnt - database_core.decrease_ref_count(connection_id); - } + refcnt_dec_connection(database_core, sink.connection_id); for dependent_relation_id in &sink.dependent_relations { - database_core.decrease_ref_count(*dependent_relation_id); + database_core.decrease_relation_ref_count(*dependent_relation_id); } } for subscription in &subscriptions_removed { - database_core.decrease_ref_count(subscription.dependent_table_id); + database_core.decrease_relation_ref_count(subscription.dependent_table_id); } let version = self @@ -2798,7 +2792,7 @@ impl CatalogManager { database_core .get_connection(connection_id) .cloned() - .ok_or_else(|| anyhow!(format!("could not find connection {}", connection_id)).into()) + .ok_or_else(|| MetaError::catalog_id_not_found("connection", connection_id)) } pub async fn finish_create_source_procedure( @@ -3009,7 +3003,7 @@ impl CatalogManager { database_core.mark_creating(&key); database_core.mark_creating_streaming_job(index_table.id, key); for &dependent_relation_id in &index_table.dependent_relations { - database_core.increase_ref_count(dependent_relation_id); + database_core.increase_relation_ref_count(dependent_relation_id); } // index table and index. user_core.increase_ref_count(index.owner, 2); @@ -3030,7 +3024,7 @@ impl CatalogManager { database_core.unmark_creating(&key); database_core.unmark_creating_streaming_job(index_table.id); for &dependent_relation_id in &index_table.dependent_relations { - database_core.decrease_ref_count(dependent_relation_id); + database_core.decrease_relation_ref_count(dependent_relation_id); } // index table and index. user_core.decrease_ref_count(index.owner, 2); @@ -3101,7 +3095,6 @@ impl CatalogManager { database_core.ensure_database_id(sink.database_id)?; database_core.ensure_schema_id(sink.schema_id)?; for dependent_id in &sink.dependent_relations { - // TODO(zehua): refactor when using SourceId. database_core.ensure_table_view_or_source_id(dependent_id)?; } let key = (sink.database_id, sink.schema_id, sink.name.clone()); @@ -3115,7 +3108,7 @@ impl CatalogManager { database_core.mark_creating(&key); database_core.mark_creating_streaming_job(sink.id, key); for &dependent_relation_id in &sink.dependent_relations { - database_core.increase_ref_count(dependent_relation_id); + database_core.increase_relation_ref_count(dependent_relation_id); } user_core.increase_ref(sink.owner); // We have validate the status of connection before starting the procedure. @@ -3189,7 +3182,7 @@ impl CatalogManager { database_core.unmark_creating(&key); database_core.unmark_creating_streaming_job(sink.id); for &dependent_relation_id in &sink.dependent_relations { - database_core.decrease_ref_count(dependent_relation_id); + database_core.decrease_relation_ref_count(dependent_relation_id); } user_core.decrease_ref(sink.owner); refcnt_dec_connection(database_core, sink.connection_id); @@ -3224,7 +3217,7 @@ impl CatalogManager { } else { database_core.mark_creating(&key); database_core.mark_creating_streaming_job(subscription.id, key); - database_core.increase_ref_count(subscription.dependent_table_id); + database_core.increase_relation_ref_count(subscription.dependent_table_id); user_core.increase_ref(subscription.owner); let mut subscriptions = BTreeMapTransaction::new(&mut database_core.subscriptions); subscriptions.insert(subscription.id, subscription.clone()); @@ -3242,7 +3235,7 @@ impl CatalogManager { let mut subscriptions = BTreeMapTransaction::new(&mut database_core.subscriptions); let mut subscription = subscriptions .get(&subscription_id) - .ok_or_else(|| anyhow!("subscription not found"))? + .ok_or_else(|| MetaError::catalog_id_not_found("subscription", subscription_id))? .clone(); subscription.created_at_cluster_version = Some(current_cluster_version()); subscription.created_at_epoch = Some(Epoch::now().0); @@ -3278,7 +3271,7 @@ impl CatalogManager { let subscriptions = BTreeMapTransaction::new(&mut database_core.subscriptions); let subscription = subscriptions .get(&subscription_id) - .ok_or_else(|| anyhow!("subscription not found"))? + .ok_or_else(|| MetaError::catalog_id_not_found("subscription", subscription_id))? .clone(); assert_eq!( subscription.subscription_state, @@ -3327,7 +3320,7 @@ impl CatalogManager { database_core.unmark_creating(&key); database_core.unmark_creating_streaming_job(subscription.id); - database_core.decrease_ref_count(subscription.dependent_table_id); + database_core.decrease_relation_ref_count(subscription.dependent_table_id); user_core.decrease_ref(subscription.owner); } Ok(()) @@ -3816,7 +3809,7 @@ impl CatalogManager { .database .subscriptions .get(&subscription_id) - .ok_or_else(|| anyhow!("cant find subscription with id {}", subscription_id))?; + .ok_or_else(|| MetaError::catalog_id_not_found("subscription", subscription_id))?; Ok(subscription.clone()) } @@ -4005,7 +3998,7 @@ impl CatalogManager { core.user_info .get(&id) .cloned() - .ok_or_else(|| anyhow!("User {} not found", id).into()) + .ok_or_else(|| MetaError::catalog_id_not_found("user", id)) } pub async fn drop_user(&self, id: UserId) -> MetaResult { @@ -4135,11 +4128,11 @@ impl CatalogManager { let grantor_info = users .get(&grantor) .cloned() - .ok_or_else(|| anyhow!("User {} does not exist", &grantor))?; + .ok_or_else(|| MetaError::catalog_id_not_found("user", grantor))?; for user_id in user_ids { let mut user = users .get_mut(*user_id) - .ok_or_else(|| anyhow!("User {} does not exist", user_id))?; + .ok_or_else(|| MetaError::catalog_id_not_found("user", user_id))?; if user.is_super { return Err(MetaError::permission_denied(format!( @@ -4262,7 +4255,7 @@ impl CatalogManager { // check revoke permission let revoke_by = users .get(&revoke_by) - .ok_or_else(|| anyhow!("User {} does not exist", &revoke_by))?; + .ok_or_else(|| MetaError::catalog_id_not_found("user", revoke_by))?; let same_user = granted_by == revoke_by.id; if !revoke_by.is_super { for privilege in revoke_grant_privileges { @@ -4297,7 +4290,7 @@ impl CatalogManager { let user = users .get(user_id) .cloned() - .ok_or_else(|| anyhow!("User {} does not exist", user_id))?; + .ok_or_else(|| MetaError::catalog_id_not_found("user", user_id))?; if user.is_super { return Err(MetaError::permission_denied(format!( "Cannot revoke privilege from supper user {}", diff --git a/src/meta/src/manager/catalog/utils.rs b/src/meta/src/manager/catalog/utils.rs index 6c73e6eeebb39..4d4257ec5a7e5 100644 --- a/src/meta/src/manager/catalog/utils.rs +++ b/src/meta/src/manager/catalog/utils.rs @@ -22,8 +22,7 @@ pub fn refcnt_inc_connection( ) -> anyhow::Result<()> { if let Some(connection_id) = connection_id { if let Some(_conn) = database_mgr.get_connection(connection_id) { - // TODO(weili): wait for yezizp to refactor ref cnt - database_mgr.increase_ref_count(connection_id); + database_mgr.increase_connection_ref_count(connection_id); } else { bail!("connection {} not found.", connection_id); } @@ -36,7 +35,6 @@ pub fn refcnt_dec_connection( connection_id: Option, ) { if let Some(connection_id) = connection_id { - // TODO: wait for yezizp to refactor ref cnt - database_mgr.decrease_ref_count(connection_id); + database_mgr.decrease_connection_ref_count(connection_id); } } diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index 838527b32382b..f2d9a43241729 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -345,9 +345,7 @@ fn make_stream_fragments() -> Vec { make_inputref(0), make_inputref(1), ], - watermark_input_cols: vec![], - watermark_output_cols: vec![], - nondecreasing_exprs: vec![], + ..Default::default() })), fields: vec![], // TODO: fill this later input: vec![simple_agg_node_1], diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index e341adad9df5c..8f73fe26ee7de 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -178,6 +178,7 @@ async fn test_merger_sum_aggr() { MultiMap::new(), vec![], 0.0, + false, ); let items = Arc::new(Mutex::new(vec![])); diff --git a/src/stream/src/executor/project.rs b/src/stream/src/executor/project.rs index 22ce33a1066d5..6ea579afea524 100644 --- a/src/stream/src/executor/project.rs +++ b/src/stream/src/executor/project.rs @@ -44,6 +44,10 @@ struct Inner { /// the selectivity threshold which should be in `[0,1]`. for the chunk with selectivity less /// than the threshold, the Project executor will construct a new chunk before expr evaluation, materialize_selectivity_threshold: f64, + + /// Whether there are likely no-op updates in the output chunks, so that eliminating them with + /// `StreamChunk::eliminate_adjacent_noop_update` could be beneficial. + noop_update_hint: bool, } impl ProjectExecutor { @@ -55,6 +59,7 @@ impl ProjectExecutor { watermark_derivations: MultiMap, nondecreasing_expr_indices: Vec, materialize_selectivity_threshold: f64, + noop_update_hint: bool, ) -> Self { let n_nondecreasing_exprs = nondecreasing_expr_indices.len(); Self { @@ -66,6 +71,7 @@ impl ProjectExecutor { nondecreasing_expr_indices, last_nondec_expr_values: vec![None; n_nondecreasing_exprs], materialize_selectivity_threshold, + noop_update_hint, }, } } @@ -103,7 +109,11 @@ impl Inner { projected_columns.push(evaluated_expr); } let (_, vis) = data_chunk.into_parts(); - let new_chunk = StreamChunk::with_visibility(ops, projected_columns, vis); + + let mut new_chunk = StreamChunk::with_visibility(ops, projected_columns, vis); + if self.noop_update_hint { + new_chunk = new_chunk.eliminate_adjacent_noop_update(); + } Ok(Some(new_chunk)) } @@ -233,6 +243,7 @@ mod tests { MultiMap::new(), vec![], 0.0, + false, ); let mut project = project.boxed().execute(); @@ -315,6 +326,7 @@ mod tests { MultiMap::from_iter(vec![(0, 0), (0, 1)].into_iter()), vec![2], 0.0, + false, ); let mut project = project.boxed().execute(); diff --git a/src/stream/src/from_proto/project.rs b/src/stream/src/from_proto/project.rs index d7f96c4dffcbf..177045b3eba7d 100644 --- a/src/stream/src/from_proto/project.rs +++ b/src/stream/src/from_proto/project.rs @@ -67,6 +67,7 @@ impl ExecutorBuilder for ProjectExecutorBuilder { watermark_derivations, nondecreasing_expr_indices, materialize_selectivity_threshold, + node.noop_update_hint, ); Ok((params.info, exec).into()) }