=
diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs
index a499a9c6ea3d3..07d2a6c7653e7 100644
--- a/src/frontend/src/optimizer/plan_node/logical_scan.rs
+++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs
@@ -273,7 +273,7 @@ impl LogicalScan {
self.output_col_idx().to_vec(),
self.core.table_desc.clone(),
self.indexes().to_vec(),
- self.base.ctx.clone(),
+ self.base.ctx().clone(),
predicate,
self.for_system_time_as_of_proctime(),
self.table_cardinality(),
@@ -288,7 +288,7 @@ impl LogicalScan {
output_col_idx,
self.core.table_desc.clone(),
self.indexes().to_vec(),
- self.base.ctx.clone(),
+ self.base.ctx().clone(),
self.predicate().clone(),
self.for_system_time_as_of_proctime(),
self.table_cardinality(),
@@ -309,7 +309,7 @@ impl_plan_tree_node_for_leaf! {LogicalScan}
impl Distill for LogicalScan {
fn distill<'a>(&self) -> XmlNode<'a> {
- let verbose = self.base.ctx.is_explain_verbose();
+ let verbose = self.base.ctx().is_explain_verbose();
let mut vec = Vec::with_capacity(5);
vec.push(("table", Pretty::from(self.table_name().to_owned())));
let key_is_columns =
@@ -440,7 +440,7 @@ impl LogicalScan {
let (scan_ranges, predicate) = self.predicate().clone().split_to_scan_ranges(
self.core.table_desc.clone(),
self.base
- .ctx
+ .ctx()
.session_ctx()
.config()
.get_max_split_range_gap(),
@@ -551,7 +551,7 @@ impl ToStream for LogicalScan {
None.into(),
)));
}
- match self.base.stream_key.is_none() {
+ match self.base.stream_key().is_none() {
true => {
let mut col_ids = HashSet::new();
diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs
index d924ee7180168..d6b5711740a98 100644
--- a/src/frontend/src/optimizer/plan_node/logical_share.rs
+++ b/src/frontend/src/optimizer/plan_node/logical_share.rs
@@ -69,7 +69,7 @@ impl LogicalShare {
}
pub(super) fn pretty_fields<'a>(base: &PlanBase, name: &'a str) -> XmlNode<'a> {
- childless_record(name, vec![("id", Pretty::debug(&base.id.0))])
+ childless_record(name, vec![("id", Pretty::debug(&base.id().0))])
}
}
diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs
index 1d37da9eaa40f..45a5fbcb2240f 100644
--- a/src/frontend/src/optimizer/plan_node/logical_source.rs
+++ b/src/frontend/src/optimizer/plan_node/logical_source.rs
@@ -28,6 +28,7 @@ use risingwave_connector::source::{ConnectorProperties, DataType};
use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn;
use risingwave_pb::plan_common::GeneratedColumnDesc;
+use super::generic::GenericPlanRef;
use super::stream_watermark_filter::StreamWatermarkFilter;
use super::utils::{childless_record, Distill};
use super::{
@@ -204,7 +205,7 @@ impl LogicalSource {
..self.core.clone()
};
let mut new_s3_plan: PlanRef = StreamSource {
- base: PlanBase::new_stream_with_logical(
+ base: PlanBase::new_stream_with_core(
&logical_source,
Distribution::Single,
true, // `list` will keep listing all objects, it must be append-only
@@ -506,7 +507,7 @@ impl PredicatePushdown for LogicalSource {
let mut new_conjunctions = Vec::with_capacity(predicate.conjunctions.len());
for expr in predicate.conjunctions {
- if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, &self.base.schema) {
+ if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, self.base.schema()) {
// Not recognized, so push back
new_conjunctions.push(e);
}
diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs
index 51e4e620cf4ca..1f02b026c0020 100644
--- a/src/frontend/src/optimizer/plan_node/logical_union.rs
+++ b/src/frontend/src/optimizer/plan_node/logical_union.rs
@@ -130,7 +130,7 @@ impl ToBatch for LogicalUnion {
if !self.all() {
let batch_union = BatchUnion::new(new_logical).into();
Ok(BatchHashAgg::new(
- generic::Agg::new(vec![], (0..self.base.schema.len()).collect(), batch_union)
+ generic::Agg::new(vec![], (0..self.base.schema().len()).collect(), batch_union)
.with_enable_two_phase(false),
)
.into())
@@ -170,7 +170,7 @@ impl ToStream for LogicalUnion {
&self,
ctx: &mut RewriteStreamContext,
) -> Result<(PlanRef, ColIndexMapping)> {
- let original_schema = self.base.schema.clone();
+ let original_schema = self.base.schema().clone();
let original_schema_len = original_schema.len();
let mut rewrites = vec![];
for input in &self.core.inputs {
@@ -353,7 +353,7 @@ mod tests {
// Check the result
let union = plan.as_logical_union().unwrap();
- assert_eq!(union.base.schema.len(), 2);
+ assert_eq!(union.base.schema().len(), 2);
}
#[tokio::test]
diff --git a/src/frontend/src/optimizer/plan_node/logical_update.rs b/src/frontend/src/optimizer/plan_node/logical_update.rs
index 80e4f350d8edb..1dbe1d3d3c5c9 100644
--- a/src/frontend/src/optimizer/plan_node/logical_update.rs
+++ b/src/frontend/src/optimizer/plan_node/logical_update.rs
@@ -15,6 +15,7 @@
use risingwave_common::catalog::TableVersionId;
use risingwave_common::error::Result;
+use super::generic::GenericPlanRef;
use super::utils::impl_distill_by_unit;
use super::{
gen_filter_and_pushdown, generic, BatchUpdate, ColPrunable, ExprRewritable, LogicalProject,
diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs
index c6a3d2ac0564e..e62c6400f2015 100644
--- a/src/frontend/src/optimizer/plan_node/logical_values.rs
+++ b/src/frontend/src/optimizer/plan_node/logical_values.rs
@@ -21,6 +21,7 @@ use risingwave_common::catalog::{Field, Schema};
use risingwave_common::error::Result;
use risingwave_common::types::{DataType, ScalarImpl};
+use super::generic::GenericPlanRef;
use super::utils::{childless_record, Distill};
use super::{
BatchValues, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown,
@@ -144,7 +145,7 @@ impl ColPrunable for LogicalValues {
.iter()
.map(|i| self.schema().fields[*i].clone())
.collect();
- Self::new(rows, Schema { fields }, self.base.ctx.clone()).into()
+ Self::new(rows, Schema { fields }, self.base.ctx().clone()).into()
}
}
diff --git a/src/frontend/src/optimizer/plan_node/merge_eq_nodes.rs b/src/frontend/src/optimizer/plan_node/merge_eq_nodes.rs
index 73f82e86aa260..9f2e8d94634be 100644
--- a/src/frontend/src/optimizer/plan_node/merge_eq_nodes.rs
+++ b/src/frontend/src/optimizer/plan_node/merge_eq_nodes.rs
@@ -15,6 +15,7 @@
use std::collections::HashMap;
use std::hash::Hash;
+use super::generic::GenericPlanRef;
use super::{EndoPlan, LogicalShare, PlanNodeId, PlanRef, PlanTreeNodeUnary, VisitPlan};
use crate::optimizer::plan_visitor;
use crate::utils::{Endo, Visit};
diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs
index 188787c93b8c0..f16ebfb0c792c 100644
--- a/src/frontend/src/optimizer/plan_node/mod.rs
+++ b/src/frontend/src/optimizer/plan_node/mod.rs
@@ -46,7 +46,7 @@ use serde::Serialize;
use smallvec::SmallVec;
use self::batch::BatchPlanRef;
-use self::generic::GenericPlanRef;
+use self::generic::{GenericPlanRef, PhysicalPlanRef};
use self::stream::StreamPlanRef;
use self::utils::Distill;
use super::property::{Distribution, FunctionalDependencySet, Order};
@@ -419,29 +419,31 @@ impl PlanTreeNode for PlanRef {
}
}
-impl StreamPlanRef for PlanRef {
- fn distribution(&self) -> &Distribution {
- &self.plan_base().dist
+impl PlanNodeMeta for PlanRef {
+ fn node_type(&self) -> PlanNodeType {
+ self.0.node_type()
}
- fn append_only(&self) -> bool {
- self.plan_base().append_only
+ fn plan_base(&self) -> &PlanBase {
+ self.0.plan_base()
}
- fn emit_on_window_close(&self) -> bool {
- self.plan_base().emit_on_window_close
+ fn convention(&self) -> Convention {
+ self.0.convention()
}
}
-impl BatchPlanRef for PlanRef {
- fn order(&self) -> &Order {
- &self.plan_base().order
+/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`].
+impl GenericPlanRef for P
+where
+ P: PlanNodeMeta + Eq + Hash,
+{
+ fn id(&self) -> PlanNodeId {
+ self.plan_base().id()
}
-}
-impl GenericPlanRef for PlanRef {
fn schema(&self) -> &Schema {
- &self.plan_base().schema
+ self.plan_base().schema()
}
fn stream_key(&self) -> Option<&[usize]> {
@@ -457,6 +459,47 @@ impl GenericPlanRef for PlanRef {
}
}
+/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`].
+// TODO: further constrain the convention to be `Stream` or `Batch`.
+impl
PhysicalPlanRef for P
+where
+ P: PlanNodeMeta + Eq + Hash,
+{
+ fn distribution(&self) -> &Distribution {
+ self.plan_base().distribution()
+ }
+}
+
+/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`].
+// TODO: further constrain the convention to be `Stream`.
+impl
StreamPlanRef for P
+where
+ P: PlanNodeMeta + Eq + Hash,
+{
+ fn append_only(&self) -> bool {
+ self.plan_base().append_only()
+ }
+
+ fn emit_on_window_close(&self) -> bool {
+ self.plan_base().emit_on_window_close()
+ }
+
+ fn watermark_columns(&self) -> &FixedBitSet {
+ self.plan_base().watermark_columns()
+ }
+}
+
+/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`].
+// TODO: further constrain the convention to be `Batch`.
+impl
BatchPlanRef for P
+where
+ P: PlanNodeMeta + Eq + Hash,
+{
+ fn order(&self) -> &Order {
+ self.plan_base().order()
+ }
+}
+
/// In order to let expression display id started from 1 for explaining, hidden column names and
/// other places. We will reset expression display id to 0 and clone the whole plan to reset the
/// schema.
@@ -512,15 +555,15 @@ pub(crate) fn pretty_config() -> PrettyConfig {
impl dyn PlanNode {
pub fn id(&self) -> PlanNodeId {
- self.plan_base().id
+ self.plan_base().id()
}
pub fn ctx(&self) -> OptimizerContextRef {
- self.plan_base().ctx.clone()
+ self.plan_base().ctx().clone()
}
pub fn schema(&self) -> &Schema {
- &self.plan_base().schema
+ self.plan_base().schema()
}
pub fn stream_key(&self) -> Option<&[usize]> {
@@ -528,27 +571,28 @@ impl dyn PlanNode {
}
pub fn order(&self) -> &Order {
- &self.plan_base().order
+ self.plan_base().order()
}
+ // TODO: avoid no manual delegation
pub fn distribution(&self) -> &Distribution {
- &self.plan_base().dist
+ self.plan_base().distribution()
}
pub fn append_only(&self) -> bool {
- self.plan_base().append_only
+ self.plan_base().append_only()
}
pub fn emit_on_window_close(&self) -> bool {
- self.plan_base().emit_on_window_close
+ self.plan_base().emit_on_window_close()
}
pub fn functional_dependency(&self) -> &FunctionalDependencySet {
- &self.plan_base().functional_dependency
+ self.plan_base().functional_dependency()
}
pub fn watermark_columns(&self) -> &FixedBitSet {
- &self.plan_base().watermark_columns
+ self.plan_base().watermark_columns()
}
/// Serialize the plan node and its children to a stream plan proto.
diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs
index e9a5bf26885bf..51b1aa5f41141 100644
--- a/src/frontend/src/optimizer/plan_node/plan_base.rs
+++ b/src/frontend/src/optimizer/plan_node/plan_base.rs
@@ -14,47 +14,132 @@
use educe::Educe;
use fixedbitset::FixedBitSet;
-use paste::paste;
use risingwave_common::catalog::Schema;
use super::generic::GenericPlanNode;
use super::*;
-use crate::for_all_plan_nodes;
use crate::optimizer::optimizer_context::OptimizerContextRef;
use crate::optimizer::property::{Distribution, FunctionalDependencySet, Order};
-/// the common fields of all nodes, please make a field named `base` in
-/// every planNode and correctly value it when construct the planNode.
-#[derive(Clone, Debug, Educe)]
-#[educe(PartialEq, Eq, Hash)]
-pub struct PlanBase {
- #[educe(PartialEq(ignore))]
- #[educe(Hash(ignore))]
- pub id: PlanNodeId,
- #[educe(PartialEq(ignore))]
- #[educe(Hash(ignore))]
- pub ctx: OptimizerContextRef,
- pub schema: Schema,
- /// the pk indices of the PlanNode's output, a empty stream key vec means there is no stream key
- pub stream_key: Option>,
- /// The order property of the PlanNode's output, store an `&Order::any()` here will not affect
- /// correctness, but insert unnecessary sort in plan
- pub order: Order,
+/// Common extra fields for physical plan nodes.
+#[derive(Clone, Debug, PartialEq, Eq, Hash)]
+struct PhysicalCommonExtra {
/// The distribution property of the PlanNode's output, store an `Distribution::any()` here
/// will not affect correctness, but insert unnecessary exchange in plan
- pub dist: Distribution,
+ dist: Distribution,
+}
+
+/// Extra fields for stream plan nodes.
+#[derive(Clone, Debug, PartialEq, Eq, Hash)]
+struct StreamExtra {
+ /// Common fields for physical plan nodes.
+ physical: PhysicalCommonExtra,
+
/// The append-only property of the PlanNode's output is a stream-only property. Append-only
/// means the stream contains only insert operation.
- pub append_only: bool,
+ append_only: bool,
/// Whether the output is emitted on window close.
- pub emit_on_window_close: bool,
- pub functional_dependency: FunctionalDependencySet,
+ emit_on_window_close: bool,
/// The watermark column indices of the PlanNode's output. There could be watermark output from
/// this stream operator.
- pub watermark_columns: FixedBitSet,
+ watermark_columns: FixedBitSet,
+}
+
+/// Extra fields for batch plan nodes.
+#[derive(Clone, Debug, PartialEq, Eq, Hash)]
+struct BatchExtra {
+ /// Common fields for physical plan nodes.
+ physical: PhysicalCommonExtra,
+
+ /// The order property of the PlanNode's output, store an `&Order::any()` here will not affect
+ /// correctness, but insert unnecessary sort in plan
+ order: Order,
+}
+
+/// Extra fields for physical plan nodes.
+#[derive(Clone, Debug, PartialEq, Eq, Hash)]
+enum PhysicalExtra {
+ Stream(StreamExtra),
+ Batch(BatchExtra),
+}
+
+impl PhysicalExtra {
+ fn common(&self) -> &PhysicalCommonExtra {
+ match self {
+ PhysicalExtra::Stream(stream) => &stream.physical,
+ PhysicalExtra::Batch(batch) => &batch.physical,
+ }
+ }
+
+ fn common_mut(&mut self) -> &mut PhysicalCommonExtra {
+ match self {
+ PhysicalExtra::Stream(stream) => &mut stream.physical,
+ PhysicalExtra::Batch(batch) => &mut batch.physical,
+ }
+ }
+
+ fn stream(&self) -> &StreamExtra {
+ match self {
+ PhysicalExtra::Stream(extra) => extra,
+ _ => panic!("access stream properties from batch plan node"),
+ }
+ }
+
+ fn batch(&self) -> &BatchExtra {
+ match self {
+ PhysicalExtra::Batch(extra) => extra,
+ _ => panic!("access batch properties from stream plan node"),
+ }
+ }
+}
+
+/// the common fields of all nodes, please make a field named `base` in
+/// every planNode and correctly value it when construct the planNode.
+///
+/// All fields are intentionally made private and immutable, as they should
+/// normally be the same as the given [`GenericPlanNode`] when constructing.
+///
+/// - To access them, use traits including [`GenericPlanRef`],
+/// [`PhysicalPlanRef`], [`StreamPlanRef`] and [`BatchPlanRef`].
+/// - To mutate them, use methods like `new_*` or `clone_with_*`.
+#[derive(Clone, Debug, Educe)]
+#[educe(PartialEq, Eq, Hash)]
+pub struct PlanBase {
+ // -- common fields --
+ #[educe(PartialEq(ignore), Hash(ignore))]
+ id: PlanNodeId,
+ #[educe(PartialEq(ignore), Hash(ignore))]
+ ctx: OptimizerContextRef,
+
+ schema: Schema,
+ /// the pk indices of the PlanNode's output, a empty stream key vec means there is no stream key
+ // TODO: this is actually a logical and stream only property
+ stream_key: Option>,
+ functional_dependency: FunctionalDependencySet,
+
+ /// Extra fields if the plan node is physical.
+ physical_extra: Option,
+}
+
+impl PlanBase {
+ fn physical_extra(&self) -> &PhysicalExtra {
+ self.physical_extra
+ .as_ref()
+ .expect("access physical properties from logical plan node")
+ }
+
+ fn physical_extra_mut(&mut self) -> &mut PhysicalExtra {
+ self.physical_extra
+ .as_mut()
+ .expect("access physical properties from logical plan node")
+ }
}
impl generic::GenericPlanRef for PlanBase {
+ fn id(&self) -> PlanNodeId {
+ self.id
+ }
+
fn schema(&self) -> &Schema {
&self.schema
}
@@ -72,23 +157,29 @@ impl generic::GenericPlanRef for PlanBase {
}
}
-impl stream::StreamPlanRef for PlanBase {
+impl generic::PhysicalPlanRef for PlanBase {
fn distribution(&self) -> &Distribution {
- &self.dist
+ &self.physical_extra().common().dist
}
+}
+impl stream::StreamPlanRef for PlanBase {
fn append_only(&self) -> bool {
- self.append_only
+ self.physical_extra().stream().append_only
}
fn emit_on_window_close(&self) -> bool {
- self.emit_on_window_close
+ self.physical_extra().stream().emit_on_window_close
+ }
+
+ fn watermark_columns(&self) -> &FixedBitSet {
+ &self.physical_extra().stream().watermark_columns
}
}
impl batch::BatchPlanRef for PlanBase {
fn order(&self) -> &Order {
- &self.order
+ &self.physical_extra().batch().order
}
}
@@ -100,47 +191,22 @@ impl PlanBase {
functional_dependency: FunctionalDependencySet,
) -> Self {
let id = ctx.next_plan_node_id();
- let watermark_columns = FixedBitSet::with_capacity(schema.len());
Self {
id,
ctx,
schema,
stream_key,
- dist: Distribution::Single,
- order: Order::any(),
- // Logical plan node won't touch `append_only` field
- append_only: true,
- emit_on_window_close: false,
functional_dependency,
- watermark_columns,
+ physical_extra: None,
}
}
- pub fn new_logical_with_core(node: &impl GenericPlanNode) -> Self {
+ pub fn new_logical_with_core(core: &impl GenericPlanNode) -> Self {
Self::new_logical(
- node.ctx(),
- node.schema(),
- node.stream_key(),
- node.functional_dependency(),
- )
- }
-
- pub fn new_stream_with_logical(
- logical: &impl GenericPlanNode,
- dist: Distribution,
- append_only: bool,
- emit_on_window_close: bool,
- watermark_columns: FixedBitSet,
- ) -> Self {
- Self::new_stream(
- logical.ctx(),
- logical.schema(),
- logical.stream_key(),
- logical.functional_dependency(),
- dist,
- append_only,
- emit_on_window_close,
- watermark_columns,
+ core.ctx(),
+ core.schema(),
+ core.stream_key(),
+ core.functional_dependency(),
)
}
@@ -160,22 +226,36 @@ impl PlanBase {
id,
ctx,
schema,
- dist,
- order: Order::any(),
stream_key,
- append_only,
- emit_on_window_close,
functional_dependency,
- watermark_columns,
+ physical_extra: Some(PhysicalExtra::Stream({
+ StreamExtra {
+ physical: PhysicalCommonExtra { dist },
+ append_only,
+ emit_on_window_close,
+ watermark_columns,
+ }
+ })),
}
}
- pub fn new_batch_from_logical(
- logical: &impl GenericPlanNode,
+ pub fn new_stream_with_core(
+ core: &impl GenericPlanNode,
dist: Distribution,
- order: Order,
+ append_only: bool,
+ emit_on_window_close: bool,
+ watermark_columns: FixedBitSet,
) -> Self {
- Self::new_batch(logical.ctx(), logical.schema(), dist, order)
+ Self::new_stream(
+ core.ctx(),
+ core.schema(),
+ core.stream_key(),
+ core.functional_dependency(),
+ dist,
+ append_only,
+ emit_on_window_close,
+ watermark_columns,
+ )
}
pub fn new_batch(
@@ -186,75 +266,49 @@ impl PlanBase {
) -> Self {
let id = ctx.next_plan_node_id();
let functional_dependency = FunctionalDependencySet::new(schema.len());
- let watermark_columns = FixedBitSet::with_capacity(schema.len());
Self {
id,
ctx,
schema,
- dist,
- order,
stream_key: None,
- // Batch plan node won't touch `append_only` field
- append_only: true,
- emit_on_window_close: false, // TODO(rc): batch EOWC support?
functional_dependency,
- watermark_columns,
+ physical_extra: Some(PhysicalExtra::Batch({
+ BatchExtra {
+ physical: PhysicalCommonExtra { dist },
+ order,
+ }
+ })),
}
}
- pub fn derive_stream_plan_base(plan_node: &PlanRef) -> Self {
- PlanBase::new_stream(
- plan_node.ctx(),
- plan_node.schema().clone(),
- plan_node.stream_key().map(|v| v.to_vec()),
- plan_node.functional_dependency().clone(),
- plan_node.distribution().clone(),
- plan_node.append_only(),
- plan_node.emit_on_window_close(),
- plan_node.watermark_columns().clone(),
- )
+ pub fn new_batch_with_core(
+ core: &impl GenericPlanNode,
+ dist: Distribution,
+ order: Order,
+ ) -> Self {
+ Self::new_batch(core.ctx(), core.schema(), dist, order)
}
pub fn clone_with_new_plan_id(&self) -> Self {
let mut new = self.clone();
- new.id = self.ctx.next_plan_node_id();
+ new.id = self.ctx().next_plan_node_id();
+ new
+ }
+
+ /// Clone the plan node with a new distribution.
+ ///
+ /// Panics if the plan node is not physical.
+ pub fn clone_with_new_distribution(&self, dist: Distribution) -> Self {
+ let mut new = self.clone();
+ new.physical_extra_mut().common_mut().dist = dist;
new
}
}
-macro_rules! impl_base_delegate {
- ($( { $convention:ident, $name:ident }),*) => {
- $(paste! {
- impl [<$convention $name>] {
- pub fn id(&self) -> PlanNodeId {
- self.plan_base().id
- }
- pub fn ctx(&self) -> OptimizerContextRef {
- self.plan_base().ctx()
- }
- pub fn schema(&self) -> &Schema {
- &self.plan_base().schema
- }
- pub fn stream_key(&self) -> Option<&[usize]> {
- self.plan_base().stream_key()
- }
- pub fn order(&self) -> &Order {
- &self.plan_base().order
- }
- pub fn distribution(&self) -> &Distribution {
- &self.plan_base().dist
- }
- pub fn append_only(&self) -> bool {
- self.plan_base().append_only
- }
- pub fn emit_on_window_close(&self) -> bool {
- self.plan_base().emit_on_window_close
- }
- pub fn functional_dependency(&self) -> &FunctionalDependencySet {
- &self.plan_base().functional_dependency
- }
- }
- })*
+// Mutators for testing only.
+#[cfg(test)]
+impl PlanBase {
+ pub fn functional_dependency_mut(&mut self) -> &mut FunctionalDependencySet {
+ &mut self.functional_dependency
}
}
-for_all_plan_nodes! { impl_base_delegate }
diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs
index 2edf997bf91fd..866c62c2413a5 100644
--- a/src/frontend/src/optimizer/plan_node/stream.rs
+++ b/src/frontend/src/optimizer/plan_node/stream.rs
@@ -12,16 +12,20 @@
// See the License for the specific language governing permissions and
// limitations under the License.
-use super::generic::GenericPlanRef;
-use crate::optimizer::property::Distribution;
+use fixedbitset::FixedBitSet;
-/// A subtrait of [`GenericPlanRef`] for stream plans.
+use super::generic::PhysicalPlanRef;
+
+/// A subtrait of [`PhysicalPlanRef`] for stream plans.
///
/// Due to the lack of refactoring, all plan nodes currently implement this trait
/// through [`super::PlanBase`]. One may still use this trait as a bound for
-/// expecting a stream plan, in contrast to [`GenericPlanRef`].
-pub trait StreamPlanRef: GenericPlanRef {
- fn distribution(&self) -> &Distribution;
+/// accessing a stream plan, in contrast to [`GenericPlanRef`] or
+/// [`PhysicalPlanRef`].
+///
+/// [`GenericPlanRef`]: super::generic::GenericPlanRef
+pub trait StreamPlanRef: PhysicalPlanRef {
fn append_only(&self) -> bool;
fn emit_on_window_close(&self) -> bool;
+ fn watermark_columns(&self) -> &FixedBitSet;
}
diff --git a/src/frontend/src/optimizer/plan_node/stream_dedup.rs b/src/frontend/src/optimizer/plan_node/stream_dedup.rs
index 6e96d0eab0e93..51b5e589e886e 100644
--- a/src/frontend/src/optimizer/plan_node/stream_dedup.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_dedup.rs
@@ -17,7 +17,7 @@ use risingwave_common::util::sort_util::OrderType;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
use risingwave_pb::stream_plan::DedupNode;
-use super::generic::{self, GenericPlanNode, GenericPlanRef};
+use super::generic::{self, GenericPlanNode, GenericPlanRef, PhysicalPlanRef};
use super::stream::StreamPlanRef;
use super::utils::{impl_distill_by_unit, TableCatalogBuilder};
use super::{ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode};
@@ -37,7 +37,7 @@ impl StreamDedup {
// A dedup operator must be append-only.
assert!(input.append_only());
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
input.distribution().clone(),
true,
diff --git a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs
index db9e6ac296bbf..bb18f9cffdf0f 100644
--- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs
@@ -20,7 +20,7 @@ use risingwave_pb::plan_common::JoinType;
use risingwave_pb::stream_plan::stream_node::NodeBody;
use risingwave_pb::stream_plan::{ArrangementInfo, DeltaIndexJoinNode};
-use super::generic::{self};
+use super::generic::{self, GenericPlanRef};
use super::utils::{childless_record, Distill};
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamNode};
use crate::expr::{Expr, ExprRewriter};
@@ -67,7 +67,7 @@ impl StreamDeltaJoin {
core.i2o_col_mapping().rewrite_bitset(&watermark_columns)
};
// TODO: derive from input
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
dist,
append_only,
@@ -90,7 +90,7 @@ impl StreamDeltaJoin {
impl Distill for StreamDeltaJoin {
fn distill<'a>(&self) -> XmlNode<'a> {
- let verbose = self.base.ctx.is_explain_verbose();
+ let verbose = self.base.ctx().is_explain_verbose();
let mut vec = Vec::with_capacity(if verbose { 3 } else { 2 });
vec.push(("type", Pretty::debug(&self.core.join_type)));
diff --git a/src/frontend/src/optimizer/plan_node/stream_dml.rs b/src/frontend/src/optimizer/plan_node/stream_dml.rs
index c9f969384c3a4..9b000974786e4 100644
--- a/src/frontend/src/optimizer/plan_node/stream_dml.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_dml.rs
@@ -17,6 +17,7 @@ use pretty_xmlish::{Pretty, XmlNode};
use risingwave_common::catalog::{ColumnDesc, INITIAL_TABLE_VERSION_ID};
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
+use super::stream::StreamPlanRef;
use super::utils::{childless_record, Distill};
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::stream_fragmenter::BuildFragmentGraphState;
diff --git a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs
index e1ca18da937e9..a4b74f37208e7 100644
--- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs
@@ -17,7 +17,8 @@ pub use risingwave_pb::expr::expr_node::Type as ExprType;
use risingwave_pb::stream_plan::stream_node::NodeBody;
use risingwave_pb::stream_plan::DynamicFilterNode;
-use super::generic::DynamicFilter;
+use super::generic::{DynamicFilter, GenericPlanRef};
+use super::stream::StreamPlanRef;
use super::utils::{childless_record, column_names_pretty, watermark_pretty, Distill};
use super::{generic, ExprRewritable};
use crate::expr::{Expr, ExprImpl};
@@ -37,7 +38,7 @@ impl StreamDynamicFilter {
let watermark_columns = core.watermark_columns(core.right().watermark_columns()[0]);
// TODO: derive from input
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
core.left().distribution().clone(),
false, /* we can have a new abstraction for append only and monotonically increasing
@@ -78,11 +79,11 @@ impl StreamDynamicFilter {
impl Distill for StreamDynamicFilter {
fn distill<'a>(&self) -> XmlNode<'a> {
- let verbose = self.base.ctx.is_explain_verbose();
+ let verbose = self.base.ctx().is_explain_verbose();
let pred = self.core.pretty_field();
let mut vec = Vec::with_capacity(if verbose { 3 } else { 2 });
vec.push(("predicate", pred));
- if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) {
+ if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) {
vec.push(("output_watermarks", ow));
}
vec.push(("output", column_names_pretty(self.schema())));
diff --git a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs
index 9418af8e4a364..d8c5a9635ce59 100644
--- a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs
@@ -18,7 +18,7 @@ use fixedbitset::FixedBitSet;
use risingwave_common::util::sort_util::OrderType;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
-use super::generic::{self, PlanWindowFunction};
+use super::generic::{self, GenericPlanRef, PlanWindowFunction};
use super::utils::{impl_distill_by_unit, TableCatalogBuilder};
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::stream_fragmenter::BuildFragmentGraphState;
@@ -50,7 +50,7 @@ impl StreamEowcOverWindow {
// ancient rows in some rarely updated partitions that are emitted at the end of time.
let watermark_columns = FixedBitSet::with_capacity(core.output_len());
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
input.distribution().clone(),
true,
diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs
index 0fa1713bf4488..99e6c3c5161a1 100644
--- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs
@@ -16,6 +16,8 @@ use pretty_xmlish::{Pretty, XmlNode};
use risingwave_pb::stream_plan::stream_node::NodeBody;
use risingwave_pb::stream_plan::{DispatchStrategy, DispatcherType, ExchangeNode};
+use super::generic::{GenericPlanRef, PhysicalPlanRef};
+use super::stream::StreamPlanRef;
use super::utils::{childless_record, plan_node_name, Distill};
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::optimizer::property::{Distribution, DistributionDisplay};
@@ -78,7 +80,7 @@ impl StreamExchange {
impl Distill for StreamExchange {
fn distill<'a>(&self) -> XmlNode<'a> {
let distribution_display = DistributionDisplay {
- distribution: &self.base.dist,
+ distribution: self.base.distribution(),
input_schema: self.input.schema(),
};
childless_record(
@@ -117,13 +119,13 @@ impl StreamNode for StreamExchange {
})
} else {
Some(DispatchStrategy {
- r#type: match &self.base.dist {
+ r#type: match &self.base.distribution() {
Distribution::HashShard(_) => DispatcherType::Hash,
Distribution::Single => DispatcherType::Simple,
Distribution::Broadcast => DispatcherType::Broadcast,
_ => panic!("Do not allow Any or AnyShard in serialization process"),
} as i32,
- dist_key_indices: match &self.base.dist {
+ dist_key_indices: match &self.base.distribution() {
Distribution::HashShard(keys) => {
keys.iter().map(|num| *num as u32).collect()
}
diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs
index e0f8852a19fb5..5959b8d6be4d2 100644
--- a/src/frontend/src/optimizer/plan_node/stream_expand.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs
@@ -48,7 +48,7 @@ impl StreamExpand {
.map(|idx| idx + input.schema().len()),
);
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
dist,
input.append_only(),
diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs
index ff4d344607776..0f000e6b8c0db 100644
--- a/src/frontend/src/optimizer/plan_node/stream_filter.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs
@@ -34,7 +34,7 @@ impl StreamFilter {
let input = core.input.clone();
let dist = input.distribution().clone();
// Filter executor won't change the append-only behavior of the stream.
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
dist,
input.append_only(),
diff --git a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs
index 190c05c0a5ba1..95fd72e9f6aa0 100644
--- a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs
@@ -48,7 +48,7 @@ impl_plan_tree_node_for_unary! { StreamFsFetch }
impl StreamFsFetch {
pub fn new(input: PlanRef, source: generic::Source) -> Self {
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&source,
Distribution::SomeShard,
source.catalog.as_ref().map_or(true, |s| s.append_only),
diff --git a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs
index 14711d353f9d8..3e8f3c00206c4 100644
--- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs
@@ -16,7 +16,8 @@ use fixedbitset::FixedBitSet;
use pretty_xmlish::XmlNode;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
-use super::generic::{DistillUnit, TopNLimit};
+use super::generic::{DistillUnit, GenericPlanRef, TopNLimit};
+use super::stream::StreamPlanRef;
use super::utils::{plan_node_name, watermark_pretty, Distill};
use super::{generic, ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode};
use crate::optimizer::plan_node::generic::GenericPlanNode;
@@ -135,7 +136,7 @@ impl Distill for StreamGroupTopN {
{ "append_only", self.input().append_only() },
);
let mut node = self.core.distill_with_name(name);
- if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) {
+ if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) {
node.fields.push(("output_watermarks".into(), ow));
}
node
diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs
index 25e1ac801f97c..55ab6b5906e59 100644
--- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs
@@ -18,10 +18,11 @@ use pretty_xmlish::XmlNode;
use risingwave_common::error::{ErrorCode, Result};
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
-use super::generic::{self, PlanAggCall};
+use super::generic::{self, GenericPlanRef, PlanAggCall};
use super::utils::{childless_record, plan_node_name, watermark_pretty, Distill};
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::expr::ExprRewriter;
+use crate::optimizer::plan_node::stream::StreamPlanRef;
use crate::stream_fragmenter::BuildFragmentGraphState;
use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, IndexSet};
@@ -85,7 +86,7 @@ impl StreamHashAgg {
}
// Hash agg executor might change the append-only behavior of the stream.
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
dist,
emit_on_window_close, // in EOWC mode, we produce append only output
@@ -142,7 +143,7 @@ impl StreamHashAgg {
impl Distill for StreamHashAgg {
fn distill<'a>(&self) -> XmlNode<'a> {
let mut vec = self.core.fields_pretty();
- if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) {
+ if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) {
vec.push(("output_watermarks", ow));
}
childless_record(
@@ -214,7 +215,7 @@ impl StreamNode for StreamHashAgg {
})
.collect(),
row_count_index: self.row_count_idx as u32,
- emit_on_window_close: self.base.emit_on_window_close,
+ emit_on_window_close: self.base.emit_on_window_close(),
})
}
}
diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs
index 0075b1730b4eb..9d9c41425c4b1 100644
--- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs
@@ -20,7 +20,8 @@ use risingwave_pb::plan_common::JoinType;
use risingwave_pb::stream_plan::stream_node::NodeBody;
use risingwave_pb::stream_plan::{DeltaExpression, HashJoinNode, PbInequalityPair};
-use super::generic::Join;
+use super::generic::{GenericPlanRef, Join};
+use super::stream::StreamPlanRef;
use super::utils::{childless_record, plan_node_name, watermark_pretty, Distill};
use super::{
generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamDeltaJoin, StreamNode,
@@ -178,7 +179,7 @@ impl StreamHashJoin {
};
// TODO: derive from input
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
dist,
append_only,
@@ -291,7 +292,7 @@ impl Distill for StreamHashJoin {
{ "interval", self.clean_left_state_conjunction_idx.is_some() && self.clean_right_state_conjunction_idx.is_some() },
{ "append_only", self.is_append_only },
);
- let verbose = self.base.ctx.is_explain_verbose();
+ let verbose = self.base.ctx().is_explain_verbose();
let mut vec = Vec::with_capacity(6);
vec.push(("type", Pretty::debug(&self.core.join_type)));
@@ -316,7 +317,7 @@ impl Distill for StreamHashJoin {
if let Some(i) = self.clean_right_state_conjunction_idx {
vec.push(("conditions_to_clean_right_state_table", get_cond(i)));
}
- if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) {
+ if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) {
vec.push(("output_watermarks", ow));
}
diff --git a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs
index c68b1b307d470..e177be6942360 100644
--- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs
@@ -17,6 +17,8 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
use risingwave_pb::stream_plan::HopWindowNode;
+use super::generic::GenericPlanRef;
+use super::stream::StreamPlanRef;
use super::utils::{childless_record, watermark_pretty, Distill};
use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::expr::{Expr, ExprImpl, ExprRewriter};
@@ -56,7 +58,7 @@ impl StreamHopWindow {
)
.rewrite_bitset(&watermark_columns);
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
dist,
input.append_only(),
@@ -75,7 +77,7 @@ impl StreamHopWindow {
impl Distill for StreamHopWindow {
fn distill<'a>(&self) -> XmlNode<'a> {
let mut vec = self.core.fields_pretty();
- if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) {
+ if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) {
vec.push(("output_watermarks", ow));
}
childless_record("StreamHopWindow", vec)
diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs
index fb17537bc90e6..d8972436d5c78 100644
--- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs
@@ -24,11 +24,13 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType};
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
use super::derive::derive_columns;
+use super::stream::StreamPlanRef;
use super::utils::{childless_record, Distill};
use super::{reorganize_elements_id, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::catalog::table_catalog::{CreateType, TableCatalog, TableType, TableVersion};
use crate::catalog::FragmentId;
use crate::optimizer::plan_node::derive::derive_pk;
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::{PlanBase, PlanNodeMeta};
use crate::optimizer::property::{Cardinality, Distribution, Order, RequiredDist};
use crate::stream_fragmenter::BuildFragmentGraphState;
@@ -273,8 +275,8 @@ impl Distill for StreamMaterialize {
vec.push(("pk_conflict", Pretty::from(pk_conflict_behavior)));
- let watermark_columns = &self.base.watermark_columns;
- if self.base.watermark_columns.count_ones(..) > 0 {
+ let watermark_columns = &self.base.watermark_columns();
+ if self.base.watermark_columns().count_ones(..) > 0 {
let watermark_column_names = watermark_columns
.ones()
.map(|i| table.columns()[i].name_with_hidden().to_string())
@@ -294,16 +296,16 @@ impl PlanTreeNodeUnary for StreamMaterialize {
fn clone_with_input(&self, input: PlanRef) -> Self {
let new = Self::new(input, self.table().clone());
new.base
- .schema
+ .schema()
.fields
.iter()
- .zip_eq_fast(self.base.schema.fields.iter())
+ .zip_eq_fast(self.base.schema().fields.iter())
.for_each(|(a, b)| {
assert_eq!(a.data_type, b.data_type);
assert_eq!(a.type_name, b.type_name);
assert_eq!(a.sub_fields, b.sub_fields);
});
- assert_eq!(new.plan_base().stream_key, self.plan_base().stream_key);
+ assert_eq!(new.plan_base().stream_key(), self.plan_base().stream_key());
new
}
}
diff --git a/src/frontend/src/optimizer/plan_node/stream_now.rs b/src/frontend/src/optimizer/plan_node/stream_now.rs
index 9eb0a0e0f143e..91ebc344fa51d 100644
--- a/src/frontend/src/optimizer/plan_node/stream_now.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_now.rs
@@ -19,8 +19,7 @@ use risingwave_common::types::DataType;
use risingwave_pb::stream_plan::stream_node::NodeBody;
use risingwave_pb::stream_plan::NowNode;
-use super::generic::GenericPlanRef;
-use super::stream::StreamPlanRef;
+use super::generic::{GenericPlanRef, PhysicalPlanRef};
use super::utils::{childless_record, Distill, TableCatalogBuilder};
use super::{ExprRewritable, LogicalNow, PlanBase, StreamNode};
use crate::optimizer::plan_node::utils::column_names_pretty;
@@ -59,7 +58,7 @@ impl StreamNow {
impl Distill for StreamNow {
fn distill<'a>(&self) -> XmlNode<'a> {
- let vec = if self.base.ctx.is_explain_verbose() {
+ let vec = if self.base.ctx().is_explain_verbose() {
vec![("output", column_names_pretty(self.schema()))]
} else {
vec![]
diff --git a/src/frontend/src/optimizer/plan_node/stream_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_over_window.rs
index 0d749f0c7b0e6..5a2f9d98f1340 100644
--- a/src/frontend/src/optimizer/plan_node/stream_over_window.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_over_window.rs
@@ -21,6 +21,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody;
use super::generic::{GenericPlanNode, PlanWindowFunction};
use super::utils::{impl_distill_by_unit, TableCatalogBuilder};
use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::stream_fragmenter::BuildFragmentGraphState;
use crate::TableCatalog;
@@ -37,7 +38,7 @@ impl StreamOverWindow {
let input = &core.input;
let watermark_columns = FixedBitSet::with_capacity(core.output_len());
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
input.distribution().clone(),
false, // general over window cannot be append-only
@@ -122,7 +123,7 @@ impl StreamNode for StreamOverWindow {
.to_internal_table_prost();
let cache_policy = self
.base
- .ctx
+ .ctx()
.session_ctx()
.config()
.get_streaming_over_window_cache_policy();
diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs
index 8a7665881e0cf..c0ff0d1cf2f43 100644
--- a/src/frontend/src/optimizer/plan_node/stream_project.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_project.rs
@@ -17,6 +17,8 @@ use pretty_xmlish::XmlNode;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
use risingwave_pb::stream_plan::ProjectNode;
+use super::generic::GenericPlanRef;
+use super::stream::StreamPlanRef;
use super::utils::{childless_record, watermark_pretty, Distill};
use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::expr::{try_derive_watermark, Expr, ExprImpl, ExprRewriter, WatermarkDerivation};
@@ -41,7 +43,7 @@ impl Distill for StreamProject {
let schema = self.schema();
let mut vec = self.core.fields_pretty(schema);
if let Some(display_output_watermarks) =
- watermark_pretty(&self.base.watermark_columns, schema)
+ watermark_pretty(self.base.watermark_columns(), schema)
{
vec.push(("output_watermarks", display_output_watermarks));
}
@@ -79,7 +81,7 @@ impl StreamProject {
}
// Project executor won't change the append-only behavior of the stream, so it depends on
// input's `append_only`.
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
distribution,
input.append_only(),
diff --git a/src/frontend/src/optimizer/plan_node/stream_project_set.rs b/src/frontend/src/optimizer/plan_node/stream_project_set.rs
index cadd600f3c3b7..ba09d79c96c60 100644
--- a/src/frontend/src/optimizer/plan_node/stream_project_set.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_project_set.rs
@@ -66,7 +66,7 @@ impl StreamProjectSet {
// ProjectSet executor won't change the append-only behavior of the stream, so it depends on
// input's `append_only`.
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
distribution,
input.append_only(),
diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs
index 8b406005f40a6..3acf0b132805e 100644
--- a/src/frontend/src/optimizer/plan_node/stream_share.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_share.rs
@@ -16,6 +16,8 @@ use pretty_xmlish::XmlNode;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
use risingwave_pb::stream_plan::PbStreamNode;
+use super::generic::GenericPlanRef;
+use super::stream::StreamPlanRef;
use super::utils::Distill;
use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamExchange, StreamNode};
use crate::optimizer::plan_node::{LogicalShare, PlanBase, PlanTreeNode};
@@ -34,7 +36,7 @@ impl StreamShare {
let input = core.input.borrow().0.clone();
let dist = input.distribution().clone();
// Filter executor won't change the append-only behavior of the stream.
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
dist,
input.append_only(),
@@ -79,7 +81,7 @@ impl StreamNode for StreamShare {
impl StreamShare {
pub fn adhoc_to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> PbStreamNode {
- let operator_id = self.base.id.0 as u32;
+ let operator_id = self.base.id().0 as u32;
match state.get_share_stream_node(operator_id) {
None => {
diff --git a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs
index 59311dd22226c..92d96fdf21b08 100644
--- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs
@@ -21,6 +21,8 @@ use super::generic::{self, PlanAggCall};
use super::utils::{childless_record, plan_node_name, Distill};
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::expr::ExprRewriter;
+use crate::optimizer::plan_node::generic::PhysicalPlanRef;
+use crate::optimizer::plan_node::stream::StreamPlanRef;
use crate::optimizer::property::Distribution;
use crate::stream_fragmenter::BuildFragmentGraphState;
@@ -48,7 +50,7 @@ impl StreamSimpleAgg {
let watermark_columns = FixedBitSet::with_capacity(core.output_len());
// Simple agg executor might change the append-only behavior of the stream.
- let base = PlanBase::new_stream_with_logical(&core, dist, false, false, watermark_columns);
+ let base = PlanBase::new_stream_with_core(&core, dist, false, false, watermark_columns);
StreamSimpleAgg {
base,
core,
@@ -99,7 +101,7 @@ impl StreamNode for StreamSimpleAgg {
.collect(),
distribution_key: self
.base
- .dist
+ .distribution()
.dist_column_indices()
.iter()
.map(|idx| *idx as u32)
diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs
index a51380d630331..32e9fb487910c 100644
--- a/src/frontend/src/optimizer/plan_node/stream_sink.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs
@@ -37,6 +37,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody;
use tracing::info;
use super::derive::{derive_columns, derive_pk};
+use super::generic::GenericPlanRef;
use super::utils::{childless_record, Distill, IndicesDisplay, TableCatalogBuilder};
use super::{ExprRewritable, PlanBase, PlanRef, StreamNode};
use crate::optimizer::plan_node::PlanTreeNodeUnary;
@@ -57,7 +58,7 @@ pub struct StreamSink {
impl StreamSink {
#[must_use]
pub fn new(input: PlanRef, sink_desc: SinkDesc) -> Self {
- let base = PlanBase::derive_stream_plan_base(&input);
+ let base = input.plan_base().clone_with_new_plan_id();
Self {
base,
input,
@@ -389,7 +390,7 @@ impl Distill for StreamSink {
.iter()
.map(|k| k.column_index)
.collect_vec(),
- schema: &self.base.schema,
+ schema: self.base.schema(),
};
vec.push(("pk", pk.distill()));
}
@@ -409,7 +410,7 @@ impl StreamNode for StreamSink {
PbNodeBody::Sink(SinkNode {
sink_desc: Some(self.sink_desc.to_proto()),
table: Some(table.to_internal_table_prost()),
- log_store_type: match self.base.ctx.session_ctx().config().get_sink_decouple() {
+ log_store_type: match self.base.ctx().session_ctx().config().get_sink_decouple() {
SinkDecouple::Default => {
let enable_sink_decouple =
match_sink_name_str!(
diff --git a/src/frontend/src/optimizer/plan_node/stream_sort.rs b/src/frontend/src/optimizer/plan_node/stream_sort.rs
index b82d71068d817..41a56a0fd5df2 100644
--- a/src/frontend/src/optimizer/plan_node/stream_sort.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_sort.rs
@@ -20,6 +20,8 @@ use risingwave_common::catalog::FieldDisplay;
use risingwave_common::util::sort_util::OrderType;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
+use super::generic::{GenericPlanRef, PhysicalPlanRef};
+use super::stream::StreamPlanRef;
use super::utils::{childless_record, Distill, TableCatalogBuilder};
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::stream_fragmenter::BuildFragmentGraphState;
@@ -84,7 +86,7 @@ impl StreamEowcSort {
tbl_builder.add_order_column(self.sort_column_index, OrderType::ascending());
order_cols.insert(self.sort_column_index);
- let dist_key = self.base.dist.dist_column_indices().to_vec();
+ let dist_key = self.base.distribution().dist_column_indices().to_vec();
for idx in &dist_key {
if !order_cols.contains(idx) {
tbl_builder.add_order_column(*idx, OrderType::ascending());
diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs
index 377e2704776bb..ae66cf568118b 100644
--- a/src/frontend/src/optimizer/plan_node/stream_source.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_source.rs
@@ -37,7 +37,7 @@ pub struct StreamSource {
impl StreamSource {
pub fn new(core: generic::Source) -> Self {
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
Distribution::SomeShard,
core.catalog.as_ref().map_or(true, |s| s.append_only),
diff --git a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs
index 0af7ebded94d9..474582ec877c7 100644
--- a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs
@@ -20,6 +20,7 @@ use super::generic::{self, PlanAggCall};
use super::utils::impl_distill_by_unit;
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::expr::ExprRewriter;
+use crate::optimizer::plan_node::generic::PhysicalPlanRef;
use crate::optimizer::property::RequiredDist;
use crate::stream_fragmenter::BuildFragmentGraphState;
@@ -49,7 +50,7 @@ impl StreamStatelessSimpleAgg {
}
}
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
input_dist.clone(),
input.append_only(),
diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs
index 907a41db28525..965ca217a3369 100644
--- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs
@@ -24,11 +24,13 @@ use risingwave_common::util::sort_util::OrderType;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
use risingwave_pb::stream_plan::{ChainType, PbStreamNode};
+use super::generic::PhysicalPlanRef;
use super::utils::{childless_record, Distill};
use super::{generic, ExprRewritable, PlanBase, PlanNodeId, PlanRef, StreamNode};
use crate::catalog::ColumnId;
use crate::expr::{ExprRewriter, FunctionCall};
use crate::optimizer::plan_node::generic::GenericPlanRef;
+use crate::optimizer::plan_node::stream::StreamPlanRef;
use crate::optimizer::plan_node::utils::{IndicesDisplay, TableCatalogBuilder};
use crate::optimizer::property::{Distribution, DistributionDisplay};
use crate::stream_fragmenter::BuildFragmentGraphState;
@@ -66,7 +68,7 @@ impl StreamTableScan {
None => Distribution::SomeShard,
}
};
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
distribution,
core.table_desc.append_only,
@@ -192,7 +194,7 @@ impl_plan_tree_node_for_leaf! { StreamTableScan }
impl Distill for StreamTableScan {
fn distill<'a>(&self) -> XmlNode<'a> {
- let verbose = self.base.ctx.is_explain_verbose();
+ let verbose = self.base.ctx().is_explain_verbose();
let mut vec = Vec::with_capacity(4);
vec.push(("table", Pretty::from(self.core.table_name.clone())));
vec.push(("columns", self.core.columns_pretty(verbose)));
@@ -200,12 +202,12 @@ impl Distill for StreamTableScan {
if verbose {
let pk = IndicesDisplay {
indices: self.stream_key().unwrap_or_default(),
- schema: &self.base.schema,
+ schema: self.base.schema(),
};
vec.push(("pk", pk.distill()));
let dist = Pretty::display(&DistributionDisplay {
distribution: self.distribution(),
- input_schema: &self.base.schema,
+ input_schema: self.base.schema(),
});
vec.push(("dist", dist));
}
@@ -325,7 +327,7 @@ impl StreamTableScan {
..Default::default()
})),
stream_key,
- operator_id: self.base.id.0 as u64,
+ operator_id: self.base.id().0 as u64,
identity: {
let s = self.distill_to_string();
s.replace("StreamTableScan", "Chain")
diff --git a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs
index 2191ca322342d..675dbeb9ab381 100644
--- a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs
@@ -18,6 +18,8 @@ use risingwave_pb::plan_common::JoinType;
use risingwave_pb::stream_plan::stream_node::NodeBody;
use risingwave_pb::stream_plan::TemporalJoinNode;
+use super::generic::GenericPlanRef;
+use super::stream::StreamPlanRef;
use super::utils::{childless_record, watermark_pretty, Distill};
use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamNode};
use crate::expr::{Expr, ExprRewriter};
@@ -61,7 +63,7 @@ impl StreamTemporalJoin {
.rewrite_bitset(core.left.watermark_columns()),
);
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
dist,
true,
@@ -88,7 +90,7 @@ impl StreamTemporalJoin {
impl Distill for StreamTemporalJoin {
fn distill<'a>(&self) -> XmlNode<'a> {
- let verbose = self.base.ctx.is_explain_verbose();
+ let verbose = self.base.ctx().is_explain_verbose();
let mut vec = Vec::with_capacity(if verbose { 3 } else { 2 });
vec.push(("type", Pretty::debug(&self.core.join_type)));
@@ -101,7 +103,7 @@ impl Distill for StreamTemporalJoin {
}),
));
- if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) {
+ if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) {
vec.push(("output_watermarks", ow));
}
diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs
index e7a880fa7d757..87890625f6be7 100644
--- a/src/frontend/src/optimizer/plan_node/stream_topn.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs
@@ -40,7 +40,7 @@ impl StreamTopN {
};
let watermark_columns = FixedBitSet::with_capacity(input.schema().len());
- let base = PlanBase::new_stream_with_logical(&core, dist, false, false, watermark_columns);
+ let base = PlanBase::new_stream_with_core(&core, dist, false, false, watermark_columns);
StreamTopN { base, core }
}
diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs
index 8f6353d6be44c..6d6dca2d8dd02 100644
--- a/src/frontend/src/optimizer/plan_node/stream_union.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_union.rs
@@ -19,6 +19,8 @@ use pretty_xmlish::XmlNode;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
use risingwave_pb::stream_plan::UnionNode;
+use super::generic::GenericPlanRef;
+use super::stream::StreamPlanRef;
use super::utils::{childless_record, watermark_pretty, Distill};
use super::{generic, ExprRewritable, PlanRef};
use crate::optimizer::plan_node::generic::GenericPlanNode;
@@ -46,7 +48,7 @@ impl StreamUnion {
|acc_watermark_columns, input| acc_watermark_columns.bitand(input.watermark_columns()),
);
- let base = PlanBase::new_stream_with_logical(
+ let base = PlanBase::new_stream_with_core(
&core,
dist,
inputs.iter().all(|x| x.append_only()),
@@ -60,7 +62,7 @@ impl StreamUnion {
impl Distill for StreamUnion {
fn distill<'a>(&self) -> XmlNode<'a> {
let mut vec = self.core.fields_pretty();
- if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) {
+ if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) {
vec.push(("output_watermarks", ow));
}
childless_record("StreamUnion", vec)
diff --git a/src/frontend/src/optimizer/plan_node/stream_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs
index fb0b844411f63..f8cc5db851159 100644
--- a/src/frontend/src/optimizer/plan_node/stream_values.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_values.rs
@@ -18,6 +18,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode;
use risingwave_pb::stream_plan::values_node::ExprTuple;
use risingwave_pb::stream_plan::ValuesNode;
+use super::generic::GenericPlanRef;
use super::utils::{childless_record, Distill};
use super::{ExprRewritable, LogicalValues, PlanBase, StreamNode};
use crate::expr::{Expr, ExprImpl};
diff --git a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs
index ed5a946603ee4..066bc9a234ca5 100644
--- a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs
@@ -21,6 +21,7 @@ use risingwave_common::util::sort_util::OrderType;
use risingwave_pb::catalog::WatermarkDesc;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
+use super::stream::StreamPlanRef;
use super::utils::{childless_record, watermark_pretty, Distill, TableCatalogBuilder};
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::expr::{ExprDisplay, ExprImpl};
@@ -85,7 +86,7 @@ impl Distill for StreamWatermarkFilter {
})
.collect();
let display_output_watermarks =
- watermark_pretty(&self.base.watermark_columns, input_schema).unwrap();
+ watermark_pretty(self.base.watermark_columns(), input_schema).unwrap();
let fields = vec![
("watermark_descs", Pretty::Array(display_watermark_descs)),
("output_watermarks", display_output_watermarks),
diff --git a/src/frontend/src/optimizer/plan_rewriter/plan_cloner.rs b/src/frontend/src/optimizer/plan_rewriter/plan_cloner.rs
index f30f3d9fa4966..7e53b903ac962 100644
--- a/src/frontend/src/optimizer/plan_rewriter/plan_cloner.rs
+++ b/src/frontend/src/optimizer/plan_rewriter/plan_cloner.rs
@@ -16,6 +16,7 @@ use std::collections::HashMap;
use itertools::Itertools;
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::{LogicalShare, PlanNodeId, PlanTreeNode, StreamShare};
use crate::optimizer::PlanRewriter;
use crate::PlanRef;
diff --git a/src/frontend/src/optimizer/plan_rewriter/share_source_rewriter.rs b/src/frontend/src/optimizer/plan_rewriter/share_source_rewriter.rs
index 9ab0d4d580ddc..5b9efb9fc7c94 100644
--- a/src/frontend/src/optimizer/plan_rewriter/share_source_rewriter.rs
+++ b/src/frontend/src/optimizer/plan_rewriter/share_source_rewriter.rs
@@ -17,6 +17,7 @@ use std::collections::{HashMap, HashSet};
use itertools::Itertools;
use crate::catalog::SourceId;
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::{
LogicalShare, LogicalSource, PlanNodeId, PlanTreeNode, StreamShare,
};
diff --git a/src/frontend/src/optimizer/plan_visitor/share_parent_counter.rs b/src/frontend/src/optimizer/plan_visitor/share_parent_counter.rs
index 7d538392f9361..7950b5d81a49c 100644
--- a/src/frontend/src/optimizer/plan_visitor/share_parent_counter.rs
+++ b/src/frontend/src/optimizer/plan_visitor/share_parent_counter.rs
@@ -15,6 +15,7 @@
use std::collections::HashMap;
use super::{DefaultBehavior, DefaultValue};
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::{LogicalShare, PlanNodeId, PlanTreeNodeUnary};
use crate::optimizer::plan_visitor::PlanVisitor;
diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs
index 4fcaf959eac87..b6e7715dd155f 100644
--- a/src/frontend/src/optimizer/property/distribution.rs
+++ b/src/frontend/src/optimizer/property/distribution.rs
@@ -295,10 +295,12 @@ impl RequiredDist {
pub fn enforce_if_not_satisfies(
&self,
- plan: PlanRef,
+ mut plan: PlanRef,
required_order: &Order,
) -> Result {
- let plan = required_order.enforce_if_not_satisfies(plan)?;
+ if let Convention::Batch = plan.convention() {
+ plan = required_order.enforce_if_not_satisfies(plan)?;
+ }
if !plan.distribution().satisfies(self) {
Ok(self.enforce(plan, required_order))
} else {
diff --git a/src/frontend/src/optimizer/property/order.rs b/src/frontend/src/optimizer/property/order.rs
index a70bffb13a8ba..19ad7586e1c11 100644
--- a/src/frontend/src/optimizer/property/order.rs
+++ b/src/frontend/src/optimizer/property/order.rs
@@ -92,7 +92,7 @@ impl Order {
}
}
- pub fn enforce(&self, plan: PlanRef) -> PlanRef {
+ fn enforce(&self, plan: PlanRef) -> PlanRef {
assert_eq!(plan.convention(), Convention::Batch);
BatchSort::new(plan, self.clone()).into()
}
diff --git a/src/frontend/src/optimizer/rule/always_false_filter_rule.rs b/src/frontend/src/optimizer/rule/always_false_filter_rule.rs
index 02165232372e4..eeba7d9f3be3b 100644
--- a/src/frontend/src/optimizer/rule/always_false_filter_rule.rs
+++ b/src/frontend/src/optimizer/rule/always_false_filter_rule.rs
@@ -15,6 +15,7 @@
use risingwave_common::types::ScalarImpl;
use super::Rule;
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::{LogicalFilter, LogicalValues};
use crate::PlanRef;
diff --git a/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs
index 66579248a76f9..7ac121692c81d 100644
--- a/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs
+++ b/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs
@@ -23,6 +23,7 @@ use crate::expr::{
CorrelatedId, CorrelatedInputRef, Expr, ExprImpl, ExprRewriter, ExprType, FunctionCall,
InputRef,
};
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::{LogicalApply, LogicalFilter, LogicalJoin, PlanTreeNodeBinary};
use crate::optimizer::plan_visitor::{ExprCorrelatedIdFinder, PlanCorrelatedIdFinder};
use crate::optimizer::rule::apply_offset_rewriter::ApplyCorrelatedIndicesConverter;
diff --git a/src/frontend/src/optimizer/rule/expand_to_project_rule.rs b/src/frontend/src/optimizer/rule/expand_to_project_rule.rs
index 1ed1da0037aba..01a39042efd98 100644
--- a/src/frontend/src/optimizer/rule/expand_to_project_rule.rs
+++ b/src/frontend/src/optimizer/rule/expand_to_project_rule.rs
@@ -36,7 +36,7 @@ impl Rule for ExpandToProjectRule {
let column_subset = column_subsets.get(0).unwrap();
// if `column_subsets` len equals 1, convert it into a project
- let mut exprs = Vec::with_capacity(expand.base.schema.len());
+ let mut exprs = Vec::with_capacity(expand.base.schema().len());
// Add original input column first
for i in 0..input.schema().len() {
exprs.push(ExprImpl::InputRef(
diff --git a/src/frontend/src/optimizer/rule/index_selection_rule.rs b/src/frontend/src/optimizer/rule/index_selection_rule.rs
index 9103d1bc906bc..323cc59ef3558 100644
--- a/src/frontend/src/optimizer/rule/index_selection_rule.rs
+++ b/src/frontend/src/optimizer/rule/index_selection_rule.rs
@@ -66,6 +66,7 @@ use crate::expr::{
FunctionCall, InputRef,
};
use crate::optimizer::optimizer_context::OptimizerContextRef;
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::{
generic, ColumnPruningContext, LogicalJoin, LogicalScan, LogicalUnion, PlanTreeNode,
PlanTreeNodeBinary, PredicatePushdown, PredicatePushdownContext,
diff --git a/src/frontend/src/optimizer/rule/left_deep_tree_join_ordering_rule.rs b/src/frontend/src/optimizer/rule/left_deep_tree_join_ordering_rule.rs
index dcbb6f7b015ee..bd2db0ac67cca 100644
--- a/src/frontend/src/optimizer/rule/left_deep_tree_join_ordering_rule.rs
+++ b/src/frontend/src/optimizer/rule/left_deep_tree_join_ordering_rule.rs
@@ -47,6 +47,7 @@ mod tests {
use super::*;
use crate::expr::{ExprImpl, FunctionCall, InputRef};
use crate::optimizer::optimizer_context::OptimizerContext;
+ use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::utils::Condition;
#[tokio::test]
diff --git a/src/frontend/src/optimizer/rule/merge_multijoin_rule.rs b/src/frontend/src/optimizer/rule/merge_multijoin_rule.rs
index c496a906400ae..8682db8491a1d 100644
--- a/src/frontend/src/optimizer/rule/merge_multijoin_rule.rs
+++ b/src/frontend/src/optimizer/rule/merge_multijoin_rule.rs
@@ -46,6 +46,7 @@ mod tests {
use super::*;
use crate::expr::{ExprImpl, FunctionCall, InputRef};
use crate::optimizer::optimizer_context::OptimizerContext;
+ use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::utils::Condition;
#[tokio::test]
diff --git a/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs b/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs
index ea8386bc227f8..c32ae40531cd0 100644
--- a/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs
+++ b/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs
@@ -27,7 +27,7 @@ use risingwave_expr::aggregate::AggKind;
use super::{BoxedRule, Rule};
use crate::expr::{ExprImpl, ExprType, FunctionCall, InputRef};
-use crate::optimizer::plan_node::generic::Agg;
+use crate::optimizer::plan_node::generic::{Agg, GenericPlanRef};
use crate::optimizer::plan_node::{
LogicalAgg, LogicalFilter, LogicalScan, LogicalTopN, PlanAggCall, PlanTreeNodeUnary,
};
diff --git a/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs b/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs
index dfb6963c7fb4f..93637d3ba8193 100644
--- a/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs
+++ b/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs
@@ -18,6 +18,7 @@ use risingwave_expr::window_function::WindowFuncKind;
use super::Rule;
use crate::expr::{collect_input_refs, ExprImpl, ExprType};
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::{LogicalFilter, LogicalTopN, PlanTreeNodeUnary};
use crate::optimizer::property::Order;
use crate::planner::LIMIT_ALL_COUNT;
diff --git a/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_rule.rs b/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_rule.rs
index dc5f9c2bc9aba..f34146ba80050 100644
--- a/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_rule.rs
+++ b/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_rule.rs
@@ -18,6 +18,7 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping;
use super::super::plan_node::*;
use super::{BoxedRule, Rule};
use crate::expr::{CorrelatedId, CorrelatedInputRef, Expr, ExprImpl, ExprRewriter, InputRef};
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_visitor::{PlanCorrelatedIdFinder, PlanVisitor};
use crate::optimizer::PlanRef;
use crate::utils::Condition;
diff --git a/src/frontend/src/optimizer/rule/table_function_to_project_set_rule.rs b/src/frontend/src/optimizer/rule/table_function_to_project_set_rule.rs
index 5a6f1187fdd02..f85ffc2318459 100644
--- a/src/frontend/src/optimizer/rule/table_function_to_project_set_rule.rs
+++ b/src/frontend/src/optimizer/rule/table_function_to_project_set_rule.rs
@@ -18,6 +18,7 @@ use risingwave_common::types::DataType;
use super::{BoxedRule, Rule};
use crate::expr::{Expr, ExprImpl, ExprType, FunctionCall, InputRef};
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::{
LogicalProject, LogicalProjectSet, LogicalTableFunction, LogicalValues, PlanTreeNodeUnary,
};
@@ -51,7 +52,7 @@ impl Rule for TableFunctionToProjectSetRule {
let logical_values = LogicalValues::create(
vec![vec![]],
Schema::new(vec![]),
- logical_table_function.base.ctx.clone(),
+ logical_table_function.base.ctx().clone(),
);
let logical_project_set = LogicalProjectSet::create(logical_values, vec![table_function]);
// We need a project to align schema type because
diff --git a/src/frontend/src/optimizer/rule/trivial_project_to_values_rule.rs b/src/frontend/src/optimizer/rule/trivial_project_to_values_rule.rs
index 9759739490fe6..a13bef3baa9d9 100644
--- a/src/frontend/src/optimizer/rule/trivial_project_to_values_rule.rs
+++ b/src/frontend/src/optimizer/rule/trivial_project_to_values_rule.rs
@@ -13,6 +13,7 @@
// limitations under the License.
use super::{BoxedRule, Rule};
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::{LogicalValues, PlanTreeNodeUnary};
use crate::optimizer::plan_visitor::{LogicalCardinalityExt, SideEffectVisitor};
use crate::optimizer::{PlanRef, PlanVisitor};
diff --git a/src/frontend/src/optimizer/rule/union_input_values_merge_rule.rs b/src/frontend/src/optimizer/rule/union_input_values_merge_rule.rs
index 8119b8847b600..7b83c017ab781 100644
--- a/src/frontend/src/optimizer/rule/union_input_values_merge_rule.rs
+++ b/src/frontend/src/optimizer/rule/union_input_values_merge_rule.rs
@@ -13,6 +13,7 @@
// limitations under the License.
use super::{BoxedRule, Rule};
+use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::LogicalValues;
use crate::optimizer::{PlanRef, PlanTreeNode};
diff --git a/src/frontend/src/optimizer/rule/union_to_distinct_rule.rs b/src/frontend/src/optimizer/rule/union_to_distinct_rule.rs
index 2a12f6b712e0d..f1d203fba1350 100644
--- a/src/frontend/src/optimizer/rule/union_to_distinct_rule.rs
+++ b/src/frontend/src/optimizer/rule/union_to_distinct_rule.rs
@@ -13,7 +13,7 @@
// limitations under the License.
use super::{BoxedRule, Rule};
-use crate::optimizer::plan_node::generic::Agg;
+use crate::optimizer::plan_node::generic::{Agg, GenericPlanRef};
use crate::optimizer::plan_node::{LogicalUnion, PlanTreeNode};
use crate::optimizer::PlanRef;
@@ -24,7 +24,7 @@ impl Rule for UnionToDistinctRule {
let union: &LogicalUnion = plan.as_logical_union()?;
if !union.all() {
let union_all = LogicalUnion::create(true, union.inputs().into_iter().collect());
- let distinct = Agg::new(vec![], (0..union.base.schema.len()).collect(), union_all)
+ let distinct = Agg::new(vec![], (0..union.base.schema().len()).collect(), union_all)
.with_enable_two_phase(false);
Some(distinct.into())
} else {
diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs
index 4e16bc6cd0b21..cb20103b3e76f 100644
--- a/src/frontend/src/scheduler/plan_fragmenter.rs
+++ b/src/frontend/src/scheduler/plan_fragmenter.rs
@@ -103,7 +103,7 @@ impl Serialize for ExecutionPlanNode {
impl From for ExecutionPlanNode {
fn from(plan_node: PlanRef) -> Self {
Self {
- plan_node_id: plan_node.plan_base().id,
+ plan_node_id: plan_node.plan_base().id(),
plan_node_type: plan_node.node_type(),
node: plan_node.to_batch_prost_body(),
children: vec![],
From 04151c24fc7b22fc04e9898c28b6473219bed9bb Mon Sep 17 00:00:00 2001
From: Yingjun Wu
Date: Mon, 23 Oct 2023 23:37:45 -0700
Subject: [PATCH 03/11] chore: beautify readme. (#13019)
Co-authored-by: hengm3467 <100685635+hengm3467@users.noreply.github.com>
---
README.md | 39 ++++++++++++++++++++++++++-------------
1 file changed, 26 insertions(+), 13 deletions(-)
diff --git a/README.md b/README.md
index 05f8ed8e3fb75..29a7d7e51888a 100644
--- a/README.md
+++ b/README.md
@@ -80,22 +80,35 @@ Learn more at [Quick Start](https://docs.risingwave.com/docs/current/get-started
## Why RisingWave for stream processing?
RisingWave adaptly tackles some of the most challenging problems in stream processing. Compared to existing stream processing systems, RisingWave shines through with the following key features:
-* **Easy to learn:** RisingWave speaks PostgreSQL-style SQL, enabling users to dive into stream processing in much the same way as operating a PostgreSQL database.
-* **Highly efficient in multi-stream joins:** RisingWave has made significant optimizations for multiple stream join scenarios. Users can easily join 10-20 streams (or more) efficiently in a production environment.
-* **High resource utilization:** Queries within RisingWave benefit from shared computational resources, obviating the need for users to manually allocate resources for individual queries.
-* **No compromise on large state management:** The decoupled compute-storage architecture of RisingWave ensures remote persistence of internal states, and users never need to worry about the size of internal states when handling complex queries.
-* **Transparent dynamic scaling:** RisingWave supports near-instantaneous dynamic scaling without any service interruptions.
-* **Instant failure recovery:** RisingWave's state management mechanism allows it to recover from failure in seconds, not minutes or hours.
-* **Easy to verify correctness:** RisingWave persists results in materialized views and allow users to break down complex stream computation programs into stacked materialized views, simplifying program development and result verification.
-* **Simplified data stack:** RisingWave's ability to store data and serve queries eliminates the need for separate maintenance of stream processors and databases. Users can effortlessly link RisingWave to their preferred BI tools or through client libraries.
-* **Simple to maintain and operate:** RisingWave abstracts away unnecessary low-level details, allowing users to concentrate solely on SQL code-level issues.
-* **Rich ecosystem:** With integrations to a diverse range of cloud systems and the PostgreSQL ecosystem, RisingWave boasts a rich and expansive ecosystem.
+* **Easy to learn**
+ * RisingWave speaks PostgreSQL-style SQL, enabling users to dive into stream processing in much the same way as operating a PostgreSQL database.
+* **Highly efficient in multi-stream joins**
+ * RisingWave has made significant optimizations for multiple stream join scenarios. Users can easily join 10-20 streams (or more) efficiently in a production environment.
+* **High resource utilization**
+ * Queries in RisingWave leverage shared computational resources, eliminating the need for users to manually allocate resources for each query.
+* **No compromise on large state management**
+ * The decoupled compute-storage architecture of RisingWave ensures remote persistence of internal states, and users never need to worry about the size of internal states when handling complex queries.
+* **Transparent dynamic scaling**
+ * RisingWave supports near-instantaneous dynamic scaling without any service interruptions.
+* **Instant failure recovery**
+ * RisingWave's state management mechanism allows it to recover from failure in seconds, not minutes or hours.
+* **Easy to verify correctness**
+ * RisingWave persists results in materialized views and allow users to break down complex stream computation programs into stacked materialized views, simplifying program development and result verification.
+* **Simplified data stack**
+ * RisingWave's ability to store data and serve queries eliminates the need for separate maintenance of stream processors and databases. Users can effortlessly connect RisingWave to their preferred BI tools or through client libraries.
+* **Simple to maintain and operate**
+ * RisingWave abstracts away unnecessary low-level details, allowing users to concentrate solely on SQL code-level issues.
+* **Rich ecosystem**
+ * With integrations to a diverse range of cloud systems and the PostgreSQL ecosystem, RisingWave boasts a rich and expansive ecosystem.
## RisingWave's limitations
RisingWave isn’t a panacea for all data engineering hurdles. It has its own set of limitations:
-* **No programmable interfaces:** RisingWave does not provide low-level APIs in languages like Java and Scala, and does not allow users to manage internal states manually (unless you want to hack!). For coding in Java, Scala, and other languages, please consider using RisingWave's User-Defined Functions (UDF).
-* **No support for transaction processing:** RisingWave isn’t cut out for transactional workloads, thus it’s not a viable substitute for operational databases dedicated to transaction processing. However, it supports read-only transactions, ensuring data freshness and consistency. It also comprehends the transactional semantics of upstream database Change Data Capture (CDC).
-* **Not tailored for ad-hoc analytical queries:** RisingWave's row store design is tailored for optimal stream processing performance rather than interactive analytical workloads. Hence, it's not a suitable replacement for OLAP databases. Yet, a reliable integration with many OLAP databases exists, and a collaborative use of RisingWave and OLAP databases is a common practice among many users.
+* **No programmable interfaces**
+ * RisingWave does not provide low-level APIs in languages like Java and Scala, and does not allow users to manage internal states manually (unless you want to hack!). For coding in Java, Scala, and other languages, please consider using RisingWave's User-Defined Functions (UDF).
+* **No support for transaction processing**
+ * RisingWave isn’t cut out for transactional workloads, thus it’s not a viable substitute for operational databases dedicated to transaction processing. However, it supports read-only transactions, ensuring data freshness and consistency. It also comprehends the transactional semantics of upstream database Change Data Capture (CDC).
+* **Not tailored for ad-hoc analytical queries**
+ * RisingWave's row store design is tailored for optimal stream processing performance rather than interactive analytical workloads. Hence, it's not a suitable replacement for OLAP databases. Yet, a reliable integration with many OLAP databases exists, and a collaborative use of RisingWave and OLAP databases is a common practice among many users.
## RisingWave Cloud
From 6939da61d9669f19d6b1193cc5eb53780b41a4b9 Mon Sep 17 00:00:00 2001
From: Noel Kwan <47273164+kwannoel@users.noreply.github.com>
Date: Tue, 24 Oct 2023 15:21:09 +0800
Subject: [PATCH 04/11] fix(ci): upload microbench only when it runs (#12988)
---
ci/scripts/run-micro-benchmarks.sh | 2 ++
ci/scripts/upload-micro-bench-results.sh | 13 +++++++++++++
2 files changed, 15 insertions(+)
diff --git a/ci/scripts/run-micro-benchmarks.sh b/ci/scripts/run-micro-benchmarks.sh
index 568c90de425ca..371cc416e7ac5 100755
--- a/ci/scripts/run-micro-benchmarks.sh
+++ b/ci/scripts/run-micro-benchmarks.sh
@@ -46,6 +46,8 @@ main() {
echo "--- Getting aws instance type"
local instance_type=$(get_instance_type)
echo "instance_type: $instance_type"
+ echo "$instance_type" > microbench_instance_type.txt
+ buildkite-agent artifact upload ./microbench_instance_type.txt
if [[ $instance_type != "m6i.4xlarge" ]]; then
echo "Only m6i.4xlarge is supported, skipping microbenchmark"
exit 0
diff --git a/ci/scripts/upload-micro-bench-results.sh b/ci/scripts/upload-micro-bench-results.sh
index 2644ca936c5da..e72b69950bb7b 100755
--- a/ci/scripts/upload-micro-bench-results.sh
+++ b/ci/scripts/upload-micro-bench-results.sh
@@ -36,6 +36,19 @@ get_commit() {
| sed 's/\"//g'
}
+get_machine() {
+ buildkite-agent artifact download microbench_instance_type.txt ./
+ cat ./microbench_instance_type.txt
+}
+
+echo "--- Checking microbench_instance_type"
+INSTANCE_TYPE=$(get_machine)
+echo "instance type: $INSTANCE_TYPE"
+if [[ $INSTANCE_TYPE != "m6i.4xlarge" ]]; then
+ echo "Only m6i.4xlarge is supported, microbenchmark was skipped"
+ exit 0
+fi
+
setup
BUILDKITE_BUILD_URL="https://buildkite.com/risingwavelabs/main-cron/builds/$BUILDKITE_BUILD_NUMBER"
From 23b33460e2df0d4efa22c1574bede5dcf82cf16a Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 24 Oct 2023 16:36:40 +0800
Subject: [PATCH 05/11] chore(deps): Bump comfy-table from 7.0.1 to 7.1.0
(#13013)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
Cargo.lock | 19 ++++++++-----------
1 file changed, 8 insertions(+), 11 deletions(-)
diff --git a/Cargo.lock b/Cargo.lock
index b7e9b6c45ec06..bbc112fa4bb5b 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -1825,13 +1825,13 @@ dependencies = [
[[package]]
name = "comfy-table"
-version = "7.0.1"
+version = "7.1.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "9ab77dbd8adecaf3f0db40581631b995f312a8a5ae3aa9993188bb8f23d83a5b"
+checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686"
dependencies = [
- "crossterm 0.26.1",
- "strum 0.24.1",
- "strum_macros 0.24.3",
+ "crossterm 0.27.0",
+ "strum 0.25.0",
+ "strum_macros 0.25.2",
"unicode-width",
]
@@ -2140,17 +2140,14 @@ dependencies = [
[[package]]
name = "crossterm"
-version = "0.26.1"
+version = "0.27.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "a84cda67535339806297f1b331d6dd6320470d2a0fe65381e79ee9e156dd3d13"
+checksum = "f476fe445d41c9e991fd07515a6f463074b782242ccf4a5b7b1d1012e70824df"
dependencies = [
- "bitflags 1.3.2",
+ "bitflags 2.4.0",
"crossterm_winapi",
"libc",
- "mio",
"parking_lot 0.12.1",
- "signal-hook",
- "signal-hook-mio",
"winapi",
]
From fcad5e1f127c13229acb7e1ef3c4fcbf8b3e69e6 Mon Sep 17 00:00:00 2001
From: Noel Kwan <47273164+kwannoel@users.noreply.github.com>
Date: Tue, 24 Oct 2023 16:57:40 +0800
Subject: [PATCH 06/11] fix(stream): clean dirty tables for barrier recovery
(#12990)
---
ci/scripts/deterministic-recovery-test.sh | 1 +
src/meta/src/barrier/mod.rs | 7 ++--
src/meta/src/barrier/recovery.rs | 9 ++----
src/meta/src/manager/catalog/mod.rs | 32 +++++++++++--------
.../recovery/background_ddl.rs | 3 ++
5 files changed, 27 insertions(+), 25 deletions(-)
diff --git a/ci/scripts/deterministic-recovery-test.sh b/ci/scripts/deterministic-recovery-test.sh
index 6514fe1f7c0c3..c5f89a2bbc7e0 100755
--- a/ci/scripts/deterministic-recovery-test.sh
+++ b/ci/scripts/deterministic-recovery-test.sh
@@ -11,6 +11,7 @@ chmod +x ./risingwave_simulation
export RUST_LOG="info,\
risingwave_meta::barrier::recovery=debug,\
+risingwave_meta::manager::catalog=debug,\
risingwave_meta::rpc::ddl_controller=debug,\
risingwave_meta::barrier::mod=debug,\
risingwave_simulation=debug"
diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs
index ed6ad289a5a68..d39dde51399d8 100644
--- a/src/meta/src/barrier/mod.rs
+++ b/src/meta/src/barrier/mod.rs
@@ -626,7 +626,7 @@ impl GlobalBarrierManager {
let paused = self.take_pause_on_bootstrap().await.unwrap_or(false);
let paused_reason = paused.then_some(PausedReason::Manual);
- self.recovery(prev_epoch, paused_reason, true)
+ self.recovery(prev_epoch, paused_reason)
.instrument(span)
.await
};
@@ -981,10 +981,7 @@ impl GlobalBarrierManager {
// No need to clean dirty tables for barrier recovery,
// The foreground stream job should cleanup their own tables.
- *state = self
- .recovery(prev_epoch, None, false)
- .instrument(span)
- .await;
+ *state = self.recovery(prev_epoch, None).instrument(span).await;
self.set_status(BarrierManagerStatus::Running).await;
} else {
panic!("failed to execute barrier: {:?}", err);
diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs
index 21197a8df98d4..3e319f0e69a52 100644
--- a/src/meta/src/barrier/recovery.rs
+++ b/src/meta/src/barrier/recovery.rs
@@ -219,7 +219,6 @@ impl GlobalBarrierManager {
&self,
prev_epoch: TracedEpoch,
paused_reason: Option,
- bootstrap_recovery: bool,
) -> BarrierManagerState {
// Mark blocked and abort buffered schedules, they might be dirty already.
self.scheduled_barriers
@@ -227,11 +226,9 @@ impl GlobalBarrierManager {
.await;
tracing::info!("recovery start!");
- if bootstrap_recovery {
- self.clean_dirty_tables()
- .await
- .expect("clean dirty tables should not fail");
- }
+ self.clean_dirty_tables()
+ .await
+ .expect("clean dirty tables should not fail");
self.clean_dirty_fragments()
.await
.expect("clean dirty fragments");
diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs
index bcac32922d180..f988646428aac 100644
--- a/src/meta/src/manager/catalog/mod.rs
+++ b/src/meta/src/manager/catalog/mod.rs
@@ -853,14 +853,18 @@ impl CatalogManager {
database_core.clear_creating_stream_jobs();
let user_core = &mut core.user;
for table in &tables_to_clean {
- // Recovered when init database manager.
- for relation_id in &table.dependent_relations {
- database_core.decrease_ref_count(*relation_id);
+ // If table type is internal, no need to update the ref count OR
+ // user ref count.
+ if table.table_type != TableType::Internal as i32 {
+ // Recovered when init database manager.
+ for relation_id in &table.dependent_relations {
+ database_core.decrease_ref_count(*relation_id);
+ }
+ // Recovered when init user manager.
+ tracing::debug!("decrease ref for {}", table.id);
+ user_core.decrease_ref(table.owner);
}
- // Recovered when init user manager.
- user_core.decrease_ref(table.owner);
}
-
Ok(())
}
@@ -919,10 +923,11 @@ impl CatalogManager {
let database_core = &mut core.database;
let tables = &mut database_core.tables;
let Some(table) = tables.get(&table_id).cloned() else {
- bail!(
- "table_id {} missing when attempting to cancel job",
+ tracing::warn!(
+ "table_id {} missing when attempting to cancel job, could be cleaned on recovery",
table_id
- )
+ );
+ return Ok(());
};
table
};
@@ -938,7 +943,8 @@ impl CatalogManager {
let tables = &mut database_core.tables;
let mut tables = BTreeMapTransaction::new(tables);
for table_id in table_ids {
- tables.remove(table_id);
+ let res = tables.remove(table_id);
+ assert!(res.is_some());
}
commit_meta!(self, tables)?;
}
@@ -2032,8 +2038,7 @@ impl CatalogManager {
let user_core = &mut core.user;
let key = (index.database_id, index.schema_id, index.name.clone());
assert!(
- !database_core.indexes.contains_key(&index.id)
- && database_core.has_in_progress_creation(&key),
+ !database_core.indexes.contains_key(&index.id),
"index must be in creating procedure"
);
@@ -2188,8 +2193,7 @@ impl CatalogManager {
let user_core = &mut core.user;
let key = (sink.database_id, sink.schema_id, sink.name.clone());
assert!(
- !database_core.sinks.contains_key(&sink.id)
- && database_core.has_in_progress_creation(&key),
+ !database_core.sinks.contains_key(&sink.id),
"sink must be in creating procedure"
);
diff --git a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs
index 1fd5c90e59e4b..89df82d4c21a0 100644
--- a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs
+++ b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs
@@ -61,6 +61,9 @@ async fn test_background_mv_barrier_recovery() -> Result<()> {
.run("create materialized view m1 as select * from t1;")
.await?;
+ // If the CN is killed before first barrier pass for the MV, the MV will be dropped.
+ // This is because it's table fragments will NOT be committed until first barrier pass.
+ sleep(Duration::from_secs(5)).await;
kill_cn_and_wait_recover(&cluster).await;
// Send some upstream updates.
From 88a8331028674fa98b4584751bc7937c18e26ab3 Mon Sep 17 00:00:00 2001
From: Bugen Zhao
Date: Tue, 24 Oct 2023 17:09:47 +0800
Subject: [PATCH 07/11] refactor(expr): type-safe infallible evaluation
(#12921)
Signed-off-by: Bugen Zhao
---
src/batch/src/executor/aggregation/filter.rs | 2 +-
src/batch/src/executor/project_set.rs | 2 +-
src/expr/core/src/expr/build.rs | 21 ++--
src/expr/core/src/expr/mod.rs | 106 ++++++++++++------
src/expr/core/src/expr/wrapper/checked.rs | 2 +-
src/expr/core/src/expr/wrapper/mod.rs | 7 +-
src/expr/core/src/expr/wrapper/non_strict.rs | 14 ++-
src/expr/impl/src/scalar/cast.rs | 4 +-
.../src/table_function/generate_series.rs | 2 +-
src/storage/src/row_serde/value_serde.rs | 3 +-
src/stream/clippy.toml | 4 +-
src/stream/src/executor/aggregation/mod.rs | 8 +-
src/stream/src/executor/dynamic_filter.rs | 6 +-
src/stream/src/executor/filter.rs | 8 +-
src/stream/src/executor/hash_join.rs | 20 ++--
src/stream/src/executor/hop_window.rs | 21 ++--
src/stream/src/executor/integration_tests.rs | 2 +-
src/stream/src/executor/mod.rs | 6 +-
src/stream/src/executor/project.rs | 11 +-
src/stream/src/executor/project_set.rs | 7 +-
src/stream/src/executor/temporal_join.rs | 6 +-
src/stream/src/executor/test_utils.rs | 10 +-
src/stream/src/executor/values.rs | 17 +--
src/stream/src/executor/watermark_filter.rs | 11 +-
src/stream/src/from_proto/hash_join.rs | 9 +-
src/stream/src/from_proto/temporal_join.rs | 4 +-
.../tests/integration_tests/hop_window.rs | 11 +-
.../tests/integration_tests/project_set.rs | 8 +-
28 files changed, 212 insertions(+), 120 deletions(-)
diff --git a/src/batch/src/executor/aggregation/filter.rs b/src/batch/src/executor/aggregation/filter.rs
index 2db2320ed3534..9cfbeabffe417 100644
--- a/src/batch/src/executor/aggregation/filter.rs
+++ b/src/batch/src/executor/aggregation/filter.rs
@@ -75,7 +75,7 @@ impl AggregateFunction for Filter {
mod tests {
use risingwave_common::test_prelude::StreamChunkTestExt;
use risingwave_expr::aggregate::{build_append_only, AggCall};
- use risingwave_expr::expr::{build_from_pretty, Expression, LiteralExpression};
+ use risingwave_expr::expr::{build_from_pretty, ExpressionBoxExt, LiteralExpression};
use super::*;
diff --git a/src/batch/src/executor/project_set.rs b/src/batch/src/executor/project_set.rs
index 670933a6bb50c..fa3dfac917e8a 100644
--- a/src/batch/src/executor/project_set.rs
+++ b/src/batch/src/executor/project_set.rs
@@ -171,7 +171,7 @@ mod tests {
use risingwave_common::catalog::{Field, Schema};
use risingwave_common::test_prelude::*;
use risingwave_common::types::DataType;
- use risingwave_expr::expr::{Expression, InputRefExpression, LiteralExpression};
+ use risingwave_expr::expr::{ExpressionBoxExt, InputRefExpression, LiteralExpression};
use risingwave_expr::table_function::repeat;
use super::*;
diff --git a/src/expr/core/src/expr/build.rs b/src/expr/core/src/expr/build.rs
index 1ea03bd36f42a..7dffbcd42d66b 100644
--- a/src/expr/core/src/expr/build.rs
+++ b/src/expr/core/src/expr/build.rs
@@ -27,8 +27,13 @@ use super::expr_in::InExpression;
use super::expr_some_all::SomeAllExpression;
use super::expr_udf::UdfExpression;
use super::expr_vnode::VnodeExpression;
-use super::wrapper::{Checked, EvalErrorReport, NonStrict};
-use crate::expr::{BoxedExpression, Expression, InputRefExpression, LiteralExpression};
+use super::wrapper::checked::Checked;
+use super::wrapper::non_strict::NonStrict;
+use super::wrapper::EvalErrorReport;
+use super::NonStrictExpression;
+use crate::expr::{
+ BoxedExpression, Expression, ExpressionBoxExt, InputRefExpression, LiteralExpression,
+};
use crate::sig::FUNCTION_REGISTRY;
use crate::{bail, ExprError, Result};
@@ -41,8 +46,10 @@ pub fn build_from_prost(prost: &ExprNode) -> Result {
pub fn build_non_strict_from_prost(
prost: &ExprNode,
error_report: impl EvalErrorReport + 'static,
-) -> Result {
- ExprBuilder::new_non_strict(error_report).build(prost)
+) -> Result {
+ ExprBuilder::new_non_strict(error_report)
+ .build(prost)
+ .map(NonStrictExpression)
}
/// Build an expression from protobuf with possibly some wrappers attached to each node.
@@ -153,7 +160,7 @@ impl BuildBoxed for E {
prost: &ExprNode,
build_child: impl Fn(&ExprNode) -> Result,
) -> Result {
- Self::build(prost, build_child).map(Expression::boxed)
+ Self::build(prost, build_child).map(ExpressionBoxExt::boxed)
}
}
@@ -217,9 +224,9 @@ pub fn build_func_non_strict(
ret_type: DataType,
children: Vec,
error_report: impl EvalErrorReport + 'static,
-) -> Result {
+) -> Result {
let expr = build_func(func, ret_type, children)?;
- let wrapped = ExprBuilder::new_non_strict(error_report).wrap(expr);
+ let wrapped = NonStrictExpression(ExprBuilder::new_non_strict(error_report).wrap(expr));
Ok(wrapped)
}
diff --git a/src/expr/core/src/expr/mod.rs b/src/expr/core/src/expr/mod.rs
index 37e0104371a3e..48a46f640bf7b 100644
--- a/src/expr/core/src/expr/mod.rs
+++ b/src/expr/core/src/expr/mod.rs
@@ -58,7 +58,7 @@ pub use self::build::*;
pub use self::expr_input_ref::InputRefExpression;
pub use self::expr_literal::LiteralExpression;
pub use self::value::{ValueImpl, ValueRef};
-pub use self::wrapper::EvalErrorReport;
+pub use self::wrapper::*;
pub use super::{ExprError, Result};
/// Interface of an expression.
@@ -67,6 +67,7 @@ pub use super::{ExprError, Result};
/// should be implemented. Prefer calling and implementing `eval_v2` instead of `eval` if possible,
/// to gain the performance benefit of scalar expression.
#[async_trait::async_trait]
+#[auto_impl::auto_impl(&, Box)]
pub trait Expression: std::fmt::Debug + Sync + Send {
/// Get the return data type.
fn return_type(&self) -> DataType;
@@ -101,23 +102,77 @@ pub trait Expression: std::fmt::Debug + Sync + Send {
fn eval_const(&self) -> Result {
Err(ExprError::NotConstant)
}
+}
+/// An owned dynamically typed [`Expression`].
+pub type BoxedExpression = Box;
+
+/// Extension trait for boxing expressions.
+///
+/// This is not directly made into [`Expression`] trait because...
+/// - an expression does not have to be `'static`,
+/// - and for the ease of `auto_impl`.
+#[easy_ext::ext(ExpressionBoxExt)]
+impl E {
/// Wrap the expression in a Box.
- fn boxed(self) -> BoxedExpression
- where
- Self: Sized + Send + 'static,
- {
+ pub fn boxed(self) -> BoxedExpression {
Box::new(self)
}
}
-// TODO: make this an extension, or implement it on a `NonStrict` newtype.
-impl dyn Expression {
+/// An type-safe wrapper that indicates the inner expression can be evaluated in a non-strict
+/// manner, i.e., developers can directly call `eval_infallible` and `eval_row_infallible` without
+/// checking the result.
+///
+/// This is usually created by non-strict build functions like [`crate::expr::build_non_strict_from_prost`]
+/// and [`crate::expr::build_func_non_strict`]. It can also be created directly by
+/// [`NonStrictExpression::new_topmost`], where only the evaluation of the topmost level expression
+/// node is non-strict and should be treated as a TODO.
+///
+/// Compared to [`crate::expr::wrapper::non_strict::NonStrict`], this is more like an indicator
+/// applied on the root of an expression tree, while the latter is a wrapper that can be applied on
+/// each node of the tree and actually changes the behavior. As a result, [`NonStrictExpression`]
+/// does not implement [`Expression`] trait and instead deals directly with developers.
+#[derive(Debug)]
+pub struct NonStrictExpression(E);
+
+impl NonStrictExpression
+where
+ E: Expression,
+{
+ /// Create a non-strict expression directly wrapping the given expression.
+ ///
+ /// Should only be used in tests as evaluation may panic.
+ pub fn for_test(inner: E) -> NonStrictExpression
+ where
+ E: 'static,
+ {
+ NonStrictExpression(inner.boxed())
+ }
+
+ /// Create a non-strict expression from the given expression, where only the evaluation of the
+ /// topmost level expression node is non-strict (which is subtly different from
+ /// [`crate::expr::build_non_strict_from_prost`] where every node is non-strict).
+ ///
+ /// This should be used as a TODO.
+ pub fn new_topmost(
+ inner: E,
+ error_report: impl EvalErrorReport,
+ ) -> NonStrictExpression {
+ let inner = wrapper::non_strict::NonStrict::new(inner, error_report);
+ NonStrictExpression(inner)
+ }
+
+ /// Get the return data type.
+ pub fn return_type(&self) -> DataType {
+ self.0.return_type()
+ }
+
/// Evaluate the expression in vectorized execution and assert it succeeds. Returns an array.
///
/// Use with expressions built in non-strict mode.
pub async fn eval_infallible(&self, input: &DataChunk) -> ArrayRef {
- self.eval(input).await.expect("evaluation failed")
+ self.0.eval(input).await.expect("evaluation failed")
}
/// Evaluate the expression in row-based execution and assert it succeeds. Returns a nullable
@@ -125,38 +180,17 @@ impl dyn Expression {
///
/// Use with expressions built in non-strict mode.
pub async fn eval_row_infallible(&self, input: &OwnedRow) -> Datum {
- self.eval_row(input).await.expect("evaluation failed")
- }
-}
-
-/// An owned dynamically typed [`Expression`].
-pub type BoxedExpression = Box;
-
-// TODO: avoid the overhead of extra boxing.
-#[async_trait::async_trait]
-impl Expression for BoxedExpression {
- fn return_type(&self) -> DataType {
- (**self).return_type()
+ self.0.eval_row(input).await.expect("evaluation failed")
}
- async fn eval(&self, input: &DataChunk) -> Result {
- (**self).eval(input).await
- }
-
- async fn eval_v2(&self, input: &DataChunk) -> Result {
- (**self).eval_v2(input).await
- }
-
- async fn eval_row(&self, input: &OwnedRow) -> Result {
- (**self).eval_row(input).await
- }
-
- fn eval_const(&self) -> Result {
- (**self).eval_const()
+ /// Unwrap the inner expression.
+ pub fn into_inner(self) -> E {
+ self.0
}
- fn boxed(self) -> BoxedExpression {
- self
+ /// Get a reference to the inner expression.
+ pub fn inner(&self) -> &E {
+ &self.0
}
}
diff --git a/src/expr/core/src/expr/wrapper/checked.rs b/src/expr/core/src/expr/wrapper/checked.rs
index 1e049ad481010..b3b1375c4fa82 100644
--- a/src/expr/core/src/expr/wrapper/checked.rs
+++ b/src/expr/core/src/expr/wrapper/checked.rs
@@ -22,7 +22,7 @@ use crate::expr::{Expression, ValueImpl};
/// A wrapper of [`Expression`] that does extra checks after evaluation.
#[derive(Debug)]
-pub struct Checked(pub E);
+pub(crate) struct Checked(pub E);
// TODO: avoid the overhead of extra boxing.
#[async_trait]
diff --git a/src/expr/core/src/expr/wrapper/mod.rs b/src/expr/core/src/expr/wrapper/mod.rs
index 48241d05de45c..16988a050ad8d 100644
--- a/src/expr/core/src/expr/wrapper/mod.rs
+++ b/src/expr/core/src/expr/wrapper/mod.rs
@@ -12,8 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
-mod checked;
-mod non_strict;
+pub(crate) mod checked;
+pub(crate) mod non_strict;
-pub use checked::Checked;
-pub use non_strict::{EvalErrorReport, NonStrict};
+pub use non_strict::{EvalErrorReport, LogReport};
diff --git a/src/expr/core/src/expr/wrapper/non_strict.rs b/src/expr/core/src/expr/wrapper/non_strict.rs
index 0859cea27aa49..782456023cdf7 100644
--- a/src/expr/core/src/expr/wrapper/non_strict.rs
+++ b/src/expr/core/src/expr/wrapper/non_strict.rs
@@ -23,7 +23,7 @@ use crate::expr::{Expression, ValueImpl};
use crate::ExprError;
/// Report an error during evaluation.
-#[auto_impl(Arc)]
+#[auto_impl(&, Arc)]
pub trait EvalErrorReport: Clone + Send + Sync {
/// Perform the error reporting.
///
@@ -42,11 +42,21 @@ impl EvalErrorReport for ! {
}
}
+/// Log the error to report an error during evaluation.
+#[derive(Clone)]
+pub struct LogReport;
+
+impl EvalErrorReport for LogReport {
+ fn report(&self, error: ExprError) {
+ tracing::error!(%error, "failed to evaluate expression");
+ }
+}
+
/// A wrapper of [`Expression`] that evaluates in a non-strict way. Basically...
/// - When an error occurs during chunk-level evaluation, recompute in row-based execution and pad
/// with NULL for each failed row.
/// - Report all error occurred during row-level evaluation to the [`EvalErrorReport`].
-pub struct NonStrict {
+pub(crate) struct NonStrict {
inner: E,
report: R,
}
diff --git a/src/expr/impl/src/scalar/cast.rs b/src/expr/impl/src/scalar/cast.rs
index 889cc43fe6b18..c173c76c330c5 100644
--- a/src/expr/impl/src/scalar/cast.rs
+++ b/src/expr/impl/src/scalar/cast.rs
@@ -22,7 +22,9 @@ use risingwave_common::cast;
use risingwave_common::row::OwnedRow;
use risingwave_common::types::{DataType, Int256, IntoOrdered, JsonbRef, ToText, F64};
use risingwave_common::util::iter_util::ZipEqFast;
-use risingwave_expr::expr::{build_func, Context, Expression, InputRefExpression};
+use risingwave_expr::expr::{
+ build_func, Context, Expression, ExpressionBoxExt, InputRefExpression,
+};
use risingwave_expr::{function, ExprError, Result};
use risingwave_pb::expr::expr_node::PbType;
diff --git a/src/expr/impl/src/table_function/generate_series.rs b/src/expr/impl/src/table_function/generate_series.rs
index 586fa60de02c2..dfa09b0e215b8 100644
--- a/src/expr/impl/src/table_function/generate_series.rs
+++ b/src/expr/impl/src/table_function/generate_series.rs
@@ -159,7 +159,7 @@ mod tests {
use risingwave_common::array::DataChunk;
use risingwave_common::types::test_utils::IntervalTestExt;
use risingwave_common::types::{DataType, Decimal, Interval, ScalarImpl, Timestamp};
- use risingwave_expr::expr::{BoxedExpression, Expression, LiteralExpression};
+ use risingwave_expr::expr::{BoxedExpression, ExpressionBoxExt, LiteralExpression};
use risingwave_expr::table_function::build;
use risingwave_expr::ExprError;
use risingwave_pb::expr::table_function::PbType;
diff --git a/src/storage/src/row_serde/value_serde.rs b/src/storage/src/row_serde/value_serde.rs
index 5d56cdba2d96d..9048b90c23a53 100644
--- a/src/storage/src/row_serde/value_serde.rs
+++ b/src/storage/src/row_serde/value_serde.rs
@@ -114,9 +114,10 @@ impl ValueRowSerdeNew for ColumnAwareSerde {
// It's okay since we previously banned impure expressions in default columns.
build_from_prost(&expr.expect("expr should not be none"))
.expect("build_from_prost error")
- .eval_row_infallible(&OwnedRow::empty())
+ .eval_row(&OwnedRow::empty())
.now_or_never()
.expect("constant expression should not be async")
+ .expect("eval_row failed")
};
Some((i, value))
} else {
diff --git a/src/stream/clippy.toml b/src/stream/clippy.toml
index a6969d5bd607b..b7257c4acb98c 100644
--- a/src/stream/clippy.toml
+++ b/src/stream/clippy.toml
@@ -3,8 +3,8 @@ disallowed-methods = [
{ path = "risingwave_expr::expr::build_from_prost", reason = "Expressions in streaming must be in non-strict mode. Please use `build_non_strict_from_prost` instead." },
{ path = "risingwave_expr::expr::build_func", reason = "Expressions in streaming must be in non-strict mode. Please use `build_func_non_strict` instead." },
- { path = "risingwave_expr::expr::Expression::eval", reason = "Please use `Expression::eval_infallible` instead." },
- { path = "risingwave_expr::expr::Expression::eval_row", reason = "Please use `Expression::eval_row_infallible` instead." },
+ { path = "risingwave_expr::expr::Expression::eval", reason = "Please use `NonStrictExpression::eval_infallible` instead." },
+ { path = "risingwave_expr::expr::Expression::eval_row", reason = "Please use `NonStrictExpression::eval_row_infallible` instead." },
{ path = "risingwave_common::error::internal_err", reason = "Please use per-crate error type instead." },
{ path = "risingwave_common::error::internal_error", reason = "Please use per-crate error type instead." },
diff --git a/src/stream/src/executor/aggregation/mod.rs b/src/stream/src/executor/aggregation/mod.rs
index dd0ce9d01c544..9bb1113152962 100644
--- a/src/stream/src/executor/aggregation/mod.rs
+++ b/src/stream/src/executor/aggregation/mod.rs
@@ -21,6 +21,7 @@ use risingwave_common::bail;
use risingwave_common::buffer::Bitmap;
use risingwave_common::catalog::{Field, Schema};
use risingwave_expr::aggregate::{AggCall, AggKind};
+use risingwave_expr::expr::{LogReport, NonStrictExpression};
use risingwave_storage::StateStore;
use crate::common::table::state_table::StateTable;
@@ -74,7 +75,12 @@ pub async fn agg_call_filter_res(
}
if let Some(ref filter) = agg_call.filter {
- if let Bool(filter_res) = filter.eval_infallible(chunk).await.as_ref() {
+ // TODO: should we build `filter` in non-strict mode?
+ if let Bool(filter_res) = NonStrictExpression::new_topmost(&**filter, LogReport)
+ .eval_infallible(chunk)
+ .await
+ .as_ref()
+ {
vis &= filter_res.to_bitmap();
} else {
bail!("Filter can only receive bool array");
diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs
index e8eb4da545f2e..ccb55b75c24fc 100644
--- a/src/stream/src/executor/dynamic_filter.rs
+++ b/src/stream/src/executor/dynamic_filter.rs
@@ -26,7 +26,7 @@ use risingwave_common::row::{self, once, OwnedRow, OwnedRow as RowData, Row};
use risingwave_common::types::{DataType, Datum, DefaultOrd, ScalarImpl, ToDatumRef, ToOwnedDatum};
use risingwave_common::util::iter_util::ZipEqDebug;
use risingwave_expr::expr::{
- build_func_non_strict, BoxedExpression, InputRefExpression, LiteralExpression,
+ build_func_non_strict, InputRefExpression, LiteralExpression, NonStrictExpression,
};
use risingwave_pb::expr::expr_node::Type as ExprNodeType;
use risingwave_pb::expr::expr_node::Type::{
@@ -97,7 +97,7 @@ impl DynamicFilterExecutor,
+ condition: Option,
) -> Result<(Vec, Bitmap), StreamExecutorError> {
let mut new_ops = Vec::with_capacity(chunk.capacity());
let mut new_visibility = BitmapBuilder::with_capacity(chunk.capacity());
@@ -265,7 +265,7 @@ impl DynamicFilterExecutor,
- expr: BoxedExpression,
+ expr: NonStrictExpression,
executor_id: u64,
) -> Self {
let input_info = input.info();
@@ -190,8 +190,8 @@ mod tests {
use risingwave_common::array::StreamChunk;
use risingwave_common::catalog::{Field, Schema};
use risingwave_common::types::DataType;
- use risingwave_expr::expr::build_from_pretty;
+ use super::super::test_utils::expr::build_from_pretty;
use super::super::test_utils::MockSource;
use super::super::*;
use super::*;
diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs
index 7aed840679c82..4178012cb9d9e 100644
--- a/src/stream/src/executor/hash_join.rs
+++ b/src/stream/src/executor/hash_join.rs
@@ -28,7 +28,7 @@ use risingwave_common::row::{OwnedRow, Row};
use risingwave_common::types::{DataType, DefaultOrd, ToOwnedDatum};
use risingwave_common::util::epoch::EpochPair;
use risingwave_common::util::iter_util::ZipEqDebug;
-use risingwave_expr::expr::BoxedExpression;
+use risingwave_expr::expr::NonStrictExpression;
use risingwave_expr::ExprError;
use risingwave_storage::StateStore;
use tokio::time::Instant;
@@ -242,9 +242,9 @@ pub struct HashJoinExecutor,
/// Optional non-equi join conditions
- cond: Option,
+ cond: Option,
/// Column indices of watermark output and offset expression of each inequality, respectively.
- inequality_pairs: Vec<(Vec, Option)>,
+ inequality_pairs: Vec<(Vec, Option)>,
/// The output watermark of each inequality condition and its value is the minimum of the
/// calculation result of both side. It will be used to generate watermark into downstream
/// and do state cleaning if `clean_state` field of that inequality is `true`.
@@ -313,7 +313,7 @@ struct EqJoinArgs<'a, K: HashKey, S: StateStore> {
side_l: &'a mut JoinSide,
side_r: &'a mut JoinSide,
actual_output_data_types: &'a [DataType],
- cond: &'a mut Option,
+ cond: &'a mut Option,
inequality_watermarks: &'a [Option],
chunk: StreamChunk,
append_only_optimize: bool,
@@ -448,8 +448,8 @@ impl HashJoinExecutor,
executor_id: u64,
- cond: Option,
- inequality_pairs: Vec<(usize, usize, bool, Option)>,
+ cond: Option,
+ inequality_pairs: Vec<(usize, usize, bool, Option)>,
op_info: String,
state_table_l: StateTable,
degree_state_table_l: StateTable,
@@ -912,7 +912,7 @@ impl HashJoinExecutor input_watermark.val = value.unwrap(),
@@ -1275,11 +1275,11 @@ mod tests {
use risingwave_common::hash::{Key128, Key64};
use risingwave_common::types::ScalarImpl;
use risingwave_common::util::sort_util::OrderType;
- use risingwave_expr::expr::build_from_pretty;
use risingwave_storage::memory::MemoryStateStore;
use super::*;
use crate::common::table::state_table::StateTable;
+ use crate::executor::test_utils::expr::build_from_pretty;
use crate::executor::test_utils::{MessageSender, MockSource, StreamExecutorTestExt};
use crate::executor::{ActorContext, Barrier, EpochPair};
@@ -1327,7 +1327,7 @@ mod tests {
(state_table, degree_state_table)
}
- fn create_cond(condition_text: Option) -> BoxedExpression {
+ fn create_cond(condition_text: Option) -> NonStrictExpression {
build_from_pretty(
condition_text
.as_deref()
@@ -1339,7 +1339,7 @@ mod tests {
with_condition: bool,
null_safe: bool,
condition_text: Option,
- inequality_pairs: Vec<(usize, usize, bool, Option)>,
+ inequality_pairs: Vec<(usize, usize, bool, Option)>,
) -> (MessageSender, MessageSender, BoxedMessageStream) {
let schema = Schema {
fields: vec![
diff --git a/src/stream/src/executor/hop_window.rs b/src/stream/src/executor/hop_window.rs
index c6fffcd94896d..42d13d790da88 100644
--- a/src/stream/src/executor/hop_window.rs
+++ b/src/stream/src/executor/hop_window.rs
@@ -19,7 +19,7 @@ use futures_async_stream::try_stream;
use itertools::Itertools;
use risingwave_common::array::{DataChunk, Op};
use risingwave_common::types::Interval;
-use risingwave_expr::expr::BoxedExpression;
+use risingwave_expr::expr::NonStrictExpression;
use risingwave_expr::ExprError;
use super::error::StreamExecutorError;
@@ -33,8 +33,8 @@ pub struct HopWindowExecutor {
pub time_col_idx: usize,
pub window_slide: Interval,
pub window_size: Interval,
- window_start_exprs: Vec,
- window_end_exprs: Vec,
+ window_start_exprs: Vec,
+ window_end_exprs: Vec,
pub output_indices: Vec,
chunk_size: usize,
}
@@ -48,8 +48,8 @@ impl HopWindowExecutor {
time_col_idx: usize,
window_slide: Interval,
window_size: Interval,
- window_start_exprs: Vec,
- window_end_exprs: Vec,
+ window_start_exprs: Vec,
+ window_end_exprs: Vec,
output_indices: Vec,
chunk_size: usize,
) -> Self {
@@ -251,6 +251,7 @@ mod tests {
use risingwave_common::types::test_utils::IntervalTestExt;
use risingwave_common::types::{DataType, Interval};
use risingwave_expr::expr::test_utils::make_hop_window_expression;
+ use risingwave_expr::expr::NonStrictExpression;
use crate::executor::test_utils::MockSource;
use crate::executor::{ActorContext, Executor, ExecutorInfo, StreamChunk};
@@ -302,8 +303,14 @@ mod tests {
2,
window_slide,
window_size,
- window_start_exprs,
- window_end_exprs,
+ window_start_exprs
+ .into_iter()
+ .map(NonStrictExpression::for_test)
+ .collect(),
+ window_end_exprs
+ .into_iter()
+ .map(NonStrictExpression::for_test)
+ .collect(),
output_indices,
CHUNK_SIZE,
)
diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs
index a9c219a25641f..cd505093294f1 100644
--- a/src/stream/src/executor/integration_tests.rs
+++ b/src/stream/src/executor/integration_tests.rs
@@ -152,7 +152,7 @@ async fn test_merger_sum_aggr() {
vec![],
vec![
// TODO: use the new streaming_if_null expression here, and add `None` tests
- Box::new(InputRefExpression::new(DataType::Int64, 1)),
+ NonStrictExpression::for_test(InputRefExpression::new(DataType::Int64, 1)),
],
3,
MultiMap::new(),
diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs
index 99b090e21a240..c28d6ec8564d9 100644
--- a/src/stream/src/executor/mod.rs
+++ b/src/stream/src/executor/mod.rs
@@ -31,7 +31,7 @@ use risingwave_common::util::epoch::{Epoch, EpochPair};
use risingwave_common::util::tracing::TracingContext;
use risingwave_common::util::value_encoding::{DatumFromProtoExt, DatumToProtoExt};
use risingwave_connector::source::SplitImpl;
-use risingwave_expr::expr::BoxedExpression;
+use risingwave_expr::expr::{Expression, NonStrictExpression};
use risingwave_pb::data::PbEpoch;
use risingwave_pb::expr::PbInputRef;
use risingwave_pb::stream_plan::barrier::{BarrierKind, PbMutation};
@@ -641,7 +641,7 @@ impl Watermark {
pub async fn transform_with_expr(
self,
- expr: &BoxedExpression,
+ expr: &NonStrictExpression,
new_col_idx: usize,
) -> Option {
let Self { col_idx, val, .. } = self;
@@ -651,7 +651,7 @@ impl Watermark {
OwnedRow::new(row)
};
let val = expr.eval_row_infallible(&row).await?;
- Some(Self::new(new_col_idx, expr.return_type(), val))
+ Some(Self::new(new_col_idx, expr.inner().return_type(), val))
}
/// Transform the watermark with the given output indices. If this watermark is not in the
diff --git a/src/stream/src/executor/project.rs b/src/stream/src/executor/project.rs
index 56a31bde901b9..8cfebfecd3f33 100644
--- a/src/stream/src/executor/project.rs
+++ b/src/stream/src/executor/project.rs
@@ -21,7 +21,7 @@ use risingwave_common::catalog::{Field, Schema};
use risingwave_common::row::{Row, RowExt};
use risingwave_common::types::ToOwnedDatum;
use risingwave_common::util::iter_util::ZipEqFast;
-use risingwave_expr::expr::BoxedExpression;
+use risingwave_expr::expr::NonStrictExpression;
use super::*;
@@ -38,7 +38,7 @@ struct Inner {
info: ExecutorInfo,
/// Expressions of the current projection.
- exprs: Vec,
+ exprs: Vec,
/// All the watermark derivations, (input_column_index, output_column_index). And the
/// derivation expression is the project's expression itself.
watermark_derivations: MultiMap,
@@ -58,7 +58,7 @@ impl ProjectExecutor {
ctx: ActorContextRef,
input: Box,
pk_indices: PkIndices,
- exprs: Vec,
+ exprs: Vec,
executor_id: u64,
watermark_derivations: MultiMap,
nondecreasing_expr_indices: Vec,
@@ -233,11 +233,12 @@ mod tests {
use risingwave_common::array::{DataChunk, StreamChunk};
use risingwave_common::catalog::{Field, Schema};
use risingwave_common::types::{DataType, Datum};
- use risingwave_expr::expr::{self, build_from_pretty, Expression, ValueImpl};
+ use risingwave_expr::expr::{self, Expression, ValueImpl};
use super::super::test_utils::MockSource;
use super::super::*;
use super::*;
+ use crate::executor::test_utils::expr::build_from_pretty;
use crate::executor::test_utils::StreamExecutorTestExt;
#[tokio::test]
@@ -345,7 +346,7 @@ mod tests {
let a_expr = build_from_pretty("(add:int8 $0:int8 1:int8)");
let b_expr = build_from_pretty("(subtract:int8 $0:int8 1:int8)");
- let c_expr = DummyNondecreasingExpr.boxed();
+ let c_expr = NonStrictExpression::for_test(DummyNondecreasingExpr);
let project = Box::new(ProjectExecutor::new(
ActorContext::create(123),
diff --git a/src/stream/src/executor/project_set.rs b/src/stream/src/executor/project_set.rs
index 6867e3d55bfde..ff3214db88eaa 100644
--- a/src/stream/src/executor/project_set.rs
+++ b/src/stream/src/executor/project_set.rs
@@ -24,6 +24,7 @@ use risingwave_common::catalog::{Field, Schema};
use risingwave_common::row::{Row, RowExt};
use risingwave_common::types::{DataType, Datum, DatumRef, ToOwnedDatum};
use risingwave_common::util::iter_util::ZipEqFast;
+use risingwave_expr::expr::{LogReport, NonStrictExpression};
use risingwave_expr::table_function::ProjectSetSelectItem;
use super::error::StreamExecutorError;
@@ -260,7 +261,11 @@ impl Inner {
ProjectSetSelectItem::Expr(expr) => {
watermark
.clone()
- .transform_with_expr(expr, expr_idx + PROJ_ROW_ID_OFFSET)
+ .transform_with_expr(
+ // TODO: should we build `expr` in non-strict mode?
+ &NonStrictExpression::new_topmost(expr, LogReport),
+ expr_idx + PROJ_ROW_ID_OFFSET,
+ )
.await
}
ProjectSetSelectItem::TableFunction(_) => {
diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs
index 3c8cde63c4ca9..82c1e56649672 100644
--- a/src/stream/src/executor/temporal_join.rs
+++ b/src/stream/src/executor/temporal_join.rs
@@ -32,7 +32,7 @@ use risingwave_common::hash::{HashKey, NullBitmap};
use risingwave_common::row::{OwnedRow, Row, RowExt};
use risingwave_common::types::DataType;
use risingwave_common::util::iter_util::ZipEqDebug;
-use risingwave_expr::expr::BoxedExpression;
+use risingwave_expr::expr::NonStrictExpression;
use risingwave_hummock_sdk::{HummockEpoch, HummockReadEpoch};
use risingwave_storage::store::PrefetchOptions;
use risingwave_storage::table::batch_table::storage_table::StorageTable;
@@ -57,7 +57,7 @@ pub struct TemporalJoinExecutor,
right_join_keys: Vec,
null_safe: Vec,
- condition: Option,
+ condition: Option,
output_indices: Vec,
pk_indices: PkIndices,
schema: Schema,
@@ -338,7 +338,7 @@ impl TemporalJoinExecutor
left_join_keys: Vec,
right_join_keys: Vec,
null_safe: Vec,
- condition: Option,
+ condition: Option,
pk_indices: PkIndices,
output_indices: Vec,
table_output_indices: Vec,
diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs
index bb4864ac04ef8..13a9237cf0159 100644
--- a/src/stream/src/executor/test_utils.rs
+++ b/src/stream/src/executor/test_utils.rs
@@ -34,11 +34,11 @@ pub mod prelude {
pub use risingwave_common::test_prelude::StreamChunkTestExt;
pub use risingwave_common::types::DataType;
pub use risingwave_common::util::sort_util::OrderType;
- pub use risingwave_expr::expr::build_from_pretty;
pub use risingwave_storage::memory::MemoryStateStore;
pub use risingwave_storage::StateStore;
pub use crate::common::table::state_table::StateTable;
+ pub use crate::executor::test_utils::expr::build_from_pretty;
pub use crate::executor::test_utils::{MessageSender, MockSource, StreamExecutorTestExt};
pub use crate::executor::{ActorContext, BoxedMessageStream, Executor, PkIndices};
}
@@ -263,6 +263,14 @@ pub trait StreamExecutorTestExt: MessageStream + Unpin {
// FIXME: implement on any `impl MessageStream` if the analyzer works well.
impl StreamExecutorTestExt for BoxedMessageStream {}
+pub mod expr {
+ use risingwave_expr::expr::NonStrictExpression;
+
+ pub fn build_from_pretty(s: impl AsRef) -> NonStrictExpression {
+ NonStrictExpression::for_test(risingwave_expr::expr::build_from_pretty(s))
+ }
+}
+
pub mod agg_executor {
use std::sync::atomic::AtomicU64;
use std::sync::Arc;
diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs
index 624b2531bf7bd..8c09b56aa3551 100644
--- a/src/stream/src/executor/values.rs
+++ b/src/stream/src/executor/values.rs
@@ -21,7 +21,7 @@ use risingwave_common::array::{DataChunk, Op, StreamChunk};
use risingwave_common::catalog::Schema;
use risingwave_common::ensure;
use risingwave_common::util::iter_util::ZipEqFast;
-use risingwave_expr::expr::BoxedExpression;
+use risingwave_expr::expr::NonStrictExpression;
use tokio::sync::mpsc::UnboundedReceiver;
use super::{
@@ -40,7 +40,7 @@ pub struct ValuesExecutor {
barrier_receiver: UnboundedReceiver,
progress: CreateMviewProgress,
- rows: vec::IntoIter>,
+ rows: vec::IntoIter>,
pk_indices: PkIndices,
identity: String,
schema: Schema,
@@ -51,7 +51,7 @@ impl ValuesExecutor {
pub fn new(
ctx: ActorContextRef,
progress: CreateMviewProgress,
- rows: Vec>,
+ rows: Vec>,
schema: Schema,
barrier_receiver: UnboundedReceiver,
executor_id: u64,
@@ -167,7 +167,7 @@ mod tests {
};
use risingwave_common::catalog::{Field, Schema};
use risingwave_common::types::{DataType, ScalarImpl, StructType};
- use risingwave_expr::expr::{BoxedExpression, LiteralExpression};
+ use risingwave_expr::expr::{BoxedExpression, LiteralExpression, NonStrictExpression};
use tokio::sync::mpsc::unbounded_channel;
use super::ValuesExecutor;
@@ -202,11 +202,11 @@ mod tests {
vec![],
),
Some(ScalarImpl::Struct(value)),
- )) as BoxedExpression,
+ )),
Box::new(LiteralExpression::new(
DataType::Int64,
Some(ScalarImpl::Int64(0)),
- )) as BoxedExpression,
+ )),
];
let fields = exprs
.iter() // for each column
@@ -215,7 +215,10 @@ mod tests {
let values_executor_struct = ValuesExecutor::new(
ActorContext::create(actor_id),
progress,
- vec![exprs],
+ vec![exprs
+ .into_iter()
+ .map(NonStrictExpression::for_test)
+ .collect()],
Schema { fields },
barrier_receiver,
10005,
diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs
index ad332112ef269..5e5454cecff93 100644
--- a/src/stream/src/executor/watermark_filter.rs
+++ b/src/stream/src/executor/watermark_filter.rs
@@ -23,7 +23,8 @@ use risingwave_common::row::{OwnedRow, Row};
use risingwave_common::types::{DataType, DefaultOrd, ScalarImpl};
use risingwave_common::{bail, row};
use risingwave_expr::expr::{
- build_func_non_strict, BoxedExpression, Expression, InputRefExpression, LiteralExpression,
+ build_func_non_strict, ExpressionBoxExt, InputRefExpression, LiteralExpression,
+ NonStrictExpression,
};
use risingwave_expr::Result as ExprResult;
use risingwave_pb::expr::expr_node::Type;
@@ -44,7 +45,7 @@ use crate::task::ActorEvalErrorReport;
pub struct WatermarkFilterExecutor {
input: BoxedExecutor,
/// The expression used to calculate the watermark value.
- watermark_expr: BoxedExpression,
+ watermark_expr: NonStrictExpression,
/// The column we should generate watermark and filter on.
event_time_col_idx: usize,
ctx: ActorContextRef,
@@ -55,7 +56,7 @@ pub struct WatermarkFilterExecutor {
impl WatermarkFilterExecutor {
pub fn new(
input: BoxedExecutor,
- watermark_expr: BoxedExpression,
+ watermark_expr: NonStrictExpression,
event_time_col_idx: usize,
ctx: ActorContextRef,
table: StateTable,
@@ -298,7 +299,7 @@ impl WatermarkFilterExecutor {
event_time_col_idx: usize,
watermark: ScalarImpl,
eval_error_report: ActorEvalErrorReport,
- ) -> ExprResult {
+ ) -> ExprResult {
build_func_non_strict(
Type::GreaterThanOrEqual,
DataType::Boolean,
@@ -350,11 +351,11 @@ mod tests {
use risingwave_common::test_prelude::StreamChunkTestExt;
use risingwave_common::types::Date;
use risingwave_common::util::sort_util::OrderType;
- use risingwave_expr::expr::build_from_pretty;
use risingwave_storage::memory::MemoryStateStore;
use risingwave_storage::table::Distribution;
use super::*;
+ use crate::executor::test_utils::expr::build_from_pretty;
use crate::executor::test_utils::{MessageSender, MockSource};
use crate::executor::ActorContext;
diff --git a/src/stream/src/from_proto/hash_join.rs b/src/stream/src/from_proto/hash_join.rs
index 44799af9405c2..87174282e517a 100644
--- a/src/stream/src/from_proto/hash_join.rs
+++ b/src/stream/src/from_proto/hash_join.rs
@@ -18,7 +18,7 @@ use std::sync::Arc;
use risingwave_common::hash::{HashKey, HashKeyDispatcher};
use risingwave_common::types::DataType;
use risingwave_expr::expr::{
- build_func_non_strict, build_non_strict_from_prost, BoxedExpression, InputRefExpression,
+ build_func_non_strict, build_non_strict_from_prost, InputRefExpression, NonStrictExpression,
};
pub use risingwave_pb::expr::expr_node::Type as ExprType;
use risingwave_pb::plan_common::JoinType as JoinTypeProto;
@@ -109,7 +109,8 @@ impl ExecutorBuilder for HashJoinExecutorBuilder {
build_non_strict_from_prost(
delta_expression.delta.as_ref().unwrap(),
params.eval_error_report.clone(),
- )?,
+ )?
+ .into_inner(),
],
params.eval_error_report.clone(),
)?)
@@ -175,8 +176,8 @@ struct HashJoinExecutorDispatcherArgs {
pk_indices: PkIndices,
output_indices: Vec,
executor_id: u64,
- cond: Option,
- inequality_pairs: Vec<(usize, usize, bool, Option)>,
+ cond: Option,
+ inequality_pairs: Vec<(usize, usize, bool, Option)>,
op_info: String,
state_table_l: StateTable,
degree_state_table_l: StateTable,
diff --git a/src/stream/src/from_proto/temporal_join.rs b/src/stream/src/from_proto/temporal_join.rs
index 8b7b3b6af1335..58699089e8c27 100644
--- a/src/stream/src/from_proto/temporal_join.rs
+++ b/src/stream/src/from_proto/temporal_join.rs
@@ -18,7 +18,7 @@ use risingwave_common::catalog::{ColumnDesc, TableId, TableOption};
use risingwave_common::hash::{HashKey, HashKeyDispatcher};
use risingwave_common::types::DataType;
use risingwave_common::util::sort_util::OrderType;
-use risingwave_expr::expr::{build_non_strict_from_prost, BoxedExpression};
+use risingwave_expr::expr::{build_non_strict_from_prost, NonStrictExpression};
use risingwave_pb::plan_common::{JoinType as JoinTypeProto, StorageTableDesc};
use risingwave_storage::table::batch_table::storage_table::StorageTable;
use risingwave_storage::table::Distribution;
@@ -190,7 +190,7 @@ struct TemporalJoinExecutorDispatcherArgs {
left_join_keys: Vec,
right_join_keys: Vec,
null_safe: Vec,
- condition: Option,
+ condition: Option,
pk_indices: PkIndices,
output_indices: Vec,
table_output_indices: Vec,
diff --git a/src/stream/tests/integration_tests/hop_window.rs b/src/stream/tests/integration_tests/hop_window.rs
index 167857cc7d9fc..9d6d879240fc0 100644
--- a/src/stream/tests/integration_tests/hop_window.rs
+++ b/src/stream/tests/integration_tests/hop_window.rs
@@ -15,6 +15,7 @@
use risingwave_common::types::test_utils::IntervalTestExt;
use risingwave_common::types::{Interval, Timestamp};
use risingwave_expr::expr::test_utils::make_hop_window_expression;
+use risingwave_expr::expr::NonStrictExpression;
use risingwave_stream::executor::{ExecutorInfo, HopWindowExecutor};
use crate::prelude::*;
@@ -55,8 +56,14 @@ fn create_executor(output_indices: Vec) -> (MessageSender, BoxedMessageSt
TIME_COL_IDX,
window_slide,
window_size,
- window_start_exprs,
- window_end_exprs,
+ window_start_exprs
+ .into_iter()
+ .map(NonStrictExpression::for_test)
+ .collect(),
+ window_end_exprs
+ .into_iter()
+ .map(NonStrictExpression::for_test)
+ .collect(),
output_indices,
CHUNK_SIZE,
)
diff --git a/src/stream/tests/integration_tests/project_set.rs b/src/stream/tests/integration_tests/project_set.rs
index bf1354c25b83b..61a879256108d 100644
--- a/src/stream/tests/integration_tests/project_set.rs
+++ b/src/stream/tests/integration_tests/project_set.rs
@@ -29,10 +29,10 @@ fn create_executor() -> (MessageSender, BoxedMessageStream) {
};
let (tx, source) = MockSource::channel(schema, PkIndices::new());
- let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)");
- let test_expr_watermark = build_from_pretty("(add:int8 $0:int8 1:int8)");
- let tf1 = repeat(build_from_pretty("1:int4"), 1);
- let tf2 = repeat(build_from_pretty("2:int4"), 2);
+ let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)").into_inner();
+ let test_expr_watermark = build_from_pretty("(add:int8 $0:int8 1:int8)").into_inner();
+ let tf1 = repeat(build_from_pretty("1:int4").into_inner(), 1);
+ let tf2 = repeat(build_from_pretty("2:int4").into_inner(), 2);
let project_set = Box::new(ProjectSetExecutor::new(
ActorContext::create(123),
From 02282b9c3fdd52933fc8baf7032777c184577e57 Mon Sep 17 00:00:00 2001
From: lmatz
Date: Tue, 24 Oct 2023 17:10:48 +0800
Subject: [PATCH 08/11] chore: upgrade the docker image version (#13020)
---
docker/docker-compose.yml | 10 +++++-----
1 file changed, 5 insertions(+), 5 deletions(-)
diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml
index 89aa99a1c8b5d..d25c94daf2670 100644
--- a/docker/docker-compose.yml
+++ b/docker/docker-compose.yml
@@ -2,7 +2,7 @@
version: "3"
services:
compactor-0:
- image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}"
+ image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}"
command:
- compactor-node
- "--listen-addr"
@@ -37,7 +37,7 @@ services:
timeout: 5s
retries: 5
compute-node-0:
- image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}"
+ image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}"
command:
- compute-node
- "--listen-addr"
@@ -122,7 +122,7 @@ services:
timeout: 5s
retries: 5
frontend-node-0:
- image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}"
+ image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}"
command:
- frontend-node
- "--listen-addr"
@@ -179,7 +179,7 @@ services:
timeout: 5s
retries: 5
meta-node-0:
- image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}"
+ image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}"
command:
- meta-node
- "--listen-addr"
@@ -295,7 +295,7 @@ services:
timeout: 5s
retries: 5
connector-node:
- image: ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}
+ image: ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}
entrypoint: "/risingwave/bin/connector-node/start-service.sh"
ports:
- 50051
From e64543f9e5a8b080ddf8d8717846ccd4c68dac80 Mon Sep 17 00:00:00 2001
From: Richard Chien
Date: Tue, 24 Oct 2023 04:15:25 -0500
Subject: [PATCH 09/11] feat(agg): introduce an internal agg kind to avoid
minput state table after #12349 (#12384)
Signed-off-by: Richard Chien
---
proto/expr.proto | 1 +
src/expr/core/src/aggregate/def.rs | 14 +-
src/expr/impl/src/aggregate/general.rs | 9 +
.../tests/testdata/output/agg.yaml | 24 +--
.../testdata/output/batch_index_join.yaml | 6 +-
.../tests/testdata/output/except.yaml | 59 +++---
.../tests/testdata/output/intersect.yaml | 59 +++---
.../tests/testdata/output/nexmark.yaml | 179 +++++++++---------
.../tests/testdata/output/subquery.yaml | 28 +--
.../output/subquery_expr_correlated.yaml | 8 +-
.../src/optimizer/plan_node/generic/agg.rs | 7 +-
.../rule/agg_group_by_simplify_rule.rs | 5 +-
12 files changed, 199 insertions(+), 200 deletions(-)
diff --git a/proto/expr.proto b/proto/expr.proto
index 769532d8dbe19..2f252d67c8400 100644
--- a/proto/expr.proto
+++ b/proto/expr.proto
@@ -348,6 +348,7 @@ message AggCall {
MODE = 24;
LAST_VALUE = 25;
GROUPING = 26;
+ INTERNAL_LAST_SEEN_VALUE = 27;
}
Type type = 1;
repeated InputRef args = 2;
diff --git a/src/expr/core/src/aggregate/def.rs b/src/expr/core/src/aggregate/def.rs
index f71bfd454a415..964ec46c9f9c4 100644
--- a/src/expr/core/src/aggregate/def.rs
+++ b/src/expr/core/src/aggregate/def.rs
@@ -233,6 +233,9 @@ pub enum AggKind {
PercentileDisc,
Mode,
Grouping,
+
+ /// Return last seen one of the input values.
+ InternalLastSeenValue,
}
impl AggKind {
@@ -264,6 +267,7 @@ impl AggKind {
PbType::PercentileDisc => Ok(AggKind::PercentileDisc),
PbType::Mode => Ok(AggKind::Mode),
PbType::Grouping => Ok(AggKind::Grouping),
+ PbType::InternalLastSeenValue => Ok(AggKind::InternalLastSeenValue),
PbType::Unspecified => bail!("Unrecognized agg."),
}
}
@@ -294,8 +298,9 @@ impl AggKind {
Self::VarSamp => PbType::VarSamp,
Self::PercentileCont => PbType::PercentileCont,
Self::PercentileDisc => PbType::PercentileDisc,
- Self::Grouping => PbType::Grouping,
Self::Mode => PbType::Mode,
+ Self::Grouping => PbType::Grouping,
+ Self::InternalLastSeenValue => PbType::InternalLastSeenValue,
}
}
}
@@ -422,6 +427,7 @@ pub mod agg_kinds {
| AggKind::BoolAnd
| AggKind::BoolOr
| AggKind::ApproxCountDistinct
+ | AggKind::InternalLastSeenValue
};
}
pub use single_value_state;
@@ -450,7 +456,11 @@ impl AggKind {
/// Get the total phase agg kind from the partial phase agg kind.
pub fn partial_to_total(self) -> Option {
match self {
- AggKind::BitXor | AggKind::Min | AggKind::Max | AggKind::Sum => Some(self),
+ AggKind::BitXor
+ | AggKind::Min
+ | AggKind::Max
+ | AggKind::Sum
+ | AggKind::InternalLastSeenValue => Some(self),
AggKind::Sum0 | AggKind::Count => Some(AggKind::Sum0),
agg_kinds::simply_cannot_two_phase!() => None,
agg_kinds::rewritten!() => None,
diff --git a/src/expr/impl/src/aggregate/general.rs b/src/expr/impl/src/aggregate/general.rs
index de1331c524063..f47c94d45f24d 100644
--- a/src/expr/impl/src/aggregate/general.rs
+++ b/src/expr/impl/src/aggregate/general.rs
@@ -62,6 +62,15 @@ fn last_value(_: T, input: T) -> T {
input
}
+#[aggregate("internal_last_seen_value(*) -> auto", state = "ref")]
+fn internal_last_seen_value(state: T, input: T, retract: bool) -> T {
+ if retract {
+ state
+ } else {
+ input
+ }
+}
+
/// Note the following corner cases:
///
/// ```slt
diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml
index aefb4df98ef4e..baa77dc79c89b 100644
--- a/src/frontend/planner_test/tests/testdata/output/agg.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml
@@ -1395,20 +1395,20 @@
sq_1.col_2;
batch_plan: |-
BatchExchange { order: [], dist: Single }
- └─BatchProject { exprs: [max(max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))))] }
- └─BatchHashAgg { group_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], aggs: [max(max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))))] }
- └─BatchExchange { order: [], dist: HashShard(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))) }
- └─BatchHashAgg { group_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], aggs: [max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)))] }
- └─BatchSortAgg { group_key: [lineitem.l_orderkey], aggs: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))] }
+ └─BatchProject { exprs: [max(max(internal_last_seen_value(lineitem.l_commitdate)))] }
+ └─BatchHashAgg { group_key: [internal_last_seen_value(lineitem.l_commitdate)], aggs: [max(max(internal_last_seen_value(lineitem.l_commitdate)))] }
+ └─BatchExchange { order: [], dist: HashShard(internal_last_seen_value(lineitem.l_commitdate)) }
+ └─BatchHashAgg { group_key: [internal_last_seen_value(lineitem.l_commitdate)], aggs: [max(internal_last_seen_value(lineitem.l_commitdate))] }
+ └─BatchSortAgg { group_key: [lineitem.l_orderkey], aggs: [internal_last_seen_value(lineitem.l_commitdate)] }
└─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_commitdate], distribution: UpstreamHashShard(lineitem.l_orderkey) }
stream_plan: |-
- StreamMaterialize { columns: [col_0, first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))(hidden)], stream_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], pk_columns: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], pk_conflict: NoCheck }
- └─StreamProject { exprs: [max(max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)))), first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))] }
- └─StreamHashAgg { group_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], aggs: [max(max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)))), count] }
- └─StreamExchange { dist: HashShard(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))) }
- └─StreamHashAgg { group_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)), $expr1], aggs: [max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))), count] }
- └─StreamProject { exprs: [lineitem.l_orderkey, first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)), Vnode(lineitem.l_orderkey) as $expr1] }
- └─StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)), count] }
+ StreamMaterialize { columns: [col_0, internal_last_seen_value(lineitem.l_commitdate)(hidden)], stream_key: [internal_last_seen_value(lineitem.l_commitdate)], pk_columns: [internal_last_seen_value(lineitem.l_commitdate)], pk_conflict: NoCheck }
+ └─StreamProject { exprs: [max(max(internal_last_seen_value(lineitem.l_commitdate))), internal_last_seen_value(lineitem.l_commitdate)] }
+ └─StreamHashAgg { group_key: [internal_last_seen_value(lineitem.l_commitdate)], aggs: [max(max(internal_last_seen_value(lineitem.l_commitdate))), count] }
+ └─StreamExchange { dist: HashShard(internal_last_seen_value(lineitem.l_commitdate)) }
+ └─StreamHashAgg { group_key: [internal_last_seen_value(lineitem.l_commitdate), $expr1], aggs: [max(internal_last_seen_value(lineitem.l_commitdate)), count] }
+ └─StreamProject { exprs: [lineitem.l_orderkey, internal_last_seen_value(lineitem.l_commitdate), Vnode(lineitem.l_orderkey) as $expr1] }
+ └─StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [internal_last_seen_value(lineitem.l_commitdate), count] }
└─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_commitdate], pk: [lineitem.l_orderkey], dist: UpstreamHashShard(lineitem.l_orderkey) }
- name: two phase agg on hop window input should use two phase agg
sql: |
diff --git a/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml b/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml
index 236bc31b2503e..2d1b0951089e8 100644
--- a/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml
@@ -60,9 +60,9 @@
select t2.c, t2.d, count(distinct t.a) from t join t2 on t.a = t2.c group by t2.c, t2.d;
batch_plan: |-
BatchExchange { order: [], dist: Single }
- └─BatchHashAgg { group_key: [first_value(t2.c order_by(t2.c ASC))], aggs: [first_value(first_value(t2.d order_by(t2.d ASC)) order_by(first_value(t2.d order_by(t2.d ASC)) ASC)), count(t.a)] }
- └─BatchExchange { order: [], dist: HashShard(first_value(t2.c order_by(t2.c ASC))) }
- └─BatchHashAgg { group_key: [t.a], aggs: [first_value(t2.c order_by(t2.c ASC)), first_value(t2.d order_by(t2.d ASC))] }
+ └─BatchHashAgg { group_key: [internal_last_seen_value(t2.c)], aggs: [internal_last_seen_value(internal_last_seen_value(t2.d)), count(t.a)] }
+ └─BatchExchange { order: [], dist: HashShard(internal_last_seen_value(t2.c)) }
+ └─BatchHashAgg { group_key: [t.a], aggs: [internal_last_seen_value(t2.c), internal_last_seen_value(t2.d)] }
└─BatchLookupJoin { type: Inner, predicate: t.a = t2.c, output: [t2.c, t2.d, t.a] }
└─BatchExchange { order: [], dist: UpstreamHashShard(t.a) }
└─BatchScan { table: t, columns: [t.a], distribution: SomeShard }
diff --git a/src/frontend/planner_test/tests/testdata/output/except.yaml b/src/frontend/planner_test/tests/testdata/output/except.yaml
index 204a1814b8db7..1e27a7b74c0f0 100644
--- a/src/frontend/planner_test/tests/testdata/output/except.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/except.yaml
@@ -108,20 +108,20 @@
create table t2 (a int, b numeric, c bigint, primary key(a));
select * from t1 except select * from t2;
optimized_logical_plan_for_batch: |-
- LogicalAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] }
+ LogicalAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] }
└─LogicalJoin { type: LeftAnti, on: IsNotDistinctFrom(t1.a, t2.a) AND IsNotDistinctFrom(t1.b, t2.b) AND IsNotDistinctFrom(t1.c, t2.c), output: all }
├─LogicalScan { table: t1, columns: [t1.a, t1.b, t1.c] }
└─LogicalScan { table: t2, columns: [t2.a, t2.b, t2.c] }
batch_plan: |-
BatchExchange { order: [], dist: Single }
- └─BatchHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] }
+ └─BatchHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] }
└─BatchLookupJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all }
└─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) }
└─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: UpstreamHashShard(t1.a) }
stream_plan: |-
StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck }
- └─StreamProject { exprs: [t1.a, first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] }
- └─StreamHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC)), count] }
+ └─StreamProject { exprs: [t1.a, internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] }
+ └─StreamHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c), count] }
└─StreamExchange { dist: HashShard(t1.a) }
└─StreamHashJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all }
├─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) }
@@ -130,83 +130,70 @@
└─StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) }
stream_dist_plan: |+
Fragment 0
- StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } { materialized table: 4294967294 }
- └── StreamProject { exprs: [t1.a, first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] }
- └── StreamHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC)), count] }
- ├── intermediate state table: 2
- ├── state tables: [ 0, 1 ]
+ StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck }
+ ├── materialized table: 4294967294
+ └── StreamProject { exprs: [t1.a, internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] }
+ └── StreamHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c), count] }
+ ├── intermediate state table: 0
+ ├── state tables: []
├── distinct tables: []
└── StreamExchange Hash([0]) from 1
Fragment 1
StreamHashJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all }
- ├── left table: 3
- ├── right table: 5
- ├── left degree table: 4
- ├── right degree table: 6
+ ├── left table: 1
+ ├── right table: 3
+ ├── left degree table: 2
+ ├── right degree table: 4
├── StreamExchange Hash([0, 1, 2]) from 2
└── StreamExchange Hash([0, 1, 2]) from 3
Fragment 2
- Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 7 }
+ Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 5 }
├── Upstream
└── BatchPlanNode
Fragment 3
- Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 8 }
+ Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 6 }
├── Upstream
└── BatchPlanNode
Table 0
- ├── columns: [ t1_a, t1_b, t1_c ]
- ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
- ├── value indices: [ 0, 1, 2 ]
- ├── distribution key: [ 0 ]
- └── read pk prefix len hint: 1
-
- Table 1
- ├── columns: [ t1_a, t1_c, t1_b ]
- ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
- ├── value indices: [ 0, 1, 2 ]
- ├── distribution key: [ 0 ]
- └── read pk prefix len hint: 1
-
- Table 2
- ├── columns: [ t1_a, first_value(t1_b order_by(t1_b ASC)), first_value(t1_c order_by(t1_c ASC)), count ]
+ ├── columns: [ t1_a, internal_last_seen_value(t1_b), internal_last_seen_value(t1_c), count ]
├── primary key: [ $0 ASC ]
├── value indices: [ 1, 2, 3 ]
├── distribution key: [ 0 ]
└── read pk prefix len hint: 1
- Table 3
+ Table 1
├── columns: [ t1_a, t1_b, t1_c ]
├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
├── value indices: [ 0, 1, 2 ]
├── distribution key: [ 0, 1, 2 ]
└── read pk prefix len hint: 3
- Table 4
+ Table 2
├── columns: [ t1_a, t1_b, t1_c, _degree ]
├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
├── value indices: [ 3 ]
├── distribution key: [ 0, 1, 2 ]
└── read pk prefix len hint: 3
- Table 5
+ Table 3
├── columns: [ t2_a, t2_b, t2_c ]
├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
├── value indices: [ 0, 1, 2 ]
├── distribution key: [ 0, 1, 2 ]
└── read pk prefix len hint: 3
- Table 6
+ Table 4
├── columns: [ t2_a, t2_b, t2_c, _degree ]
├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
├── value indices: [ 3 ]
├── distribution key: [ 0, 1, 2 ]
└── read pk prefix len hint: 3
- Table 7
+ Table 5
├── columns: [ vnode, a, t1_backfill_finished, t1_row_count ]
├── primary key: [ $0 ASC ]
├── value indices: [ 1, 2, 3 ]
@@ -214,7 +201,7 @@
├── read pk prefix len hint: 1
└── vnode column idx: 0
- Table 8
+ Table 6
├── columns: [ vnode, a, t2_backfill_finished, t2_row_count ]
├── primary key: [ $0 ASC ]
├── value indices: [ 1, 2, 3 ]
diff --git a/src/frontend/planner_test/tests/testdata/output/intersect.yaml b/src/frontend/planner_test/tests/testdata/output/intersect.yaml
index c203f1f953814..91839346824ec 100644
--- a/src/frontend/planner_test/tests/testdata/output/intersect.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/intersect.yaml
@@ -108,20 +108,20 @@
create table t2 (a int, b numeric, c bigint, primary key(a));
select * from t1 intersect select * from t2;
optimized_logical_plan_for_batch: |-
- LogicalAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] }
+ LogicalAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] }
└─LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(t1.a, t2.a) AND IsNotDistinctFrom(t1.b, t2.b) AND IsNotDistinctFrom(t1.c, t2.c), output: all }
├─LogicalScan { table: t1, columns: [t1.a, t1.b, t1.c] }
└─LogicalScan { table: t2, columns: [t2.a, t2.b, t2.c] }
batch_plan: |-
BatchExchange { order: [], dist: Single }
- └─BatchHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] }
+ └─BatchHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] }
└─BatchLookupJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all }
└─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) }
└─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: UpstreamHashShard(t1.a) }
stream_plan: |-
StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck }
- └─StreamProject { exprs: [t1.a, first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] }
- └─StreamHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC)), count] }
+ └─StreamProject { exprs: [t1.a, internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] }
+ └─StreamHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c), count] }
└─StreamExchange { dist: HashShard(t1.a) }
└─StreamHashJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all }
├─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) }
@@ -130,83 +130,70 @@
└─StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) }
stream_dist_plan: |+
Fragment 0
- StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } { materialized table: 4294967294 }
- └── StreamProject { exprs: [t1.a, first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] }
- └── StreamHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC)), count] }
- ├── intermediate state table: 2
- ├── state tables: [ 0, 1 ]
+ StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck }
+ ├── materialized table: 4294967294
+ └── StreamProject { exprs: [t1.a, internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] }
+ └── StreamHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c), count] }
+ ├── intermediate state table: 0
+ ├── state tables: []
├── distinct tables: []
└── StreamExchange Hash([0]) from 1
Fragment 1
StreamHashJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all }
- ├── left table: 3
- ├── right table: 5
- ├── left degree table: 4
- ├── right degree table: 6
+ ├── left table: 1
+ ├── right table: 3
+ ├── left degree table: 2
+ ├── right degree table: 4
├── StreamExchange Hash([0, 1, 2]) from 2
└── StreamExchange Hash([0, 1, 2]) from 3
Fragment 2
- Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 7 }
+ Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 5 }
├── Upstream
└── BatchPlanNode
Fragment 3
- Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 8 }
+ Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 6 }
├── Upstream
└── BatchPlanNode
Table 0
- ├── columns: [ t1_a, t1_b, t1_c ]
- ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
- ├── value indices: [ 0, 1, 2 ]
- ├── distribution key: [ 0 ]
- └── read pk prefix len hint: 1
-
- Table 1
- ├── columns: [ t1_a, t1_c, t1_b ]
- ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
- ├── value indices: [ 0, 1, 2 ]
- ├── distribution key: [ 0 ]
- └── read pk prefix len hint: 1
-
- Table 2
- ├── columns: [ t1_a, first_value(t1_b order_by(t1_b ASC)), first_value(t1_c order_by(t1_c ASC)), count ]
+ ├── columns: [ t1_a, internal_last_seen_value(t1_b), internal_last_seen_value(t1_c), count ]
├── primary key: [ $0 ASC ]
├── value indices: [ 1, 2, 3 ]
├── distribution key: [ 0 ]
└── read pk prefix len hint: 1
- Table 3
+ Table 1
├── columns: [ t1_a, t1_b, t1_c ]
├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
├── value indices: [ 0, 1, 2 ]
├── distribution key: [ 0, 1, 2 ]
└── read pk prefix len hint: 3
- Table 4
+ Table 2
├── columns: [ t1_a, t1_b, t1_c, _degree ]
├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
├── value indices: [ 3 ]
├── distribution key: [ 0, 1, 2 ]
└── read pk prefix len hint: 3
- Table 5
+ Table 3
├── columns: [ t2_a, t2_b, t2_c ]
├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
├── value indices: [ 0, 1, 2 ]
├── distribution key: [ 0, 1, 2 ]
└── read pk prefix len hint: 3
- Table 6
+ Table 4
├── columns: [ t2_a, t2_b, t2_c, _degree ]
├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
├── value indices: [ 3 ]
├── distribution key: [ 0, 1, 2 ]
└── read pk prefix len hint: 3
- Table 7
+ Table 5
├── columns: [ vnode, a, t1_backfill_finished, t1_row_count ]
├── primary key: [ $0 ASC ]
├── value indices: [ 1, 2, 3 ]
@@ -214,7 +201,7 @@
├── read pk prefix len hint: 1
└── vnode column idx: 0
- Table 8
+ Table 6
├── columns: [ vnode, a, t2_backfill_finished, t2_row_count ]
├── primary key: [ $0 ASC ]
├── value indices: [ 1, 2, 3 ]
diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml
index f4b9e28ce0775..2f23b0674fa84 100644
--- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml
@@ -834,9 +834,9 @@
AND P.endtime = A.endtime;
batch_plan: |-
BatchExchange { order: [], dist: Single }
- └─BatchHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, first_value(person.name order_by(person.name ASC)), $expr1] }
+ └─BatchHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1] }
├─BatchExchange { order: [], dist: HashShard(person.id, $expr1, $expr2) }
- │ └─BatchHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [first_value(person.name order_by(person.name ASC))] }
+ │ └─BatchHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name)] }
│ └─BatchProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] }
│ └─BatchProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] }
│ └─BatchScan { table: person, columns: [person.id, person.name, person.date_time], distribution: UpstreamHashShard(person.id) }
@@ -847,10 +847,10 @@
└─BatchScan { table: auction, columns: [auction.date_time, auction.seller], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2], pk_columns: [id, starttime, $expr2], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, first_value(person.name order_by(person.name ASC)), $expr1, $expr2, auction.seller, $expr3, $expr4] }
+ └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] }
├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) }
- │ └─StreamProject { exprs: [person.id, $expr1, $expr2, first_value(person.name order_by(person.name ASC))] }
- │ └─StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [first_value(person.name order_by(person.name ASC)), count] }
+ │ └─StreamProject { exprs: [person.id, $expr1, $expr2, internal_last_seen_value(person.name)] }
+ │ └─StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] }
│ └─StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] }
│ └─StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] }
│ └─StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) }
@@ -864,33 +864,33 @@
Fragment 0
StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2], pk_columns: [id, starttime, $expr2], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, first_value(person.name order_by(person.name ASC)), $expr1, $expr2, auction.seller, $expr3, $expr4] }
+ └── StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] }
├── left table: 0
├── right table: 2
├── left degree table: 1
├── right degree table: 3
├── StreamExchange Hash([0, 1, 2]) from 1
└── StreamProject { exprs: [auction.seller, $expr3, $expr4] }
- └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { intermediate state table: 7, state tables: [], distinct tables: [] }
+ └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] }
└── StreamExchange Hash([0, 1, 2]) from 2
Fragment 1
- StreamProject { exprs: [person.id, $expr1, $expr2, first_value(person.name order_by(person.name ASC))] }
- └── StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [first_value(person.name order_by(person.name ASC)), count] } { intermediate state table: 5, state tables: [ 4 ], distinct tables: [] }
+ StreamProject { exprs: [person.id, $expr1, $expr2, internal_last_seen_value(person.name)] }
+ └── StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] } { intermediate state table: 4, state tables: [], distinct tables: [] }
└── StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] }
└── StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] }
- └── Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 6 }
+ └── Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 5 }
├── Upstream
└── BatchPlanNode
Fragment 2
StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] }
└── StreamProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, auction.id] }
- └── Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 8 }
+ └── Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 }
├── Upstream
└── BatchPlanNode
- Table 0 { columns: [ person_id, $expr1, $expr2, first_value(person_name order_by(person_name ASC)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 }
+ Table 0 { columns: [ person_id, $expr1, $expr2, internal_last_seen_value(person_name) ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 }
Table 1 { columns: [ person_id, $expr1, $expr2, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 }
@@ -898,17 +898,20 @@
Table 3 { columns: [ auction_seller, $expr3, $expr4, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 }
- Table 4 { columns: [ person_id, $expr1, $expr2, person_name ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 3 ], distribution key: [ 0 ], read pk prefix len hint: 3 }
+ Table 4 { columns: [ person_id, $expr1, $expr2, internal_last_seen_value(person_name), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 }
- Table 5 { columns: [ person_id, $expr1, $expr2, first_value(person_name order_by(person_name ASC)), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 }
+ Table 5 { columns: [ vnode, id, person_backfill_finished, person_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 }
- Table 6 { columns: [ vnode, id, person_backfill_finished, person_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 }
+ Table 6 { columns: [ auction_seller, $expr3, $expr4, count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 }
- Table 7 { columns: [ auction_seller, $expr3, $expr4, count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 }
+ Table 7 { columns: [ vnode, id, auction_backfill_finished, auction_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 }
- Table 8 { columns: [ vnode, id, auction_backfill_finished, auction_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 }
-
- Table 4294967294 { columns: [ id, name, starttime, $expr2, auction.seller, $expr3, $expr4 ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 }
+ Table 4294967294
+ ├── columns: [ id, name, starttime, $expr2, auction.seller, $expr3, $expr4 ]
+ ├── primary key: [ $0 ASC, $2 ASC, $3 ASC ]
+ ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ]
+ ├── distribution key: [ 0, 2, 3 ]
+ └── read pk prefix len hint: 3
- id: nexmark_q9
before:
@@ -2026,9 +2029,9 @@
SELECT COUNT(*) / COUNT(DISTINCT auction) FROM bid
)
batch_plan: |-
- BatchNestedLoopJoin { type: Inner, predicate: (count(bid.auction) >= $expr1), output: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] }
+ BatchNestedLoopJoin { type: Inner, predicate: (count(bid.auction) >= $expr1), output: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] }
├─BatchExchange { order: [], dist: Single }
- │ └─BatchHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] }
+ │ └─BatchHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction)] }
│ └─BatchHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all }
│ ├─BatchExchange { order: [], dist: HashShard(auction.id) }
│ │ └─BatchScan { table: auction, columns: [auction.id, auction.item_name], distribution: UpstreamHashShard(auction.id) }
@@ -2043,9 +2046,9 @@
└─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck }
- └─StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] }
- ├─StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] }
- │ └─StreamHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), count] }
+ └─StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] }
+ ├─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] }
+ │ └─StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] }
│ └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all }
│ ├─StreamExchange { dist: HashShard(auction.id) }
│ │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
@@ -2063,50 +2066,50 @@
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] }
+ └── StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] }
├── left table: 0
├── right table: 1
- ├── StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] }
- │ └── StreamHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), count] }
- │ ├── intermediate state table: 3
- │ ├── state tables: [ 2 ]
+ ├── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] }
+ │ └── StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] }
+ │ ├── intermediate state table: 2
+ │ ├── state tables: []
│ ├── distinct tables: []
│ └── StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all }
- │ ├── left table: 4
- │ ├── right table: 6
- │ ├── left degree table: 5
- │ ├── right degree table: 7
+ │ ├── left table: 3
+ │ ├── right table: 5
+ │ ├── left degree table: 4
+ │ ├── right degree table: 6
│ ├── StreamExchange Hash([0]) from 1
│ └── StreamExchange Hash([0]) from 2
└── StreamExchange Broadcast from 3
Fragment 1
- Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 8 }
+ Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 }
├── Upstream
└── BatchPlanNode
Fragment 2
- Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 9 }
+ Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 8 }
├── Upstream
└── BatchPlanNode
Fragment 3
StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(bid.auction))) as $expr1] }
- └── StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(bid.auction)), count] } { intermediate state table: 10, state tables: [], distinct tables: [] }
+ └── StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(bid.auction)), count] } { intermediate state table: 9, state tables: [], distinct tables: [] }
└── StreamExchange Single from 4
Fragment 4
StreamStatelessSimpleAgg { aggs: [sum0(count), count(bid.auction)] }
- └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } { intermediate state table: 11, state tables: [], distinct tables: [] }
+ └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } { intermediate state table: 10, state tables: [], distinct tables: [] }
└── StreamExchange Hash([0]) from 5
Fragment 5
- Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 12 }
+ Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 11 }
├── Upstream
└── BatchPlanNode
Table 0
- ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction) ]
+ ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction) ]
├── primary key: [ $2 ASC, $0 ASC ]
├── value indices: [ 0, 1, 2 ]
├── distribution key: [ 0 ]
@@ -2115,28 +2118,36 @@
Table 1 { columns: [ $expr1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 }
Table 2
- ├── columns: [ auction_id, auction_item_name, bid__row_id ]
- ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
- ├── value indices: [ 0, 1, 2 ]
+ ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), count ]
+ ├── primary key: [ $0 ASC ]
+ ├── value indices: [ 1, 2, 3 ]
├── distribution key: [ 0 ]
└── read pk prefix len hint: 1
Table 3
- ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction), count ]
+ ├── columns: [ auction_id, auction_item_name ]
├── primary key: [ $0 ASC ]
- ├── value indices: [ 1, 2, 3 ]
+ ├── value indices: [ 0, 1 ]
├── distribution key: [ 0 ]
└── read pk prefix len hint: 1
- Table 4 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
-
- Table 5 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
+ Table 4 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
- Table 6 { columns: [ bid_auction, bid__row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
+ Table 5
+ ├── columns: [ bid_auction, bid__row_id ]
+ ├── primary key: [ $0 ASC, $1 ASC ]
+ ├── value indices: [ 0, 1 ]
+ ├── distribution key: [ 0 ]
+ └── read pk prefix len hint: 1
- Table 7 { columns: [ bid_auction, bid__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
+ Table 6
+ ├── columns: [ bid_auction, bid__row_id, _degree ]
+ ├── primary key: [ $0 ASC, $1 ASC ]
+ ├── value indices: [ 2 ]
+ ├── distribution key: [ 0 ]
+ └── read pk prefix len hint: 1
- Table 8
+ Table 7
├── columns: [ vnode, id, auction_backfill_finished, auction_row_count ]
├── primary key: [ $0 ASC ]
├── value indices: [ 1, 2, 3 ]
@@ -2144,7 +2155,7 @@
├── read pk prefix len hint: 1
└── vnode column idx: 0
- Table 9
+ Table 8
├── columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ]
├── primary key: [ $0 ASC ]
├── value indices: [ 1, 2, 3 ]
@@ -2152,11 +2163,16 @@
├── read pk prefix len hint: 1
└── vnode column idx: 0
- Table 10 { columns: [ sum0(sum0(count)), sum0(count(bid_auction)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 }
+ Table 9
+ ├── columns: [ sum0(sum0(count)), sum0(count(bid_auction)), count ]
+ ├── primary key: []
+ ├── value indices: [ 0, 1, 2 ]
+ ├── distribution key: []
+ └── read pk prefix len hint: 0
- Table 11 { columns: [ bid_auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
+ Table 10 { columns: [ bid_auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
- Table 12
+ Table 11
├── columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ]
├── primary key: [ $0 ASC ]
├── value indices: [ 1, 2, 3 ]
@@ -2427,7 +2443,7 @@
BatchTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 }
└─BatchExchange { order: [], dist: Single }
└─BatchTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 }
- └─BatchHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] }
+ └─BatchHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction)] }
└─BatchHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all }
├─BatchExchange { order: [], dist: HashShard(auction.id) }
│ └─BatchScan { table: auction, columns: [auction.id, auction.item_name], distribution: UpstreamHashShard(auction.id) }
@@ -2435,12 +2451,12 @@
└─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [bid_count, auction_id], pk_conflict: NoCheck }
- └─StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] }
+ └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] }
└─StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 }
└─StreamExchange { dist: Single }
└─StreamGroupTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] }
- └─StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), Vnode(auction.id) as $expr1] }
- └─StreamHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), count] }
+ └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction), Vnode(auction.id) as $expr1] }
+ └─StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] }
└─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all }
├─StreamExchange { dist: HashShard(auction.id) }
│ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
@@ -2450,44 +2466,44 @@
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [bid_count, auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] }
+ └── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] }
└── StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } { state table: 0 }
└── StreamExchange Single from 1
Fragment 1
StreamGroupTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } { state table: 1 }
- └── StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), Vnode(auction.id) as $expr1] }
- └── StreamHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), count] }
- ├── intermediate state table: 3
- ├── state tables: [ 2 ]
+ └── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction), Vnode(auction.id) as $expr1] }
+ └── StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] }
+ ├── intermediate state table: 2
+ ├── state tables: []
├── distinct tables: []
└── StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all }
- ├── left table: 4
- ├── right table: 6
- ├── left degree table: 5
- ├── right degree table: 7
+ ├── left table: 3
+ ├── right table: 5
+ ├── left degree table: 4
+ ├── right degree table: 6
├── StreamExchange Hash([0]) from 2
└── StreamExchange Hash([0]) from 3
Fragment 2
- Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 8 }
+ Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 }
├── Upstream
└── BatchPlanNode
Fragment 3
- Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 9 }
+ Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 8 }
├── Upstream
└── BatchPlanNode
Table 0
- ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction), $expr1 ]
+ ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), $expr1 ]
├── primary key: [ $2 DESC, $0 ASC ]
├── value indices: [ 0, 1, 2, 3 ]
├── distribution key: []
└── read pk prefix len hint: 0
Table 1
- ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction), $expr1 ]
+ ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), $expr1 ]
├── primary key: [ $3 ASC, $2 DESC, $0 ASC ]
├── value indices: [ 0, 1, 2, 3 ]
├── distribution key: [ 0 ]
@@ -2495,38 +2511,31 @@
└── vnode column idx: 3
Table 2
- ├── columns: [ auction_id, auction_item_name, bid__row_id ]
- ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ]
- ├── value indices: [ 0, 1, 2 ]
- ├── distribution key: [ 0 ]
- └── read pk prefix len hint: 1
-
- Table 3
- ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction), count ]
+ ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), count ]
├── primary key: [ $0 ASC ]
├── value indices: [ 1, 2, 3 ]
├── distribution key: [ 0 ]
└── read pk prefix len hint: 1
- Table 4 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
+ Table 3 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
- Table 5 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
+ Table 4 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
- Table 6
+ Table 5
├── columns: [ bid_auction, bid__row_id ]
├── primary key: [ $0 ASC, $1 ASC ]
├── value indices: [ 0, 1 ]
├── distribution key: [ 0 ]
└── read pk prefix len hint: 1
- Table 7
+ Table 6
├── columns: [ bid_auction, bid__row_id, _degree ]
├── primary key: [ $0 ASC, $1 ASC ]
├── value indices: [ 2 ]
├── distribution key: [ 0 ]
└── read pk prefix len hint: 1
- Table 8
+ Table 7
├── columns: [ vnode, id, auction_backfill_finished, auction_row_count ]
├── primary key: [ $0 ASC ]
├── value indices: [ 1, 2, 3 ]
@@ -2534,7 +2543,7 @@
├── read pk prefix len hint: 1
└── vnode column idx: 0
- Table 9
+ Table 8
├── columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ]
├── primary key: [ $0 ASC ]
├── value indices: [ 1, 2, 3 ]
diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml
index 1ad1d9f92c418..003e4185ada81 100644
--- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml
@@ -582,29 +582,29 @@
create table t(x int[], y int[], k int primary key);
select *, (select sum(i) from (select unnest(x) i, 1 c) Q where k = c ) as sum_x from t;
optimized_logical_plan_for_batch: |-
- LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t.x, first_value(t.x order_by(t.x ASC))) AND IsNotDistinctFrom(t.k, t.k), output: [t.x, t.y, t.k, sum(Unnest($0))] }
+ LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t.x, internal_last_seen_value(t.x)) AND IsNotDistinctFrom(t.k, t.k), output: [t.x, t.y, t.k, sum(Unnest($0))] }
├─LogicalScan { table: t, columns: [t.x, t.y, t.k] }
- └─LogicalAgg { group_key: [first_value(t.x order_by(t.x ASC)), t.k], aggs: [sum(Unnest($0))] }
- └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(first_value(t.x order_by(t.x ASC)), first_value(t.x order_by(t.x ASC))) AND IsNotDistinctFrom(t.k, t.k), output: [first_value(t.x order_by(t.x ASC)), t.k, Unnest($0)] }
- ├─LogicalAgg { group_key: [t.k], aggs: [first_value(t.x order_by(t.x ASC))] }
+ └─LogicalAgg { group_key: [internal_last_seen_value(t.x), t.k], aggs: [sum(Unnest($0))] }
+ └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(internal_last_seen_value(t.x), internal_last_seen_value(t.x)) AND IsNotDistinctFrom(t.k, t.k), output: [internal_last_seen_value(t.x), t.k, Unnest($0)] }
+ ├─LogicalAgg { group_key: [t.k], aggs: [internal_last_seen_value(t.x)] }
│ └─LogicalScan { table: t, columns: [t.x, t.k] }
- └─LogicalProject { exprs: [first_value(t.x order_by(t.x ASC)), t.k, Unnest($0)] }
+ └─LogicalProject { exprs: [internal_last_seen_value(t.x), t.k, Unnest($0)] }
└─LogicalProjectSet { select_list: [$0, $1, Unnest($0)] }
- └─LogicalJoin { type: Inner, on: true, output: [first_value(t.x order_by(t.x ASC)), t.k] }
- ├─LogicalAgg { group_key: [t.k], aggs: [first_value(t.x order_by(t.x ASC))] }
+ └─LogicalJoin { type: Inner, on: true, output: [internal_last_seen_value(t.x), t.k] }
+ ├─LogicalAgg { group_key: [t.k], aggs: [internal_last_seen_value(t.x)] }
│ └─LogicalScan { table: t, columns: [t.x, t.k], predicate: (t.k = 1:Int32) }
└─LogicalValues { rows: [[]], schema: Schema { fields: [] } }
optimized_logical_plan_for_stream: |-
- LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t.x, first_value(t.x order_by(t.x ASC))) AND IsNotDistinctFrom(t.k, t.k), output: [t.x, t.y, t.k, sum(Unnest($0))] }
+ LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t.x, internal_last_seen_value(t.x)) AND IsNotDistinctFrom(t.k, t.k), output: [t.x, t.y, t.k, sum(Unnest($0))] }
├─LogicalScan { table: t, columns: [t.x, t.y, t.k] }
- └─LogicalAgg { group_key: [first_value(t.x order_by(t.x ASC)), t.k], aggs: [sum(Unnest($0))] }
- └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(first_value(t.x order_by(t.x ASC)), first_value(t.x order_by(t.x ASC))) AND IsNotDistinctFrom(t.k, t.k), output: [first_value(t.x order_by(t.x ASC)), t.k, Unnest($0)] }
- ├─LogicalAgg { group_key: [t.k], aggs: [first_value(t.x order_by(t.x ASC))] }
+ └─LogicalAgg { group_key: [internal_last_seen_value(t.x), t.k], aggs: [sum(Unnest($0))] }
+ └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(internal_last_seen_value(t.x), internal_last_seen_value(t.x)) AND IsNotDistinctFrom(t.k, t.k), output: [internal_last_seen_value(t.x), t.k, Unnest($0)] }
+ ├─LogicalAgg { group_key: [t.k], aggs: [internal_last_seen_value(t.x)] }
│ └─LogicalScan { table: t, columns: [t.x, t.k] }
- └─LogicalProject { exprs: [first_value(t.x order_by(t.x ASC)), t.k, Unnest($0)] }
+ └─LogicalProject { exprs: [internal_last_seen_value(t.x), t.k, Unnest($0)] }
└─LogicalProjectSet { select_list: [$0, $1, Unnest($0)] }
- └─LogicalJoin { type: Inner, on: true, output: [first_value(t.x order_by(t.x ASC)), t.k] }
- ├─LogicalAgg { group_key: [t.k], aggs: [first_value(t.x order_by(t.x ASC))] }
+ └─LogicalJoin { type: Inner, on: true, output: [internal_last_seen_value(t.x), t.k] }
+ ├─LogicalAgg { group_key: [t.k], aggs: [internal_last_seen_value(t.x)] }
│ └─LogicalScan { table: t, columns: [t.x, t.k], predicate: (t.k = 1:Int32) }
└─LogicalValues { rows: [[]], schema: Schema { fields: [] } }
- name: CorrelatedInputRef in ProjectSet and apply on condition refers to table function.
diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml
index 6d216ad9c81c4..3b84def33abfa 100644
--- a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml
@@ -466,14 +466,14 @@
└─LogicalScan { table: c, columns: [c.c1, c.c2, c.c3, c._row_id] }
optimized_logical_plan_for_batch: |-
LogicalAgg { aggs: [count] }
- └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a3, first_value(a.a3 order_by(a.a3 ASC))) AND IsNotDistinctFrom(b.b2, b.b2), output: [] }
+ └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a3, internal_last_seen_value(a.a3)) AND IsNotDistinctFrom(b.b2, b.b2), output: [] }
├─LogicalJoin { type: Inner, on: (a.a3 = b.b2), output: all }
│ ├─LogicalScan { table: a, columns: [a.a3] }
│ └─LogicalScan { table: b, columns: [b.b2] }
└─LogicalFilter { predicate: (3:Int32 = count(1:Int32)) }
- └─LogicalAgg { group_key: [first_value(a.a3 order_by(a.a3 ASC)), b.b2], aggs: [count(1:Int32)] }
- └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(first_value(a.a3 order_by(a.a3 ASC)), c.c3) AND IsNotDistinctFrom(b.b2, c.c2), output: [first_value(a.a3 order_by(a.a3 ASC)), b.b2, 1:Int32] }
- ├─LogicalAgg { group_key: [b.b2], aggs: [first_value(a.a3 order_by(a.a3 ASC))] }
+ └─LogicalAgg { group_key: [internal_last_seen_value(a.a3), b.b2], aggs: [count(1:Int32)] }
+ └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(internal_last_seen_value(a.a3), c.c3) AND IsNotDistinctFrom(b.b2, c.c2), output: [internal_last_seen_value(a.a3), b.b2, 1:Int32] }
+ ├─LogicalAgg { group_key: [b.b2], aggs: [internal_last_seen_value(a.a3)] }
│ └─LogicalJoin { type: Inner, on: (a.a3 = b.b2), output: all }
│ ├─LogicalScan { table: a, columns: [a.a3] }
│ └─LogicalScan { table: b, columns: [b.b2] }
diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs
index 2fb251ca89aa6..e0c7e339ee6a6 100644
--- a/src/frontend/src/optimizer/plan_node/generic/agg.rs
+++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs
@@ -516,11 +516,8 @@ impl Agg {
// we use materialized input state for non-retractable aggregate function.
// for backward compatibility, the state type is same as the return type.
// its values in the intermediate state table are always null.
- } else {
- field.data_type = sig
- .state_type
- .clone()
- .unwrap_or(sig.ret_type.as_exact().clone());
+ } else if let Some(state_type) = &sig.state_type {
+ field.data_type = state_type.clone();
}
}
let in_dist_key = self.input.distribution().dist_column_indices().to_vec();
diff --git a/src/frontend/src/optimizer/rule/agg_group_by_simplify_rule.rs b/src/frontend/src/optimizer/rule/agg_group_by_simplify_rule.rs
index 34025eca43032..3e22348e27b49 100644
--- a/src/frontend/src/optimizer/rule/agg_group_by_simplify_rule.rs
+++ b/src/frontend/src/optimizer/rule/agg_group_by_simplify_rule.rs
@@ -12,7 +12,6 @@
// See the License for the specific language governing permissions and
// limitations under the License.
-use risingwave_common::util::sort_util::{ColumnOrder, OrderType};
use risingwave_expr::aggregate::AggKind;
use super::super::plan_node::*;
@@ -48,11 +47,11 @@ impl Rule for AggGroupBySimplifyRule {
if !new_group_key.contains(i) {
let data_type = agg_input.schema().fields[i].data_type();
new_agg_calls.push(PlanAggCall {
- agg_kind: AggKind::FirstValue,
+ agg_kind: AggKind::InternalLastSeenValue,
return_type: data_type.clone(),
inputs: vec![InputRef::new(i, data_type)],
distinct: false,
- order_by: vec![ColumnOrder::new(i, OrderType::ascending())],
+ order_by: vec![],
filter: Condition::true_cond(),
direct_args: vec![],
});
From 447e1422a2a0989d873ddd21f660460d0f0e26d2 Mon Sep 17 00:00:00 2001
From: Croxx
Date: Tue, 24 Oct 2023 18:23:13 +0800
Subject: [PATCH 10/11] chore: bump foyer (#13029)
Signed-off-by: MrCroxx
---
Cargo.lock | 15 ++++++++-------
src/storage/Cargo.toml | 2 +-
src/storage/src/hummock/file_cache/store.rs | 1 +
3 files changed, 10 insertions(+), 8 deletions(-)
diff --git a/Cargo.lock b/Cargo.lock
index bbc112fa4bb5b..ce71e2bf98950 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -3015,7 +3015,7 @@ dependencies = [
[[package]]
name = "foyer"
version = "0.1.0"
-source = "git+https://github.com/mrcroxx/foyer?rev=438eec8#438eec87e90c7a80cb53a06b711c6ea1ad7a0f41"
+source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783"
dependencies = [
"foyer-common",
"foyer-intrusive",
@@ -3026,10 +3026,11 @@ dependencies = [
[[package]]
name = "foyer-common"
version = "0.1.0"
-source = "git+https://github.com/mrcroxx/foyer?rev=438eec8#438eec87e90c7a80cb53a06b711c6ea1ad7a0f41"
+source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783"
dependencies = [
"bytes",
"foyer-workspace-hack",
+ "itertools 0.11.0",
"madsim-tokio",
"parking_lot 0.12.1",
"paste",
@@ -3040,13 +3041,13 @@ dependencies = [
[[package]]
name = "foyer-intrusive"
version = "0.1.0"
-source = "git+https://github.com/mrcroxx/foyer?rev=438eec8#438eec87e90c7a80cb53a06b711c6ea1ad7a0f41"
+source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783"
dependencies = [
"bytes",
"cmsketch",
"foyer-common",
"foyer-workspace-hack",
- "itertools 0.10.5",
+ "itertools 0.11.0",
"memoffset",
"parking_lot 0.12.1",
"paste",
@@ -3057,7 +3058,7 @@ dependencies = [
[[package]]
name = "foyer-storage"
version = "0.1.0"
-source = "git+https://github.com/mrcroxx/foyer?rev=438eec8#438eec87e90c7a80cb53a06b711c6ea1ad7a0f41"
+source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783"
dependencies = [
"anyhow",
"async-channel",
@@ -3086,7 +3087,7 @@ dependencies = [
[[package]]
name = "foyer-workspace-hack"
version = "0.1.0"
-source = "git+https://github.com/mrcroxx/foyer?rev=438eec8#438eec87e90c7a80cb53a06b711c6ea1ad7a0f41"
+source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783"
dependencies = [
"crossbeam-utils",
"either",
@@ -3095,7 +3096,7 @@ dependencies = [
"futures-sink",
"futures-util",
"hyper",
- "itertools 0.10.5",
+ "itertools 0.11.0",
"libc",
"memchr",
"parking_lot 0.12.1",
diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml
index f1022ab2fd935..c6fc5531acd33 100644
--- a/src/storage/Cargo.toml
+++ b/src/storage/Cargo.toml
@@ -25,7 +25,7 @@ dyn-clone = "1.0.14"
either = "1"
enum-as-inner = "0.6"
fail = "0.5"
-foyer = { git = "https://github.com/mrcroxx/foyer", rev = "438eec8" }
+foyer = { git = "https://github.com/mrcroxx/foyer", rev = "5d0134b" }
futures = { version = "0.3", default-features = false, features = ["alloc"] }
futures-async-stream = { workspace = true }
hex = "0.4"
diff --git a/src/storage/src/hummock/file_cache/store.rs b/src/storage/src/hummock/file_cache/store.rs
index 9de54552ae077..222700c8376b2 100644
--- a/src/storage/src/hummock/file_cache/store.rs
+++ b/src/storage/src/hummock/file_cache/store.rs
@@ -256,6 +256,7 @@ where
io_size: config.device_io_size,
},
allocator_bits: config.allocator_bits,
+ catalog_bits: 6,
admissions,
reinsertions: config.reinsertions,
buffer_pool_size: config.buffer_pool_size,
From e81850870a8026fd41184344c01f71bc67a7cb5d Mon Sep 17 00:00:00 2001
From: Dylan
Date: Tue, 24 Oct 2023 20:33:32 +0800
Subject: [PATCH 11/11] feat(optimizer): change stream join mv distribution key
(#13022)
---
.../tests/testdata/output/append_only.yaml | 11 +-
.../tests/testdata/output/basic_query.yaml | 13 +-
.../tests/testdata/output/ch_benchmark.yaml | 302 ++++++-----
.../output/common_table_expressions.yaml | 22 +-
.../testdata/output/distribution_derive.yaml | 110 ++--
.../tests/testdata/output/dynamic_filter.yaml | 21 +-
.../tests/testdata/output/join.yaml | 133 ++---
.../tests/testdata/output/join_ordering.yaml | 108 ++--
.../testdata/output/lateral_subquery.yaml | 84 +--
.../tests/testdata/output/mv_on_mv.yaml | 11 +-
.../tests/testdata/output/nexmark.yaml | 282 +++++-----
.../tests/testdata/output/nexmark_source.yaml | 246 +++++----
.../output/nexmark_temporal_filter.yaml | 395 +++++++-------
.../testdata/output/nexmark_watermark.yaml | 510 ++++++++++--------
.../testdata/output/over_window_function.yaml | 32 +-
.../tests/testdata/output/pk_derive.yaml | 38 +-
.../testdata/output/predicate_pushdown.yaml | 36 +-
.../tests/testdata/output/project_set.yaml | 23 +-
.../tests/testdata/output/select_except.yaml | 22 +-
.../tests/testdata/output/share.yaml | 43 +-
.../tests/testdata/output/shared_views.yaml | 33 +-
.../tests/testdata/output/subquery.yaml | 179 +++---
.../output/subquery_expr_correlated.yaml | 74 +--
.../testdata/output/temporal_filter.yaml | 81 +--
.../tests/testdata/output/temporal_join.yaml | 171 +++---
.../tests/testdata/output/tpch_variant.yaml | 440 +++++++--------
.../tests/testdata/output/watermark.yaml | 52 +-
.../tests/testdata/output/window_join.yaml | 38 +-
.../optimizer/plan_node/stream_materialize.rs | 17 +-
.../src/optimizer/property/distribution.rs | 2 +-
.../scale/cascade_materialized_view.rs | 4 +-
.../tests/integration_tests/scale/plan.rs | 8 +-
32 files changed, 1882 insertions(+), 1659 deletions(-)
diff --git a/src/frontend/planner_test/tests/testdata/output/append_only.yaml b/src/frontend/planner_test/tests/testdata/output/append_only.yaml
index 184abd564c32b..d693d3fc942df 100644
--- a/src/frontend/planner_test/tests/testdata/output/append_only.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/append_only.yaml
@@ -14,11 +14,12 @@
select t1.v1 as id, v2, v3 from t1 join t2 on t1.v1=t2.v1;
stream_plan: |-
StreamMaterialize { columns: [id, v2, v3, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, id], pk_columns: [t1._row_id, t2._row_id, id], pk_conflict: NoCheck }
- └─StreamHashJoin [append_only] { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v3, t1._row_id, t2._row_id] }
- ├─StreamExchange { dist: HashShard(t1.v1) }
- │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard(t2.v1) }
- └─StreamTableScan { table: t2, columns: [t2.v1, t2.v3, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) }
+ └─StreamHashJoin [append_only] { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v3, t1._row_id, t2._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(t2.v1) }
+ └─StreamTableScan { table: t2, columns: [t2.v1, t2.v3, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- sql: |
create table t1 (v1 int, v2 int) append only;
select v1 from t1 order by v1 limit 3 offset 3;
diff --git a/src/frontend/planner_test/tests/testdata/output/basic_query.yaml b/src/frontend/planner_test/tests/testdata/output/basic_query.yaml
index fde09972bb66b..ce6724dc91c37 100644
--- a/src/frontend/planner_test/tests/testdata/output/basic_query.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/basic_query.yaml
@@ -234,9 +234,10 @@
└─BatchValues { rows: [] }
stream_plan: |-
StreamMaterialize { columns: [v, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, v], pk_columns: [t._row_id, t._row_id#1, v], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t.v = t.v, output: [t.v, t._row_id, t._row_id] }
- ├─StreamExchange { dist: HashShard(t.v) }
- │ └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- └─StreamExchange { dist: HashShard(t.v) }
- └─StreamFilter { predicate: false:Boolean }
- └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.v, t._row_id, t._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t.v = t.v, output: [t.v, t._row_id, t._row_id] }
+ ├─StreamExchange { dist: HashShard(t.v) }
+ │ └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.v) }
+ └─StreamFilter { predicate: false:Boolean }
+ └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
diff --git a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml
index 6f4f8a673c996..e7196f7cf4fea 100644
--- a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml
@@ -134,141 +134,145 @@
└─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) }
stream_plan: |-
StreamMaterialize { columns: [s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id(hidden), stock.s_w_id(hidden), min(stock.s_quantity)(hidden), $expr2(hidden), region.r_regionkey(hidden), supplier.s_nationkey(hidden)], stream_key: [stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_columns: [n_name, s_name, i_id, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] }
- ├─StreamExchange { dist: HashShard($expr2) }
- │ └─StreamProject { exprs: [item.i_id, item.i_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_i_id, stock.s_w_id, min(stock.s_quantity)] }
- │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND min(stock.s_quantity) = stock.s_quantity AND stock.s_i_id = stock.s_i_id, output: [item.i_id, item.i_name, stock.s_i_id, stock.s_w_id, stock.s_i_id, min(stock.s_quantity)] }
- │ ├─StreamProject { exprs: [stock.s_i_id, min(stock.s_quantity)] }
- │ │ └─StreamHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity), count] }
- │ │ └─StreamExchange { dist: HashShard(stock.s_i_id) }
- │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] }
- │ │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) }
- │ │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] }
- │ │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) }
- │ │ │ │ └─StreamProject { exprs: [region.r_regionkey] }
- │ │ │ │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) }
- │ │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) }
- │ │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) }
- │ │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] }
- │ │ │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) }
- │ │ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) }
- │ │ │ └─StreamExchange { dist: HashShard(nation.n_nationkey) }
- │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) }
- │ │ └─StreamExchange { dist: HashShard($expr1) }
- │ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] }
- │ │ └─StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) }
- │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) }
- │ └─StreamHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all }
- │ ├─StreamExchange { dist: HashShard(item.i_id) }
- │ │ └─StreamProject { exprs: [item.i_id, item.i_name] }
- │ │ └─StreamFilter { predicate: Like(item.i_data, '%b':Varchar) }
- │ │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) }
- │ └─StreamExchange { dist: HashShard(stock.s_i_id) }
- │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) }
- └─StreamExchange { dist: HashShard(supplier.s_suppkey) }
- └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] }
- ├─StreamExchange { dist: HashShard(region.r_regionkey) }
- │ └─StreamProject { exprs: [region.r_regionkey] }
- │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) }
- │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) }
- └─StreamExchange { dist: HashShard(nation.n_regionkey) }
- └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] }
- ├─StreamExchange { dist: HashShard(supplier.s_nationkey) }
- │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) }
- └─StreamExchange { dist: HashShard(nation.n_nationkey) }
- └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) }
+ └─StreamExchange { dist: HashShard(stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey) }
+ └─StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] }
+ ├─StreamExchange { dist: HashShard($expr2) }
+ │ └─StreamProject { exprs: [item.i_id, item.i_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_i_id, stock.s_w_id, min(stock.s_quantity)] }
+ │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND min(stock.s_quantity) = stock.s_quantity AND stock.s_i_id = stock.s_i_id, output: [item.i_id, item.i_name, stock.s_i_id, stock.s_w_id, stock.s_i_id, min(stock.s_quantity)] }
+ │ ├─StreamProject { exprs: [stock.s_i_id, min(stock.s_quantity)] }
+ │ │ └─StreamHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity), count] }
+ │ │ └─StreamExchange { dist: HashShard(stock.s_i_id) }
+ │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] }
+ │ │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) }
+ │ │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] }
+ │ │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) }
+ │ │ │ │ └─StreamProject { exprs: [region.r_regionkey] }
+ │ │ │ │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) }
+ │ │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) }
+ │ │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) }
+ │ │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] }
+ │ │ │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) }
+ │ │ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) }
+ │ │ │ └─StreamExchange { dist: HashShard(nation.n_nationkey) }
+ │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) }
+ │ │ └─StreamExchange { dist: HashShard($expr1) }
+ │ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] }
+ │ │ └─StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) }
+ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) }
+ │ └─StreamHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all }
+ │ ├─StreamExchange { dist: HashShard(item.i_id) }
+ │ │ └─StreamProject { exprs: [item.i_id, item.i_name] }
+ │ │ └─StreamFilter { predicate: Like(item.i_data, '%b':Varchar) }
+ │ │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) }
+ │ └─StreamExchange { dist: HashShard(stock.s_i_id) }
+ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) }
+ └─StreamExchange { dist: HashShard(supplier.s_suppkey) }
+ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] }
+ ├─StreamExchange { dist: HashShard(region.r_regionkey) }
+ │ └─StreamProject { exprs: [region.r_regionkey] }
+ │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) }
+ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) }
+ └─StreamExchange { dist: HashShard(nation.n_regionkey) }
+ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] }
+ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) }
+ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) }
+ └─StreamExchange { dist: HashShard(nation.n_nationkey) }
+ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id(hidden), stock.s_w_id(hidden), min(stock.s_quantity)(hidden), $expr2(hidden), region.r_regionkey(hidden), supplier.s_nationkey(hidden)], stream_key: [stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_columns: [n_name, s_name, i_id, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([2]) from 1
- └── StreamExchange Hash([0]) from 11
+ └── StreamExchange Hash([8, 9, 10, 11, 12, 13]) from 1
Fragment 1
+ StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([2]) from 2
+ └── StreamExchange Hash([0]) from 12
+
+ Fragment 2
StreamProject { exprs: [item.i_id, item.i_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_i_id, stock.s_w_id, min(stock.s_quantity)] }
└── StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND min(stock.s_quantity) = stock.s_quantity AND stock.s_i_id = stock.s_i_id, output: [item.i_id, item.i_name, stock.s_i_id, stock.s_w_id, stock.s_i_id, min(stock.s_quantity)] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 }
├── StreamProject { exprs: [stock.s_i_id, min(stock.s_quantity)] }
│ └── StreamHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity), count] } { intermediate state table: 9, state tables: [ 8 ], distinct tables: [] }
- │ └── StreamExchange Hash([0]) from 2
+ │ └── StreamExchange Hash([0]) from 3
└── StreamHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all } { left table: 26, right table: 28, left degree table: 27, right degree table: 29 }
- ├── StreamExchange Hash([0]) from 9
- └── StreamExchange Hash([0]) from 10
-
- Fragment 2
- StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } { left table: 10, right table: 12, left degree table: 11, right degree table: 13 }
- ├── StreamExchange Hash([0]) from 3
- └── StreamExchange Hash([2]) from 8
+ ├── StreamExchange Hash([0]) from 10
+ └── StreamExchange Hash([0]) from 11
Fragment 3
- StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 }
+ StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } { left table: 10, right table: 12, left degree table: 11, right degree table: 13 }
├── StreamExchange Hash([0]) from 4
- └── StreamExchange Hash([1]) from 5
+ └── StreamExchange Hash([2]) from 9
Fragment 4
+ StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 }
+ ├── StreamExchange Hash([0]) from 5
+ └── StreamExchange Hash([1]) from 6
+
+ Fragment 5
StreamProject { exprs: [region.r_regionkey] }
└── StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) }
└── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 18 }
├── Upstream
└── BatchPlanNode
- Fragment 5
+ Fragment 6
StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } { left table: 19, right table: 21, left degree table: 20, right degree table: 22 }
- ├── StreamExchange Hash([1]) from 6
- └── StreamExchange Hash([0]) from 7
+ ├── StreamExchange Hash([1]) from 7
+ └── StreamExchange Hash([0]) from 8
- Fragment 6
+ Fragment 7
Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 23 }
├── Upstream
└── BatchPlanNode
- Fragment 7
+ Fragment 8
Chain { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 24 }
├── Upstream
└── BatchPlanNode
- Fragment 8
+ Fragment 9
StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] }
└── StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) }
└── Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 25 }
├── Upstream
└── BatchPlanNode
- Fragment 9
+ Fragment 10
StreamProject { exprs: [item.i_id, item.i_name] }
└── StreamFilter { predicate: Like(item.i_data, '%b':Varchar) }
└── Chain { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 30 }
├── Upstream
└── BatchPlanNode
- Fragment 10
+ Fragment 11
Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 31 }
├── Upstream
└── BatchPlanNode
- Fragment 11
+ Fragment 12
StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] } { left table: 32, right table: 34, left degree table: 33, right degree table: 35 }
- ├── StreamExchange Hash([0]) from 12
- └── StreamExchange Hash([6]) from 13
+ ├── StreamExchange Hash([0]) from 13
+ └── StreamExchange Hash([6]) from 14
- Fragment 12
+ Fragment 13
StreamProject { exprs: [region.r_regionkey] }
└── StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) }
└── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 36 }
├── Upstream
└── BatchPlanNode
- Fragment 13
+ Fragment 14
StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } { left table: 37, right table: 39, left degree table: 38, right degree table: 40 }
- ├── StreamExchange Hash([3]) from 14
- └── StreamExchange Hash([0]) from 15
+ ├── StreamExchange Hash([3]) from 15
+ └── StreamExchange Hash([0]) from 16
- Fragment 14
+ Fragment 15
Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 41 }
├── Upstream
└── BatchPlanNode
- Fragment 15
+ Fragment 16
Chain { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 42 }
├── Upstream
└── BatchPlanNode
@@ -359,7 +363,7 @@
Table 42 { columns: [ vnode, n_nationkey, nation_backfill_finished, nation_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 }
- Table 4294967294 { columns: [ s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 11 ], read pk prefix len hint: 9 }
+ Table 4294967294 { columns: [ s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 8, 9, 10, 11, 12, 13 ], read pk prefix len hint: 9 }
- id: ch_q3
before:
@@ -2496,59 +2500,63 @@
└─LogicalScan { table: revenue1, columns: [revenue1.total_revenue] }
stream_plan: |-
StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no(hidden)], stream_key: [s_suppkey, revenue1.supplier_no, total_revenue], pk_columns: [s_suppkey, revenue1.supplier_no, total_revenue], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] }
- ├─StreamExchange { dist: HashShard(revenue1.total_revenue) }
- │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] }
- │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) }
- │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) }
- │ └─StreamExchange { dist: HashShard($expr1) }
- │ └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no::Int64 as $expr1, revenue1.supplier_no] }
- │ └─StreamTableScan { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) }
- └─StreamExchange { dist: HashShard(max(max(revenue1.total_revenue))) }
- └─StreamProject { exprs: [max(max(revenue1.total_revenue))] }
- └─StreamSimpleAgg { aggs: [max(max(revenue1.total_revenue)), count] }
- └─StreamExchange { dist: Single }
- └─StreamHashAgg { group_key: [$expr2], aggs: [max(revenue1.total_revenue), count] }
- └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no, Vnode(revenue1.supplier_no) as $expr2] }
- └─StreamTableScan { table: revenue1, columns: [revenue1.total_revenue, revenue1.supplier_no], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) }
+ └─StreamExchange { dist: HashShard(supplier.s_suppkey, revenue1.total_revenue, revenue1.supplier_no) }
+ └─StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] }
+ ├─StreamExchange { dist: HashShard(revenue1.total_revenue) }
+ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] }
+ │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) }
+ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) }
+ │ └─StreamExchange { dist: HashShard($expr1) }
+ │ └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no::Int64 as $expr1, revenue1.supplier_no] }
+ │ └─StreamTableScan { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) }
+ └─StreamExchange { dist: HashShard(max(max(revenue1.total_revenue))) }
+ └─StreamProject { exprs: [max(max(revenue1.total_revenue))] }
+ └─StreamSimpleAgg { aggs: [max(max(revenue1.total_revenue)), count] }
+ └─StreamExchange { dist: Single }
+ └─StreamHashAgg { group_key: [$expr2], aggs: [max(revenue1.total_revenue), count] }
+ └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no, Vnode(revenue1.supplier_no) as $expr2] }
+ └─StreamTableScan { table: revenue1, columns: [revenue1.total_revenue, revenue1.supplier_no], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no(hidden)], stream_key: [s_suppkey, revenue1.supplier_no, total_revenue], pk_columns: [s_suppkey, revenue1.supplier_no, total_revenue], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([4]) from 1
- └── StreamExchange Hash([0]) from 4
+ └── StreamExchange Hash([0, 4, 5]) from 1
Fragment 1
+ StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamExchange Hash([4]) from 2
+ └── StreamExchange Hash([0]) from 5
+
+ Fragment 2
StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] }
├── left table: 4
├── right table: 6
├── left degree table: 5
├── right degree table: 7
- ├── StreamExchange Hash([0]) from 2
- └── StreamExchange Hash([1]) from 3
+ ├── StreamExchange Hash([0]) from 3
+ └── StreamExchange Hash([1]) from 4
- Fragment 2
+ Fragment 3
Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 8 }
├── Upstream
└── BatchPlanNode
- Fragment 3
+ Fragment 4
StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no::Int64 as $expr1, revenue1.supplier_no] }
└── Chain { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } { state table: 9 }
├── Upstream
└── BatchPlanNode
- Fragment 4
+ Fragment 5
StreamProject { exprs: [max(max(revenue1.total_revenue))] }
└── StreamSimpleAgg { aggs: [max(max(revenue1.total_revenue)), count] } { intermediate state table: 11, state tables: [ 10 ], distinct tables: [] }
- └── StreamExchange Single from 5
+ └── StreamExchange Single from 6
- Fragment 5
+ Fragment 6
StreamHashAgg { group_key: [$expr2], aggs: [max(revenue1.total_revenue), count] } { intermediate state table: 13, state tables: [ 12 ], distinct tables: [] }
└── StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no, Vnode(revenue1.supplier_no) as $expr2] }
└── Chain { table: revenue1, columns: [revenue1.total_revenue, revenue1.supplier_no], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } { state table: 14 }
@@ -2590,7 +2598,7 @@
Table 14 { columns: [ vnode, supplier_no, revenue1_backfill_finished, revenue1_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 }
- Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no ], primary key: [ $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 4 ], read pk prefix len hint: 3 }
+ Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no ], primary key: [ $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 4, 5 ], read pk prefix len hint: 3 }
- id: ch_q16
before:
@@ -3174,58 +3182,62 @@
└─BatchScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_delivery_d], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [supplier.s_suppkey, supplier.s_nationkey], pk_columns: [s_name, supplier.s_suppkey, supplier.s_nationkey], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] }
- ├─StreamExchange { dist: HashShard(supplier.s_suppkey) }
- │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all }
- │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) }
- │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) }
- │ └─StreamExchange { dist: HashShard(nation.n_nationkey) }
- │ └─StreamProject { exprs: [nation.n_nationkey] }
- │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) }
- │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) }
- └─StreamExchange { dist: HashShard($expr1) }
- └─StreamProject { exprs: [((stock.s_i_id * stock.s_w_id) % 10000:Int32)::Int64 as $expr1, stock.s_i_id, stock.s_w_id, stock.s_quantity] }
- └─StreamFilter { predicate: ((2:Int32 * stock.s_quantity) > sum(order_line.ol_quantity)) }
- └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, stock.s_quantity, sum(order_line.ol_quantity)] }
- └─StreamHashAgg { group_key: [stock.s_i_id, stock.s_w_id, stock.s_quantity], aggs: [sum(order_line.ol_quantity), count] }
- └─StreamHashJoin { type: LeftSemi, predicate: stock.s_i_id = item.i_id, output: all }
- ├─StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, stock.s_quantity, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] }
- │ ├─StreamExchange { dist: HashShard(stock.s_i_id) }
- │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) }
- │ └─StreamExchange { dist: HashShard(order_line.ol_i_id) }
- │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] }
- │ └─StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) }
- │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) }
- └─StreamExchange { dist: HashShard(item.i_id) }
- └─StreamProject { exprs: [item.i_id] }
- └─StreamFilter { predicate: Like(item.i_data, 'co%':Varchar) }
- └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) }
+ └─StreamExchange { dist: HashShard(supplier.s_suppkey, supplier.s_nationkey) }
+ └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] }
+ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) }
+ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all }
+ │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) }
+ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) }
+ │ └─StreamExchange { dist: HashShard(nation.n_nationkey) }
+ │ └─StreamProject { exprs: [nation.n_nationkey] }
+ │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) }
+ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) }
+ └─StreamExchange { dist: HashShard($expr1) }
+ └─StreamProject { exprs: [((stock.s_i_id * stock.s_w_id) % 10000:Int32)::Int64 as $expr1, stock.s_i_id, stock.s_w_id, stock.s_quantity] }
+ └─StreamFilter { predicate: ((2:Int32 * stock.s_quantity) > sum(order_line.ol_quantity)) }
+ └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, stock.s_quantity, sum(order_line.ol_quantity)] }
+ └─StreamHashAgg { group_key: [stock.s_i_id, stock.s_w_id, stock.s_quantity], aggs: [sum(order_line.ol_quantity), count] }
+ └─StreamHashJoin { type: LeftSemi, predicate: stock.s_i_id = item.i_id, output: all }
+ ├─StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, stock.s_quantity, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] }
+ │ ├─StreamExchange { dist: HashShard(stock.s_i_id) }
+ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) }
+ │ └─StreamExchange { dist: HashShard(order_line.ol_i_id) }
+ │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] }
+ │ └─StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) }
+ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) }
+ └─StreamExchange { dist: HashShard(item.i_id) }
+ └─StreamProject { exprs: [item.i_id] }
+ └─StreamFilter { predicate: Like(item.i_data, 'co%':Varchar) }
+ └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [supplier.s_suppkey, supplier.s_nationkey], pk_columns: [s_name, supplier.s_suppkey, supplier.s_nationkey], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamExchange Hash([0]) from 4
+ └── StreamExchange Hash([2, 3]) from 1
Fragment 1
- StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 }
- ├── StreamExchange Hash([3]) from 2
- └── StreamExchange Hash([0]) from 3
+ StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0]) from 5
Fragment 2
+ StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 }
+ ├── StreamExchange Hash([3]) from 3
+ └── StreamExchange Hash([0]) from 4
+
+ Fragment 3
Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 8 }
├── Upstream
└── BatchPlanNode
- Fragment 3
+ Fragment 4
StreamProject { exprs: [nation.n_nationkey] }
└── StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) }
└── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 9 }
├── Upstream
└── BatchPlanNode
- Fragment 4
+ Fragment 5
StreamProject { exprs: [((stock.s_i_id * stock.s_w_id) % 10000:Int32)::Int64 as $expr1, stock.s_i_id, stock.s_w_id, stock.s_quantity] }
└── StreamFilter { predicate: ((2:Int32 * stock.s_quantity) > sum(order_line.ol_quantity)) }
└── StreamProject { exprs: [stock.s_i_id, stock.s_w_id, stock.s_quantity, sum(order_line.ol_quantity)] }
@@ -3236,16 +3248,16 @@
│ ├── right table: 17
│ ├── left degree table: 16
│ ├── right degree table: 18
- │ ├── StreamExchange Hash([0]) from 5
- │ └── StreamExchange Hash([0]) from 6
- └── StreamExchange Hash([0]) from 7
+ │ ├── StreamExchange Hash([0]) from 6
+ │ └── StreamExchange Hash([0]) from 7
+ └── StreamExchange Hash([0]) from 8
- Fragment 5
+ Fragment 6
Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 19 }
├── Upstream
└── BatchPlanNode
- Fragment 6
+ Fragment 7
StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] }
└── StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) }
└── Chain { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) }
@@ -3253,7 +3265,7 @@
├── Upstream
└── BatchPlanNode
- Fragment 7
+ Fragment 8
StreamProject { exprs: [item.i_id] }
└── StreamFilter { predicate: Like(item.i_data, 'co%':Varchar) }
└── Chain { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 21 }
@@ -3304,7 +3316,7 @@
Table 21 { columns: [ vnode, i_id, item_backfill_finished, item_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 }
- Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 3 }
+ Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 }
- id: ch_q21
before:
diff --git a/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml b/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml
index 9c961429276a3..dece27002b19b 100644
--- a/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml
@@ -23,11 +23,12 @@
└─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] }
stream_plan: |-
StreamMaterialize { columns: [v3, v4, v1, t2._row_id(hidden), t1._row_id(hidden)], stream_key: [t2._row_id, t1._row_id, v3], pk_columns: [t2._row_id, t1._row_id, v3], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t2._row_id, t1._row_id] }
- ├─StreamExchange { dist: HashShard(t2.v3) }
- │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- └─StreamExchange { dist: HashShard(t1.v1) }
- └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(t2.v3, t2._row_id, t1._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t2._row_id, t1._row_id] }
+ ├─StreamExchange { dist: HashShard(t2.v3) }
+ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1) }
+ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- sql: |
create table t1 (v1 int, v2 int);
create table t2 (v3 int, v4 int);
@@ -79,8 +80,9 @@
└─LogicalValues { rows: [['cn':Varchar, 'China':Varchar], ['us':Varchar, 'United States':Varchar]], schema: Schema { fields: [*VALUES*_0.column_0:Varchar, *VALUES*_0.column_1:Varchar] } }
stream_plan: |-
StreamMaterialize { columns: [v, c, abbr, real, t._row_id(hidden), _row_id(hidden)], stream_key: [t._row_id, _row_id, c], pk_columns: [t._row_id, _row_id, c], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t.c = *VALUES*_0.column_0, output: [t.v, t.c, *VALUES*_0.column_0, *VALUES*_0.column_1, t._row_id, _row_id] }
- ├─StreamExchange { dist: HashShard(t.c) }
- │ └─StreamTableScan { table: t, columns: [t.v, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- └─StreamExchange { dist: HashShard(*VALUES*_0.column_0) }
- └─StreamValues { rows: [['cn':Varchar, 'China':Varchar, 0:Int64], ['us':Varchar, 'United States':Varchar, 1:Int64]] }
+ └─StreamExchange { dist: HashShard(t.c, t._row_id, _row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t.c = *VALUES*_0.column_0, output: [t.v, t.c, *VALUES*_0.column_0, *VALUES*_0.column_1, t._row_id, _row_id] }
+ ├─StreamExchange { dist: HashShard(t.c) }
+ │ └─StreamTableScan { table: t, columns: [t.v, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(*VALUES*_0.column_0) }
+ └─StreamValues { rows: [['cn':Varchar, 'China':Varchar, 0:Int64], ['us':Varchar, 'United States':Varchar, 1:Int64]] }
diff --git a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml
index 85d76188f3e76..818fd88b30a20 100644
--- a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml
@@ -963,31 +963,35 @@
└─BatchScan { table: a, columns: [a.k1], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], stream_key: [ak1.a._row_id, ak1.k1], pk_columns: [ak1.a._row_id, ak1.k1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] }
- ├─StreamExchange { dist: HashShard(ak1.k1) }
- │ └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) }
- └─StreamHashAgg { group_key: [a.k1], aggs: [count] }
- └─StreamExchange { dist: HashShard(a.k1) }
- └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) }
+ └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1) }
+ └─StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] }
+ ├─StreamExchange { dist: HashShard(ak1.k1) }
+ │ └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) }
+ └─StreamHashAgg { group_key: [a.k1], aggs: [count] }
+ └─StreamExchange { dist: HashShard(a.k1) }
+ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], stream_key: [ak1.a._row_id, ak1.k1], pk_columns: [ak1.a._row_id, ak1.k1], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([0]) from 1
- └── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([2, 3]) from 1
Fragment 1
+ StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 4 }
├── Upstream
└── BatchPlanNode
- Fragment 2
+ Fragment 3
Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 6 }
├── Upstream
└── BatchPlanNode
@@ -1022,7 +1026,7 @@
├── columns: [ v, bv, ak1.a._row_id, ak1.k1, a.k1 ]
├── primary key: [ $2 ASC, $3 ASC ]
├── value indices: [ 0, 1, 2, 3, 4 ]
- ├── distribution key: [ 3 ]
+ ├── distribution key: [ 2, 3 ]
└── read pk prefix len hint: 2
- id: aggk1_join_Ak1_onk1
@@ -1054,31 +1058,35 @@
└─BatchScan { table: a, columns: [a.k1], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden)], stream_key: [a.k1, ak1.a._row_id], pk_columns: [a.k1, ak1.a._row_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] }
- ├─StreamHashAgg { group_key: [a.k1], aggs: [count] }
- │ └─StreamExchange { dist: HashShard(a.k1) }
- │ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) }
- └─StreamExchange { dist: HashShard(ak1.k1) }
- └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) }
+ └─StreamExchange { dist: HashShard(a.k1, ak1.a._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] }
+ ├─StreamHashAgg { group_key: [a.k1], aggs: [count] }
+ │ └─StreamExchange { dist: HashShard(a.k1) }
+ │ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) }
+ └─StreamExchange { dist: HashShard(ak1.k1) }
+ └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden)], stream_key: [a.k1, ak1.a._row_id], pk_columns: [a.k1, ak1.a._row_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 4, state tables: [], distinct tables: [] }
- │ └── StreamExchange Hash([0]) from 1
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([2, 3]) from 1
Fragment 1
+ StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 4, state tables: [], distinct tables: [] }
+ │ └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 5 }
├── Upstream
└── BatchPlanNode
- Fragment 2
+ Fragment 3
Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 6 }
├── Upstream
└── BatchPlanNode
@@ -1113,7 +1121,7 @@
├── columns: [ v, bv, a.k1, ak1.a._row_id ]
├── primary key: [ $2 ASC, $3 ASC ]
├── value indices: [ 0, 1, 2, 3 ]
- ├── distribution key: [ 2 ]
+ ├── distribution key: [ 2, 3 ]
└── read pk prefix len hint: 2
- id: aggk1_join_aggk1_onk1
@@ -1156,33 +1164,37 @@
└─BatchScan { table: b, columns: [b.k1], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], stream_key: [a.k1], pk_columns: [a.k1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] }
- ├─StreamHashAgg { group_key: [a.k1], aggs: [count] }
- │ └─StreamExchange { dist: HashShard(a.k1) }
- │ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) }
- └─StreamHashAgg { group_key: [b.k1], aggs: [count] }
- └─StreamExchange { dist: HashShard(b.k1) }
- └─StreamTableScan { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) }
+ └─StreamExchange { dist: HashShard(a.k1) }
+ └─StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] }
+ ├─StreamHashAgg { group_key: [a.k1], aggs: [count] }
+ │ └─StreamExchange { dist: HashShard(a.k1) }
+ │ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) }
+ └─StreamHashAgg { group_key: [b.k1], aggs: [count] }
+ └─StreamExchange { dist: HashShard(b.k1) }
+ └─StreamTableScan { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], stream_key: [a.k1], pk_columns: [a.k1], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 4, state tables: [], distinct tables: [] }
- │ └── StreamExchange Hash([0]) from 1
- └── StreamHashAgg { group_key: [b.k1], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([2]) from 1
Fragment 1
+ StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 4, state tables: [], distinct tables: [] }
+ │ └── StreamExchange Hash([0]) from 2
+ └── StreamHashAgg { group_key: [b.k1], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] }
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 5 }
├── Upstream
└── BatchPlanNode
- Fragment 2
+ Fragment 3
Chain { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } { state table: 7 }
├── Upstream
└── BatchPlanNode
diff --git a/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml b/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml
index a340014298c47..ab282ebe3858a 100644
--- a/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml
@@ -124,16 +124,17 @@
└─LogicalScan { table: t2, columns: [t2.v2] }
stream_plan: |-
StreamMaterialize { columns: [v1, max, t1._row_id(hidden)], stream_key: [t1._row_id, v1], pk_columns: [t1._row_id, v1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t1.v1 = max(max(t2.v2)), output: [t1.v1, max(max(t2.v2)), t1._row_id] }
- ├─StreamExchange { dist: HashShard(t1.v1) }
- │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard(max(max(t2.v2))) }
- └─StreamProject { exprs: [max(max(t2.v2))] }
- └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] }
- └─StreamExchange { dist: Single }
- └─StreamHashAgg { group_key: [$expr1], aggs: [max(t2.v2), count] }
- └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] }
- └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t1._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t1.v1 = max(max(t2.v2)), output: [t1.v1, max(max(t2.v2)), t1._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(max(max(t2.v2))) }
+ └─StreamProject { exprs: [max(max(t2.v2))] }
+ └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] }
+ └─StreamExchange { dist: Single }
+ └─StreamHashAgg { group_key: [$expr1], aggs: [max(t2.v2), count] }
+ └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] }
+ └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- name: Dynamic filter join on unequal types
sql: |
create table t1 (v1 int);
diff --git a/src/frontend/planner_test/tests/testdata/output/join.yaml b/src/frontend/planner_test/tests/testdata/output/join.yaml
index 4ef01cc84ab47..a61d2a0d73327 100644
--- a/src/frontend/planner_test/tests/testdata/output/join.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/join.yaml
@@ -14,14 +14,15 @@
└─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id] }
stream_plan: |-
StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1, t3._row_id], pk_columns: [t1._row_id, t2._row_id, v1, t3._row_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t1.v1 = t3.v5, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] }
- ├─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] }
- │ ├─StreamExchange { dist: HashShard(t1.v1) }
- │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- │ └─StreamExchange { dist: HashShard(t2.v3) }
- │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- └─StreamExchange { dist: HashShard(t3.v5) }
- └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id, t3._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t3.v5, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] }
+ ├─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] }
+ │ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ │ └─StreamExchange { dist: HashShard(t2.v3) }
+ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t3.v5) }
+ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) }
- name: self join
sql: |
create table t (v1 int, v2 int);
@@ -33,11 +34,12 @@
└─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] }
stream_plan: |-
StreamMaterialize { columns: [t1v1, t2v1, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, t1v1], pk_columns: [t._row_id, t._row_id#1, t1v1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v1, t._row_id, t._row_id] }
- ├─StreamExchange { dist: HashShard(t.v1) }
- │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- └─StreamExchange { dist: HashShard(t.v1) }
- └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.v1, t._row_id, t._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v1, t._row_id, t._row_id] }
+ ├─StreamExchange { dist: HashShard(t.v1) }
+ │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.v1) }
+ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- sql: |
create table t1 (v1 int, v2 int);
create table t2 (v1 int, v2 int);
@@ -65,15 +67,16 @@
└─BatchScan { table: t3, columns: [t3.v1, t3.v2], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t3_v1, t3_v2, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1, t3._row_id, t2_v2], pk_columns: [t1._row_id, t2._row_id, t1_v1, t3._row_id, t2_v2], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t2.v2 = t3.v2, output: [t1.v1, t1.v2, t2.v1, t2.v2, t3.v1, t3.v2, t1._row_id, t2._row_id, t3._row_id] }
- ├─StreamExchange { dist: HashShard(t2.v2) }
- │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] }
- │ ├─StreamExchange { dist: HashShard(t1.v1) }
- │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- │ └─StreamExchange { dist: HashShard(t2.v1) }
- │ └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- └─StreamExchange { dist: HashShard(t3.v2) }
- └─StreamTableScan { table: t3, columns: [t3.v1, t3.v2, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t2.v2, t1._row_id, t2._row_id, t3._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t2.v2 = t3.v2, output: [t1.v1, t1.v2, t2.v1, t2.v2, t3.v1, t3.v2, t1._row_id, t2._row_id, t3._row_id] }
+ ├─StreamExchange { dist: HashShard(t2.v2) }
+ │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] }
+ │ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ │ └─StreamExchange { dist: HashShard(t2.v1) }
+ │ └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t3.v2) }
+ └─StreamTableScan { table: t3, columns: [t3.v1, t3.v2, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) }
- sql: |
create table t1 (v1 int, v2 int);
create table t2 (v1 int, v2 int);
@@ -93,11 +96,12 @@
└─BatchScan { table: t2, columns: [t2.v1, t2.v2], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [t1_v2, t2_v2, t1._row_id(hidden), t1.v1(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1.v1], pk_columns: [t1._row_id, t2._row_id, t1.v1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v2, t2.v2, t1._row_id, t1.v1, t2._row_id] }
- ├─StreamExchange { dist: HashShard(t1.v1) }
- │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard(t2.v1) }
- └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1._row_id, t1.v1, t2._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v2, t2.v2, t1._row_id, t1.v1, t2._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(t2.v1) }
+ └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- sql: |
create table t1 (v1 int, v2 int);
create table t2 (v1 int, v2 int);
@@ -154,11 +158,12 @@
└─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) }
stream_plan: |-
StreamMaterialize { columns: [ix, iix, i.t._row_id(hidden), i.t._row_id#1(hidden)], stream_key: [i.t._row_id, i.t._row_id#1, ix], pk_columns: [i.t._row_id, i.t._row_id#1, ix], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id] }
- ├─StreamExchange { dist: HashShard(i.x) }
- │ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) }
- └─StreamExchange { dist: HashShard(i.x) }
- └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) }
+ └─StreamExchange { dist: HashShard(i.x, i.t._row_id, i.t._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id] }
+ ├─StreamExchange { dist: HashShard(i.x) }
+ │ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) }
+ └─StreamExchange { dist: HashShard(i.x) }
+ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) }
- name: Left & right has same SomeShard distribution. There should still be exchanges below hash join
sql: |
create table t(x int);
@@ -170,11 +175,12 @@
└─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) }
stream_plan: |-
StreamMaterialize { columns: [ix, tx, i.t._row_id(hidden), t._row_id(hidden)], stream_key: [i.t._row_id, t._row_id, ix], pk_columns: [i.t._row_id, t._row_id, ix], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: i.x = t.x, output: [i.x, t.x, i.t._row_id, t._row_id] }
- ├─StreamExchange { dist: HashShard(i.x) }
- │ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) }
- └─StreamExchange { dist: HashShard(t.x) }
- └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(i.x, i.t._row_id, t._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: i.x = t.x, output: [i.x, t.x, i.t._row_id, t._row_id] }
+ ├─StreamExchange { dist: HashShard(i.x) }
+ │ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) }
+ └─StreamExchange { dist: HashShard(t.x) }
+ └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- name: Left & right has same HashShard distribution. There should be no exchange below hash join
sql: |
create table t(x int);
@@ -628,12 +634,13 @@
└─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr1(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, $expr1], pk_columns: [t1._row_id, t2._row_id, $expr1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: $expr1 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr1, t2._row_id] }
- ├─StreamExchange { dist: HashShard($expr1) }
- │ └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] }
- │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard(t2.v2) }
- └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1._row_id, $expr1, t2._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: $expr1 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr1, t2._row_id] }
+ ├─StreamExchange { dist: HashShard($expr1) }
+ │ └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] }
+ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(t2.v2) }
+ └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- name: Repeated columns in project should not interfere with join result (https://github.com/risingwavelabs/risingwave/issues/8216)
sql: |
create table t(x int);
@@ -652,39 +659,43 @@
select t1.src p1, t1.dst p2, t2.dst p3 from t t1, t t2, t t3 where t1.dst = t2.src and t2.src = t3.dst and t3.dst = t1.src;
stream_plan: |-
StreamMaterialize { columns: [p1, p2, p3, t._row_id(hidden), t._row_id#1(hidden), t.src(hidden), t._row_id#2(hidden)], stream_key: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_columns: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] }
- ├─StreamExchange { dist: HashShard(t.src) }
- │ └─StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] }
- │ ├─StreamExchange { dist: HashShard(t.dst) }
- │ │ └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- │ └─StreamExchange { dist: HashShard(t.src) }
- │ └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- └─StreamExchange { dist: HashShard(t.dst) }
- └─StreamTableScan { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.src, t.dst, t._row_id, t._row_id, t.src, t._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] }
+ ├─StreamExchange { dist: HashShard(t.src) }
+ │ └─StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] }
+ │ ├─StreamExchange { dist: HashShard(t.dst) }
+ │ │ └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ │ └─StreamExchange { dist: HashShard(t.src) }
+ │ └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.dst) }
+ └─StreamTableScan { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [p1, p2, p3, t._row_id(hidden), t._row_id#1(hidden), t.src(hidden), t._row_id#2(hidden)], stream_key: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_columns: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamExchange Hash([0]) from 4
+ └── StreamExchange Hash([0, 1, 3, 4, 5, 6]) from 1
Fragment 1
- StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 }
- ├── StreamExchange Hash([1]) from 2
- └── StreamExchange Hash([0]) from 3
+ StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0]) from 5
Fragment 2
+ StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 }
+ ├── StreamExchange Hash([1]) from 3
+ └── StreamExchange Hash([0]) from 4
+
+ Fragment 3
Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 8 }
├── Upstream
└── BatchPlanNode
- Fragment 3
+ Fragment 4
Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 9 }
├── Upstream
└── BatchPlanNode
- Fragment 4
+ Fragment 5
Chain { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 10 }
├── Upstream
└── BatchPlanNode
@@ -711,5 +722,5 @@
Table 10 { columns: [ vnode, _row_id, t_backfill_finished, t_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 }
- Table 4294967294 { columns: [ p1, p2, p3, t._row_id, t._row_id#1, t.src, t._row_id#2 ], primary key: [ $3 ASC, $4 ASC, $1 ASC, $6 ASC, $5 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 6 }
+ Table 4294967294 { columns: [ p1, p2, p3, t._row_id, t._row_id#1, t.src, t._row_id#2 ], primary key: [ $3 ASC, $4 ASC, $1 ASC, $6 ASC, $5 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 1, 3, 4, 5, 6 ], read pk prefix len hint: 6 }
diff --git a/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml b/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml
index b24fc18c6b513..31c53d02a9a18 100644
--- a/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml
@@ -34,19 +34,20 @@
└─BatchScan { table: t4, columns: [t4.v7, t4.v8], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, v7, v8, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden), t4._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1, t3._row_id, v2, t4._row_id, v5], pk_columns: [t1._row_id, t2._row_id, v1, t3._row_id, v2, t4._row_id, v5], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] }
- ├─StreamExchange { dist: HashShard(t3.v5) }
- │ └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] }
- │ ├─StreamExchange { dist: HashShard(t1.v2) }
- │ │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] }
- │ │ ├─StreamExchange { dist: HashShard(t1.v1) }
- │ │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- │ │ └─StreamExchange { dist: HashShard(t2.v3) }
- │ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- │ └─StreamExchange { dist: HashShard(t3.v6) }
- │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) }
- └─StreamExchange { dist: HashShard(t4.v7) }
- └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t1.v2, t3.v5, t1._row_id, t2._row_id, t3._row_id, t4._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] }
+ ├─StreamExchange { dist: HashShard(t3.v5) }
+ │ └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] }
+ │ ├─StreamExchange { dist: HashShard(t1.v2) }
+ │ │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] }
+ │ │ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ │ │ └─StreamExchange { dist: HashShard(t2.v3) }
+ │ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ │ └─StreamExchange { dist: HashShard(t3.v6) }
+ │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) }
+ └─StreamExchange { dist: HashShard(t4.v7) }
+ └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) }
- name: bushy tree join ordering
sql: |
create table t1 (v1 int, v2 int);
@@ -81,19 +82,20 @@
└─BatchScan { table: t4, columns: [t4.v7, t4.v8], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, v7, v8, t2._row_id(hidden), t1._row_id(hidden), t4._row_id(hidden), t3._row_id(hidden)], stream_key: [t2._row_id, t1._row_id, v3, t4._row_id, t3._row_id, v7, v2], pk_columns: [t2._row_id, t1._row_id, v3, t4._row_id, t3._row_id, v7, v2], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t2._row_id, t1._row_id, t4._row_id, t3._row_id] }
- ├─StreamExchange { dist: HashShard(t1.v2) }
- │ └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t1.v2, t2._row_id, t1._row_id] }
- │ ├─StreamExchange { dist: HashShard(t2.v3) }
- │ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- │ └─StreamExchange { dist: HashShard(t1.v1) }
- │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard(t3.v6) }
- └─StreamHashJoin { type: Inner, predicate: t4.v7 = t3.v5, output: [t4.v7, t4.v8, t3.v5, t3.v6, t4._row_id, t3._row_id] }
- ├─StreamExchange { dist: HashShard(t4.v7) }
- │ └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) }
- └─StreamExchange { dist: HashShard(t3.v5) }
- └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v2, t2.v3, t4.v7, t2._row_id, t1._row_id, t4._row_id, t3._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t2._row_id, t1._row_id, t4._row_id, t3._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.v2) }
+ │ └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t1.v2, t2._row_id, t1._row_id] }
+ │ ├─StreamExchange { dist: HashShard(t2.v3) }
+ │ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ │ └─StreamExchange { dist: HashShard(t1.v1) }
+ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(t3.v6) }
+ └─StreamHashJoin { type: Inner, predicate: t4.v7 = t3.v5, output: [t4.v7, t4.v8, t3.v5, t3.v6, t4._row_id, t3._row_id] }
+ ├─StreamExchange { dist: HashShard(t4.v7) }
+ │ └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) }
+ └─StreamExchange { dist: HashShard(t3.v5) }
+ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) }
- name: bushy tree join ordering manually
sql: |
set rw_enable_join_ordering = false;
@@ -128,19 +130,20 @@
└─BatchScan { table: t4, columns: [t4.v7, t4.v8], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, v7, v8, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden), t4._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1, t3._row_id, t4._row_id, v5, v2], pk_columns: [t1._row_id, t2._row_id, v1, t3._row_id, t4._row_id, v5, v2], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] }
- ├─StreamExchange { dist: HashShard(t1.v2) }
- │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] }
- │ ├─StreamExchange { dist: HashShard(t1.v1) }
- │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- │ └─StreamExchange { dist: HashShard(t2.v3) }
- │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- └─StreamExchange { dist: HashShard(t3.v6) }
- └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t3.v5, t3.v6, t4.v7, t4.v8, t3._row_id, t4._row_id] }
- ├─StreamExchange { dist: HashShard(t3.v5) }
- │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) }
- └─StreamExchange { dist: HashShard(t4.v7) }
- └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t1.v2, t3.v5, t1._row_id, t2._row_id, t3._row_id, t4._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.v2) }
+ │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] }
+ │ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ │ └─StreamExchange { dist: HashShard(t2.v3) }
+ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t3.v6) }
+ └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t3.v5, t3.v6, t4.v7, t4.v8, t3._row_id, t4._row_id] }
+ ├─StreamExchange { dist: HashShard(t3.v5) }
+ │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) }
+ └─StreamExchange { dist: HashShard(t4.v7) }
+ └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) }
- name: right deep tree join ordering manually
sql: |
set rw_enable_join_ordering = false;
@@ -175,16 +178,17 @@
└─BatchScan { table: t4, columns: [t4.v7, t4.v8], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, v7, v8, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden), t4._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t3._row_id, t4._row_id, v5, v4, v1], pk_columns: [t1._row_id, t2._row_id, t3._row_id, t4._row_id, v5, v4, v1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] }
- ├─StreamExchange { dist: HashShard(t1.v1) }
- │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard(t2.v3) }
- └─StreamHashJoin { type: Inner, predicate: t2.v4 = t3.v6, output: [t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t2._row_id, t3._row_id, t4._row_id] }
- ├─StreamExchange { dist: HashShard(t2.v4) }
- │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- └─StreamExchange { dist: HashShard(t3.v6) }
- └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t3.v5, t3.v6, t4.v7, t4.v8, t3._row_id, t4._row_id] }
- ├─StreamExchange { dist: HashShard(t3.v5) }
- │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) }
- └─StreamExchange { dist: HashShard(t4.v7) }
- └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t2.v4, t3.v5, t1._row_id, t2._row_id, t3._row_id, t4._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(t2.v3) }
+ └─StreamHashJoin { type: Inner, predicate: t2.v4 = t3.v6, output: [t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t2._row_id, t3._row_id, t4._row_id] }
+ ├─StreamExchange { dist: HashShard(t2.v4) }
+ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t3.v6) }
+ └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t3.v5, t3.v6, t4.v7, t4.v8, t3._row_id, t4._row_id] }
+ ├─StreamExchange { dist: HashShard(t3.v5) }
+ │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) }
+ └─StreamExchange { dist: HashShard(t4.v7) }
+ └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) }
diff --git a/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml b/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml
index 8e63beb9798c1..85bfb1a6cda36 100644
--- a/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml
@@ -42,22 +42,23 @@
└─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), all_sales._row_id(hidden), salesperson.id(hidden), all_sales.amount(hidden), salesperson.id#1(hidden)], stream_key: [salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount], pk_columns: [salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM salesperson.id AND all_sales.amount = max(all_sales.amount), output: [salesperson.name, max(all_sales.amount), all_sales.customer_name, salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount, salesperson.id] }
- ├─StreamHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id, output: [salesperson.id, salesperson.name, all_sales.customer_name, all_sales.amount, salesperson._row_id, all_sales._row_id] }
- │ ├─StreamExchange { dist: HashShard(salesperson.id) }
- │ │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) }
- │ └─StreamExchange { dist: HashShard(all_sales.salesperson_id) }
- │ └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) }
- └─StreamProject { exprs: [salesperson.id, max(all_sales.amount)] }
- └─StreamHashAgg { group_key: [salesperson.id], aggs: [max(all_sales.amount), count] }
- └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.id, all_sales.amount, all_sales._row_id] }
- ├─StreamProject { exprs: [salesperson.id] }
- │ └─StreamHashAgg { group_key: [salesperson.id], aggs: [count] }
- │ └─StreamExchange { dist: HashShard(salesperson.id) }
- │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) }
- └─StreamExchange { dist: HashShard(all_sales.salesperson_id) }
- └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) }
- └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) }
+ └─StreamExchange { dist: HashShard(salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount) }
+ └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM salesperson.id AND all_sales.amount = max(all_sales.amount), output: [salesperson.name, max(all_sales.amount), all_sales.customer_name, salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount, salesperson.id] }
+ ├─StreamHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id, output: [salesperson.id, salesperson.name, all_sales.customer_name, all_sales.amount, salesperson._row_id, all_sales._row_id] }
+ │ ├─StreamExchange { dist: HashShard(salesperson.id) }
+ │ │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) }
+ │ └─StreamExchange { dist: HashShard(all_sales.salesperson_id) }
+ │ └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) }
+ └─StreamProject { exprs: [salesperson.id, max(all_sales.amount)] }
+ └─StreamHashAgg { group_key: [salesperson.id], aggs: [max(all_sales.amount), count] }
+ └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.id, all_sales.amount, all_sales._row_id] }
+ ├─StreamProject { exprs: [salesperson.id] }
+ │ └─StreamHashAgg { group_key: [salesperson.id], aggs: [count] }
+ │ └─StreamExchange { dist: HashShard(salesperson.id) }
+ │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) }
+ └─StreamExchange { dist: HashShard(all_sales.salesperson_id) }
+ └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) }
+ └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) }
- name: lateral join 2
sql: |
create table all_sales (salesperson_id int, customer_name varchar, amount int );
@@ -87,14 +88,15 @@
└─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), salesperson.id(hidden), all_sales.salesperson_id(hidden)], stream_key: [salesperson._row_id, salesperson.id], pk_columns: [salesperson._row_id, salesperson.id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] }
- ├─StreamExchange { dist: HashShard(salesperson.id) }
- │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) }
- └─StreamGroupTopN { order: [all_sales.amount DESC], limit: 1, offset: 0, group_key: [all_sales.salesperson_id] }
- └─StreamExchange { dist: HashShard(all_sales.salesperson_id) }
- └─StreamProject { exprs: [all_sales.salesperson_id, all_sales.amount, all_sales.customer_name, all_sales._row_id] }
- └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) }
- └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) }
+ └─StreamExchange { dist: HashShard(salesperson._row_id, salesperson.id) }
+ └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] }
+ ├─StreamExchange { dist: HashShard(salesperson.id) }
+ │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) }
+ └─StreamGroupTopN { order: [all_sales.amount DESC], limit: 1, offset: 0, group_key: [all_sales.salesperson_id] }
+ └─StreamExchange { dist: HashShard(all_sales.salesperson_id) }
+ └─StreamProject { exprs: [all_sales.salesperson_id, all_sales.amount, all_sales.customer_name, all_sales._row_id] }
+ └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) }
+ └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) }
- name: lateral join 2 (left join)
sql: |
create table all_sales (salesperson_id int, customer_name varchar, amount int );
@@ -124,14 +126,15 @@
└─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), salesperson.id(hidden), all_sales.salesperson_id(hidden)], stream_key: [salesperson._row_id, salesperson.id], pk_columns: [salesperson._row_id, salesperson.id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] }
- ├─StreamExchange { dist: HashShard(salesperson.id) }
- │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) }
- └─StreamGroupTopN { order: [all_sales.amount DESC], limit: 1, offset: 0, group_key: [all_sales.salesperson_id] }
- └─StreamExchange { dist: HashShard(all_sales.salesperson_id) }
- └─StreamProject { exprs: [all_sales.salesperson_id, all_sales.amount, all_sales.customer_name, all_sales._row_id] }
- └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) }
- └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) }
+ └─StreamExchange { dist: HashShard(salesperson._row_id, salesperson.id) }
+ └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] }
+ ├─StreamExchange { dist: HashShard(salesperson.id) }
+ │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) }
+ └─StreamGroupTopN { order: [all_sales.amount DESC], limit: 1, offset: 0, group_key: [all_sales.salesperson_id] }
+ └─StreamExchange { dist: HashShard(all_sales.salesperson_id) }
+ └─StreamProject { exprs: [all_sales.salesperson_id, all_sales.amount, all_sales.customer_name, all_sales._row_id] }
+ └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) }
+ └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) }
- name: lateral join 2 (right join) should throw an error
sql: |
create table all_sales (salesperson_id int, customer_name varchar, amount int );
@@ -165,14 +168,15 @@
└─BatchScan { table: t, columns: [t.arr], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [x, arr, unnest, t._row_id(hidden), t.arr(hidden), projected_row_id(hidden)], stream_key: [t._row_id, projected_row_id, arr], pk_columns: [t._row_id, projected_row_id, arr], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, Unnest($0), t._row_id, t.arr, projected_row_id] }
- ├─StreamExchange { dist: HashShard(t.arr) }
- │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- └─StreamProjectSet { select_list: [$0, Unnest($0)] }
- └─StreamProject { exprs: [t.arr] }
- └─StreamHashAgg { group_key: [t.arr], aggs: [count] }
- └─StreamExchange { dist: HashShard(t.arr) }
- └─StreamTableScan { table: t, columns: [t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.arr, t._row_id, projected_row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, Unnest($0), t._row_id, t.arr, projected_row_id] }
+ ├─StreamExchange { dist: HashShard(t.arr) }
+ │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamProjectSet { select_list: [$0, Unnest($0)] }
+ └─StreamProject { exprs: [t.arr] }
+ └─StreamHashAgg { group_key: [t.arr], aggs: [count] }
+ └─StreamExchange { dist: HashShard(t.arr) }
+ └─StreamTableScan { table: t, columns: [t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- name: https://github.com/risingwavelabs/risingwave/issues/12298
sql: |
create table t1(c varchar, n varchar, id varchar, d varchar);
diff --git a/src/frontend/planner_test/tests/testdata/output/mv_on_mv.yaml b/src/frontend/planner_test/tests/testdata/output/mv_on_mv.yaml
index 2f7d9e5e75b3b..6838ddb331939 100644
--- a/src/frontend/planner_test/tests/testdata/output/mv_on_mv.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/mv_on_mv.yaml
@@ -12,8 +12,9 @@
select m1.v1 as m1v1, m1.v2 as m1v2, m2.v1 as m2v1, m2.v2 as m2v2 from m1 join m2 on m1.v1 = m2.v1;
stream_plan: |-
StreamMaterialize { columns: [m1v1, m1v2, m2v1, m2v2, m1.t1._row_id(hidden), m2.t1._row_id(hidden)], stream_key: [m1.t1._row_id, m2.t1._row_id, m1v1], pk_columns: [m1.t1._row_id, m2.t1._row_id, m1v1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: m1.v1 = m2.v1, output: [m1.v1, m1.v2, m2.v1, m2.v2, m1.t1._row_id, m2.t1._row_id] }
- ├─StreamExchange { dist: HashShard(m1.v1) }
- │ └─StreamTableScan { table: m1, columns: [m1.v1, m1.v2, m1.t1._row_id], pk: [m1.t1._row_id], dist: UpstreamHashShard(m1.t1._row_id) }
- └─StreamExchange { dist: HashShard(m2.v1) }
- └─StreamTableScan { table: m2, columns: [m2.v1, m2.v2, m2.t1._row_id], pk: [m2.t1._row_id], dist: UpstreamHashShard(m2.t1._row_id) }
+ └─StreamExchange { dist: HashShard(m1.v1, m1.t1._row_id, m2.t1._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: m1.v1 = m2.v1, output: [m1.v1, m1.v2, m2.v1, m2.v2, m1.t1._row_id, m2.t1._row_id] }
+ ├─StreamExchange { dist: HashShard(m1.v1) }
+ │ └─StreamTableScan { table: m1, columns: [m1.v1, m1.v2, m1.t1._row_id], pk: [m1.t1._row_id], dist: UpstreamHashShard(m1.t1._row_id) }
+ └─StreamExchange { dist: HashShard(m2.v1) }
+ └─StreamTableScan { table: m2, columns: [m2.v1, m2.v2, m2.t1._row_id], pk: [m2.t1._row_id], dist: UpstreamHashShard(m2.t1._row_id) }
diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml
index 2f23b0674fa84..8d452bf45bc36 100644
--- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml
@@ -181,34 +181,38 @@
└─BatchScan { table: auction, columns: [auction.id, auction.seller, auction.category], distribution: UpstreamHashShard(auction.id) }
stream_plan: |-
StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], stream_key: [id, auction.seller], pk_columns: [id, auction.seller], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] }
- ├─StreamExchange { dist: HashShard(auction.seller) }
- │ └─StreamProject { exprs: [auction.id, auction.seller] }
- │ └─StreamFilter { predicate: (auction.category = 10:Int32) }
- │ └─StreamTableScan { table: auction, columns: [auction.id, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
- └─StreamExchange { dist: HashShard(person.id) }
- └─StreamFilter { predicate: (((person.state = 'or':Varchar) OR (person.state = 'id':Varchar)) OR (person.state = 'ca':Varchar)) }
- └─StreamTableScan { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) }
+ └─StreamExchange { dist: HashShard(auction.id, auction.seller) }
+ └─StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] }
+ ├─StreamExchange { dist: HashShard(auction.seller) }
+ │ └─StreamProject { exprs: [auction.id, auction.seller] }
+ │ └─StreamFilter { predicate: (auction.category = 10:Int32) }
+ │ └─StreamTableScan { table: auction, columns: [auction.id, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
+ └─StreamExchange { dist: HashShard(person.id) }
+ └─StreamFilter { predicate: (((person.state = 'or':Varchar) OR (person.state = 'id':Varchar)) OR (person.state = 'ca':Varchar)) }
+ └─StreamTableScan { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], stream_key: [id, auction.seller], pk_columns: [id, auction.seller], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([1]) from 1
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([3, 4]) from 1
Fragment 1
+ StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamExchange Hash([1]) from 2
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
StreamProject { exprs: [auction.id, auction.seller] }
└── StreamFilter { predicate: (auction.category = 10:Int32) }
└── Chain { table: auction, columns: [auction.id, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 4 }
├── Upstream
└── BatchPlanNode
- Fragment 2
+ Fragment 3
StreamFilter { predicate: (((person.state = 'or':Varchar) OR (person.state = 'id':Varchar)) OR (person.state = 'ca':Varchar)) }
└── Chain { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 5 }
├── Upstream
@@ -242,7 +246,7 @@
├── columns: [ name, city, state, id, auction.seller, person.id ]
├── primary key: [ $3 ASC, $4 ASC ]
├── value indices: [ 0, 1, 2, 3, 4, 5 ]
- ├── distribution key: [ 4 ]
+ ├── distribution key: [ 3, 4 ]
└── read pk prefix len hint: 2
- id: nexmark_q4
@@ -847,34 +851,38 @@
└─BatchScan { table: auction, columns: [auction.date_time, auction.seller], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2], pk_columns: [id, starttime, $expr2], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] }
- ├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) }
- │ └─StreamProject { exprs: [person.id, $expr1, $expr2, internal_last_seen_value(person.name)] }
- │ └─StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] }
- │ └─StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] }
- │ └─StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] }
- │ └─StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) }
- └─StreamProject { exprs: [auction.seller, $expr3, $expr4] }
- └─StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] }
- └─StreamExchange { dist: HashShard(auction.seller, $expr3, $expr4) }
- └─StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] }
- └─StreamProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, auction.id] }
- └─StreamTableScan { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
+ └─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) }
+ └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] }
+ ├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) }
+ │ └─StreamProject { exprs: [person.id, $expr1, $expr2, internal_last_seen_value(person.name)] }
+ │ └─StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] }
+ │ └─StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] }
+ │ └─StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] }
+ │ └─StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) }
+ └─StreamProject { exprs: [auction.seller, $expr3, $expr4] }
+ └─StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] }
+ └─StreamExchange { dist: HashShard(auction.seller, $expr3, $expr4) }
+ └─StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] }
+ └─StreamProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, auction.id] }
+ └─StreamTableScan { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2], pk_columns: [id, starttime, $expr2], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([0, 1, 2]) from 1
- └── StreamProject { exprs: [auction.seller, $expr3, $expr4] }
- └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0, 1, 2]) from 2
+ └── StreamExchange Hash([0, 2, 3]) from 1
Fragment 1
+ StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamExchange Hash([0, 1, 2]) from 2
+ └── StreamProject { exprs: [auction.seller, $expr3, $expr4] }
+ └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] }
+ └── StreamExchange Hash([0, 1, 2]) from 3
+
+ Fragment 2
StreamProject { exprs: [person.id, $expr1, $expr2, internal_last_seen_value(person.name)] }
└── StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] } { intermediate state table: 4, state tables: [], distinct tables: [] }
└── StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] }
@@ -883,7 +891,7 @@
├── Upstream
└── BatchPlanNode
- Fragment 2
+ Fragment 3
StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] }
└── StreamProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, auction.id] }
└── Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 }
@@ -1133,27 +1141,31 @@
└─StreamTableScan { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) }
stream_plan: |-
StreamMaterialize { columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr1(hidden), side_input.key(hidden)], stream_key: [bid._row_id, $expr1], pk_columns: [bid._row_id, $expr1], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] }
- ├─StreamExchange { dist: HashShard($expr1) }
- │ └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] }
- │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(side_input.key) }
- └─StreamTableScan { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) }
+ └─StreamExchange { dist: HashShard(bid._row_id, $expr1) }
+ └─StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] }
+ ├─StreamExchange { dist: HashShard($expr1) }
+ │ └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] }
+ │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(side_input.key) }
+ └─StreamTableScan { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr1(hidden), side_input.key(hidden)], stream_key: [bid._row_id, $expr1], pk_columns: [bid._row_id, $expr1], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] }
- ├── StreamExchange Hash([4]) from 1
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange Hash([5, 6]) from 1
Fragment 1
+ StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] }
+ ├── StreamExchange Hash([4]) from 2
+ └── StreamExchange NoShuffle from 3
+
+ Fragment 2
StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] }
└── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 }
├── Upstream
└── BatchPlanNode
- Fragment 2
+ Fragment 3
Chain { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } { state table: 1 }
├── Upstream
└── BatchPlanNode
@@ -1166,7 +1178,7 @@
├── columns: [ auction, bidder, price, date_time, value, bid._row_id, $expr1, side_input.key ]
├── primary key: [ $5 ASC, $6 ASC ]
├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ]
- ├── distribution key: [ 6 ]
+ ├── distribution key: [ 5, 6 ]
└── read pk prefix len hint: 2
- id: nexmark_q14
@@ -1795,30 +1807,34 @@
└─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], stream_key: [bid._row_id, auction], pk_columns: [bid._row_id, auction], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] }
- ├─StreamExchange { dist: HashShard(bid.auction) }
- │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
- └─StreamExchange { dist: HashShard(auction.id) }
- └─StreamFilter { predicate: (auction.category = 10:Int32) }
- └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
+ └─StreamExchange { dist: HashShard(bid.auction, bid._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] }
+ ├─StreamExchange { dist: HashShard(bid.auction) }
+ │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
+ └─StreamExchange { dist: HashShard(auction.id) }
+ └─StreamFilter { predicate: (auction.category = 10:Int32) }
+ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], stream_key: [bid._row_id, auction], pk_columns: [bid._row_id, auction], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([0]) from 1
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0, 14]) from 1
Fragment 1
+ StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 4 }
├── Upstream
└── BatchPlanNode
- Fragment 2
+ Fragment 3
StreamFilter { predicate: (auction.category = 10:Int32) }
└── Chain { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 5 }
├── Upstream
@@ -1840,7 +1856,7 @@
├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id, auction.id ]
├── primary key: [ $14 ASC, $0 ASC ]
├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ]
- ├── distribution key: [ 0 ]
+ ├── distribution key: [ 0, 14 ]
└── read pk prefix len hint: 2
- id: nexmark_q21
@@ -1946,33 +1962,37 @@
└─BatchScan { table: bid, columns: [bid.auction, bid.price], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] }
- ├─StreamExchange { dist: HashShard(auction.id) }
- │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
- └─StreamProject { exprs: [bid.auction, max(bid.price)] }
- └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [max(bid.price), count] }
- └─StreamExchange { dist: HashShard(bid.auction) }
- └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
+ └─StreamExchange { dist: HashShard(auction.id) }
+ └─StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] }
+ ├─StreamExchange { dist: HashShard(auction.id) }
+ │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
+ └─StreamProject { exprs: [bid.auction, max(bid.price)] }
+ └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [max(bid.price), count] }
+ └─StreamExchange { dist: HashShard(bid.auction) }
+ └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [bid.auction, max(bid.price)] }
- └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [max(bid.price), count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0]) from 1
Fragment 1
+ StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [bid.auction, max(bid.price)] }
+ └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [max(bid.price), count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 4 }
├── Upstream
└── BatchPlanNode
- Fragment 2
+ Fragment 3
Chain { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 }
├── Upstream
└── BatchPlanNode
@@ -2215,39 +2235,43 @@
└─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all }
- ├─StreamExchange { dist: HashShard(auction.id) }
- │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
- └─StreamProject { exprs: [bid.auction] }
- └─StreamFilter { predicate: (count >= 20:Int32) }
- └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] }
- └─StreamExchange { dist: HashShard(bid.auction) }
- └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
+ └─StreamExchange { dist: HashShard(auction.id) }
+ └─StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all }
+ ├─StreamExchange { dist: HashShard(auction.id) }
+ │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
+ └─StreamProject { exprs: [bid.auction] }
+ └─StreamFilter { predicate: (count >= 20:Int32) }
+ └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] }
+ └─StreamExchange { dist: HashShard(bid.auction) }
+ └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [bid.auction] }
- └── StreamFilter { predicate: (count >= 20:Int32) }
- └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] }
- ├── intermediate state table: 5
- ├── state tables: []
- ├── distinct tables: []
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0]) from 1
Fragment 1
+ StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [bid.auction] }
+ └── StreamFilter { predicate: (count >= 20:Int32) }
+ └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] }
+ ├── intermediate state table: 5
+ ├── state tables: []
+ ├── distinct tables: []
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
├── state table: 4
├── Upstream
└── BatchPlanNode
- Fragment 2
+ Fragment 3
Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 }
├── Upstream
└── BatchPlanNode
@@ -2333,39 +2357,43 @@
└─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all }
- ├─StreamExchange { dist: HashShard(auction.id) }
- │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
- └─StreamProject { exprs: [bid.auction] }
- └─StreamFilter { predicate: (count < 20:Int32) }
- └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] }
- └─StreamExchange { dist: HashShard(bid.auction) }
- └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
+ └─StreamExchange { dist: HashShard(auction.id) }
+ └─StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all }
+ ├─StreamExchange { dist: HashShard(auction.id) }
+ │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
+ └─StreamProject { exprs: [bid.auction] }
+ └─StreamFilter { predicate: (count < 20:Int32) }
+ └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] }
+ └─StreamExchange { dist: HashShard(bid.auction) }
+ └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [bid.auction] }
- └── StreamFilter { predicate: (count < 20:Int32) }
- └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] }
- ├── intermediate state table: 5
- ├── state tables: []
- ├── distinct tables: []
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0]) from 1
Fragment 1
+ StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [bid.auction] }
+ └── StreamFilter { predicate: (count < 20:Int32) }
+ └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] }
+ ├── intermediate state table: 5
+ ├── state tables: []
+ ├── distinct tables: []
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) }
├── state table: 4
├── Upstream
└── BatchPlanNode
- Fragment 2
+ Fragment 3
Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 }
├── Upstream
└── BatchPlanNode
diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml
index 7c694fad1fa67..31be64b2c480a 100644
--- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml
@@ -158,29 +158,33 @@
└─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, seller], pk_columns: [_row_id, _row_id#1, seller], pk_conflict: NoCheck }
- └─StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] }
- ├─StreamExchange { dist: HashShard(seller) }
- │ └─StreamFilter { predicate: (category = 10:Int32) }
- │ └─StreamRowIdGen { row_id_index: 10 }
- │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
- └─StreamExchange { dist: HashShard(id) }
- └─StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) }
- └─StreamRowIdGen { row_id_index: 8 }
- └─StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] }
+ └─StreamExchange { dist: HashShard(_row_id, seller, _row_id) }
+ └─StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] }
+ ├─StreamExchange { dist: HashShard(seller) }
+ │ └─StreamFilter { predicate: (category = 10:Int32) }
+ │ └─StreamRowIdGen { row_id_index: 10 }
+ │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
+ └─StreamExchange { dist: HashShard(id) }
+ └─StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) }
+ └─StreamRowIdGen { row_id_index: 8 }
+ └─StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, seller], pk_columns: [_row_id, _row_id#1, seller], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([7]) from 1
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([4, 5, 6]) from 1
Fragment 1
+ StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([7]) from 2
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
StreamFilter { predicate: (category = 10:Int32) }
└── StreamRowIdGen { row_id_index: 10 }
└── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { source state table: 4 }
- Fragment 2
+ Fragment 3
StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) }
└── StreamRowIdGen { row_id_index: 8 }
└── StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } { source state table: 5 }
@@ -211,7 +215,7 @@
├── columns: [ name, city, state, id, _row_id, seller, _row_id#1 ]
├── primary key: [ $4 ASC, $6 ASC, $5 ASC ]
├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ]
- ├── distribution key: [ 5 ]
+ ├── distribution key: [ 4, 5, 6 ]
└── read pk prefix len hint: 3
- id: nexmark_q4
@@ -737,40 +741,44 @@
└─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, name, starttime, $expr2], pk_columns: [id, name, starttime, $expr2], pk_conflict: NoCheck }
- └─StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all }
- ├─StreamExchange { dist: HashShard(id, $expr1, $expr2) }
- │ └─StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] }
- │ └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) }
- │ └─StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] }
- │ └─StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] }
- │ └─StreamRowIdGen { row_id_index: 8 }
- │ └─StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] }
- └─StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] }
- └─StreamExchange { dist: HashShard(seller, $expr3, $expr4) }
- └─StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] }
- └─StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] }
- └─StreamRowIdGen { row_id_index: 10 }
- └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
+ └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) }
+ └─StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all }
+ ├─StreamExchange { dist: HashShard(id, $expr1, $expr2) }
+ │ └─StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] }
+ │ └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) }
+ │ └─StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] }
+ │ └─StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] }
+ │ └─StreamRowIdGen { row_id_index: 8 }
+ │ └─StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] }
+ └─StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] }
+ └─StreamExchange { dist: HashShard(seller, $expr3, $expr4) }
+ └─StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] }
+ └─StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] }
+ └─StreamRowIdGen { row_id_index: 10 }
+ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, name, starttime, $expr2], pk_columns: [id, name, starttime, $expr2], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0, 2, 3]) from 1
- └── StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } { state table: 6 }
- └── StreamExchange Hash([0, 1, 2]) from 3
+ └── StreamExchange Hash([0, 1, 2, 3]) from 1
Fragment 1
- StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } { state table: 4 }
- └── StreamExchange Hash([0, 1, 2, 3]) from 2
+ StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0, 2, 3]) from 2
+ └── StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } { state table: 6 }
+ └── StreamExchange Hash([0, 1, 2]) from 4
Fragment 2
+ StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } { state table: 4 }
+ └── StreamExchange Hash([0, 1, 2, 3]) from 3
+
+ Fragment 3
StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] }
└── StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] }
└── StreamRowIdGen { row_id_index: 8 }
└── StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } { source state table: 5 }
- Fragment 3
+ Fragment 4
StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] }
└── StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] }
└── StreamRowIdGen { row_id_index: 10 }
@@ -796,7 +804,7 @@
├── columns: [ id, name, starttime, $expr2, seller, $expr3, $expr4 ]
├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ]
├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ]
- ├── distribution key: [ 0, 2, 3 ]
+ ├── distribution key: [ 0, 1, 2, 3 ]
└── read pk prefix len hint: 4
- id: nexmark_q9
@@ -1629,31 +1637,31 @@
└─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck }
- └─StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] }
- ├─StreamExchange { dist: HashShard(auction) }
- │ └─StreamRowIdGen { row_id_index: 7 }
- │ └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] }
- └─StreamExchange { dist: HashShard(id) }
- └─StreamFilter { predicate: (category = 10:Int32) }
- └─StreamRowIdGen { row_id_index: 10 }
- └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
+ └─StreamExchange { dist: HashShard(auction, _row_id, _row_id) }
+ └─StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] }
+ ├─StreamExchange { dist: HashShard(auction) }
+ │ └─StreamRowIdGen { row_id_index: 7 }
+ │ └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] }
+ └─StreamExchange { dist: HashShard(id) }
+ └─StreamFilter { predicate: (category = 10:Int32) }
+ └─StreamRowIdGen { row_id_index: 10 }
+ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([0]) from 1
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0, 14, 15]) from 1
Fragment 1
+ StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
StreamRowIdGen { row_id_index: 7 }
└── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 4 }
- Fragment 2
+ Fragment 3
StreamFilter { predicate: (category = 10:Int32) }
└── StreamRowIdGen { row_id_index: 10 }
└── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { source state table: 5 }
@@ -1674,7 +1682,7 @@
├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ]
├── primary key: [ $14 ASC, $15 ASC, $0 ASC ]
├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ]
- ├── distribution key: [ 0 ]
+ ├── distribution key: [ 0, 14, 15 ]
└── read pk prefix len hint: 3
- id: nexmark_q21
@@ -1775,30 +1783,34 @@
└─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] }
- ├─StreamExchange { dist: HashShard(id) }
- │ └─StreamRowIdGen { row_id_index: 10 }
- │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
- └─StreamProject { exprs: [auction, max(price)] }
- └─StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] }
- └─StreamExchange { dist: HashShard(auction) }
- └─StreamRowIdGen { row_id_index: 7 }
- └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] }
+ └─StreamExchange { dist: HashShard(id, _row_id) }
+ └─StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] }
+ ├─StreamExchange { dist: HashShard(id) }
+ │ └─StreamRowIdGen { row_id_index: 10 }
+ │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
+ └─StreamProject { exprs: [auction, max(price)] }
+ └─StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] }
+ └─StreamExchange { dist: HashShard(auction) }
+ └─StreamRowIdGen { row_id_index: 7 }
+ └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [auction, max(price)] }
- └── StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0, 3]) from 1
Fragment 1
+ StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [auction, max(price)] }
+ └── StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
StreamRowIdGen { row_id_index: 10 }
└── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { source state table: 4 }
- Fragment 2
+ Fragment 3
StreamRowIdGen { row_id_index: 7 }
└── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 6 }
@@ -1825,7 +1837,7 @@
├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, auction ]
├── primary key: [ $3 ASC, $0 ASC ]
├── value indices: [ 0, 1, 2, 3, 4 ]
- ├── distribution key: [ 0 ]
+ ├── distribution key: [ 0, 3 ]
└── read pk prefix len hint: 2
- id: nexmark_q102
@@ -1992,37 +2004,41 @@
└─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] }
- ├─StreamExchange { dist: HashShard(id) }
- │ └─StreamRowIdGen { row_id_index: 10 }
- │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
- └─StreamProject { exprs: [auction] }
- └─StreamFilter { predicate: (count >= 20:Int32) }
- └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] }
- └─StreamExchange { dist: HashShard(auction) }
- └─StreamRowIdGen { row_id_index: 7 }
- └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] }
+ └─StreamExchange { dist: HashShard(id, _row_id) }
+ └─StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] }
+ ├─StreamExchange { dist: HashShard(id) }
+ │ └─StreamRowIdGen { row_id_index: 10 }
+ │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
+ └─StreamProject { exprs: [auction] }
+ └─StreamFilter { predicate: (count >= 20:Int32) }
+ └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] }
+ └─StreamExchange { dist: HashShard(auction) }
+ └─StreamRowIdGen { row_id_index: 7 }
+ └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [auction] }
- └── StreamFilter { predicate: (count >= 20:Int32) }
- └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0, 2]) from 1
Fragment 1
+ StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [auction] }
+ └── StreamFilter { predicate: (count >= 20:Int32) }
+ └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
StreamRowIdGen { row_id_index: 10 }
└── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
└── source state table: 4
- Fragment 2
+ Fragment 3
StreamRowIdGen { row_id_index: 7 }
└── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 6 }
@@ -2049,7 +2065,7 @@
├── columns: [ auction_id, auction_item_name, _row_id ]
├── primary key: [ $2 ASC, $0 ASC ]
├── value indices: [ 0, 1, 2 ]
- ├── distribution key: [ 0 ]
+ ├── distribution key: [ 0, 2 ]
└── read pk prefix len hint: 2
- id: nexmark_q104
@@ -2080,37 +2096,41 @@
└─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] }
- ├─StreamExchange { dist: HashShard(id) }
- │ └─StreamRowIdGen { row_id_index: 10 }
- │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
- └─StreamProject { exprs: [auction] }
- └─StreamFilter { predicate: (count < 20:Int32) }
- └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] }
- └─StreamExchange { dist: HashShard(auction) }
- └─StreamRowIdGen { row_id_index: 7 }
- └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] }
+ └─StreamExchange { dist: HashShard(id, _row_id) }
+ └─StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] }
+ ├─StreamExchange { dist: HashShard(id) }
+ │ └─StreamRowIdGen { row_id_index: 10 }
+ │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
+ └─StreamProject { exprs: [auction] }
+ └─StreamFilter { predicate: (count < 20:Int32) }
+ └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] }
+ └─StreamExchange { dist: HashShard(auction) }
+ └─StreamRowIdGen { row_id_index: 7 }
+ └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [auction] }
- └── StreamFilter { predicate: (count < 20:Int32) }
- └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0, 2]) from 1
Fragment 1
+ StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [auction] }
+ └── StreamFilter { predicate: (count < 20:Int32) }
+ └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
+ └── StreamExchange Hash([0]) from 3
+
+ Fragment 2
StreamRowIdGen { row_id_index: 10 }
└── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
└── source state table: 4
- Fragment 2
+ Fragment 3
StreamRowIdGen { row_id_index: 7 }
└── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 6 }
@@ -2137,7 +2157,7 @@
├── columns: [ auction_id, auction_item_name, _row_id ]
├── primary key: [ $2 ASC, $0 ASC ]
├── value indices: [ 0, 1, 2 ]
- ├── distribution key: [ 0 ]
+ ├── distribution key: [ 0, 2 ]
└── read pk prefix len hint: 2
- id: nexmark_q105
diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml
index ccdde39e76764..c6c3ffd4f5ad6 100644
--- a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml
@@ -717,65 +717,69 @@
AND P.endtime = A.endtime;
stream_plan: |-
StreamMaterialize { columns: [id, name, starttime, $expr6(hidden), $expr8(hidden), $expr9(hidden), $expr10(hidden)], stream_key: [id, name, starttime, $expr6], pk_columns: [id, name, starttime, $expr6], pk_conflict: NoCheck }
- └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all }
- ├─StreamExchange { dist: HashShard($expr2, $expr5, $expr6) }
- │ └─StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] }
- │ └─StreamExchange { dist: HashShard($expr2, $expr3, $expr5, $expr6) }
- │ └─StreamProject { exprs: [$expr2, $expr3, $expr5, ($expr5 + '00:00:10':Interval) as $expr6] }
- │ └─StreamProject { exprs: [$expr2, $expr3, $expr4, TumbleStart($expr4, '00:00:10':Interval) as $expr5, _row_id] }
- │ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr2, Field(person, 1:Int32) as $expr3, Field(person, 6:Int32) as $expr4, _row_id] }
- │ └─StreamFilter { predicate: (event_type = 0:Int32) }
- │ └─StreamShare { id: 5 }
- │ └─StreamProject { exprs: [event_type, person, auction, _row_id] }
- │ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamAppendOnlyDedup { dedup_cols: [$expr8, $expr9, $expr10] }
- └─StreamExchange { dist: HashShard($expr8, $expr9, $expr10) }
- └─StreamProject { exprs: [$expr8, $expr9, ($expr9 + '00:00:10':Interval) as $expr10] }
- └─StreamProject { exprs: [$expr7, $expr8, TumbleStart($expr7, '00:00:10':Interval) as $expr9, _row_id] }
- └─StreamProject { exprs: [Field(auction, 5:Int32) as $expr7, Field(auction, 7:Int32) as $expr8, _row_id] }
- └─StreamFilter { predicate: (event_type = 1:Int32) }
- └─StreamShare { id: 5 }
- └─StreamProject { exprs: [event_type, person, auction, _row_id] }
- └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) }
- └─StreamRowIdGen { row_id_index: 5 }
- └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
- └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: HashShard($expr2, $expr3, $expr5, $expr6) }
+ └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all }
+ ├─StreamExchange { dist: HashShard($expr2, $expr5, $expr6) }
+ │ └─StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] }
+ │ └─StreamExchange { dist: HashShard($expr2, $expr3, $expr5, $expr6) }
+ │ └─StreamProject { exprs: [$expr2, $expr3, $expr5, ($expr5 + '00:00:10':Interval) as $expr6] }
+ │ └─StreamProject { exprs: [$expr2, $expr3, $expr4, TumbleStart($expr4, '00:00:10':Interval) as $expr5, _row_id] }
+ │ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr2, Field(person, 1:Int32) as $expr3, Field(person, 6:Int32) as $expr4, _row_id] }
+ │ └─StreamFilter { predicate: (event_type = 0:Int32) }
+ │ └─StreamShare { id: 5 }
+ │ └─StreamProject { exprs: [event_type, person, auction, _row_id] }
+ │ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamAppendOnlyDedup { dedup_cols: [$expr8, $expr9, $expr10] }
+ └─StreamExchange { dist: HashShard($expr8, $expr9, $expr10) }
+ └─StreamProject { exprs: [$expr8, $expr9, ($expr9 + '00:00:10':Interval) as $expr10] }
+ └─StreamProject { exprs: [$expr7, $expr8, TumbleStart($expr7, '00:00:10':Interval) as $expr9, _row_id] }
+ └─StreamProject { exprs: [Field(auction, 5:Int32) as $expr7, Field(auction, 7:Int32) as $expr8, _row_id] }
+ └─StreamFilter { predicate: (event_type = 1:Int32) }
+ └─StreamShare { id: 5 }
+ └─StreamProject { exprs: [event_type, person, auction, _row_id] }
+ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) }
+ └─StreamRowIdGen { row_id_index: 5 }
+ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
+ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [id, name, starttime, $expr6(hidden), $expr8(hidden), $expr9(hidden), $expr10(hidden)], stream_key: [id, name, starttime, $expr6], pk_columns: [id, name, starttime, $expr6], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0, 2, 3]) from 1
- └── StreamAppendOnlyDedup { dedup_cols: [$expr8, $expr9, $expr10] } { state table: 6 }
- └── StreamExchange Hash([0, 1, 2]) from 4
+ └── StreamExchange Hash([0, 1, 2, 3]) from 1
Fragment 1
- StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] } { state table: 4 }
- └── StreamExchange Hash([0, 1, 2, 3]) from 2
+ StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0, 2, 3]) from 2
+ └── StreamAppendOnlyDedup { dedup_cols: [$expr8, $expr9, $expr10] } { state table: 6 }
+ └── StreamExchange Hash([0, 1, 2]) from 5
Fragment 2
+ StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] } { state table: 4 }
+ └── StreamExchange Hash([0, 1, 2, 3]) from 3
+
+ Fragment 3
StreamProject { exprs: [$expr2, $expr3, $expr5, ($expr5 + '00:00:10':Interval) as $expr6] }
└── StreamProject { exprs: [$expr2, $expr3, $expr4, TumbleStart($expr4, '00:00:10':Interval) as $expr5, _row_id] }
└── StreamProject { exprs: [Field(person, 0:Int32) as $expr2, Field(person, 1:Int32) as $expr3, Field(person, 6:Int32) as $expr4, _row_id] }
└── StreamFilter { predicate: (event_type = 0:Int32) }
- └── StreamExchange NoShuffle from 3
+ └── StreamExchange NoShuffle from 4
- Fragment 3
+ Fragment 4
StreamProject { exprs: [event_type, person, auction, _row_id] }
└── StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) }
└── StreamRowIdGen { row_id_index: 5 }
└── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 }
- Fragment 4
+ Fragment 5
StreamProject { exprs: [$expr8, $expr9, ($expr9 + '00:00:10':Interval) as $expr10] }
└── StreamProject { exprs: [$expr7, $expr8, TumbleStart($expr7, '00:00:10':Interval) as $expr9, _row_id] }
└── StreamProject { exprs: [Field(auction, 5:Int32) as $expr7, Field(auction, 7:Int32) as $expr8, _row_id] }
└── StreamFilter { predicate: (event_type = 1:Int32) }
- └── StreamExchange NoShuffle from 3
+ └── StreamExchange NoShuffle from 4
Table 0 { columns: [ $expr2, $expr3, $expr5, $expr6 ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 }
@@ -795,7 +799,7 @@
├── columns: [ id, name, starttime, $expr6, $expr8, $expr9, $expr10 ]
├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ]
├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ]
- ├── distribution key: [ 0, 2, 3 ]
+ ├── distribution key: [ 0, 1, 2, 3 ]
└── read pk prefix len hint: 4
- id: nexmark_q9
@@ -1180,59 +1184,63 @@
WHERE A.category = 10;
stream_plan: |-
StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: $expr3 = $expr9, output: [$expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _row_id] }
- ├─StreamExchange { dist: HashShard($expr3) }
- │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr3, Field(bid, 1:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, Field(bid, 3:Int32) as $expr6, Field(bid, 4:Int32) as $expr7, Field(bid, 5:Int32) as $expr8, _row_id] }
- │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true }
- │ ├─StreamFilter { predicate: (event_type = 2:Int32) }
- │ │ └─StreamShare { id: 5 }
- │ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
- │ │ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) }
- │ │ └─StreamRowIdGen { row_id_index: 5 }
- │ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
- │ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- │ └─StreamExchange { dist: Broadcast }
- │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] }
- │ └─StreamNow { output: [now] }
- └─StreamExchange { dist: HashShard($expr9) }
- └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr9, Field(auction, 1:Int32) as $expr10, Field(auction, 2:Int32) as $expr11, Field(auction, 3:Int32) as $expr12, Field(auction, 4:Int32) as $expr13, Field(auction, 5:Int32) as $expr14, Field(auction, 6:Int32) as $expr15, Field(auction, 7:Int32) as $expr16, Field(auction, 8:Int32) as $expr17, _row_id] }
- └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) }
- └─StreamShare { id: 5 }
- └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
- └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) }
- └─StreamRowIdGen { row_id_index: 5 }
- └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
- └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: HashShard($expr3, _row_id, _row_id) }
+ └─StreamHashJoin { type: Inner, predicate: $expr3 = $expr9, output: [$expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _row_id] }
+ ├─StreamExchange { dist: HashShard($expr3) }
+ │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr3, Field(bid, 1:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, Field(bid, 3:Int32) as $expr6, Field(bid, 4:Int32) as $expr7, Field(bid, 5:Int32) as $expr8, _row_id] }
+ │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true }
+ │ ├─StreamFilter { predicate: (event_type = 2:Int32) }
+ │ │ └─StreamShare { id: 5 }
+ │ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
+ │ │ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) }
+ │ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
+ │ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ │ └─StreamExchange { dist: Broadcast }
+ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] }
+ │ └─StreamNow { output: [now] }
+ └─StreamExchange { dist: HashShard($expr9) }
+ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr9, Field(auction, 1:Int32) as $expr10, Field(auction, 2:Int32) as $expr11, Field(auction, 3:Int32) as $expr12, Field(auction, 4:Int32) as $expr13, Field(auction, 5:Int32) as $expr14, Field(auction, 6:Int32) as $expr15, Field(auction, 7:Int32) as $expr16, Field(auction, 8:Int32) as $expr17, _row_id] }
+ └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) }
+ └─StreamShare { id: 5 }
+ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
+ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) }
+ └─StreamRowIdGen { row_id_index: 5 }
+ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
+ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: Inner, predicate: $expr3 = $expr9, output: [$expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamExchange Hash([0]) from 4
+ └── StreamExchange Hash([0, 14, 15]) from 1
Fragment 1
+ StreamHashJoin { type: Inner, predicate: $expr3 = $expr9, output: [$expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0]) from 5
+
+ Fragment 2
StreamProject { exprs: [Field(bid, 0:Int32) as $expr3, Field(bid, 1:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, Field(bid, 3:Int32) as $expr6, Field(bid, 4:Int32) as $expr7, Field(bid, 5:Int32) as $expr8, _row_id] }
└── StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } { left table: 4, right table: 5 }
├── StreamFilter { predicate: (event_type = 2:Int32) }
- │ └── StreamExchange NoShuffle from 2
- └── StreamExchange Broadcast from 3
+ │ └── StreamExchange NoShuffle from 3
+ └── StreamExchange Broadcast from 4
- Fragment 2
+ Fragment 3
StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) }
└── StreamRowIdGen { row_id_index: 5 }
└── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 6 }
- Fragment 3
+ Fragment 4
StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] }
└── StreamNow { output: [now] } { state table: 7 }
- Fragment 4
+ Fragment 5
StreamProject { exprs: [Field(auction, 0:Int32) as $expr9, Field(auction, 1:Int32) as $expr10, Field(auction, 2:Int32) as $expr11, Field(auction, 3:Int32) as $expr12, Field(auction, 4:Int32) as $expr13, Field(auction, 5:Int32) as $expr14, Field(auction, 6:Int32) as $expr15, Field(auction, 7:Int32) as $expr16, Field(auction, 8:Int32) as $expr17, _row_id] }
└── StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
Table 0 { columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, _row_id ], primary key: [ $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
@@ -1250,7 +1258,12 @@
Table 7 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 }
- Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ], primary key: [ $14 ASC, $15 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ], distribution key: [ 0 ], read pk prefix len hint: 3 }
+ Table 4294967294
+ ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ]
+ ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ]
+ ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ]
+ ├── distribution key: [ 0, 14, 15 ]
+ └── read pk prefix len hint: 3
- id: nexmark_q21
before:
@@ -1375,61 +1388,65 @@
) b ON a.id = b.auction;
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] }
- ├─StreamExchange { dist: HashShard($expr2) }
- │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
- │ └─StreamFilter { predicate: (event_type = 1:Int32) }
- │ └─StreamShare { id: 5 }
- │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamProject { exprs: [$expr5, max($expr6)] }
- └─StreamHashAgg { group_key: [$expr5], aggs: [max($expr6), count] }
- └─StreamExchange { dist: HashShard($expr5) }
- └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6, _row_id] }
- └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true }
- ├─StreamFilter { predicate: (event_type = 2:Int32) }
- │ └─StreamShare { id: 5 }
- │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamExchange { dist: Broadcast }
- └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] }
- └─StreamNow { output: [now] }
+ └─StreamExchange { dist: HashShard($expr2, _row_id) }
+ └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] }
+ ├─StreamExchange { dist: HashShard($expr2) }
+ │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
+ │ └─StreamFilter { predicate: (event_type = 1:Int32) }
+ │ └─StreamShare { id: 5 }
+ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamProject { exprs: [$expr5, max($expr6)] }
+ └─StreamHashAgg { group_key: [$expr5], aggs: [max($expr6), count] }
+ └─StreamExchange { dist: HashShard($expr5) }
+ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6, _row_id] }
+ └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true }
+ ├─StreamFilter { predicate: (event_type = 2:Int32) }
+ │ └─StreamShare { id: 5 }
+ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: Broadcast }
+ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] }
+ └─StreamNow { output: [now] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [$expr5, max($expr6)] }
- └── StreamHashAgg { group_key: [$expr5], aggs: [max($expr6), count] } { intermediate state table: 6, state tables: [ 5 ], distinct tables: [] }
- └── StreamExchange Hash([0]) from 3
+ └── StreamExchange Hash([0, 3]) from 1
Fragment 1
+ StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [$expr5, max($expr6)] }
+ └── StreamHashAgg { group_key: [$expr5], aggs: [max($expr6), count] } { intermediate state table: 6, state tables: [ 5 ], distinct tables: [] }
+ └── StreamExchange Hash([0]) from 4
+
+ Fragment 2
StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
└── StreamFilter { predicate: (event_type = 1:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
- Fragment 2
+ Fragment 3
StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
└── StreamRowIdGen { row_id_index: 5 }
└── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 4 }
- Fragment 3
+ Fragment 4
StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6, _row_id] }
└── StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } { left table: 7, right table: 8 }
├── StreamFilter { predicate: (event_type = 2:Int32) }
- │ └── StreamExchange NoShuffle from 2
- └── StreamExchange Broadcast from 4
+ │ └── StreamExchange NoShuffle from 3
+ └── StreamExchange Broadcast from 5
- Fragment 4
+ Fragment 5
StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] }
└── StreamNow { output: [now] } { state table: 9 }
@@ -1457,7 +1474,7 @@
├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, $expr5 ]
├── primary key: [ $3 ASC, $0 ASC ]
├── value indices: [ 0, 1, 2, 3, 4 ]
- ├── distribution key: [ 0 ]
+ ├── distribution key: [ 0, 3 ]
└── read pk prefix len hint: 2
- id: nexmark_q102
@@ -1642,65 +1659,69 @@
);
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr5, output: all }
- ├─StreamExchange { dist: HashShard($expr2) }
- │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
- │ └─StreamFilter { predicate: (event_type = 1:Int32) }
- │ └─StreamShare { id: 5 }
- │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamProject { exprs: [$expr5] }
- └─StreamFilter { predicate: (count >= 20:Int32) }
- └─StreamHashAgg { group_key: [$expr5], aggs: [count] }
- └─StreamExchange { dist: HashShard($expr5) }
- └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] }
- └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true }
- ├─StreamFilter { predicate: (event_type = 2:Int32) }
- │ └─StreamShare { id: 5 }
- │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamExchange { dist: Broadcast }
- └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] }
- └─StreamNow { output: [now] }
+ └─StreamExchange { dist: HashShard($expr2, _row_id) }
+ └─StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr5, output: all }
+ ├─StreamExchange { dist: HashShard($expr2) }
+ │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
+ │ └─StreamFilter { predicate: (event_type = 1:Int32) }
+ │ └─StreamShare { id: 5 }
+ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamProject { exprs: [$expr5] }
+ └─StreamFilter { predicate: (count >= 20:Int32) }
+ └─StreamHashAgg { group_key: [$expr5], aggs: [count] }
+ └─StreamExchange { dist: HashShard($expr5) }
+ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] }
+ └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true }
+ ├─StreamFilter { predicate: (event_type = 2:Int32) }
+ │ └─StreamShare { id: 5 }
+ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: Broadcast }
+ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] }
+ └─StreamNow { output: [now] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr5, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [$expr5] }
- └── StreamFilter { predicate: (count >= 20:Int32) }
- └── StreamHashAgg { group_key: [$expr5], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0]) from 3
+ └── StreamExchange Hash([0, 2]) from 1
Fragment 1
+ StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr5, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [$expr5] }
+ └── StreamFilter { predicate: (count >= 20:Int32) }
+ └── StreamHashAgg { group_key: [$expr5], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
+ └── StreamExchange Hash([0]) from 4
+
+ Fragment 2
StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
└── StreamFilter { predicate: (event_type = 1:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
- Fragment 2
+ Fragment 3
StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
└── StreamRowIdGen { row_id_index: 5 }
└── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 4 }
- Fragment 3
+ Fragment 4
StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] }
└── StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true }
├── left table: 6
├── right table: 7
├── StreamFilter { predicate: (event_type = 2:Int32) }
- │ └── StreamExchange NoShuffle from 2
- └── StreamExchange Broadcast from 4
+ │ └── StreamExchange NoShuffle from 3
+ └── StreamExchange Broadcast from 5
- Fragment 4
+ Fragment 5
StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] }
└── StreamNow { output: [now] } { state table: 8 }
@@ -1731,7 +1752,7 @@
├── columns: [ auction_id, auction_item_name, _row_id ]
├── primary key: [ $2 ASC, $0 ASC ]
├── value indices: [ 0, 1, 2 ]
- ├── distribution key: [ 0 ]
+ ├── distribution key: [ 0, 2 ]
└── read pk prefix len hint: 2
- id: nexmark_q104
@@ -1752,65 +1773,69 @@
);
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr5, output: all }
- ├─StreamExchange { dist: HashShard($expr2) }
- │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
- │ └─StreamFilter { predicate: (event_type = 1:Int32) }
- │ └─StreamShare { id: 5 }
- │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamProject { exprs: [$expr5] }
- └─StreamFilter { predicate: (count < 20:Int32) }
- └─StreamHashAgg { group_key: [$expr5], aggs: [count] }
- └─StreamExchange { dist: HashShard($expr5) }
- └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] }
- └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true }
- ├─StreamFilter { predicate: (event_type = 2:Int32) }
- │ └─StreamShare { id: 5 }
- │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamExchange { dist: Broadcast }
- └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] }
- └─StreamNow { output: [now] }
+ └─StreamExchange { dist: HashShard($expr2, _row_id) }
+ └─StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr5, output: all }
+ ├─StreamExchange { dist: HashShard($expr2) }
+ │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
+ │ └─StreamFilter { predicate: (event_type = 1:Int32) }
+ │ └─StreamShare { id: 5 }
+ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamProject { exprs: [$expr5] }
+ └─StreamFilter { predicate: (count < 20:Int32) }
+ └─StreamHashAgg { group_key: [$expr5], aggs: [count] }
+ └─StreamExchange { dist: HashShard($expr5) }
+ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] }
+ └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true }
+ ├─StreamFilter { predicate: (event_type = 2:Int32) }
+ │ └─StreamShare { id: 5 }
+ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: Broadcast }
+ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] }
+ └─StreamNow { output: [now] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr5, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [$expr5] }
- └── StreamFilter { predicate: (count < 20:Int32) }
- └── StreamHashAgg { group_key: [$expr5], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0]) from 3
+ └── StreamExchange Hash([0, 2]) from 1
Fragment 1
+ StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr5, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [$expr5] }
+ └── StreamFilter { predicate: (count < 20:Int32) }
+ └── StreamHashAgg { group_key: [$expr5], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] }
+ └── StreamExchange Hash([0]) from 4
+
+ Fragment 2
StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
└── StreamFilter { predicate: (event_type = 1:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
- Fragment 2
+ Fragment 3
StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
└── StreamRowIdGen { row_id_index: 5 }
└── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 4 }
- Fragment 3
+ Fragment 4
StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] }
└── StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true }
├── left table: 6
├── right table: 7
├── StreamFilter { predicate: (event_type = 2:Int32) }
- │ └── StreamExchange NoShuffle from 2
- └── StreamExchange Broadcast from 4
+ │ └── StreamExchange NoShuffle from 3
+ └── StreamExchange Broadcast from 5
- Fragment 4
+ Fragment 5
StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] }
└── StreamNow { output: [now] } { state table: 8 }
@@ -1841,7 +1866,7 @@
├── columns: [ auction_id, auction_item_name, _row_id ]
├── primary key: [ $2 ASC, $0 ASC ]
├── value indices: [ 0, 1, 2 ]
- ├── distribution key: [ 0 ]
+ ├── distribution key: [ 0, 2 ]
└── read pk prefix len hint: 2
- id: nexmark_q105
diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml
index 3554e31d281ec..39adc39a16653 100644
--- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml
@@ -131,45 +131,49 @@
└─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), $expr3(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, $expr3], pk_columns: [_row_id, _row_id#1, $expr3], pk_conflict: NoCheck }
- └─StreamHashJoin [append_only] { type: Inner, predicate: $expr3 = $expr4, output: [$expr5, $expr6, $expr7, $expr2, _row_id, $expr3, _row_id] }
- ├─StreamExchange { dist: HashShard($expr3) }
- │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 7:Int32) as $expr3, _row_id] }
- │ └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) }
- │ └─StreamShare { id: 6 }
- │ └─StreamProject { exprs: [event_type, person, auction, _row_id] }
- │ └─StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32))) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamExchange { dist: HashShard($expr4) }
- └─StreamProject { exprs: [Field(person, 0:Int32) as $expr4, Field(person, 1:Int32) as $expr5, Field(person, 4:Int32) as $expr6, Field(person, 5:Int32) as $expr7, _row_id] }
- └─StreamFilter { predicate: (((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32) }
- └─StreamShare { id: 6 }
- └─StreamProject { exprs: [event_type, person, auction, _row_id] }
- └─StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32))) }
- └─StreamRowIdGen { row_id_index: 5 }
- └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: HashShard(_row_id, $expr3, _row_id) }
+ └─StreamHashJoin [append_only] { type: Inner, predicate: $expr3 = $expr4, output: [$expr5, $expr6, $expr7, $expr2, _row_id, $expr3, _row_id] }
+ ├─StreamExchange { dist: HashShard($expr3) }
+ │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 7:Int32) as $expr3, _row_id] }
+ │ └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) }
+ │ └─StreamShare { id: 6 }
+ │ └─StreamProject { exprs: [event_type, person, auction, _row_id] }
+ │ └─StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32))) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: HashShard($expr4) }
+ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr4, Field(person, 1:Int32) as $expr5, Field(person, 4:Int32) as $expr6, Field(person, 5:Int32) as $expr7, _row_id] }
+ └─StreamFilter { predicate: (((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32) }
+ └─StreamShare { id: 6 }
+ └─StreamProject { exprs: [event_type, person, auction, _row_id] }
+ └─StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32))) }
+ └─StreamRowIdGen { row_id_index: 5 }
+ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), $expr3(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, $expr3], pk_columns: [_row_id, _row_id#1, $expr3], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin [append_only] { type: Inner, predicate: $expr3 = $expr4, output: [$expr5, $expr6, $expr7, $expr2, _row_id, $expr3, _row_id] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([1]) from 1
- └── StreamExchange Hash([0]) from 3
+ └── StreamExchange Hash([4, 5, 6]) from 1
Fragment 1
+ StreamHashJoin [append_only] { type: Inner, predicate: $expr3 = $expr4, output: [$expr5, $expr6, $expr7, $expr2, _row_id, $expr3, _row_id] }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamExchange Hash([1]) from 2
+ └── StreamExchange Hash([0]) from 4
+
+ Fragment 2
StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 7:Int32) as $expr3, _row_id] }
└── StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
- Fragment 2
+ Fragment 3
StreamProject { exprs: [event_type, person, auction, _row_id] }
└── StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32))) }
└── StreamRowIdGen { row_id_index: 5 }
@@ -177,10 +181,10 @@
└── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
└── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 }
- Fragment 3
+ Fragment 4
StreamProject { exprs: [Field(person, 0:Int32) as $expr4, Field(person, 1:Int32) as $expr5, Field(person, 4:Int32) as $expr6, Field(person, 5:Int32) as $expr7, _row_id] }
└── StreamFilter { predicate: (((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 }
@@ -194,7 +198,7 @@
Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 }
- Table 4294967294 { columns: [ name, city, state, id, _row_id, $expr3, _row_id#1 ], primary key: [ $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 5 ], read pk prefix len hint: 3 }
+ Table 4294967294 { columns: [ name, city, state, id, _row_id, $expr3, _row_id#1 ], primary key: [ $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 4, 5, 6 ], read pk prefix len hint: 3 }
eowc_stream_error: |-
Not supported: The query cannot be executed in Emit-On-Window-Close mode.
@@ -696,43 +700,48 @@
└─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time, $expr5(hidden)] }
- └─StreamHashJoin [interval] { type: Inner, predicate: $expr4 = max($expr4) AND ($expr1 >= $expr6) AND ($expr1 <= $expr5), conditions_to_clean_left_state_table: ($expr1 >= $expr6), conditions_to_clean_right_state_table: ($expr1 <= $expr5), output_watermarks: [$expr1, $expr5], output: [$expr2, $expr4, $expr3, $expr1, _row_id, $expr5] }
- ├─StreamExchange { dist: HashShard($expr4) }
- │ └─StreamShare { id: 6 }
- │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamFilter { predicate: (event_type = 2:Int32) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamExchange { dist: HashShard(max($expr4)) }
- └─StreamProject { exprs: [$expr5, max($expr4), ($expr5 - '00:00:10':Interval) as $expr6], output_watermarks: [$expr5, $expr6] }
- └─StreamHashAgg [append_only] { group_key: [$expr5], aggs: [max($expr4), count], output_watermarks: [$expr5] }
- └─StreamExchange { dist: HashShard($expr5) }
- └─StreamProject { exprs: [(TumbleStart($expr1, '00:00:10':Interval) + '00:00:10':Interval) as $expr5, $expr4, _row_id], output_watermarks: [$expr5] }
- └─StreamShare { id: 6 }
- └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] }
- └─StreamFilter { predicate: (event_type = 2:Int32) }
- └─StreamRowIdGen { row_id_index: 5 }
- └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: HashShard($expr4, _row_id, $expr5) }
+ └─StreamHashJoin [interval] { type: Inner, predicate: $expr4 = max($expr4) AND ($expr1 >= $expr6) AND ($expr1 <= $expr5), conditions_to_clean_left_state_table: ($expr1 >= $expr6), conditions_to_clean_right_state_table: ($expr1 <= $expr5), output_watermarks: [$expr1, $expr5], output: [$expr2, $expr4, $expr3, $expr1, _row_id, $expr5] }
+ ├─StreamExchange { dist: HashShard($expr4) }
+ │ └─StreamShare { id: 6 }
+ │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamFilter { predicate: (event_type = 2:Int32) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: HashShard(max($expr4)) }
+ └─StreamProject { exprs: [$expr5, max($expr4), ($expr5 - '00:00:10':Interval) as $expr6], output_watermarks: [$expr5, $expr6] }
+ └─StreamHashAgg [append_only] { group_key: [$expr5], aggs: [max($expr4), count], output_watermarks: [$expr5] }
+ └─StreamExchange { dist: HashShard($expr5) }
+ └─StreamProject { exprs: [(TumbleStart($expr1, '00:00:10':Interval) + '00:00:10':Interval) as $expr5, $expr4, _row_id], output_watermarks: [$expr5] }
+ └─StreamShare { id: 6 }
+ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] }
+ └─StreamFilter { predicate: (event_type = 2:Int32) }
+ └─StreamRowIdGen { row_id_index: 5 }
+ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
stream_dist_plan: |+
Fragment 0
- StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time, $expr5(hidden)] } { materialized table: 4294967294 }
- └── StreamHashJoin [interval] { type: Inner, predicate: $expr4 = max($expr4) AND ($expr1 >= $expr6) AND ($expr1 <= $expr5), conditions_to_clean_left_state_table: ($expr1 >= $expr6), conditions_to_clean_right_state_table: ($expr1 <= $expr5), output_watermarks: [$expr1, $expr5], output: [$expr2, $expr4, $expr3, $expr1, _row_id, $expr5] }
- ├── left table: 0
- ├── right table: 2
- ├── left degree table: 1
- ├── right degree table: 3
- ├── StreamExchange Hash([2]) from 1
- └── StreamExchange Hash([1]) from 3
+ StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time, $expr5(hidden)] }
+ ├── materialized table: 4294967294
+ └── StreamExchange Hash([1, 4, 5]) from 1
Fragment 1
- StreamNoOp
- └── StreamExchange NoShuffle from 2
+ StreamHashJoin [interval] { type: Inner, predicate: $expr4 = max($expr4) AND ($expr1 >= $expr6) AND ($expr1 <= $expr5), conditions_to_clean_left_state_table: ($expr1 >= $expr6), conditions_to_clean_right_state_table: ($expr1 <= $expr5), output_watermarks: [$expr1, $expr5], output: [$expr2, $expr4, $expr3, $expr1, _row_id, $expr5] }
+ ├── left table: 0
+ ├── right table: 2
+ ├── left degree table: 1
+ ├── right degree table: 3
+ ├── StreamExchange Hash([2]) from 2
+ └── StreamExchange Hash([1]) from 4
Fragment 2
+ StreamNoOp
+ └── StreamExchange NoShuffle from 3
+
+ Fragment 3
StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamFilter { predicate: (event_type = 2:Int32) }
└── StreamRowIdGen { row_id_index: 5 }
@@ -740,14 +749,14 @@
└── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
└── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 }
- Fragment 3
+ Fragment 4
StreamProject { exprs: [$expr5, max($expr4), ($expr5 - '00:00:10':Interval) as $expr6], output_watermarks: [$expr5, $expr6] }
└── StreamHashAgg [append_only] { group_key: [$expr5], aggs: [max($expr4), count], output_watermarks: [$expr5] } { intermediate state table: 6, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0]) from 4
+ └── StreamExchange Hash([0]) from 5
- Fragment 4
+ Fragment 5
StreamProject { exprs: [(TumbleStart($expr1, '00:00:10':Interval) + '00:00:10':Interval) as $expr5, $expr4, _row_id], output_watermarks: [$expr5] }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
Table 0 { columns: [ $expr2, $expr3, $expr4, $expr1, _row_id ], primary key: [ $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 }
@@ -763,7 +772,7 @@
Table 6 { columns: [ $expr5, max($expr4), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
- Table 4294967294 { columns: [ auction, price, bidder, date_time, _row_id, $expr5 ], primary key: [ $4 ASC, $5 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1 ], read pk prefix len hint: 3 }
+ Table 4294967294 { columns: [ auction, price, bidder, date_time, _row_id, $expr5 ], primary key: [ $4 ASC, $5 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1, 4, 5 ], read pk prefix len hint: 3 }
eowc_stream_plan: |-
StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time] }
@@ -845,52 +854,56 @@
└─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime, $expr5(hidden), $expr6(hidden), $expr8(hidden)] }
- └─StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all }
- ├─StreamExchange { dist: HashShard($expr3, $expr2, $expr5) }
- │ └─StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] }
- │ └─StreamExchange { dist: HashShard($expr3, $expr4, $expr2, $expr5) }
- │ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, $expr2, ($expr2 + '00:00:10':Interval) as $expr5], output_watermarks: [$expr2, $expr5] }
- │ └─StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr2, _row_id], output_watermarks: [$expr1, $expr2] }
- │ └─StreamFilter { predicate: (event_type = 0:Int32) }
- │ └─StreamShare { id: 6 }
- │ └─StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamAppendOnlyDedup { dedup_cols: [$expr7, $expr6, $expr8] }
- └─StreamExchange { dist: HashShard($expr7, $expr6, $expr8) }
- └─StreamProject { exprs: [Field(auction, 7:Int32) as $expr7, $expr6, ($expr6 + '00:00:10':Interval) as $expr8], output_watermarks: [$expr6, $expr8] }
- └─StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr6, _row_id], output_watermarks: [$expr1, $expr6] }
- └─StreamFilter { predicate: (event_type = 1:Int32) }
- └─StreamShare { id: 6 }
- └─StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] }
- └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) }
- └─StreamRowIdGen { row_id_index: 5 }
- └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: HashShard($expr3, $expr4, $expr2, $expr5) }
+ └─StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all }
+ ├─StreamExchange { dist: HashShard($expr3, $expr2, $expr5) }
+ │ └─StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] }
+ │ └─StreamExchange { dist: HashShard($expr3, $expr4, $expr2, $expr5) }
+ │ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, $expr2, ($expr2 + '00:00:10':Interval) as $expr5], output_watermarks: [$expr2, $expr5] }
+ │ └─StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr2, _row_id], output_watermarks: [$expr1, $expr2] }
+ │ └─StreamFilter { predicate: (event_type = 0:Int32) }
+ │ └─StreamShare { id: 6 }
+ │ └─StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamAppendOnlyDedup { dedup_cols: [$expr7, $expr6, $expr8] }
+ └─StreamExchange { dist: HashShard($expr7, $expr6, $expr8) }
+ └─StreamProject { exprs: [Field(auction, 7:Int32) as $expr7, $expr6, ($expr6 + '00:00:10':Interval) as $expr8], output_watermarks: [$expr6, $expr8] }
+ └─StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr6, _row_id], output_watermarks: [$expr1, $expr6] }
+ └─StreamFilter { predicate: (event_type = 1:Int32) }
+ └─StreamShare { id: 6 }
+ └─StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] }
+ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) }
+ └─StreamRowIdGen { row_id_index: 5 }
+ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime, $expr5(hidden), $expr6(hidden), $expr8(hidden)] }
├── materialized table: 4294967294
- └── StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0, 2, 3]) from 1
- └── StreamAppendOnlyDedup { dedup_cols: [$expr7, $expr6, $expr8] } { state table: 7 }
- └── StreamExchange Hash([0, 1, 2]) from 4
+ └── StreamExchange Hash([0, 1, 2, 3]) from 1
Fragment 1
- StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] } { state table: 4 }
- └── StreamExchange Hash([0, 1, 2, 3]) from 2
+ StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0, 2, 3]) from 2
+ └── StreamAppendOnlyDedup { dedup_cols: [$expr7, $expr6, $expr8] } { state table: 7 }
+ └── StreamExchange Hash([0, 1, 2]) from 5
Fragment 2
+ StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] } { state table: 4 }
+ └── StreamExchange Hash([0, 1, 2, 3]) from 3
+
+ Fragment 3
StreamProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, $expr2, ($expr2 + '00:00:10':Interval) as $expr5], output_watermarks: [$expr2, $expr5] }
└── StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr2, _row_id], output_watermarks: [$expr1, $expr2] }
└── StreamFilter { predicate: (event_type = 0:Int32) }
- └── StreamExchange NoShuffle from 3
+ └── StreamExchange NoShuffle from 4
- Fragment 3
+ Fragment 4
StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) }
└── StreamRowIdGen { row_id_index: 5 }
@@ -898,11 +911,11 @@
└── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
└── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 6 }
- Fragment 4
+ Fragment 5
StreamProject { exprs: [Field(auction, 7:Int32) as $expr7, $expr6, ($expr6 + '00:00:10':Interval) as $expr8], output_watermarks: [$expr6, $expr8] }
└── StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr6, _row_id], output_watermarks: [$expr1, $expr6] }
└── StreamFilter { predicate: (event_type = 1:Int32) }
- └── StreamExchange NoShuffle from 3
+ └── StreamExchange NoShuffle from 4
Table 0 { columns: [ $expr3, $expr4, $expr2, $expr5 ], primary key: [ $2 ASC, $3 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 }
@@ -920,7 +933,7 @@
Table 7 { columns: [ $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 }
- Table 4294967294 { columns: [ id, name, starttime, $expr5, $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 4 }
+ Table 4294967294 { columns: [ id, name, starttime, $expr5, $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 }
eowc_stream_plan: |-
StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime] }
@@ -1715,41 +1728,45 @@
└─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck }
- └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr7, output: [$expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _row_id] }
- ├─StreamExchange { dist: HashShard($expr2) }
- │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamFilter { predicate: (event_type = 2:Int32) }
- │ └─StreamShare { id: 6 }
- │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
- │ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamExchange { dist: HashShard($expr7) }
- └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr7, Field(auction, 1:Int32) as $expr8, Field(auction, 2:Int32) as $expr9, Field(auction, 3:Int32) as $expr10, Field(auction, 4:Int32) as $expr11, $expr1, Field(auction, 6:Int32) as $expr12, Field(auction, 7:Int32) as $expr13, Field(auction, 8:Int32) as $expr14, _row_id], output_watermarks: [$expr1] }
- └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) }
- └─StreamShare { id: 6 }
- └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
- └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) }
- └─StreamRowIdGen { row_id_index: 5 }
- └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: HashShard($expr2, _row_id, _row_id) }
+ └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr7, output: [$expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _row_id] }
+ ├─StreamExchange { dist: HashShard($expr2) }
+ │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamFilter { predicate: (event_type = 2:Int32) }
+ │ └─StreamShare { id: 6 }
+ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
+ │ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: HashShard($expr7) }
+ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr7, Field(auction, 1:Int32) as $expr8, Field(auction, 2:Int32) as $expr9, Field(auction, 3:Int32) as $expr10, Field(auction, 4:Int32) as $expr11, $expr1, Field(auction, 6:Int32) as $expr12, Field(auction, 7:Int32) as $expr13, Field(auction, 8:Int32) as $expr14, _row_id], output_watermarks: [$expr1] }
+ └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) }
+ └─StreamShare { id: 6 }
+ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
+ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) }
+ └─StreamRowIdGen { row_id_index: 5 }
+ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr7, output: [$expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamExchange Hash([0]) from 3
+ └── StreamExchange Hash([0, 14, 15]) from 1
Fragment 1
+ StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr7, output: [$expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0]) from 4
+
+ Fragment 2
StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamFilter { predicate: (event_type = 2:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
- Fragment 2
+ Fragment 3
StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] }
└── StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) }
└── StreamRowIdGen { row_id_index: 5 }
@@ -1757,10 +1774,10 @@
└── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
└── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 }
- Fragment 3
+ Fragment 4
StreamProject { exprs: [Field(auction, 0:Int32) as $expr7, Field(auction, 1:Int32) as $expr8, Field(auction, 2:Int32) as $expr9, Field(auction, 3:Int32) as $expr10, Field(auction, 4:Int32) as $expr11, $expr1, Field(auction, 6:Int32) as $expr12, Field(auction, 7:Int32) as $expr13, Field(auction, 8:Int32) as $expr14, _row_id], output_watermarks: [$expr1] }
└── StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, _row_id ], primary key: [ $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
@@ -1774,7 +1791,12 @@
Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 }
- Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ], primary key: [ $14 ASC, $15 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ], distribution key: [ 0 ], read pk prefix len hint: 3 }
+ Table 4294967294
+ ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ]
+ ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ]
+ ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ]
+ ├── distribution key: [ 0, 14, 15 ]
+ └── read pk prefix len hint: 3
eowc_stream_error: |-
Not supported: The query cannot be executed in Emit-On-Window-Close mode.
@@ -1909,45 +1931,49 @@
└─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr4(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] }
- ├─StreamExchange { dist: HashShard($expr2) }
- │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
- │ └─StreamFilter { predicate: (event_type = 1:Int32) }
- │ └─StreamShare { id: 6 }
- │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] }
- │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamProject { exprs: [$expr4, max($expr5)] }
- └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [max($expr5), count] }
- └─StreamExchange { dist: HashShard($expr4) }
- └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, _row_id] }
- └─StreamFilter { predicate: (event_type = 2:Int32) }
- └─StreamShare { id: 6 }
- └─StreamProject { exprs: [event_type, auction, bid, _row_id] }
- └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
- └─StreamRowIdGen { row_id_index: 5 }
- └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: HashShard($expr2, _row_id) }
+ └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] }
+ ├─StreamExchange { dist: HashShard($expr2) }
+ │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
+ │ └─StreamFilter { predicate: (event_type = 1:Int32) }
+ │ └─StreamShare { id: 6 }
+ │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] }
+ │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamProject { exprs: [$expr4, max($expr5)] }
+ └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [max($expr5), count] }
+ └─StreamExchange { dist: HashShard($expr4) }
+ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, _row_id] }
+ └─StreamFilter { predicate: (event_type = 2:Int32) }
+ └─StreamShare { id: 6 }
+ └─StreamProject { exprs: [event_type, auction, bid, _row_id] }
+ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
+ └─StreamRowIdGen { row_id_index: 5 }
+ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr4(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [$expr4, max($expr5)] }
- └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [max($expr5), count] } { intermediate state table: 6, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0]) from 3
+ └── StreamExchange Hash([0, 3]) from 1
Fragment 1
+ StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [$expr4, max($expr5)] }
+ └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [max($expr5), count] } { intermediate state table: 6, state tables: [], distinct tables: [] }
+ └── StreamExchange Hash([0]) from 4
+
+ Fragment 2
StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
└── StreamFilter { predicate: (event_type = 1:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
- Fragment 2
+ Fragment 3
StreamProject { exprs: [event_type, auction, bid, _row_id] }
└── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
└── StreamRowIdGen { row_id_index: 5 }
@@ -1955,10 +1981,10 @@
└── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
└── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 }
- Fragment 3
+ Fragment 4
StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, _row_id] }
└── StreamFilter { predicate: (event_type = 2:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
@@ -1978,7 +2004,7 @@
├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, $expr4 ]
├── primary key: [ $3 ASC, $0 ASC ]
├── value indices: [ 0, 1, 2, 3, 4 ]
- ├── distribution key: [ 0 ]
+ ├── distribution key: [ 0, 3 ]
└── read pk prefix len hint: 2
eowc_stream_error: |-
@@ -2184,47 +2210,51 @@
└─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr4, output: all }
- ├─StreamExchange { dist: HashShard($expr2) }
- │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
- │ └─StreamFilter { predicate: (event_type = 1:Int32) }
- │ └─StreamShare { id: 6 }
- │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] }
- │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamProject { exprs: [$expr4] }
- └─StreamFilter { predicate: (count >= 20:Int32) }
- └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] }
- └─StreamExchange { dist: HashShard($expr4) }
- └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] }
- └─StreamFilter { predicate: (event_type = 2:Int32) }
- └─StreamShare { id: 6 }
- └─StreamProject { exprs: [event_type, auction, bid, _row_id] }
- └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
- └─StreamRowIdGen { row_id_index: 5 }
- └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: HashShard($expr2, _row_id) }
+ └─StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr4, output: all }
+ ├─StreamExchange { dist: HashShard($expr2) }
+ │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
+ │ └─StreamFilter { predicate: (event_type = 1:Int32) }
+ │ └─StreamShare { id: 6 }
+ │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] }
+ │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamProject { exprs: [$expr4] }
+ └─StreamFilter { predicate: (count >= 20:Int32) }
+ └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] }
+ └─StreamExchange { dist: HashShard($expr4) }
+ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] }
+ └─StreamFilter { predicate: (event_type = 2:Int32) }
+ └─StreamShare { id: 6 }
+ └─StreamProject { exprs: [event_type, auction, bid, _row_id] }
+ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
+ └─StreamRowIdGen { row_id_index: 5 }
+ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [$expr4] }
- └── StreamFilter { predicate: (count >= 20:Int32) }
- └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0]) from 3
+ └── StreamExchange Hash([0, 2]) from 1
Fragment 1
+ StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [$expr4] }
+ └── StreamFilter { predicate: (count >= 20:Int32) }
+ └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] }
+ └── StreamExchange Hash([0]) from 4
+
+ Fragment 2
StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
└── StreamFilter { predicate: (event_type = 1:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
- Fragment 2
+ Fragment 3
StreamProject { exprs: [event_type, auction, bid, _row_id] }
└── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
└── StreamRowIdGen { row_id_index: 5 }
@@ -2232,10 +2262,10 @@
└── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
└── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 }
- Fragment 3
+ Fragment 4
StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] }
└── StreamFilter { predicate: (event_type = 2:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
@@ -2251,7 +2281,7 @@
Table 6 { columns: [ $expr4, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
- Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 }
+ Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 2 }
eowc_stream_error: |-
Not supported: The query cannot be executed in Emit-On-Window-Close mode.
@@ -2290,47 +2320,51 @@
└─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr4, output: all }
- ├─StreamExchange { dist: HashShard($expr2) }
- │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
- │ └─StreamFilter { predicate: (event_type = 1:Int32) }
- │ └─StreamShare { id: 6 }
- │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] }
- │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
- │ └─StreamRowIdGen { row_id_index: 5 }
- │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
- └─StreamProject { exprs: [$expr4] }
- └─StreamFilter { predicate: (count < 20:Int32) }
- └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] }
- └─StreamExchange { dist: HashShard($expr4) }
- └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] }
- └─StreamFilter { predicate: (event_type = 2:Int32) }
- └─StreamShare { id: 6 }
- └─StreamProject { exprs: [event_type, auction, bid, _row_id] }
- └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
- └─StreamRowIdGen { row_id_index: 5 }
- └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
- └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
- └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamExchange { dist: HashShard($expr2, _row_id) }
+ └─StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr4, output: all }
+ ├─StreamExchange { dist: HashShard($expr2) }
+ │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
+ │ └─StreamFilter { predicate: (event_type = 1:Int32) }
+ │ └─StreamShare { id: 6 }
+ │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] }
+ │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
+ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
+ └─StreamProject { exprs: [$expr4] }
+ └─StreamFilter { predicate: (count < 20:Int32) }
+ └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] }
+ └─StreamExchange { dist: HashShard($expr4) }
+ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] }
+ └─StreamFilter { predicate: (event_type = 2:Int32) }
+ └─StreamShare { id: 6 }
+ └─StreamProject { exprs: [event_type, auction, bid, _row_id] }
+ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
+ └─StreamRowIdGen { row_id_index: 5 }
+ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
+ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
+ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [$expr4] }
- └── StreamFilter { predicate: (count < 20:Int32) }
- └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] }
- └── StreamExchange Hash([0]) from 3
+ └── StreamExchange Hash([0, 2]) from 1
Fragment 1
+ StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [$expr4] }
+ └── StreamFilter { predicate: (count < 20:Int32) }
+ └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] }
+ └── StreamExchange Hash([0]) from 4
+
+ Fragment 2
StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] }
└── StreamFilter { predicate: (event_type = 1:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
- Fragment 2
+ Fragment 3
StreamProject { exprs: [event_type, auction, bid, _row_id] }
└── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) }
└── StreamRowIdGen { row_id_index: 5 }
@@ -2338,10 +2372,10 @@
└── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
└── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 }
- Fragment 3
+ Fragment 4
StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] }
└── StreamFilter { predicate: (event_type = 2:Int32) }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
@@ -2357,7 +2391,7 @@
Table 6 { columns: [ $expr4, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
- Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 }
+ Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 2 }
eowc_stream_error: |-
Not supported: The query cannot be executed in Emit-On-Window-Close mode.
diff --git a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml
index 5cc81578f829c..733a19f4ba05c 100644
--- a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml
@@ -185,15 +185,16 @@
└─BatchScan { table: t, columns: [t.x, t.y, t.w], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [x, y, sum, max, min, t._row_id(hidden), t.y(hidden)], stream_key: [t._row_id, y], pk_columns: [t._row_id, y], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t.y = t.y, output: [t.x, t.y, sum(t.x), max(t.x), min(t.w), t._row_id, t.y] }
- ├─StreamExchange { dist: HashShard(t.y) }
- │ └─StreamShare { id: 1 }
- │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.w, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- └─StreamProject { exprs: [t.y, sum(t.x), max(t.x), min(t.w)] }
- └─StreamHashAgg { group_key: [t.y], aggs: [sum(t.x), max(t.x), min(t.w), count] }
- └─StreamExchange { dist: HashShard(t.y) }
- └─StreamShare { id: 1 }
- └─StreamTableScan { table: t, columns: [t.x, t.y, t.w, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.y, t._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t.y = t.y, output: [t.x, t.y, sum(t.x), max(t.x), min(t.w), t._row_id, t.y] }
+ ├─StreamExchange { dist: HashShard(t.y) }
+ │ └─StreamShare { id: 1 }
+ │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.w, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamProject { exprs: [t.y, sum(t.x), max(t.x), min(t.w)] }
+ └─StreamHashAgg { group_key: [t.y], aggs: [sum(t.x), max(t.x), min(t.w), count] }
+ └─StreamExchange { dist: HashShard(t.y) }
+ └─StreamShare { id: 1 }
+ └─StreamTableScan { table: t, columns: [t.x, t.y, t.w, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- id: aggregate with over clause, rows frame definition with implicit current row, without ORDER BY
sql: |
create table t(x int, y int);
@@ -913,12 +914,13 @@
└─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [t1x, t2x, t1z, t2y, t2z, t._row_id(hidden)], stream_key: [t1x, t._row_id], pk_columns: [t1x, t._row_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t.x = t.x, output: [t.x, t.x, t.z, t.y, t.z, t._row_id] }
- ├─StreamGroupTopN { order: [t.y ASC], limit: 1, offset: 0, group_key: [t.x] }
- │ └─StreamExchange { dist: HashShard(t.x) }
- │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- └─StreamExchange { dist: HashShard(t.x) }
- └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.x, t._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t.x = t.x, output: [t.x, t.x, t.z, t.y, t.z, t._row_id] }
+ ├─StreamGroupTopN { order: [t.y ASC], limit: 1, offset: 0, group_key: [t.x] }
+ │ └─StreamExchange { dist: HashShard(t.x) }
+ │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.x) }
+ └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- id: split calls with different ORDER BY or PARTITION BY
sql: |
create table t(x int, y int, z int);
diff --git a/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml b/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml
index 55131ed1614cd..65469e7754e6b 100644
--- a/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml
@@ -21,15 +21,16 @@
Tone.id = Ttwo.id;
stream_plan: |-
StreamMaterialize { columns: [max_v1, max_v2, t1.id(hidden), t2.id(hidden)], stream_key: [t1.id], pk_columns: [t1.id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t1.id = t2.id, output: [max(t1.v1), max(t2.v2), t1.id, t2.id] }
- ├─StreamProject { exprs: [t1.id, max(t1.v1)] }
- │ └─StreamHashAgg { group_key: [t1.id], aggs: [max(t1.v1), count] }
- │ └─StreamExchange { dist: HashShard(t1.id) }
- │ └─StreamTableScan { table: t1, columns: [t1.id, t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamProject { exprs: [t2.id, max(t2.v2)] }
- └─StreamHashAgg { group_key: [t2.id], aggs: [max(t2.v2), count] }
- └─StreamExchange { dist: HashShard(t2.id) }
- └─StreamTableScan { table: t2, columns: [t2.id, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.id) }
+ └─StreamHashJoin { type: Inner, predicate: t1.id = t2.id, output: [max(t1.v1), max(t2.v2), t1.id, t2.id] }
+ ├─StreamProject { exprs: [t1.id, max(t1.v1)] }
+ │ └─StreamHashAgg { group_key: [t1.id], aggs: [max(t1.v1), count] }
+ │ └─StreamExchange { dist: HashShard(t1.id) }
+ │ └─StreamTableScan { table: t1, columns: [t1.id, t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamProject { exprs: [t2.id, max(t2.v2)] }
+ └─StreamHashAgg { group_key: [t2.id], aggs: [max(t2.v2), count] }
+ └─StreamExchange { dist: HashShard(t2.id) }
+ └─StreamTableScan { table: t2, columns: [t2.id, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- sql: |
create table t (id int, v int);
SELECT Tone.max_v, Ttwo.min_v
@@ -51,15 +52,16 @@
Tone.id = Ttwo.id;
stream_plan: |-
StreamMaterialize { columns: [max_v, min_v, t.id(hidden), t.id#1(hidden)], stream_key: [t.id], pk_columns: [t.id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t.id = t.id, output: [max(t.v), min(t.v), t.id, t.id] }
- ├─StreamProject { exprs: [t.id, max(t.v)] }
- │ └─StreamHashAgg { group_key: [t.id], aggs: [max(t.v), count] }
- │ └─StreamExchange { dist: HashShard(t.id) }
- │ └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- └─StreamProject { exprs: [t.id, min(t.v)] }
- └─StreamHashAgg { group_key: [t.id], aggs: [min(t.v), count] }
- └─StreamExchange { dist: HashShard(t.id) }
- └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.id) }
+ └─StreamHashJoin { type: Inner, predicate: t.id = t.id, output: [max(t.v), min(t.v), t.id, t.id] }
+ ├─StreamProject { exprs: [t.id, max(t.v)] }
+ │ └─StreamHashAgg { group_key: [t.id], aggs: [max(t.v), count] }
+ │ └─StreamExchange { dist: HashShard(t.id) }
+ │ └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamProject { exprs: [t.id, min(t.v)] }
+ └─StreamHashAgg { group_key: [t.id], aggs: [min(t.v), count] }
+ └─StreamExchange { dist: HashShard(t.id) }
+ └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- sql: |
create table t (v1 varchar, v2 varchar, v3 varchar);
select
diff --git a/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml
index ae37459ef7bed..91dff73df0e6a 100644
--- a/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml
@@ -260,15 +260,16 @@
└─LogicalScan { table: t2, columns: [t2.v2], predicate: (t2.v2 > ('2021-04-01 00:00:00+00:00':Timestamptz + '01:00:00':Interval)) }
stream_plan: |-
StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1], pk_columns: [t1._row_id, t2._row_id, v1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] }
- ├─StreamExchange { dist: HashShard(t1.v1) }
- │ └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id], cleaned_by_watermark: true }
- │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- │ └─StreamExchange { dist: Broadcast }
- │ └─StreamProject { exprs: [AddWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] }
- │ └─StreamNow { output: [now] }
- └─StreamExchange { dist: HashShard(t2.v2) }
- └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id], cleaned_by_watermark: true }
+ │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ │ └─StreamExchange { dist: Broadcast }
+ │ └─StreamProject { exprs: [AddWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] }
+ │ └─StreamNow { output: [now] }
+ └─StreamExchange { dist: HashShard(t2.v2) }
+ └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- name: now() in a complex cmp expr does not get pushed down
sql: |
create table t1(v1 timestamp with time zone);
@@ -343,14 +344,15 @@
└─LogicalScan { table: t2, columns: [t2.v2], predicate: (t2.v2 > '2021-04-01 00:00:00+00:00':Timestamptz) }
stream_plan: |-
StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1], pk_columns: [t1._row_id, t2._row_id, v1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] }
- ├─StreamExchange { dist: HashShard(t1.v1) }
- │ └─StreamDynamicFilter { predicate: (t1.v1 > now), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id], cleaned_by_watermark: true }
- │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- │ └─StreamExchange { dist: Broadcast }
- │ └─StreamNow { output: [now] }
- └─StreamExchange { dist: HashShard(t2.v2) }
- └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ └─StreamDynamicFilter { predicate: (t1.v1 > now), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id], cleaned_by_watermark: true }
+ │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ │ └─StreamExchange { dist: Broadcast }
+ │ └─StreamNow { output: [now] }
+ └─StreamExchange { dist: HashShard(t2.v2) }
+ └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- name: eq-predicate derived condition is banned for mismatching types
sql: |
create table t1(v1 int, v2 int);
diff --git a/src/frontend/planner_test/tests/testdata/output/project_set.yaml b/src/frontend/planner_test/tests/testdata/output/project_set.yaml
index 23db668a070df..676772d99d72e 100644
--- a/src/frontend/planner_test/tests/testdata/output/project_set.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/project_set.yaml
@@ -155,17 +155,18 @@
└─BatchScan { table: t, columns: [t.x], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [unnest, t._row_id(hidden), projected_row_id(hidden), t._row_id#1(hidden), projected_row_id#1(hidden)], stream_key: [t._row_id, projected_row_id, t._row_id#1, projected_row_id#1, unnest], pk_columns: [t._row_id, projected_row_id, t._row_id#1, projected_row_id#1, unnest], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: Unnest($0) = Unnest($0), output: [Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id] }
- ├─StreamExchange { dist: HashShard(Unnest($0)) }
- │ └─StreamShare { id: 3 }
- │ └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] }
- │ └─StreamProjectSet { select_list: [Unnest($0), $1] }
- │ └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- └─StreamExchange { dist: HashShard(Unnest($0)) }
- └─StreamShare { id: 3 }
- └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] }
- └─StreamProjectSet { select_list: [Unnest($0), $1] }
- └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id) }
+ └─StreamHashJoin { type: Inner, predicate: Unnest($0) = Unnest($0), output: [Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id] }
+ ├─StreamExchange { dist: HashShard(Unnest($0)) }
+ │ └─StreamShare { id: 3 }
+ │ └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] }
+ │ └─StreamProjectSet { select_list: [Unnest($0), $1] }
+ │ └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(Unnest($0)) }
+ └─StreamShare { id: 3 }
+ └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] }
+ └─StreamProjectSet { select_list: [Unnest($0), $1] }
+ └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- name: issue-10080
sql: |
with cte as (SELECT 1 as v1, unnest(array[1,2,3,4,5]) AS v2) select v1 from cte;
diff --git a/src/frontend/planner_test/tests/testdata/output/select_except.yaml b/src/frontend/planner_test/tests/testdata/output/select_except.yaml
index 2193524b7076f..ffd6da30b90bc 100644
--- a/src/frontend/planner_test/tests/testdata/output/select_except.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/select_except.yaml
@@ -34,11 +34,12 @@
└─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [v1, v3, v2, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, v1], pk_columns: [t._row_id, t._row_id#1, v1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v3, t.v2, t._row_id, t._row_id] }
- ├─StreamExchange { dist: HashShard(t.v1) }
- │ └─StreamTableScan { table: t, columns: [t.v1, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- └─StreamExchange { dist: HashShard(t.v1) }
- └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.v1, t._row_id, t._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v3, t.v2, t._row_id, t._row_id] }
+ ├─StreamExchange { dist: HashShard(t.v1) }
+ │ └─StreamTableScan { table: t, columns: [t.v1, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.v1) }
+ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- name: qualified wildcard
sql: |
create table t (v1 int, v2 int, v3 int);
@@ -52,11 +53,12 @@
└─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [v1, v2, v3, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, v1], pk_columns: [t._row_id, t._row_id#1, v1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v2, t.v3, t._row_id, t._row_id] }
- ├─StreamExchange { dist: HashShard(t.v1) }
- │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- └─StreamExchange { dist: HashShard(t.v1) }
- └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.v1, t._row_id, t._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v2, t.v3, t._row_id, t._row_id] }
+ ├─StreamExchange { dist: HashShard(t.v1) }
+ │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
+ └─StreamExchange { dist: HashShard(t.v1) }
+ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- name: except with unknown column
sql: |
create table t (v1 int, v2 int, v3 int);
diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml
index 2815b00784b1d..15404d6d863ab 100644
--- a/src/frontend/planner_test/tests/testdata/output/share.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/share.yaml
@@ -343,24 +343,25 @@
└─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [a_id, b_id, a_ts, b_ts, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, a_id], pk_columns: [_row_id, _row_id#1, a_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: id = id, output: [id, id, date_time, date_time, _row_id, _row_id] }
- ├─StreamExchange { dist: HashShard(id) }
- │ └─StreamProject { exprs: [id, date_time, _row_id] }
- │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [id, date_time, $expr1, _row_id], cleaned_by_watermark: true }
- │ ├─StreamProject { exprs: [id, date_time, AtTimeZone(date_time, 'UTC':Varchar) as $expr1, _row_id] }
- │ │ └─StreamFilter { predicate: (initial_bid = 1:Int32) }
- │ │ └─StreamShare { id: 4 }
- │ │ └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] }
- │ │ └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }
- │ │ └─StreamRowIdGen { row_id_index: 10 }
- │ │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
- │ └─StreamExchange { dist: Broadcast }
- │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:00:01':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] }
- │ └─StreamNow { output: [now] }
- └─StreamExchange { dist: HashShard(id) }
- └─StreamFilter { predicate: (initial_bid = 2:Int32) }
- └─StreamShare { id: 4 }
- └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] }
- └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }
- └─StreamRowIdGen { row_id_index: 10 }
- └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
+ └─StreamExchange { dist: HashShard(id, _row_id, _row_id) }
+ └─StreamHashJoin { type: Inner, predicate: id = id, output: [id, id, date_time, date_time, _row_id, _row_id] }
+ ├─StreamExchange { dist: HashShard(id) }
+ │ └─StreamProject { exprs: [id, date_time, _row_id] }
+ │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [id, date_time, $expr1, _row_id], cleaned_by_watermark: true }
+ │ ├─StreamProject { exprs: [id, date_time, AtTimeZone(date_time, 'UTC':Varchar) as $expr1, _row_id] }
+ │ │ └─StreamFilter { predicate: (initial_bid = 1:Int32) }
+ │ │ └─StreamShare { id: 4 }
+ │ │ └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] }
+ │ │ └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }
+ │ │ └─StreamRowIdGen { row_id_index: 10 }
+ │ │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
+ │ └─StreamExchange { dist: Broadcast }
+ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:00:01':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] }
+ │ └─StreamNow { output: [now] }
+ └─StreamExchange { dist: HashShard(id) }
+ └─StreamFilter { predicate: (initial_bid = 2:Int32) }
+ └─StreamShare { id: 4 }
+ └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] }
+ └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }
+ └─StreamRowIdGen { row_id_index: 10 }
+ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] }
diff --git a/src/frontend/planner_test/tests/testdata/output/shared_views.yaml b/src/frontend/planner_test/tests/testdata/output/shared_views.yaml
index 775812f77b59c..3777705c97ced 100644
--- a/src/frontend/planner_test/tests/testdata/output/shared_views.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/shared_views.yaml
@@ -23,19 +23,20 @@
└─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] }
stream_plan: |-
StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden)], stream_key: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, z], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, z], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x] }
- ├─StreamExchange { dist: HashShard($expr1) }
- │ └─StreamShare { id: 3 }
- │ └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] }
- │ └─StreamFilter { predicate: (t1.y > 0:Int32) }
- │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard($expr2) }
- └─StreamProject { exprs: [(t1.x * $expr1) as $expr2, (t1.y * $expr1) as $expr3, t1._row_id, t1._row_id, t1.x] }
- └─StreamHashJoin { type: Inner, predicate: t1.x = $expr1, output: [t1.x, t1.y, $expr1, t1._row_id, t1._row_id] }
- ├─StreamExchange { dist: HashShard(t1.x) }
- │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard($expr1) }
- └─StreamShare { id: 3 }
- └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] }
- └─StreamFilter { predicate: (t1.y > 0:Int32) }
- └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard($expr1, t1._row_id, t1._row_id, t1._row_id, t1.x) }
+ └─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x] }
+ ├─StreamExchange { dist: HashShard($expr1) }
+ │ └─StreamShare { id: 3 }
+ │ └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] }
+ │ └─StreamFilter { predicate: (t1.y > 0:Int32) }
+ │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard($expr2) }
+ └─StreamProject { exprs: [(t1.x * $expr1) as $expr2, (t1.y * $expr1) as $expr3, t1._row_id, t1._row_id, t1.x] }
+ └─StreamHashJoin { type: Inner, predicate: t1.x = $expr1, output: [t1.x, t1.y, $expr1, t1._row_id, t1._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.x) }
+ │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard($expr1) }
+ └─StreamShare { id: 3 }
+ └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] }
+ └─StreamFilter { predicate: (t1.y > 0:Int32) }
+ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml
index 003e4185ada81..e07e84e040929 100644
--- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml
@@ -346,21 +346,22 @@
└─BatchScan { table: auction, columns: [auction.date_time], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [date_time, window_start, window_end, auction._row_id(hidden)], stream_key: [auction._row_id, window_start, window_end, date_time], pk_columns: [auction._row_id, window_start, window_end, date_time], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: auction.date_time IS NOT DISTINCT FROM auction.date_time, output: all }
- ├─StreamExchange { dist: HashShard(auction.date_time) }
- │ └─StreamShare { id: 3 }
- │ └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] }
- │ └─StreamFilter { predicate: IsNotNull(auction.date_time) }
- │ └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], pk: [auction._row_id], dist: UpstreamHashShard(auction._row_id) }
- └─StreamProject { exprs: [auction.date_time] }
- └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] }
- └─StreamProject { exprs: [auction.date_time] }
- └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] }
- └─StreamExchange { dist: HashShard(auction.date_time) }
- └─StreamShare { id: 3 }
- └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] }
- └─StreamFilter { predicate: IsNotNull(auction.date_time) }
- └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], pk: [auction._row_id], dist: UpstreamHashShard(auction._row_id) }
+ └─StreamExchange { dist: HashShard(auction.date_time, window_start, window_end, auction._row_id) }
+ └─StreamHashJoin { type: LeftSemi, predicate: auction.date_time IS NOT DISTINCT FROM auction.date_time, output: all }
+ ├─StreamExchange { dist: HashShard(auction.date_time) }
+ │ └─StreamShare { id: 3 }
+ │ └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] }
+ │ └─StreamFilter { predicate: IsNotNull(auction.date_time) }
+ │ └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], pk: [auction._row_id], dist: UpstreamHashShard(auction._row_id) }
+ └─StreamProject { exprs: [auction.date_time] }
+ └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] }
+ └─StreamProject { exprs: [auction.date_time] }
+ └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] }
+ └─StreamExchange { dist: HashShard(auction.date_time) }
+ └─StreamShare { id: 3 }
+ └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] }
+ └─StreamFilter { predicate: IsNotNull(auction.date_time) }
+ └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], pk: [auction._row_id], dist: UpstreamHashShard(auction._row_id) }
- sql: |
CREATE TABLE t (v int);
SELECT 1 FROM t AS t_inner WHERE EXISTS ( SELECT 1 HAVING t_inner.v > 1);
@@ -535,22 +536,23 @@
└─BatchScan { table: t, columns: [t.x], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [x, y, k, sum_x, t.x(hidden)], stream_key: [k, x], pk_columns: [k, x], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, t.y, t.k, sum(Unnest($0)), t.x] }
- ├─StreamExchange { dist: HashShard(t.x) }
- │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) }
- └─StreamProject { exprs: [t.x, sum(Unnest($0))] }
- └─StreamHashAgg { group_key: [t.x], aggs: [sum(Unnest($0)), count] }
- └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, Unnest($0), t.x, projected_row_id] }
- ├─StreamProject { exprs: [t.x] }
- │ └─StreamHashAgg { group_key: [t.x], aggs: [count] }
- │ └─StreamExchange { dist: HashShard(t.x) }
- │ └─StreamTableScan { table: t, columns: [t.x, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) }
- └─StreamProject { exprs: [t.x, Unnest($0), projected_row_id] }
- └─StreamProjectSet { select_list: [$0, Unnest($0)] }
- └─StreamProject { exprs: [t.x] }
- └─StreamHashAgg { group_key: [t.x], aggs: [count] }
- └─StreamExchange { dist: HashShard(t.x) }
- └─StreamTableScan { table: t, columns: [t.x, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) }
+ └─StreamExchange { dist: HashShard(t.x, t.k) }
+ └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, t.y, t.k, sum(Unnest($0)), t.x] }
+ ├─StreamExchange { dist: HashShard(t.x) }
+ │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) }
+ └─StreamProject { exprs: [t.x, sum(Unnest($0))] }
+ └─StreamHashAgg { group_key: [t.x], aggs: [sum(Unnest($0)), count] }
+ └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, Unnest($0), t.x, projected_row_id] }
+ ├─StreamProject { exprs: [t.x] }
+ │ └─StreamHashAgg { group_key: [t.x], aggs: [count] }
+ │ └─StreamExchange { dist: HashShard(t.x) }
+ │ └─StreamTableScan { table: t, columns: [t.x, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) }
+ └─StreamProject { exprs: [t.x, Unnest($0), projected_row_id] }
+ └─StreamProjectSet { select_list: [$0, Unnest($0)] }
+ └─StreamProject { exprs: [t.x] }
+ └─StreamHashAgg { group_key: [t.x], aggs: [count] }
+ └─StreamExchange { dist: HashShard(t.x) }
+ └─StreamTableScan { table: t, columns: [t.x, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) }
- name: CorrelatedInputRef in ProjectSet and apply on condition is true.
sql: |
create table t(x int[], y int[], k int primary key);
@@ -632,16 +634,17 @@
└─BatchScan { table: integers, columns: [integers.i], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [i, col, integers._row_id(hidden), integers.i(hidden)], stream_key: [integers._row_id, i], pk_columns: [i, integers._row_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, row_number, integers._row_id, integers.i] }
- ├─StreamExchange { dist: HashShard(integers.i) }
- │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
- └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] }
- └─StreamProject { exprs: [integers.i, row_number, integers._row_id] }
- └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] }
- └─StreamExchange { dist: HashShard(integers.i) }
- └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] }
- └─StreamFilter { predicate: IsNotNull(integers.i) }
- └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
+ └─StreamExchange { dist: HashShard(integers.i, integers._row_id) }
+ └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, row_number, integers._row_id, integers.i] }
+ ├─StreamExchange { dist: HashShard(integers.i) }
+ │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
+ └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] }
+ └─StreamProject { exprs: [integers.i, row_number, integers._row_id] }
+ └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] }
+ └─StreamExchange { dist: HashShard(integers.i) }
+ └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] }
+ └─StreamFilter { predicate: IsNotNull(integers.i) }
+ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
- name: test over window subquery 2 (with nested loop join so cannot be transformed into a stream plan)
sql: |
CREATE TABLE integers(i INTEGER);
@@ -690,16 +693,17 @@
└─BatchScan { table: integers, columns: [integers.i], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [i, col, integers._row_id(hidden), integers.i(hidden)], stream_key: [integers._row_id, i], pk_columns: [i, integers._row_id], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, sum, integers._row_id, integers.i] }
- ├─StreamExchange { dist: HashShard(integers.i) }
- │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
- └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] }
- └─StreamProject { exprs: [integers.i, sum, integers._row_id] }
- └─StreamOverWindow { window_functions: [sum(integers.i) OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] }
- └─StreamExchange { dist: HashShard(integers.i) }
- └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] }
- └─StreamFilter { predicate: IsNotNull(integers.i) }
- └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
+ └─StreamExchange { dist: HashShard(integers.i, integers._row_id) }
+ └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, sum, integers._row_id, integers.i] }
+ ├─StreamExchange { dist: HashShard(integers.i) }
+ │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
+ └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] }
+ └─StreamProject { exprs: [integers.i, sum, integers._row_id] }
+ └─StreamOverWindow { window_functions: [sum(integers.i) OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] }
+ └─StreamExchange { dist: HashShard(integers.i) }
+ └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] }
+ └─StreamFilter { predicate: IsNotNull(integers.i) }
+ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
- name: test over window subquery 4 (with nested loop join so cannot be transformed into a stream plan)
sql: |
CREATE TABLE integers(i INTEGER);
@@ -747,17 +751,18 @@
└─BatchScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [i, integers._row_id(hidden), $expr1(hidden), integers.correlated_col(hidden)], stream_key: [integers._row_id, $expr1, integers.correlated_col], pk_columns: [integers._row_id, $expr1, integers.correlated_col], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: $expr1 = sum AND integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.i, integers._row_id, $expr1, integers.correlated_col] }
- ├─StreamExchange { dist: HashShard(integers.correlated_col, $expr1) }
- │ └─StreamProject { exprs: [integers.i, integers.correlated_col, integers.i::Int64 as $expr1, integers._row_id] }
- │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
- └─StreamExchange { dist: HashShard(rows.correlated_col, sum) }
- └─StreamProject { exprs: [rows.correlated_col, sum, rows._row_id, rows.k] }
- └─StreamOverWindow { window_functions: [sum(rows.v) OVER(PARTITION BY rows.correlated_col, rows.k ORDER BY rows.v ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] }
- └─StreamExchange { dist: HashShard(rows.correlated_col, rows.k) }
- └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] }
- └─StreamFilter { predicate: IsNotNull(rows.correlated_col) }
- └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) }
+ └─StreamExchange { dist: HashShard(integers._row_id, $expr1, integers.correlated_col) }
+ └─StreamHashJoin { type: LeftSemi, predicate: $expr1 = sum AND integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.i, integers._row_id, $expr1, integers.correlated_col] }
+ ├─StreamExchange { dist: HashShard(integers.correlated_col, $expr1) }
+ │ └─StreamProject { exprs: [integers.i, integers.correlated_col, integers.i::Int64 as $expr1, integers._row_id] }
+ │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
+ └─StreamExchange { dist: HashShard(rows.correlated_col, sum) }
+ └─StreamProject { exprs: [rows.correlated_col, sum, rows._row_id, rows.k] }
+ └─StreamOverWindow { window_functions: [sum(rows.v) OVER(PARTITION BY rows.correlated_col, rows.k ORDER BY rows.v ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] }
+ └─StreamExchange { dist: HashShard(rows.correlated_col, rows.k) }
+ └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] }
+ └─StreamFilter { predicate: IsNotNull(rows.correlated_col) }
+ └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) }
- name: test cardinality visitor with correlated filter
sql: |
CREATE TABLE t1(i INT);
@@ -818,21 +823,22 @@
└─BatchScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [i, correlated_col, integers._row_id(hidden), 2:Int64(hidden)], stream_key: [integers._row_id, correlated_col, 2:Int64], pk_columns: [integers._row_id, correlated_col, 2:Int64], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: integers.correlated_col IS NOT DISTINCT FROM integers.correlated_col AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col, integers._row_id, 2:Int64] }
- ├─StreamExchange { dist: HashShard(integers.correlated_col) }
- │ └─StreamProject { exprs: [integers.i, integers.correlated_col, 2:Int64, integers._row_id] }
- │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
- └─StreamProject { exprs: [integers.correlated_col, (count(distinct rows.k) + count(distinct rows.v)) as $expr1] }
- └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count(distinct rows.k), count(distinct rows.v), count] }
- └─StreamHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, rows._row_id] }
- ├─StreamProject { exprs: [integers.correlated_col] }
- │ └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count] }
- │ └─StreamExchange { dist: HashShard(integers.correlated_col) }
- │ └─StreamTableScan { table: integers, columns: [integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
- └─StreamExchange { dist: HashShard(rows.correlated_col) }
- └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] }
- └─StreamFilter { predicate: IsNotNull(rows.correlated_col) }
- └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) }
+ └─StreamExchange { dist: HashShard(integers.correlated_col, integers._row_id, 2:Int64) }
+ └─StreamHashJoin { type: LeftSemi, predicate: integers.correlated_col IS NOT DISTINCT FROM integers.correlated_col AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col, integers._row_id, 2:Int64] }
+ ├─StreamExchange { dist: HashShard(integers.correlated_col) }
+ │ └─StreamProject { exprs: [integers.i, integers.correlated_col, 2:Int64, integers._row_id] }
+ │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
+ └─StreamProject { exprs: [integers.correlated_col, (count(distinct rows.k) + count(distinct rows.v)) as $expr1] }
+ └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count(distinct rows.k), count(distinct rows.v), count] }
+ └─StreamHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, rows._row_id] }
+ ├─StreamProject { exprs: [integers.correlated_col] }
+ │ └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count] }
+ │ └─StreamExchange { dist: HashShard(integers.correlated_col) }
+ │ └─StreamTableScan { table: integers, columns: [integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) }
+ └─StreamExchange { dist: HashShard(rows.correlated_col) }
+ └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] }
+ └─StreamFilter { predicate: IsNotNull(rows.correlated_col) }
+ └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) }
- name: test hop window subquery 1
sql: |
create table t1 (k int primary key, ts timestamp);
@@ -848,12 +854,13 @@
└─BatchValues { rows: [[1:Int32], [2:Int32]] }
stream_plan: |-
StreamMaterialize { columns: [col, k, ts, window_start, window_end], stream_key: [col, window_start, window_end], pk_columns: [col, window_start, window_end], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: 1:Int32 = t1.k, output: all }
- ├─StreamAppendOnlyDedup { dedup_cols: [1:Int32] }
- │ └─StreamExchange { dist: HashShard(1:Int32) }
- │ └─StreamProject { exprs: [1:Int32] }
- │ └─StreamValues { rows: [[1:Int32, 0:Int64], [2:Int32, 1:Int64]] }
- └─StreamExchange { dist: HashShard(t1.k) }
- └─StreamHopWindow { time_col: t1.ts, slide: 00:10:00, size: 00:30:00, output: all }
- └─StreamFilter { predicate: IsNotNull(t1.ts) }
- └─StreamTableScan { table: t1, columns: [t1.k, t1.ts], pk: [t1.k], dist: UpstreamHashShard(t1.k) }
+ └─StreamExchange { dist: HashShard(1:Int32, window_start, window_end) }
+ └─StreamHashJoin { type: Inner, predicate: 1:Int32 = t1.k, output: all }
+ ├─StreamAppendOnlyDedup { dedup_cols: [1:Int32] }
+ │ └─StreamExchange { dist: HashShard(1:Int32) }
+ │ └─StreamProject { exprs: [1:Int32] }
+ │ └─StreamValues { rows: [[1:Int32, 0:Int64], [2:Int32, 1:Int64]] }
+ └─StreamExchange { dist: HashShard(t1.k) }
+ └─StreamHopWindow { time_col: t1.ts, slide: 00:10:00, size: 00:30:00, output: all }
+ └─StreamFilter { predicate: IsNotNull(t1.ts) }
+ └─StreamTableScan { table: t1, columns: [t1.k, t1.ts], pk: [t1.k], dist: UpstreamHashShard(t1.k) }
diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml
index 3b84def33abfa..0d393c378ff85 100644
--- a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml
@@ -717,15 +717,16 @@
└─BatchScan { table: t2, columns: [t2.x], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [x, y, t1._row_id(hidden)], stream_key: [t1._row_id, x], pk_columns: [t1._row_id, x], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: t1.x IS NOT DISTINCT FROM t2.x, output: all }
- ├─StreamExchange { dist: HashShard(t1.x) }
- │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamProject { exprs: [t2.x] }
- └─StreamGroupTopN { order: [t2.x ASC], limit: 1, offset: 0, group_key: [t2.x] }
- └─StreamExchange { dist: HashShard(t2.x) }
- └─StreamProject { exprs: [t2.x, t2.x, t2._row_id] }
- └─StreamFilter { predicate: IsNotNull(t2.x) }
- └─StreamTableScan { table: t2, columns: [t2.x, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.x, t1._row_id) }
+ └─StreamHashJoin { type: LeftSemi, predicate: t1.x IS NOT DISTINCT FROM t2.x, output: all }
+ ├─StreamExchange { dist: HashShard(t1.x) }
+ │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamProject { exprs: [t2.x] }
+ └─StreamGroupTopN { order: [t2.x ASC], limit: 1, offset: 0, group_key: [t2.x] }
+ └─StreamExchange { dist: HashShard(t2.x) }
+ └─StreamProject { exprs: [t2.x, t2.x, t2._row_id] }
+ └─StreamFilter { predicate: IsNotNull(t2.x) }
+ └─StreamTableScan { table: t2, columns: [t2.x, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- sql: |
create table t1(x int, y int);
create table t2(x int, y int);
@@ -883,14 +884,15 @@
└─BatchScan { table: t2, columns: [t2.v2, t2.k2], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [v1, k1, t1._row_id(hidden)], stream_key: [t1._row_id, v1, k1], pk_columns: [t1._row_id, v1, k1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2 AND t1.k1 IS NOT DISTINCT FROM t2.k2, output: all }
- ├─StreamExchange { dist: HashShard(t1.k1) }
- │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamGroupTopN { order: [t2.v2 ASC], limit: 1, offset: 0, group_key: [t2.k2] }
- └─StreamExchange { dist: HashShard(t2.k2) }
- └─StreamProject { exprs: [t2.k2, t2.v2, t2._row_id] }
- └─StreamFilter { predicate: IsNotNull(t2.k2) }
- └─StreamTableScan { table: t2, columns: [t2.v2, t2.k2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t1.k1, t1._row_id) }
+ └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2 AND t1.k1 IS NOT DISTINCT FROM t2.k2, output: all }
+ ├─StreamExchange { dist: HashShard(t1.k1) }
+ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamGroupTopN { order: [t2.v2 ASC], limit: 1, offset: 0, group_key: [t2.k2] }
+ └─StreamExchange { dist: HashShard(t2.k2) }
+ └─StreamProject { exprs: [t2.k2, t2.v2, t2._row_id] }
+ └─StreamFilter { predicate: IsNotNull(t2.k2) }
+ └─StreamTableScan { table: t2, columns: [t2.v2, t2.k2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- name: test ApplyTopNTransposeRule case 2
sql: |
create table t1 (v1 int, k1 int);
@@ -908,16 +910,17 @@
└─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [v1, k1, t1._row_id(hidden)], stream_key: [t1._row_id, v1], pk_columns: [t1._row_id, v1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2, output: all }
- ├─StreamExchange { dist: HashShard(t1.v1) }
- │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard(t2.v2) }
- └─StreamProject { exprs: [t2.v2, t2._row_id] }
- └─StreamTopN { order: [t2.v2 ASC], limit: 1, offset: 0 }
- └─StreamExchange { dist: Single }
- └─StreamGroupTopN { order: [t2.v2 ASC], limit: 1, offset: 0, group_key: [$expr1] }
- └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] }
- └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t1._row_id) }
+ └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2, output: all }
+ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(t2.v2) }
+ └─StreamProject { exprs: [t2.v2, t2._row_id] }
+ └─StreamTopN { order: [t2.v2 ASC], limit: 1, offset: 0 }
+ └─StreamExchange { dist: Single }
+ └─StreamGroupTopN { order: [t2.v2 ASC], limit: 1, offset: 0, group_key: [$expr1] }
+ └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] }
+ └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- name: test ApplyLimitTransposeRule case 1
sql: |
create table t1 (v1 int, k1 int);
@@ -935,11 +938,12 @@
└─BatchScan { table: t2, columns: [t2.v2, t2.k2], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [v1, k1, t1._row_id(hidden)], stream_key: [t1._row_id, v1, k1], pk_columns: [t1._row_id, v1, k1], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2 AND t1.k1 IS NOT DISTINCT FROM t2.k2, output: all }
- ├─StreamExchange { dist: HashShard(t1.k1) }
- │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamGroupTopN { order: [t2.k2 ASC], limit: 1, offset: 0, group_key: [t2.k2] }
- └─StreamExchange { dist: HashShard(t2.k2) }
- └─StreamProject { exprs: [t2.k2, t2.v2, t2._row_id] }
- └─StreamFilter { predicate: IsNotNull(t2.k2) }
- └─StreamTableScan { table: t2, columns: [t2.v2, t2.k2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t1.k1, t1._row_id) }
+ └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2 AND t1.k1 IS NOT DISTINCT FROM t2.k2, output: all }
+ ├─StreamExchange { dist: HashShard(t1.k1) }
+ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamGroupTopN { order: [t2.k2 ASC], limit: 1, offset: 0, group_key: [t2.k2] }
+ └─StreamExchange { dist: HashShard(t2.k2) }
+ └─StreamProject { exprs: [t2.k2, t2.v2, t2._row_id] }
+ └─StreamFilter { predicate: IsNotNull(t2.k2) }
+ └─StreamTableScan { table: t2, columns: [t2.v2, t2.k2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml
index 6673d86fd9745..29e391853cf8a 100644
--- a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml
@@ -122,19 +122,20 @@
select * from t1 join t2 on a = b AND ta < now() - interval '1 hour' and ta >= now() - interval '2 hour';
stream_plan: |-
StreamMaterialize { columns: [a, ta, b, tb, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, a], pk_columns: [t1._row_id, t2._row_id, a], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] }
- ├─StreamExchange { dist: HashShard(t1.a) }
- │ └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id] }
- │ ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true }
- │ │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- │ │ └─StreamExchange { dist: Broadcast }
- │ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] }
- │ │ └─StreamNow { output: [now] }
- │ └─StreamExchange { dist: Broadcast }
- │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] }
- │ └─StreamNow { output: [now] }
- └─StreamExchange { dist: HashShard(t2.b) }
- └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.a, t1._row_id, t2._row_id) }
+ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.a) }
+ │ └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id] }
+ │ ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true }
+ │ │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ │ │ └─StreamExchange { dist: Broadcast }
+ │ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] }
+ │ │ └─StreamNow { output: [now] }
+ │ └─StreamExchange { dist: Broadcast }
+ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] }
+ │ └─StreamNow { output: [now] }
+ └─StreamExchange { dist: HashShard(t2.b) }
+ └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- name: Temporal filter in on clause for left join's left side
sql: |
create table t1 (a int, ta timestamp with time zone);
@@ -150,19 +151,20 @@
select * from t1 right join t2 on a = b AND ta < now() - interval '1 hour' and ta >= now() - interval '2 hour';
stream_plan: |-
StreamMaterialize { columns: [a, ta, b, tb, t2._row_id(hidden), t1._row_id(hidden)], stream_key: [t2._row_id, t1._row_id, b], pk_columns: [t2._row_id, t1._row_id, b], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftOuter, predicate: t2.b = t1.a, output: [t1.a, t1.ta, t2.b, t2.tb, t2._row_id, t1._row_id] }
- ├─StreamExchange { dist: HashShard(t2.b) }
- │ └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- └─StreamExchange { dist: HashShard(t1.a) }
- └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id] }
- ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true }
- │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- │ └─StreamExchange { dist: Broadcast }
- │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] }
- │ └─StreamNow { output: [now] }
- └─StreamExchange { dist: Broadcast }
- └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] }
- └─StreamNow { output: [now] }
+ └─StreamExchange { dist: HashShard(t2.b, t2._row_id, t1._row_id) }
+ └─StreamHashJoin { type: LeftOuter, predicate: t2.b = t1.a, output: [t1.a, t1.ta, t2.b, t2.tb, t2._row_id, t1._row_id] }
+ ├─StreamExchange { dist: HashShard(t2.b) }
+ │ └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.a) }
+ └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id] }
+ ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true }
+ │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ │ └─StreamExchange { dist: Broadcast }
+ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] }
+ │ └─StreamNow { output: [now] }
+ └─StreamExchange { dist: Broadcast }
+ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] }
+ └─StreamNow { output: [now] }
- name: Temporal filter in on clause for full join's left side
sql: |
create table t1 (a int, ta timestamp with time zone);
@@ -178,19 +180,20 @@
select * from t1 left join t2 on a = b AND tb < now() - interval '1 hour' and tb >= now() - interval '2 hour';
stream_plan: |-
StreamMaterialize { columns: [a, ta, b, tb, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, a], pk_columns: [t1._row_id, t2._row_id, a], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftOuter, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] }
- ├─StreamExchange { dist: HashShard(t1.a) }
- │ └─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard(t2.b) }
- └─StreamDynamicFilter { predicate: (t2.tb < $expr2), output: [t2.b, t2.tb, t2._row_id] }
- ├─StreamDynamicFilter { predicate: (t2.tb >= $expr1), output_watermarks: [t2.tb], output: [t2.b, t2.tb, t2._row_id], cleaned_by_watermark: true }
- │ ├─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- │ └─StreamExchange { dist: Broadcast }
- │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] }
- │ └─StreamNow { output: [now] }
- └─StreamExchange { dist: Broadcast }
- └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] }
- └─StreamNow { output: [now] }
+ └─StreamExchange { dist: HashShard(t1.a, t1._row_id, t2._row_id) }
+ └─StreamHashJoin { type: LeftOuter, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.a) }
+ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(t2.b) }
+ └─StreamDynamicFilter { predicate: (t2.tb < $expr2), output: [t2.b, t2.tb, t2._row_id] }
+ ├─StreamDynamicFilter { predicate: (t2.tb >= $expr1), output_watermarks: [t2.tb], output: [t2.b, t2.tb, t2._row_id], cleaned_by_watermark: true }
+ │ ├─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ │ └─StreamExchange { dist: Broadcast }
+ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] }
+ │ └─StreamNow { output: [now] }
+ └─StreamExchange { dist: Broadcast }
+ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] }
+ └─StreamNow { output: [now] }
- name: Temporal filter in on clause for right join's right side
sql: |
create table t1 (a int, ta timestamp with time zone);
diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml
index a1020b8d16ee5..f49a82be2dd78 100644
--- a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml
@@ -6,11 +6,12 @@
select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on id1= id2
stream_plan: |-
StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] }
- ├─StreamExchange { dist: HashShard(stream.id1) }
- │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) }
- └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) }
+ └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) }
+ └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] }
+ ├─StreamExchange { dist: HashShard(stream.id1) }
+ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) }
+ └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) }
batch_error: |-
Not supported: do not support temporal join for batch queries
HINT: please use temporal join in streaming queries
@@ -21,11 +22,12 @@
select id1, a1, id2, a2 from stream join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 where a2 < 10;
stream_plan: |-
StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] }
- ├─StreamExchange { dist: HashShard(stream.id1) }
- │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) }
- └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) }
+ └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) }
+ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] }
+ ├─StreamExchange { dist: HashShard(stream.id1) }
+ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) }
+ └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) }
- name: implicit join with temporal tables
sql: |
create table stream(id1 int, a1 int, b1 int) APPEND ONLY;
@@ -33,11 +35,12 @@
select id1, a1, id2, a2 from stream, version FOR SYSTEM_TIME AS OF PROCTIME() where id1 = id2 AND a2 < 10;
stream_plan: |-
StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] }
- ├─StreamExchange { dist: HashShard(stream.id1) }
- │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) }
- └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) }
+ └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) }
+ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] }
+ ├─StreamExchange { dist: HashShard(stream.id1) }
+ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) }
+ └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) }
- name: Multi join key for temporal join
sql: |
create table stream(id1 int, a1 int, b1 int) APPEND ONLY;
@@ -45,11 +48,12 @@
select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and id1 = id2 where b2 != a2;
stream_plan: |-
StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1, a1], pk_columns: [stream._row_id, id1, a1], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND stream.a1 = version.a2 AND (version.b2 <> version.a2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] }
- ├─StreamExchange { dist: HashShard(stream.id1, stream.a1) }
- │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2, version.a2) }
- └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2], pk: [version.id2, version.a2], dist: UpstreamHashShard(version.id2, version.a2) }
+ └─StreamExchange { dist: HashShard(stream.id1, stream.a1, stream._row_id) }
+ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND stream.a1 = version.a2 AND (version.b2 <> version.a2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] }
+ ├─StreamExchange { dist: HashShard(stream.id1, stream.a1) }
+ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2, version.a2) }
+ └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2], pk: [version.id2, version.a2], dist: UpstreamHashShard(version.id2, version.a2) }
- name: Temporal join with Aggregation
sql: |
create table stream(id1 int, a1 int, b1 int) APPEND ONLY;
@@ -101,15 +105,16 @@
join version2 FOR SYSTEM_TIME AS OF PROCTIME() on stream.k = version2.k where a1 < 10;
stream_plan: |-
StreamMaterialize { columns: [k, x1, x2, a1, b1, stream._row_id(hidden), version2.k(hidden)], stream_key: [stream._row_id, k], pk_columns: [stream._row_id, k], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: Inner, predicate: stream.k = version2.k, output: [stream.k, version1.x1, version2.x2, stream.a1, stream.b1, stream._row_id, version2.k] }
- ├─StreamTemporalJoin { type: Inner, predicate: stream.k = version1.k, output: [stream.k, stream.a1, stream.b1, version1.x1, stream._row_id, version1.k] }
- │ ├─StreamExchange { dist: HashShard(stream.k) }
- │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) }
- │ │ └─StreamTableScan { table: stream, columns: [stream.k, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.k) }
- │ └─StreamTableScan { table: version1, columns: [version1.k, version1.x1], pk: [version1.k], dist: UpstreamHashShard(version1.k) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.k) }
- └─StreamTableScan { table: version2, columns: [version2.k, version2.x2], pk: [version2.k], dist: UpstreamHashShard(version2.k) }
+ └─StreamExchange { dist: HashShard(stream.k, stream._row_id) }
+ └─StreamTemporalJoin { type: Inner, predicate: stream.k = version2.k, output: [stream.k, version1.x1, version2.x2, stream.a1, stream.b1, stream._row_id, version2.k] }
+ ├─StreamTemporalJoin { type: Inner, predicate: stream.k = version1.k, output: [stream.k, stream.a1, stream.b1, version1.x1, stream._row_id, version1.k] }
+ │ ├─StreamExchange { dist: HashShard(stream.k) }
+ │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) }
+ │ │ └─StreamTableScan { table: stream, columns: [stream.k, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.k) }
+ │ └─StreamTableScan { table: version1, columns: [version1.k, version1.x1], pk: [version1.k], dist: UpstreamHashShard(version1.k) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.k) }
+ └─StreamTableScan { table: version2, columns: [version2.k, version2.x2], pk: [version2.k], dist: UpstreamHashShard(version2.k) }
- name: multi-way temporal join with different keys
sql: |
create table stream(id1 int, id2 int, a1 int, b1 int) APPEND ONLY;
@@ -121,16 +126,17 @@
join version2 FOR SYSTEM_TIME AS OF PROCTIME() on stream.id2 = version2.id2 where a1 < 10;
stream_plan: |-
StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version2.id2(hidden)], stream_key: [stream._row_id, id1, id2], pk_columns: [stream._row_id, id1, id2], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] }
- ├─StreamExchange { dist: HashShard(stream.id2) }
- │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] }
- │ ├─StreamExchange { dist: HashShard(stream.id1) }
- │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) }
- │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.id1) }
- │ └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.id2) }
- └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) }
+ └─StreamExchange { dist: HashShard(stream.id1, stream.id2, stream._row_id) }
+ └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] }
+ ├─StreamExchange { dist: HashShard(stream.id2) }
+ │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] }
+ │ ├─StreamExchange { dist: HashShard(stream.id1) }
+ │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) }
+ │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.id1) }
+ │ └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.id2) }
+ └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) }
- name: multi-way temporal join with different keys
sql: |
create table stream(id1 int, id2 int, a1 int, b1 int) APPEND ONLY;
@@ -142,16 +148,17 @@
join version2 FOR SYSTEM_TIME AS OF PROCTIME() on stream.id2 = version2.id2 where a1 < 10;
stream_plan: |-
StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version2.id2(hidden)], stream_key: [stream._row_id, id1, id2], pk_columns: [stream._row_id, id1, id2], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] }
- ├─StreamExchange { dist: HashShard(stream.id2) }
- │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] }
- │ ├─StreamExchange { dist: HashShard(stream.id1) }
- │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) }
- │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.id1) }
- │ └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.id2) }
- └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) }
+ └─StreamExchange { dist: HashShard(stream.id1, stream.id2, stream._row_id) }
+ └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] }
+ ├─StreamExchange { dist: HashShard(stream.id2) }
+ │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] }
+ │ ├─StreamExchange { dist: HashShard(stream.id1) }
+ │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) }
+ │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.id1) }
+ │ └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.id2) }
+ └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) }
- name: temporal join with an index (distribution key size = 1)
sql: |
create table stream(id1 int, a1 int, b1 int) APPEND ONLY;
@@ -160,11 +167,12 @@
select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2;
stream_plan: |-
StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] }
- ├─StreamExchange { dist: HashShard(stream.a1) }
- │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) }
- └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) }
+ └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) }
+ └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] }
+ ├─StreamExchange { dist: HashShard(stream.a1) }
+ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) }
+ └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) }
- name: temporal join with an index (distribution key size = 2)
sql: |
create table stream(id1 int, a1 int, b1 int) APPEND ONLY;
@@ -173,11 +181,12 @@
select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2;
stream_plan: |-
StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] }
- ├─StreamExchange { dist: HashShard(stream.a1) }
- │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) }
- └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) }
+ └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) }
+ └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] }
+ ├─StreamExchange { dist: HashShard(stream.a1) }
+ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) }
+ └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) }
- name: temporal join with an index (index column size = 1)
sql: |
create table stream(id1 int, a1 int, b1 int) APPEND ONLY;
@@ -186,11 +195,12 @@
select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2;
stream_plan: |-
StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, stream.b1, a1], pk_columns: [stream._row_id, id2, stream.b1, a1], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: LeftOuter, predicate: stream.b1 = idx2.b2 AND (stream.a1 = idx2.a2), output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] }
- ├─StreamExchange { dist: HashShard(stream.b1) }
- │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.b2) }
- └─StreamTableScan { table: idx2, columns: [idx2.b2, idx2.id2, idx2.a2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.b2) }
+ └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) }
+ └─StreamTemporalJoin { type: LeftOuter, predicate: stream.b1 = idx2.b2 AND (stream.a1 = idx2.a2), output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] }
+ ├─StreamExchange { dist: HashShard(stream.b1) }
+ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.b2) }
+ └─StreamTableScan { table: idx2, columns: [idx2.b2, idx2.id2, idx2.a2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.b2) }
- name: temporal join with singleton table
sql: |
create table t (a int) append only;
@@ -212,11 +222,12 @@
select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2;
stream_plan: |-
StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx.a2 AND stream.b1 = idx.b2, output: [stream.id1, stream.a1, idx.id2, idx.a2, stream._row_id, stream.b1] }
- ├─StreamExchange { dist: HashShard(stream.a1) }
- │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx.a2) }
- └─StreamTableScan { table: idx, columns: [idx.id2, idx.a2, idx.b2], pk: [idx.id2], dist: UpstreamHashShard(idx.a2) }
+ └─StreamExchange { dist: HashShard(stream.a1, idx.id2, stream._row_id, stream.b1) }
+ └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx.a2 AND stream.b1 = idx.b2, output: [stream.id1, stream.a1, idx.id2, idx.a2, stream._row_id, stream.b1] }
+ ├─StreamExchange { dist: HashShard(stream.a1) }
+ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx.a2) }
+ └─StreamTableScan { table: idx, columns: [idx.id2, idx.a2, idx.b2], pk: [idx.id2], dist: UpstreamHashShard(idx.a2) }
- name: index selection for temporal join (with two indexes) and should choose the index with a longer prefix..
sql: |
create table stream(id1 int, a1 int, b1 int) APPEND ONLY;
@@ -226,11 +237,12 @@
select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2;
stream_plan: |-
StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] }
- ├─StreamExchange { dist: HashShard(stream.a1) }
- │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) }
- └─StreamTableScan { table: idx2, columns: [idx2.id2, idx2.a2, idx2.b2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) }
+ └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) }
+ └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] }
+ ├─StreamExchange { dist: HashShard(stream.a1) }
+ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) }
+ └─StreamTableScan { table: idx2, columns: [idx2.id2, idx2.a2, idx2.b2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) }
- name: index selection for temporal join (with three indexes) and should choose primary table.
sql: |
create table stream(id1 int, a1 int, b1 int, c1 int) APPEND ONLY;
@@ -241,11 +253,12 @@
select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2 and c1 = c2 and id1 = id2;
stream_plan: |-
StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden), stream.c1(hidden)], stream_key: [stream._row_id, id1, a1, stream.b1, stream.c1], pk_columns: [stream._row_id, id1, a1, stream.b1, stream.c1], pk_conflict: NoCheck }
- └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2 AND (stream.a1 = version.a2) AND (stream.b1 = version.b2) AND (stream.c1 = version.c2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id, stream.b1, stream.c1] }
- ├─StreamExchange { dist: HashShard(stream.id1) }
- │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream.c1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
- └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) }
- └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2, version.c2], pk: [version.id2], dist: UpstreamHashShard(version.id2) }
+ └─StreamExchange { dist: HashShard(stream.id1, stream.a1, stream._row_id, stream.b1, stream.c1) }
+ └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2 AND (stream.a1 = version.a2) AND (stream.b1 = version.b2) AND (stream.c1 = version.c2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id, stream.b1, stream.c1] }
+ ├─StreamExchange { dist: HashShard(stream.id1) }
+ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream.c1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) }
+ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) }
+ └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2, version.c2], pk: [version.id2], dist: UpstreamHashShard(version.id2) }
- name: index selection for temporal join (two index) and no one matches.
sql: |
create table stream(id1 int, a1 int, b1 int, c1 int) APPEND ONLY;
diff --git a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml
index eaaa1f8e5c8d5..fdf928a0c9c84 100644
--- a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml
@@ -243,214 +243,218 @@
└─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), r_regionkey(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_suppkey(hidden), n_nationkey(hidden), ps_supplycost(hidden), p_partkey#1(hidden)], stream_key: [_row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, p_partkey, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_conflict: NoCheck }
- └─StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] }
- ├─StreamExchange { dist: HashShard(p_partkey) }
- │ └─StreamShare { id: 26 }
- │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] }
- │ ├─StreamExchange { dist: HashShard(n_nationkey) }
- │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] }
- │ │ ├─StreamExchange { dist: HashShard(r_regionkey) }
- │ │ │ └─StreamShare { id: 3 }
- │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] }
- │ │ │ └─StreamRowIdGen { row_id_index: 3 }
- │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] }
- │ │ └─StreamExchange { dist: HashShard(n_regionkey) }
- │ │ └─StreamShare { id: 7 }
- │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] }
- │ │ └─StreamRowIdGen { row_id_index: 4 }
- │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] }
- │ └─StreamExchange { dist: HashShard(s_nationkey) }
- │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] }
- │ ├─StreamExchange { dist: HashShard(ps_suppkey) }
- │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] }
- │ │ ├─StreamExchange { dist: HashShard(p_partkey) }
- │ │ │ └─StreamRowIdGen { row_id_index: 9 }
- │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] }
- │ │ └─StreamExchange { dist: HashShard(ps_partkey) }
- │ │ └─StreamShare { id: 15 }
- │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] }
- │ │ └─StreamRowIdGen { row_id_index: 5 }
- │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] }
- │ └─StreamExchange { dist: HashShard(s_suppkey) }
- │ └─StreamShare { id: 21 }
- │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
- │ └─StreamRowIdGen { row_id_index: 7 }
- │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
- └─StreamProject { exprs: [p_partkey, min(ps_supplycost)] }
- └─StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] }
- └─StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] }
- ├─StreamAppendOnlyDedup { dedup_cols: [p_partkey] }
- │ └─StreamExchange { dist: HashShard(p_partkey) }
- │ └─StreamProject { exprs: [p_partkey] }
- │ └─StreamShare { id: 26 }
- │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] }
- │ ├─StreamExchange { dist: HashShard(n_nationkey) }
- │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] }
- │ │ ├─StreamExchange { dist: HashShard(r_regionkey) }
- │ │ │ └─StreamShare { id: 3 }
- │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] }
- │ │ │ └─StreamRowIdGen { row_id_index: 3 }
- │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] }
- │ │ └─StreamExchange { dist: HashShard(n_regionkey) }
- │ │ └─StreamShare { id: 7 }
- │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] }
- │ │ └─StreamRowIdGen { row_id_index: 4 }
- │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] }
- │ └─StreamExchange { dist: HashShard(s_nationkey) }
- │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] }
- │ ├─StreamExchange { dist: HashShard(ps_suppkey) }
- │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] }
- │ │ ├─StreamExchange { dist: HashShard(p_partkey) }
- │ │ │ └─StreamRowIdGen { row_id_index: 9 }
- │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] }
- │ │ └─StreamExchange { dist: HashShard(ps_partkey) }
- │ │ └─StreamShare { id: 15 }
- │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] }
- │ │ └─StreamRowIdGen { row_id_index: 5 }
- │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] }
- │ └─StreamExchange { dist: HashShard(s_suppkey) }
- │ └─StreamShare { id: 21 }
- │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
- │ └─StreamRowIdGen { row_id_index: 7 }
- │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
- └─StreamExchange { dist: HashShard(ps_partkey) }
- └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] }
- ├─StreamExchange { dist: HashShard(s_nationkey) }
- │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] }
- │ ├─StreamExchange { dist: HashShard(ps_suppkey) }
- │ │ └─StreamFilter { predicate: IsNotNull(ps_partkey) }
- │ │ └─StreamShare { id: 15 }
- │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] }
- │ │ └─StreamRowIdGen { row_id_index: 5 }
- │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] }
- │ └─StreamExchange { dist: HashShard(s_suppkey) }
- │ └─StreamShare { id: 21 }
- │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
- │ └─StreamRowIdGen { row_id_index: 7 }
- │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
- └─StreamExchange { dist: HashShard(n_nationkey) }
- └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] }
- ├─StreamExchange { dist: HashShard(r_regionkey) }
- │ └─StreamShare { id: 3 }
- │ └─StreamProject { exprs: [r_regionkey, _row_id] }
- │ └─StreamRowIdGen { row_id_index: 3 }
- │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] }
- └─StreamExchange { dist: HashShard(n_regionkey) }
- └─StreamShare { id: 7 }
- └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] }
- └─StreamRowIdGen { row_id_index: 4 }
- └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] }
+ └─StreamExchange { dist: HashShard(p_partkey, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost) }
+ └─StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] }
+ ├─StreamExchange { dist: HashShard(p_partkey) }
+ │ └─StreamShare { id: 26 }
+ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] }
+ │ ├─StreamExchange { dist: HashShard(n_nationkey) }
+ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] }
+ │ │ ├─StreamExchange { dist: HashShard(r_regionkey) }
+ │ │ │ └─StreamShare { id: 3 }
+ │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] }
+ │ │ │ └─StreamRowIdGen { row_id_index: 3 }
+ │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] }
+ │ │ └─StreamExchange { dist: HashShard(n_regionkey) }
+ │ │ └─StreamShare { id: 7 }
+ │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] }
+ │ │ └─StreamRowIdGen { row_id_index: 4 }
+ │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] }
+ │ └─StreamExchange { dist: HashShard(s_nationkey) }
+ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] }
+ │ ├─StreamExchange { dist: HashShard(ps_suppkey) }
+ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] }
+ │ │ ├─StreamExchange { dist: HashShard(p_partkey) }
+ │ │ │ └─StreamRowIdGen { row_id_index: 9 }
+ │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] }
+ │ │ └─StreamExchange { dist: HashShard(ps_partkey) }
+ │ │ └─StreamShare { id: 15 }
+ │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] }
+ │ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] }
+ │ └─StreamExchange { dist: HashShard(s_suppkey) }
+ │ └─StreamShare { id: 21 }
+ │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
+ │ └─StreamRowIdGen { row_id_index: 7 }
+ │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
+ └─StreamProject { exprs: [p_partkey, min(ps_supplycost)] }
+ └─StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] }
+ └─StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] }
+ ├─StreamAppendOnlyDedup { dedup_cols: [p_partkey] }
+ │ └─StreamExchange { dist: HashShard(p_partkey) }
+ │ └─StreamProject { exprs: [p_partkey] }
+ │ └─StreamShare { id: 26 }
+ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] }
+ │ ├─StreamExchange { dist: HashShard(n_nationkey) }
+ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] }
+ │ │ ├─StreamExchange { dist: HashShard(r_regionkey) }
+ │ │ │ └─StreamShare { id: 3 }
+ │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] }
+ │ │ │ └─StreamRowIdGen { row_id_index: 3 }
+ │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] }
+ │ │ └─StreamExchange { dist: HashShard(n_regionkey) }
+ │ │ └─StreamShare { id: 7 }
+ │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] }
+ │ │ └─StreamRowIdGen { row_id_index: 4 }
+ │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] }
+ │ └─StreamExchange { dist: HashShard(s_nationkey) }
+ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] }
+ │ ├─StreamExchange { dist: HashShard(ps_suppkey) }
+ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] }
+ │ │ ├─StreamExchange { dist: HashShard(p_partkey) }
+ │ │ │ └─StreamRowIdGen { row_id_index: 9 }
+ │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] }
+ │ │ └─StreamExchange { dist: HashShard(ps_partkey) }
+ │ │ └─StreamShare { id: 15 }
+ │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] }
+ │ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] }
+ │ └─StreamExchange { dist: HashShard(s_suppkey) }
+ │ └─StreamShare { id: 21 }
+ │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
+ │ └─StreamRowIdGen { row_id_index: 7 }
+ │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
+ └─StreamExchange { dist: HashShard(ps_partkey) }
+ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] }
+ ├─StreamExchange { dist: HashShard(s_nationkey) }
+ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] }
+ │ ├─StreamExchange { dist: HashShard(ps_suppkey) }
+ │ │ └─StreamFilter { predicate: IsNotNull(ps_partkey) }
+ │ │ └─StreamShare { id: 15 }
+ │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] }
+ │ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] }
+ │ └─StreamExchange { dist: HashShard(s_suppkey) }
+ │ └─StreamShare { id: 21 }
+ │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
+ │ └─StreamRowIdGen { row_id_index: 7 }
+ │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
+ └─StreamExchange { dist: HashShard(n_nationkey) }
+ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] }
+ ├─StreamExchange { dist: HashShard(r_regionkey) }
+ │ └─StreamShare { id: 3 }
+ │ └─StreamProject { exprs: [r_regionkey, _row_id] }
+ │ └─StreamRowIdGen { row_id_index: 3 }
+ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] }
+ └─StreamExchange { dist: HashShard(n_regionkey) }
+ └─StreamShare { id: 7 }
+ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] }
+ └─StreamRowIdGen { row_id_index: 4 }
+ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), r_regionkey(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_suppkey(hidden), n_nationkey(hidden), ps_supplycost(hidden), p_partkey#1(hidden)], stream_key: [_row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, p_partkey, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamProject { exprs: [p_partkey, min(ps_supplycost)] }
- └── StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] } { intermediate state table: 26, state tables: [ 25 ], distinct tables: [] }
- └── StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] } { left table: 27, right table: 29, left degree table: 28, right degree table: 30 }
- ├── StreamAppendOnlyDedup { dedup_cols: [p_partkey] } { state table: 31 }
- │ └── StreamExchange Hash([0]) from 15
- └── StreamExchange Hash([0]) from 16
+ └── StreamExchange Hash([3, 8, 9, 10, 11, 12, 13, 14, 15, 16]) from 1
Fragment 1
- StreamNoOp
- └── StreamExchange NoShuffle from 2
+ StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamProject { exprs: [p_partkey, min(ps_supplycost)] }
+ └── StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] } { intermediate state table: 26, state tables: [ 25 ], distinct tables: [] }
+ └── StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] } { left table: 27, right table: 29, left degree table: 28, right degree table: 30 }
+ ├── StreamAppendOnlyDedup { dedup_cols: [p_partkey] } { state table: 31 }
+ │ └── StreamExchange Hash([0]) from 16
+ └── StreamExchange Hash([0]) from 17
Fragment 2
- StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 }
- ├── StreamExchange Hash([0]) from 3
- └── StreamExchange Hash([5]) from 8
+ StreamNoOp
+ └── StreamExchange NoShuffle from 3
Fragment 3
- StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } { left table: 8, right table: 10, left degree table: 9, right degree table: 11 }
+ StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 }
├── StreamExchange Hash([0]) from 4
- └── StreamExchange Hash([2]) from 6
+ └── StreamExchange Hash([5]) from 9
Fragment 4
- StreamNoOp
- └── StreamExchange NoShuffle from 5
+ StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } { left table: 8, right table: 10, left degree table: 9, right degree table: 11 }
+ ├── StreamExchange Hash([0]) from 5
+ └── StreamExchange Hash([2]) from 7
Fragment 5
+ StreamNoOp
+ └── StreamExchange NoShuffle from 6
+
+ Fragment 6
StreamProject { exprs: [r_regionkey, _row_id] }
└── StreamRowIdGen { row_id_index: 3 }
└── StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } { source state table: 12 }
- Fragment 6
+ Fragment 7
StreamNoOp
- └── StreamExchange NoShuffle from 7
+ └── StreamExchange NoShuffle from 8
- Fragment 7
+ Fragment 8
StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] }
└── StreamRowIdGen { row_id_index: 4 }
└── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { source state table: 13 }
- Fragment 8
+ Fragment 9
StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 }
- ├── StreamExchange Hash([2]) from 9
- └── StreamExchange Hash([0]) from 13
+ ├── StreamExchange Hash([2]) from 10
+ └── StreamExchange Hash([0]) from 14
- Fragment 9
+ Fragment 10
StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } { left table: 18, right table: 20, left degree table: 19, right degree table: 21 }
- ├── StreamExchange Hash([0]) from 10
- └── StreamExchange Hash([0]) from 11
+ ├── StreamExchange Hash([0]) from 11
+ └── StreamExchange Hash([0]) from 12
- Fragment 10
+ Fragment 11
StreamRowIdGen { row_id_index: 9 }
└── StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } { source state table: 22 }
- Fragment 11
+ Fragment 12
StreamNoOp
- └── StreamExchange NoShuffle from 12
+ └── StreamExchange NoShuffle from 13
- Fragment 12
+ Fragment 13
StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] }
└── StreamRowIdGen { row_id_index: 5 }
└── StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } { source state table: 23 }
- Fragment 13
+ Fragment 14
StreamNoOp
- └── StreamExchange NoShuffle from 14
+ └── StreamExchange NoShuffle from 15
- Fragment 14
+ Fragment 15
StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
└── StreamRowIdGen { row_id_index: 7 }
└── StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } { source state table: 24 }
- Fragment 15
+ Fragment 16
StreamProject { exprs: [p_partkey] }
- └── StreamExchange NoShuffle from 2
+ └── StreamExchange NoShuffle from 3
- Fragment 16
+ Fragment 17
StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } { left table: 32, right table: 34, left degree table: 33, right degree table: 35 }
- ├── StreamExchange Hash([2]) from 17
- └── StreamExchange Hash([0]) from 20
+ ├── StreamExchange Hash([2]) from 18
+ └── StreamExchange Hash([0]) from 21
- Fragment 17
+ Fragment 18
StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } { left table: 36, right table: 38, left degree table: 37, right degree table: 39 }
- ├── StreamExchange Hash([1]) from 18
- └── StreamExchange Hash([0]) from 19
+ ├── StreamExchange Hash([1]) from 19
+ └── StreamExchange Hash([0]) from 20
- Fragment 18
+ Fragment 19
StreamFilter { predicate: IsNotNull(ps_partkey) }
- └── StreamExchange NoShuffle from 12
+ └── StreamExchange NoShuffle from 13
- Fragment 19
+ Fragment 20
StreamNoOp
- └── StreamExchange NoShuffle from 14
+ └── StreamExchange NoShuffle from 15
- Fragment 20
+ Fragment 21
StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } { left table: 40, right table: 42, left degree table: 41, right degree table: 43 }
- ├── StreamExchange Hash([0]) from 21
- └── StreamExchange Hash([2]) from 22
+ ├── StreamExchange Hash([0]) from 22
+ └── StreamExchange Hash([2]) from 23
- Fragment 21
+ Fragment 22
StreamNoOp
- └── StreamExchange NoShuffle from 5
+ └── StreamExchange NoShuffle from 6
- Fragment 22
+ Fragment 23
StreamNoOp
- └── StreamExchange NoShuffle from 7
+ └── StreamExchange NoShuffle from 8
Table 0 { columns: [ p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id_0, r_regionkey, n_nationkey, _row_id_1, _row_id_2, _row_id_3, ps_suppkey ], primary key: [ $0 ASC, $7 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $12 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16 ], distribution key: [ 0 ], read pk prefix len hint: 2 }
@@ -540,7 +544,7 @@
Table 43 { columns: [ n_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
- Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost, p_partkey#1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3 ], read pk prefix len hint: 13 }
+ Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost, p_partkey#1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3, 8, 9, 10, 11, 12, 13, 14, 15, 16 ], read pk prefix len hint: 13 }
- id: tpch_q5
before:
@@ -1797,76 +1801,80 @@
└─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) }
stream_plan: |-
StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [_row_id, _row_id#1, s_nationkey, s_suppkey], pk_columns: [s_name, _row_id, _row_id#1, s_nationkey, s_suppkey], pk_conflict: NoCheck }
- └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] }
- ├─StreamExchange { dist: HashShard(s_suppkey) }
- │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] }
- │ ├─StreamExchange { dist: HashShard(s_nationkey) }
- │ │ └─StreamRowIdGen { row_id_index: 7 }
- │ │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
- │ └─StreamExchange { dist: HashShard(n_nationkey) }
- │ └─StreamRowIdGen { row_id_index: 4 }
- │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] }
- └─StreamExchange { dist: HashShard(ps_suppkey) }
- └─StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, ps_partkey, ps_suppkey] }
- └─StreamFilter { predicate: ($expr1 > $expr2) }
- └─StreamHashJoin { type: Inner, predicate: ps_partkey IS NOT DISTINCT FROM ps_partkey AND ps_suppkey IS NOT DISTINCT FROM ps_suppkey, output: all }
- ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) }
- │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] }
- │ └─StreamShare { id: 13 }
- │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] }
- │ ├─StreamExchange { dist: HashShard(ps_partkey) }
- │ │ └─StreamRowIdGen { row_id_index: 5 }
- │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] }
- │ └─StreamExchange { dist: HashShard(p_partkey) }
- │ └─StreamProject { exprs: [p_partkey, _row_id] }
- │ └─StreamRowIdGen { row_id_index: 9 }
- │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] }
- └─StreamProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] }
- └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity), count] }
- └─StreamHashJoin { type: LeftOuter, predicate: ps_partkey IS NOT DISTINCT FROM l_partkey AND ps_suppkey IS NOT DISTINCT FROM l_suppkey, output: [ps_partkey, ps_suppkey, l_quantity, _row_id] }
- ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) }
- │ └─StreamProject { exprs: [ps_partkey, ps_suppkey] }
- │ └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [count] }
- │ └─StreamShare { id: 13 }
- │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] }
- │ ├─StreamExchange { dist: HashShard(ps_partkey) }
- │ │ └─StreamRowIdGen { row_id_index: 5 }
- │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] }
- │ └─StreamExchange { dist: HashShard(p_partkey) }
- │ └─StreamProject { exprs: [p_partkey, _row_id] }
- │ └─StreamRowIdGen { row_id_index: 9 }
- │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] }
- └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) }
- └─StreamProject { exprs: [l_partkey, l_suppkey, l_quantity, _row_id] }
- └─StreamFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) }
- └─StreamRowIdGen { row_id_index: 16 }
- └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] }
+ └─StreamExchange { dist: HashShard(_row_id, _row_id, s_nationkey, s_suppkey) }
+ └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] }
+ ├─StreamExchange { dist: HashShard(s_suppkey) }
+ │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] }
+ │ ├─StreamExchange { dist: HashShard(s_nationkey) }
+ │ │ └─StreamRowIdGen { row_id_index: 7 }
+ │ │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] }
+ │ └─StreamExchange { dist: HashShard(n_nationkey) }
+ │ └─StreamRowIdGen { row_id_index: 4 }
+ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] }
+ └─StreamExchange { dist: HashShard(ps_suppkey) }
+ └─StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, ps_partkey, ps_suppkey] }
+ └─StreamFilter { predicate: ($expr1 > $expr2) }
+ └─StreamHashJoin { type: Inner, predicate: ps_partkey IS NOT DISTINCT FROM ps_partkey AND ps_suppkey IS NOT DISTINCT FROM ps_suppkey, output: all }
+ ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) }
+ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] }
+ │ └─StreamShare { id: 13 }
+ │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] }
+ │ ├─StreamExchange { dist: HashShard(ps_partkey) }
+ │ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] }
+ │ └─StreamExchange { dist: HashShard(p_partkey) }
+ │ └─StreamProject { exprs: [p_partkey, _row_id] }
+ │ └─StreamRowIdGen { row_id_index: 9 }
+ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] }
+ └─StreamProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] }
+ └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity), count] }
+ └─StreamHashJoin { type: LeftOuter, predicate: ps_partkey IS NOT DISTINCT FROM l_partkey AND ps_suppkey IS NOT DISTINCT FROM l_suppkey, output: [ps_partkey, ps_suppkey, l_quantity, _row_id] }
+ ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) }
+ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey] }
+ │ └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [count] }
+ │ └─StreamShare { id: 13 }
+ │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] }
+ │ ├─StreamExchange { dist: HashShard(ps_partkey) }
+ │ │ └─StreamRowIdGen { row_id_index: 5 }
+ │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] }
+ │ └─StreamExchange { dist: HashShard(p_partkey) }
+ │ └─StreamProject { exprs: [p_partkey, _row_id] }
+ │ └─StreamRowIdGen { row_id_index: 9 }
+ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] }
+ └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) }
+ └─StreamProject { exprs: [l_partkey, l_suppkey, l_quantity, _row_id] }
+ └─StreamFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) }
+ └─StreamRowIdGen { row_id_index: 16 }
+ └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [_row_id, _row_id#1, s_nationkey, s_suppkey], pk_columns: [s_name, _row_id, _row_id#1, s_nationkey, s_suppkey], pk_conflict: NoCheck }
├── materialized table: 4294967294
- └── StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
- ├── StreamExchange Hash([0]) from 1
- └── StreamExchange Hash([0]) from 4
+ └── StreamExchange Hash([2, 3, 4, 5]) from 1
Fragment 1
- StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 }
- ├── StreamExchange Hash([3]) from 2
- └── StreamExchange Hash([0]) from 3
+ StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 }
+ ├── StreamExchange Hash([0]) from 2
+ └── StreamExchange Hash([0]) from 5
Fragment 2
+ StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 }
+ ├── StreamExchange Hash([3]) from 3
+ └── StreamExchange Hash([0]) from 4
+
+ Fragment 3
StreamRowIdGen { row_id_index: 7 }
└── StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } { source state table: 8 }
- Fragment 3
+ Fragment 4
StreamRowIdGen { row_id_index: 4 }
└── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { source state table: 9 }
- Fragment 4
+ Fragment 5
StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, ps_partkey, ps_suppkey] }
└── StreamFilter { predicate: ($expr1 > $expr2) }
└── StreamHashJoin { type: Inner, predicate: ps_partkey IS NOT DISTINCT FROM ps_partkey AND ps_suppkey IS NOT DISTINCT FROM ps_suppkey, output: all } { left table: 10, right table: 12, left degree table: 11, right degree table: 13 }
- ├── StreamExchange Hash([0, 1]) from 5
+ ├── StreamExchange Hash([0, 1]) from 6
└── StreamProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] }
└── StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity), count] } { intermediate state table: 20, state tables: [], distinct tables: [] }
└── StreamHashJoin { type: LeftOuter, predicate: ps_partkey IS NOT DISTINCT FROM l_partkey AND ps_suppkey IS NOT DISTINCT FROM l_suppkey, output: [ps_partkey, ps_suppkey, l_quantity, _row_id] }
@@ -1874,33 +1882,33 @@
├── right table: 23
├── left degree table: 22
├── right degree table: 24
- ├── StreamExchange Hash([0, 1]) from 9
- └── StreamExchange Hash([0, 1]) from 10
+ ├── StreamExchange Hash([0, 1]) from 10
+ └── StreamExchange Hash([0, 1]) from 11
- Fragment 5
+ Fragment 6
StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] }
- └── StreamExchange NoShuffle from 6
+ └── StreamExchange NoShuffle from 7
- Fragment 6
+ Fragment 7
StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 }
- ├── StreamExchange Hash([0]) from 7
- └── StreamExchange Hash([0]) from 8
+ ├── StreamExchange Hash([0]) from 8
+ └── StreamExchange Hash([0]) from 9
- Fragment 7
+ Fragment 8
StreamRowIdGen { row_id_index: 5 }
└── StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } { source state table: 18 }
- Fragment 8
+ Fragment 9
StreamProject { exprs: [p_partkey, _row_id] }
└── StreamRowIdGen { row_id_index: 9 }
└── StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } { source state table: 19 }
- Fragment 9
+ Fragment 10
StreamProject { exprs: [ps_partkey, ps_suppkey] }
└── StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [count] } { intermediate state table: 25, state tables: [], distinct tables: [] }
- └── StreamExchange NoShuffle from 6
+ └── StreamExchange NoShuffle from 7
- Fragment 10
+ Fragment 11
StreamProject { exprs: [l_partkey, l_suppkey, l_quantity, _row_id] }
└── StreamFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) }
└── StreamRowIdGen { row_id_index: 16 }
@@ -1961,7 +1969,7 @@
Table 26 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 }
- Table 4294967294 { columns: [ s_name, s_address, _row_id, _row_id#1, s_nationkey, s_suppkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 5 }
+ Table 4294967294 { columns: [ s_name, s_address, _row_id, _row_id#1, s_nationkey, s_suppkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2, 3, 4, 5 ], read pk prefix len hint: 5 }
- id: tpch_q21
before:
diff --git a/src/frontend/planner_test/tests/testdata/output/watermark.yaml b/src/frontend/planner_test/tests/testdata/output/watermark.yaml
index e4ef42b121528..d57d41fa76bc3 100644
--- a/src/frontend/planner_test/tests/testdata/output/watermark.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/watermark.yaml
@@ -79,11 +79,12 @@
select t1.ts as t1_ts, t2.ts as ts2, t1.v1 as t1_v1, t1.v2 as t1_v2, t2.v1 as t2_v1, t2.v2 as t2_v2 from t1, t2 where t1.ts = t2.ts;
stream_plan: |-
StreamMaterialize { columns: [t1_ts, ts2, t1_v1, t1_v2, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_ts], pk_columns: [t1._row_id, t2._row_id, t1_ts], pk_conflict: NoCheck, watermark_columns: [t1_ts, ts2] }
- └─StreamHashJoin [window, append_only] { type: Inner, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t2.ts, t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] }
- ├─StreamExchange { dist: HashShard(t1.ts) }
- │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard(t2.ts) }
- └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.ts, t1._row_id, t2._row_id) }
+ └─StreamHashJoin [window, append_only] { type: Inner, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t2.ts, t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.ts) }
+ │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(t2.ts) }
+ └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- name: left semi window join
sql: |
create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only;
@@ -91,11 +92,12 @@
select t1.ts as t1_ts, t1.v1 as t1_v1, t1.v2 as t1_v2 from t1 where exists (select * from t2 where t1.ts = t2.ts);
stream_plan: |-
StreamMaterialize { columns: [t1_ts, t1_v1, t1_v2, t1._row_id(hidden)], stream_key: [t1._row_id, t1_ts], pk_columns: [t1._row_id, t1_ts], pk_conflict: NoCheck, watermark_columns: [t1_ts] }
- └─StreamHashJoin [window] { type: LeftSemi, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts], output: all }
- ├─StreamExchange { dist: HashShard(t1.ts) }
- │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard(t2.ts) }
- └─StreamTableScan { table: t2, columns: [t2.ts, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.ts, t1._row_id) }
+ └─StreamHashJoin [window] { type: LeftSemi, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts], output: all }
+ ├─StreamExchange { dist: HashShard(t1.ts) }
+ │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(t2.ts) }
+ └─StreamTableScan { table: t2, columns: [t2.ts, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- name: interval join(left outer join)
sql: |
create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only;
@@ -108,13 +110,14 @@
└─LogicalScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id] }
stream_plan: |-
StreamMaterialize { columns: [t1_ts, t1_v1, t1_v2, t2_ts, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1], pk_columns: [t1._row_id, t2._row_id, t1_v1], pk_conflict: NoCheck, watermark_columns: [t1_ts, t2_ts] }
- └─StreamHashJoin [interval] { type: LeftOuter, predicate: t1.v1 = t2.v1 AND (t1.ts >= $expr2) AND ($expr1 <= t2.ts), conditions_to_clean_left_state_table: (t1.ts >= $expr2), conditions_to_clean_right_state_table: ($expr1 <= t2.ts), output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2, t1._row_id, t2._row_id] }
- ├─StreamExchange { dist: HashShard(t1.v1) }
- │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, AddWithTimeZone(t1.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr1, t1._row_id], output_watermarks: [t1.ts, $expr1] }
- │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard(t2.v1) }
- └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, AddWithTimeZone(t2.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr2, t2._row_id], output_watermarks: [t2.ts, $expr2] }
- └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) }
+ └─StreamHashJoin [interval] { type: LeftOuter, predicate: t1.v1 = t2.v1 AND (t1.ts >= $expr2) AND ($expr1 <= t2.ts), conditions_to_clean_left_state_table: (t1.ts >= $expr2), conditions_to_clean_right_state_table: ($expr1 <= t2.ts), output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2, t1._row_id, t2._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, AddWithTimeZone(t1.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr1, t1._row_id], output_watermarks: [t1.ts, $expr1] }
+ │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(t2.v1) }
+ └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, AddWithTimeZone(t2.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr2, t2._row_id], output_watermarks: [t2.ts, $expr2] }
+ └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- name: interval join (inner join)
sql: |
create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only;
@@ -127,13 +130,14 @@
└─LogicalScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id] }
stream_plan: |-
StreamMaterialize { columns: [t1_ts, t1_v1, t1_v2, t2_ts, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1], pk_columns: [t1._row_id, t2._row_id, t1_v1], pk_conflict: NoCheck, watermark_columns: [t1_ts, t2_ts] }
- └─StreamHashJoin [interval, append_only] { type: Inner, predicate: t1.v1 = t2.v1 AND (t1.ts >= $expr2) AND ($expr1 <= t2.ts), conditions_to_clean_left_state_table: (t1.ts >= $expr2), conditions_to_clean_right_state_table: ($expr1 <= t2.ts), output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2, t1._row_id, t2._row_id] }
- ├─StreamExchange { dist: HashShard(t1.v1) }
- │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, AddWithTimeZone(t1.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr1, t1._row_id], output_watermarks: [t1.ts, $expr1] }
- │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
- └─StreamExchange { dist: HashShard(t2.v1) }
- └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, AddWithTimeZone(t2.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr2, t2._row_id], output_watermarks: [t2.ts, $expr2] }
- └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
+ └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) }
+ └─StreamHashJoin [interval, append_only] { type: Inner, predicate: t1.v1 = t2.v1 AND (t1.ts >= $expr2) AND ($expr1 <= t2.ts), conditions_to_clean_left_state_table: (t1.ts >= $expr2), conditions_to_clean_right_state_table: ($expr1 <= t2.ts), output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2, t1._row_id, t2._row_id] }
+ ├─StreamExchange { dist: HashShard(t1.v1) }
+ │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, AddWithTimeZone(t1.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr1, t1._row_id], output_watermarks: [t1.ts, $expr1] }
+ │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
+ └─StreamExchange { dist: HashShard(t2.v1) }
+ └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, AddWithTimeZone(t2.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr2, t2._row_id], output_watermarks: [t2.ts, $expr2] }
+ └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) }
- name: union all
sql: |
create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only;
diff --git a/src/frontend/planner_test/tests/testdata/output/window_join.yaml b/src/frontend/planner_test/tests/testdata/output/window_join.yaml
index 4113a6021e866..17c5e76f6e806 100644
--- a/src/frontend/planner_test/tests/testdata/output/window_join.yaml
+++ b/src/frontend/planner_test/tests/testdata/output/window_join.yaml
@@ -12,15 +12,16 @@
select * from t1, t2 where ts1 = ts2 and a1 = a2;
stream_plan: |-
StreamMaterialize { columns: [ts1, a1, b1, ts2, a2, b2, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, ts1, a1], pk_columns: [_row_id, _row_id#1, ts1, a1], pk_conflict: NoCheck, watermark_columns: [ts1, ts2] }
- └─StreamHashJoin [window, append_only] { type: Inner, predicate: ts1 = ts2 AND a1 = a2, output_watermarks: [ts1, ts2], output: [ts1, a1, b1, ts2, a2, b2, _row_id, _row_id] }
- ├─StreamExchange { dist: HashShard(ts1, a1) }
- │ └─StreamRowIdGen { row_id_index: 3 }
- │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts1, expr: (ts1 - '00:00:01':Interval) }], output_watermarks: [ts1] }
- │ └─StreamSource { source: t1, columns: [ts1, a1, b1, _row_id] }
- └─StreamExchange { dist: HashShard(ts2, a2) }
- └─StreamRowIdGen { row_id_index: 3 }
- └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts2, expr: (ts2 - '00:00:01':Interval) }], output_watermarks: [ts2] }
- └─StreamSource { source: t2, columns: [ts2, a2, b2, _row_id] }
+ └─StreamExchange { dist: HashShard(ts1, a1, _row_id, _row_id) }
+ └─StreamHashJoin [window, append_only] { type: Inner, predicate: ts1 = ts2 AND a1 = a2, output_watermarks: [ts1, ts2], output: [ts1, a1, b1, ts2, a2, b2, _row_id, _row_id] }
+ ├─StreamExchange { dist: HashShard(ts1, a1) }
+ │ └─StreamRowIdGen { row_id_index: 3 }
+ │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts1, expr: (ts1 - '00:00:01':Interval) }], output_watermarks: [ts1] }
+ │ └─StreamSource { source: t1, columns: [ts1, a1, b1, _row_id] }
+ └─StreamExchange { dist: HashShard(ts2, a2) }
+ └─StreamRowIdGen { row_id_index: 3 }
+ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts2, expr: (ts2 - '00:00:01':Interval) }], output_watermarks: [ts2] }
+ └─StreamSource { source: t2, columns: [ts2, a2, b2, _row_id] }
- name: Window join expression reorder
sql: |
create source t1 (ts1 timestamp with time zone, a1 int, b1 int, watermark for ts1 as ts1 - INTERVAL '1' SECOND) with (
@@ -34,12 +35,13 @@
select * from t1, t2 where a1 = a2 and ts1 = ts2;
stream_plan: |-
StreamMaterialize { columns: [ts1, a1, b1, ts2, a2, b2, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, a1, ts1], pk_columns: [_row_id, _row_id#1, a1, ts1], pk_conflict: NoCheck, watermark_columns: [ts1, ts2] }
- └─StreamHashJoin [window, append_only] { type: Inner, predicate: ts1 = ts2 AND a1 = a2, output_watermarks: [ts1, ts2], output: [ts1, a1, b1, ts2, a2, b2, _row_id, _row_id] }
- ├─StreamExchange { dist: HashShard(ts1, a1) }
- │ └─StreamRowIdGen { row_id_index: 3 }
- │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts1, expr: (ts1 - '00:00:01':Interval) }], output_watermarks: [ts1] }
- │ └─StreamSource { source: t1, columns: [ts1, a1, b1, _row_id] }
- └─StreamExchange { dist: HashShard(ts2, a2) }
- └─StreamRowIdGen { row_id_index: 3 }
- └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts2, expr: (ts2 - '00:00:01':Interval) }], output_watermarks: [ts2] }
- └─StreamSource { source: t2, columns: [ts2, a2, b2, _row_id] }
+ └─StreamExchange { dist: HashShard(ts1, a1, _row_id, _row_id) }
+ └─StreamHashJoin [window, append_only] { type: Inner, predicate: ts1 = ts2 AND a1 = a2, output_watermarks: [ts1, ts2], output: [ts1, a1, b1, ts2, a2, b2, _row_id, _row_id] }
+ ├─StreamExchange { dist: HashShard(ts1, a1) }
+ │ └─StreamRowIdGen { row_id_index: 3 }
+ │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts1, expr: (ts1 - '00:00:01':Interval) }], output_watermarks: [ts1] }
+ │ └─StreamSource { source: t1, columns: [ts1, a1, b1, _row_id] }
+ └─StreamExchange { dist: HashShard(ts2, a2) }
+ └─StreamRowIdGen { row_id_index: 3 }
+ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts2, expr: (ts2 - '00:00:01':Interval) }], output_watermarks: [ts2] }
+ └─StreamSource { source: t2, columns: [ts2, a2, b2, _row_id] }
diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs
index d8972436d5c78..9c87f1a34abbd 100644
--- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs
+++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs
@@ -151,7 +151,22 @@ impl StreamMaterialize {
TableType::MaterializedView => {
assert_matches!(user_distributed_by, RequiredDist::Any);
// ensure the same pk will not shuffle to different node
- RequiredDist::shard_by_key(input.schema().len(), input.expect_stream_key())
+ let required_dist =
+ RequiredDist::shard_by_key(input.schema().len(), input.expect_stream_key());
+
+ // If the input is a stream join, enforce the stream key as the materialized
+ // view distribution key to avoid slow backfilling caused by
+ // data skew of the dimension table join key.
+ // See for more information.
+ let is_stream_join = matches!(input.as_stream_hash_join(), Some(_join))
+ || matches!(input.as_stream_temporal_join(), Some(_join))
+ || matches!(input.as_stream_delta_join(), Some(_join));
+
+ if is_stream_join {
+ return Ok(required_dist.enforce(input, &Order::any()));
+ }
+
+ required_dist
}
TableType::Index => {
assert_matches!(
diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs
index b6e7715dd155f..2df1d7ae00bc3 100644
--- a/src/frontend/src/optimizer/property/distribution.rs
+++ b/src/frontend/src/optimizer/property/distribution.rs
@@ -331,7 +331,7 @@ impl RequiredDist {
}
}
- fn enforce(&self, plan: PlanRef, required_order: &Order) -> PlanRef {
+ pub fn enforce(&self, plan: PlanRef, required_order: &Order) -> PlanRef {
let dist = self.to_dist();
match plan.convention() {
Convention::Batch => BatchExchange::new(plan, required_order.clone(), dist).into(),
diff --git a/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs b/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs
index c05e52c927424..776692b2fab90 100644
--- a/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs
+++ b/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs
@@ -25,7 +25,7 @@ const ROOT_TABLE_CREATE: &str = "create table t1 (v1 int);";
const MV1: &str = "create materialized view m1 as select * from t1 where v1 > 5;";
const MV2: &str = "create materialized view m2 as select * from t1 where v1 > 10;";
const MV3: &str = "create materialized view m3 as select * from m2 where v1 < 15;";
-const MV4: &str = "create materialized view m4 as select m1.v1 as m1v, m3.v1 as m3v from m1 join m3 on m1.v1 = m3.v1;";
+const MV4: &str = "create materialized view m4 as select m1.v1 as m1v, m3.v1 as m3v from m1 join m3 on m1.v1 = m3.v1 limit 100;";
const MV5: &str = "create materialized view m5 as select * from m4;";
#[tokio::test]
@@ -40,6 +40,7 @@ async fn test_simple_cascade_materialized_view() -> Result<()> {
.locate_one_fragment([
identity_contains("materialize"),
no_identity_contains("chain"),
+ no_identity_contains("topn"),
no_identity_contains("hashjoin"),
])
.await?;
@@ -129,6 +130,7 @@ async fn test_diamond_cascade_materialized_view() -> Result<()> {
.locate_one_fragment([
identity_contains("materialize"),
no_identity_contains("chain"),
+ no_identity_contains("topn"),
no_identity_contains("hashjoin"),
])
.await?;
diff --git a/src/tests/simulation/tests/integration_tests/scale/plan.rs b/src/tests/simulation/tests/integration_tests/scale/plan.rs
index c7244dc826b42..8b62a58998a3f 100644
--- a/src/tests/simulation/tests/integration_tests/scale/plan.rs
+++ b/src/tests/simulation/tests/integration_tests/scale/plan.rs
@@ -39,10 +39,7 @@ async fn test_resize_normal() -> Result<()> {
.await?;
let join_fragment = cluster
- .locate_one_fragment([
- identity_contains("hashJoin"),
- identity_contains("materialize"),
- ])
+ .locate_one_fragment([identity_contains("hashJoin")])
.await?;
let join_fragment_id = join_fragment.inner.fragment_id;
@@ -270,7 +267,7 @@ async fn test_resize_no_shuffle() -> Result<()> {
session
.run(
"create materialized view mv7 as select mv1.v as mv1v, mv5.v as mv5v from mv1
-join mv5 on mv1.v = mv5.v;",
+join mv5 on mv1.v = mv5.v limit 1;",
)
.await?;
@@ -316,6 +313,7 @@ join mv5 on mv1.v = mv5.v;",
let top_materialize_fragment = cluster
.locate_one_fragment([
identity_contains("materialize"),
+ no_identity_contains("topn"),
no_identity_contains("chain"),
no_identity_contains("hashJoin"),
])